You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by tr...@apache.org on 2016/09/08 15:28:23 UTC

[01/50] [abbrv] flink git commit: [FLINK-4566] [network runtime] Properly preserve exception causes for ProducerFailedException [Forced Update!]

Repository: flink
Updated Branches:
  refs/heads/flip-6 0cf2a822b -> 1f3256b59 (forced update)


[FLINK-4566] [network runtime] Properly preserve exception causes for ProducerFailedException


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

Branch: refs/heads/flip-6
Commit: e227b10134e387f3c49804dc0cc4c223c30702e3
Parents: 761d0a0
Author: Stephan Ewen <se...@apache.org>
Authored: Fri Sep 2 11:45:25 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Sep 2 17:32:57 2016 +0200

----------------------------------------------------------------------
 .../partition/ProducerFailedException.java       | 19 +++++--------------
 .../partition/ProducerFailedExceptionTest.java   | 12 ++++++------
 2 files changed, 11 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/e227b101/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ProducerFailedException.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ProducerFailedException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ProducerFailedException.java
index 2b2acab..934234d 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ProducerFailedException.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ProducerFailedException.java
@@ -19,7 +19,7 @@
 package org.apache.flink.runtime.io.network.partition;
 
 import org.apache.flink.runtime.execution.CancelTaskException;
-import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.runtime.util.SerializedThrowable;
 
 /**
  * Network-stack level Exception to notify remote receiver about a failed
@@ -29,23 +29,14 @@ public class ProducerFailedException extends CancelTaskException {
 
 	private static final long serialVersionUID = -1555492656299526395L;
 
-	private final String causeAsString;
-
 	/**
 	 * The cause of the producer failure.
 	 *
-	 * Note: The cause will be stringified, because it might be an instance of
-	 * a user level Exception, which can not be deserialized by the remote
-	 * receiver's system class loader.
+	 * <p>The cause will be stored as a {@link SerializedThrowable}, because it might
+	 * be an instance of a user level Exception, which may not be possible to deserialize
+	 * by the remote receiver's system class loader.
 	 */
 	public ProducerFailedException(Throwable cause) {
-		this.causeAsString = cause != null ? ExceptionUtils.stringifyException(cause) : null;
-	}
-
-	/**
-	 * Returns the stringified cause of the producer failure.
-	 */
-	public String getCauseAsString() {
-		return causeAsString;
+		super(new SerializedThrowable(cause));
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/e227b101/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ProducerFailedExceptionTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ProducerFailedExceptionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ProducerFailedExceptionTest.java
index 042c136..ca2de0c 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ProducerFailedExceptionTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ProducerFailedExceptionTest.java
@@ -19,27 +19,27 @@
 package org.apache.flink.runtime.io.network.partition;
 
 import org.apache.flink.runtime.execution.CancelTaskException;
+import org.apache.flink.runtime.util.SerializedThrowable;
+
 import org.junit.Test;
 
 import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 
 public class ProducerFailedExceptionTest {
 
 	@Test
 	public void testInstanceOfCancelTaskException() throws Exception {
-		ProducerFailedException e = new ProducerFailedException(new Exception());
-		assertTrue(e instanceof CancelTaskException);
+		assertTrue(CancelTaskException.class.isAssignableFrom(ProducerFailedException.class));
 	}
 
 	@Test
-	public void testCauseIsStringified() throws Exception {
+	public void testCauseIsSerialized() throws Exception {
 		// Tests that the cause is stringified, because it might be an instance
 		// of a user level Exception, which can not be deserialized by the
 		// remote receiver's system class loader.
 		ProducerFailedException e = new ProducerFailedException(new Exception());
-		assertNull(e.getCause());
-		assertNotNull(e.getCauseAsString());
+		assertNotNull(e.getCause());
+		assertTrue(e.getCause() instanceof SerializedThrowable);
 	}
 }


[15/50] [abbrv] flink git commit: [FLINK-4458] Replace ForkableFlinkMiniCluster by LocalFlinkMiniCluster

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualExactlyOnceWithStreamReshardingTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualExactlyOnceWithStreamReshardingTest.java b/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualExactlyOnceWithStreamReshardingTest.java
index 934a795..6abea2a 100644
--- a/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualExactlyOnceWithStreamReshardingTest.java
+++ b/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualExactlyOnceWithStreamReshardingTest.java
@@ -27,11 +27,11 @@ import org.apache.commons.lang3.RandomStringUtils;
 import org.apache.flink.api.java.utils.ParameterTool;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
 import org.apache.flink.streaming.connectors.kinesis.config.ConsumerConfigConstants;
 import org.apache.flink.streaming.connectors.kinesis.testutils.ExactlyOnceValidatingConsumerThread;
 import org.apache.flink.streaming.connectors.kinesis.testutils.KinesisShardIdGenerator;
 import org.apache.flink.streaming.connectors.kinesis.util.AWSUtil;
-import org.apache.flink.test.util.ForkableFlinkMiniCluster;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -92,7 +92,7 @@ public class ManualExactlyOnceWithStreamReshardingTest {
 		flinkConfig.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 16);
 		flinkConfig.setString(ConfigConstants.RESTART_STRATEGY_FIXED_DELAY_DELAY, "0 s");
 
-		ForkableFlinkMiniCluster flink = new ForkableFlinkMiniCluster(flinkConfig, false);
+		LocalFlinkMiniCluster flink = new LocalFlinkMiniCluster(flinkConfig, false);
 		flink.start();
 
 		final int flinkPort = flink.getLeaderRPCPort();

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/ScalaStreamingMultipleProgramsTestBase.scala
----------------------------------------------------------------------
diff --git a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/ScalaStreamingMultipleProgramsTestBase.scala b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/ScalaStreamingMultipleProgramsTestBase.scala
index ee415d1..29b3a3e 100644
--- a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/ScalaStreamingMultipleProgramsTestBase.scala
+++ b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/ScalaStreamingMultipleProgramsTestBase.scala
@@ -18,8 +18,9 @@
 
 package org.apache.flink.streaming.api.scala
 
+import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster
 import org.apache.flink.streaming.util.TestStreamEnvironment
-import org.apache.flink.test.util.{ForkableFlinkMiniCluster, TestBaseUtils}
+import org.apache.flink.test.util.TestBaseUtils
 import org.scalatest.BeforeAndAfterAll
 import org.scalatest.junit.JUnitSuiteLike
 
@@ -29,7 +30,7 @@ trait ScalaStreamingMultipleProgramsTestBase
   with BeforeAndAfterAll {
 
   val parallelism = 4
-  var cluster: Option[ForkableFlinkMiniCluster] = None
+  var cluster: Option[LocalFlinkMiniCluster] = None
 
   override protected def beforeAll(): Unit = {
     val cluster = Some(

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/flink-test-utils-parent/flink-test-utils/pom.xml
----------------------------------------------------------------------
diff --git a/flink-test-utils-parent/flink-test-utils/pom.xml b/flink-test-utils-parent/flink-test-utils/pom.xml
index 2ab52b5..18ecfde 100644
--- a/flink-test-utils-parent/flink-test-utils/pom.xml
+++ b/flink-test-utils-parent/flink-test-utils/pom.xml
@@ -79,153 +79,4 @@ under the License.
 		</dependency>
 
 	</dependencies>
-
-	<build>
-		<plugins>
-			<!-- Scala Compiler -->
-			<plugin>
-				<groupId>net.alchim31.maven</groupId>
-				<artifactId>scala-maven-plugin</artifactId>
-				<version>3.1.4</version>
-				<executions>
-					<!-- Run scala compiler in the process-resources phase, so that dependencies 
-						on scala classes can be resolved later in the (Java) compile phase -->
-					<execution>
-						<id>scala-compile-first</id>
-						<phase>process-resources</phase>
-						<goals>
-							<goal>compile</goal>
-						</goals>
-					</execution>
-
-					<!-- Run scala compiler in the process-test-resources phase, so that 
-						dependencies on scala classes can be resolved later in the (Java) test-compile 
-						phase -->
-					<execution>
-						<id>scala-test-compile</id>
-						<phase>process-test-resources</phase>
-						<goals>
-							<goal>testCompile</goal>
-						</goals>
-					</execution>
-				</executions>
-				<configuration>
-					<jvmArgs>
-						<jvmArg>-Xms128m</jvmArg>
-						<jvmArg>-Xmx512m</jvmArg>
-					</jvmArgs>
-				</configuration>
-			</plugin>
-
-			<!-- Eclipse Integration -->
-			<plugin>
-				<groupId>org.apache.maven.plugins</groupId>
-				<artifactId>maven-eclipse-plugin</artifactId>
-				<version>2.8</version>
-				<configuration>
-					<downloadSources>true</downloadSources>
-					<projectnatures>
-						<projectnature>org.scala-ide.sdt.core.scalanature</projectnature>
-						<projectnature>org.eclipse.jdt.core.javanature</projectnature>
-					</projectnatures>
-					<buildcommands>
-						<buildcommand>org.scala-ide.sdt.core.scalabuilder</buildcommand>
-					</buildcommands>
-					<classpathContainers>
-						<classpathContainer>org.scala-ide.sdt.launching.SCALA_CONTAINER</classpathContainer>
-						<classpathContainer>org.eclipse.jdt.launching.JRE_CONTAINER</classpathContainer>
-					</classpathContainers>
-					<excludes>
-						<exclude>org.scala-lang:scala-library</exclude>
-						<exclude>org.scala-lang:scala-compiler</exclude>
-					</excludes>
-					<sourceIncludes>
-						<sourceInclude>**/*.scala</sourceInclude>
-						<sourceInclude>**/*.java</sourceInclude>
-					</sourceIncludes>
-				</configuration>
-			</plugin>
-
-			<!-- Adding scala source directories to build path -->
-			<plugin>
-				<groupId>org.codehaus.mojo</groupId>
-				<artifactId>build-helper-maven-plugin</artifactId>
-				<version>1.7</version>
-				<executions>
-					<!-- Add src/main/scala to eclipse build path -->
-					<execution>
-						<id>add-source</id>
-						<phase>generate-sources</phase>
-						<goals>
-							<goal>add-source</goal>
-						</goals>
-						<configuration>
-							<sources>
-								<source>src/main/scala</source>
-							</sources>
-						</configuration>
-					</execution>
-					<!-- Add src/test/scala to eclipse build path -->
-					<execution>
-						<id>add-test-source</id>
-						<phase>generate-test-sources</phase>
-						<goals>
-							<goal>add-test-source</goal>
-						</goals>
-						<configuration>
-							<sources>
-								<source>src/test/scala</source>
-							</sources>
-						</configuration>
-					</execution>
-				</executions>
-			</plugin>
-
-			<!-- Scala Code Style, most of the configuration done via plugin management -->
-			<plugin>
-				<groupId>org.scalastyle</groupId>
-				<artifactId>scalastyle-maven-plugin</artifactId>
-				<configuration>
-					<configLocation>${project.basedir}/../../tools/maven/scalastyle-config.xml</configLocation>
-				</configuration>
-			</plugin>
-
-		</plugins>
-		<pluginManagement>
-			<plugins>
-				<!--This plugin's configuration is used to store Eclipse m2e settings only. It has no influence on the Maven build itself.-->
-				<plugin>
-					<groupId>org.eclipse.m2e</groupId>
-					<artifactId>lifecycle-mapping</artifactId>
-					<version>1.0.0</version>
-					<configuration>
-						<lifecycleMappingMetadata>
-							<pluginExecutions>
-								<pluginExecution>
-									<pluginExecutionFilter>
-										<groupId>
-											net.alchim31.maven
-										</groupId>
-										<artifactId>
-											scala-maven-plugin
-										</artifactId>
-										<versionRange>
-											[3.1.4,)
-										</versionRange>
-										<goals>
-											<goal>compile</goal>
-											<goal>testCompile</goal>
-										</goals>
-									</pluginExecutionFilter>
-									<action>
-										<ignore/>
-									</action>
-								</pluginExecution>
-							</pluginExecutions>
-						</lifecycleMappingMetadata>
-					</configuration>
-				</plugin>
-			</plugins>
-		</pluginManagement>
-	</build>
 </project>

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/streaming/util/StreamingMultipleProgramsTestBase.java
----------------------------------------------------------------------
diff --git a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/streaming/util/StreamingMultipleProgramsTestBase.java b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/streaming/util/StreamingMultipleProgramsTestBase.java
index c5fbaf0..a478908 100644
--- a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/streaming/util/StreamingMultipleProgramsTestBase.java
+++ b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/streaming/util/StreamingMultipleProgramsTestBase.java
@@ -19,8 +19,8 @@
 package org.apache.flink.streaming.util;
 
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
 import org.apache.flink.test.util.AbstractTestBase;
-import org.apache.flink.test.util.ForkableFlinkMiniCluster;
 import org.apache.flink.test.util.TestBaseUtils;
 
 import org.junit.AfterClass;
@@ -61,7 +61,7 @@ public class StreamingMultipleProgramsTestBase extends AbstractTestBase {
 
 	protected static final int DEFAULT_PARALLELISM = 4;
 
-	protected static ForkableFlinkMiniCluster cluster;
+	protected static LocalFlinkMiniCluster cluster;
 
 	public StreamingMultipleProgramsTestBase() {
 		super(new Configuration());

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/streaming/util/TestStreamEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/streaming/util/TestStreamEnvironment.java b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/streaming/util/TestStreamEnvironment.java
index c700102..64c68dc 100644
--- a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/streaming/util/TestStreamEnvironment.java
+++ b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/streaming/util/TestStreamEnvironment.java
@@ -20,10 +20,10 @@ package org.apache.flink.streaming.util;
 
 import org.apache.flink.api.common.JobExecutionResult;
 import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironmentFactory;
 import org.apache.flink.streaming.api.graph.StreamGraph;
-import org.apache.flink.test.util.ForkableFlinkMiniCluster;
 import org.apache.flink.util.Preconditions;
 
 /**
@@ -32,10 +32,10 @@ import org.apache.flink.util.Preconditions;
 public class TestStreamEnvironment extends StreamExecutionEnvironment {
 	
 	/** The mini cluster in which this environment executes its jobs */
-	private ForkableFlinkMiniCluster executor;
+	private LocalFlinkMiniCluster executor;
 	
 
-	public TestStreamEnvironment(ForkableFlinkMiniCluster executor, int parallelism) {
+	public TestStreamEnvironment(LocalFlinkMiniCluster executor, int parallelism) {
 		this.executor = Preconditions.checkNotNull(executor);
 		setParallelism(parallelism);
 	}
@@ -57,7 +57,7 @@ public class TestStreamEnvironment extends StreamExecutionEnvironment {
 	 * @param cluster The test cluster to run the test program on.
 	 * @param parallelism The default parallelism for the test programs.
 	 */
-	public static void setAsContext(final ForkableFlinkMiniCluster cluster, final int parallelism) {
+	public static void setAsContext(final LocalFlinkMiniCluster cluster, final int parallelism) {
 		
 		StreamExecutionEnvironmentFactory factory = new StreamExecutionEnvironmentFactory() {
 			@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/AbstractTestBase.java
----------------------------------------------------------------------
diff --git a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/AbstractTestBase.java b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/AbstractTestBase.java
index c2da691..316fd21 100644
--- a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/AbstractTestBase.java
+++ b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/AbstractTestBase.java
@@ -21,6 +21,7 @@ package org.apache.flink.test.util;
 import com.google.common.base.Charsets;
 import com.google.common.io.Files;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
 import scala.concurrent.duration.FiniteDuration;
 
 import java.io.File;
@@ -48,7 +49,7 @@ public abstract class AbstractTestBase extends TestBaseUtils {
 	protected int numTaskManagers = 1;
 	
 	/** The mini cluster that runs the test programs */
-	protected ForkableFlinkMiniCluster executor;
+	protected LocalFlinkMiniCluster executor;
 	
 
 	public AbstractTestBase(Configuration config) {

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/MultipleProgramsTestBase.java
----------------------------------------------------------------------
diff --git a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/MultipleProgramsTestBase.java b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/MultipleProgramsTestBase.java
index d7f09bd..4e83245 100644
--- a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/MultipleProgramsTestBase.java
+++ b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/MultipleProgramsTestBase.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.test.util;
 
+import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.runners.Parameterized;
@@ -72,7 +73,7 @@ public class MultipleProgramsTestBase extends TestBaseUtils {
 
 	protected static boolean startWebServer = false;
 
-	protected static ForkableFlinkMiniCluster cluster = null;
+	protected static LocalFlinkMiniCluster cluster = null;
 	
 	// ------------------------------------------------------------------------
 	

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/TestBaseUtils.java
----------------------------------------------------------------------
diff --git a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/TestBaseUtils.java b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/TestBaseUtils.java
index 4014b80..b774f97 100644
--- a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/TestBaseUtils.java
+++ b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/TestBaseUtils.java
@@ -32,7 +32,8 @@ import org.apache.commons.io.IOUtils;
 import org.apache.flink.api.java.tuple.Tuple;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.testingUtils.TestingTaskManagerMessages;
+import org.apache.flink.runtime.messages.TaskManagerMessages;
+import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
 import org.apache.flink.util.TestLogger;
 
 import org.apache.hadoop.fs.FileSystem;
@@ -104,7 +105,7 @@ public class TestBaseUtils extends TestLogger {
 	}
 	
 	
-	public static ForkableFlinkMiniCluster startCluster(
+	public static LocalFlinkMiniCluster startCluster(
 		int numTaskManagers,
 		int taskManagerNumSlots,
 		boolean startWebserver,
@@ -126,7 +127,7 @@ public class TestBaseUtils extends TestLogger {
 		return startCluster(config, singleActorSystem);
 	}
 
-	public static ForkableFlinkMiniCluster startCluster(
+	public static LocalFlinkMiniCluster startCluster(
 		Configuration config,
 		boolean singleActorSystem) throws Exception {
 
@@ -147,7 +148,7 @@ public class TestBaseUtils extends TestLogger {
 		
 		config.setString(ConfigConstants.TASK_MANAGER_LOG_PATH_KEY, logFile.toString());
 
-		ForkableFlinkMiniCluster cluster =  new ForkableFlinkMiniCluster(config, singleActorSystem);
+		LocalFlinkMiniCluster cluster =  new LocalFlinkMiniCluster(config, singleActorSystem);
 
 		cluster.start();
 
@@ -155,7 +156,7 @@ public class TestBaseUtils extends TestLogger {
 	}
 
 
-	public static void stopCluster(ForkableFlinkMiniCluster executor, FiniteDuration timeout) throws Exception {
+	public static void stopCluster(LocalFlinkMiniCluster executor, FiniteDuration timeout) throws Exception {
 		if (logDir != null) {
 			FileUtils.deleteDirectory(logDir);
 		}
@@ -169,11 +170,15 @@ public class TestBaseUtils extends TestLogger {
 				List<Future<Object>> numActiveConnectionsResponseFutures = new ArrayList<>();
 
 				for (ActorRef tm : tms) {
-					bcVariableManagerResponseFutures.add(Patterns.ask(tm, TestingTaskManagerMessages
-							.RequestBroadcastVariablesWithReferences$.MODULE$, new Timeout(timeout)));
-
-					numActiveConnectionsResponseFutures.add(Patterns.ask(tm, TestingTaskManagerMessages
-							.RequestNumActiveConnections$.MODULE$, new Timeout(timeout)));
+					bcVariableManagerResponseFutures.add(Patterns.ask(
+						tm,
+						TaskManagerMessages.getRequestBroadcastVariablesWithReferences(),
+						new Timeout(timeout)));
+
+					numActiveConnectionsResponseFutures.add(Patterns.ask(
+						tm,
+						TaskManagerMessages.getRequestNumActiveConnections(),
+						new Timeout(timeout)));
 				}
 
 				Future<Iterable<Object>> bcVariableManagerFutureResponses = Futures.sequence(
@@ -182,8 +187,7 @@ public class TestBaseUtils extends TestLogger {
 				Iterable<Object> responses = Await.result(bcVariableManagerFutureResponses, timeout);
 
 				for (Object response : responses) {
-					numUnreleasedBCVars += ((TestingTaskManagerMessages
-							.ResponseBroadcastVariablesWithReferences) response).number();
+					numUnreleasedBCVars += ((TaskManagerMessages.ResponseBroadcastVariablesWithReferences) response).number();
 				}
 
 				Future<Iterable<Object>> numActiveConnectionsFutureResponses = Futures.sequence(
@@ -192,8 +196,7 @@ public class TestBaseUtils extends TestLogger {
 				responses = Await.result(numActiveConnectionsFutureResponses, timeout);
 
 				for (Object response : responses) {
-					numActiveConnections += ((TestingTaskManagerMessages
-							.ResponseNumActiveConnections) response).number();
+					numActiveConnections += ((TaskManagerMessages.ResponseNumActiveConnections) response).number();
 				}
 			}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/TestEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/TestEnvironment.java b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/TestEnvironment.java
index 7cb88be..aea8152 100644
--- a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/TestEnvironment.java
+++ b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/util/TestEnvironment.java
@@ -29,10 +29,11 @@ import org.apache.flink.optimizer.plan.OptimizedPlan;
 import org.apache.flink.optimizer.plandump.PlanJSONDumpGenerator;
 import org.apache.flink.optimizer.plantranslate.JobGraphGenerator;
 import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
 
 public class TestEnvironment extends ExecutionEnvironment {
 
-	private final ForkableFlinkMiniCluster executor;
+	private final LocalFlinkMiniCluster executor;
 
 	private TestEnvironment lastEnv = null;
 
@@ -46,7 +47,7 @@ public class TestEnvironment extends ExecutionEnvironment {
 		}
 	}
 
-	public TestEnvironment(ForkableFlinkMiniCluster executor, int parallelism) {
+	public TestEnvironment(LocalFlinkMiniCluster executor, int parallelism) {
 		this.executor = executor;
 		setParallelism(parallelism);
 
@@ -54,7 +55,7 @@ public class TestEnvironment extends ExecutionEnvironment {
 		getConfig().setCodeAnalysisMode(CodeAnalysisMode.DISABLE);
 	}
 
-	public TestEnvironment(ForkableFlinkMiniCluster executor, int parallelism, boolean isObjectReuseEnabled) {
+	public TestEnvironment(LocalFlinkMiniCluster executor, int parallelism, boolean isObjectReuseEnabled) {
 		this(executor, parallelism);
 
 		if (isObjectReuseEnabled) {

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/flink-test-utils-parent/flink-test-utils/src/main/scala/org/apache/flink/test/util/ForkableFlinkMiniCluster.scala
----------------------------------------------------------------------
diff --git a/flink-test-utils-parent/flink-test-utils/src/main/scala/org/apache/flink/test/util/ForkableFlinkMiniCluster.scala b/flink-test-utils-parent/flink-test-utils/src/main/scala/org/apache/flink/test/util/ForkableFlinkMiniCluster.scala
deleted file mode 100644
index fa3135a..0000000
--- a/flink-test-utils-parent/flink-test-utils/src/main/scala/org/apache/flink/test/util/ForkableFlinkMiniCluster.scala
+++ /dev/null
@@ -1,335 +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.util
-
-import java.util.concurrent.TimeoutException
-
-import akka.actor.{ActorRef, ActorSystem}
-import akka.pattern.Patterns._
-import akka.pattern.ask
-
-import org.apache.curator.test.TestingCluster
-import org.apache.flink.configuration.{ConfigConstants, Configuration}
-import org.apache.flink.runtime.akka.AkkaUtils
-import org.apache.flink.runtime.clusterframework.FlinkResourceManager
-import org.apache.flink.runtime.clusterframework.types.ResourceID
-import org.apache.flink.runtime.jobmanager.{JobManager, HighAvailabilityMode}
-import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster
-import org.apache.flink.runtime.taskmanager.TaskManager
-import org.apache.flink.runtime.testingUtils.TestingTaskManagerMessages.NotifyWhenRegisteredAtJobManager
-import org.apache.flink.runtime.testingUtils.{TestingJobManager, TestingMemoryArchivist, TestingTaskManager}
-import org.apache.flink.runtime.testutils.TestingResourceManager
-
-import scala.concurrent.{Await, Future}
-import scala.concurrent.duration._
-
-/**
- * A forkable mini cluster is a special case of the mini cluster, used for parallel test execution
- * on build servers. If multiple tests run in parallel, the cluster picks up the fork number and
- * uses it to avoid port conflicts.
- *
- * @param userConfiguration Configuration object with the user provided configuration values
- * @param singleActorSystem true, if all actors (JobManager and TaskManager) shall be run in the
- *                          same [[ActorSystem]], otherwise false.
- */
-class ForkableFlinkMiniCluster(
-    userConfiguration: Configuration,
-    singleActorSystem: Boolean)
-  extends LocalFlinkMiniCluster(userConfiguration, singleActorSystem) {
-
-  def this(userConfiguration: Configuration) = this(userConfiguration, true)
-
-  // --------------------------------------------------------------------------
-
-  var zookeeperCluster: Option[TestingCluster] = None
-
-  override def generateConfiguration(userConfiguration: Configuration): Configuration = {
-    val forkNumberString = System.getProperty("forkNumber")
-
-    val forkNumber = try {
-      Integer.parseInt(forkNumberString)
-    }
-    catch {
-      case e: NumberFormatException => -1
-    }
-
-    val config = userConfiguration.clone()
-
-    if (forkNumber != -1) {
-      val jobManagerRPC = 1024 + forkNumber*400
-      val taskManagerRPC = 1024 + forkNumber*400 + 100
-      val taskManagerData = 1024 + forkNumber*400 + 200
-      val resourceManagerRPC = 1024 + forkNumber*400 + 300
-
-      config.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, jobManagerRPC)
-      config.setInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, taskManagerRPC)
-      config.setInteger(ConfigConstants.TASK_MANAGER_DATA_PORT_KEY, taskManagerData)
-      config.setInteger(ConfigConstants.RESOURCE_MANAGER_IPC_PORT_KEY, resourceManagerRPC)
-    }
-
-    super.generateConfiguration(config)
-  }
-
-  override def startJobManager(index: Int, actorSystem: ActorSystem): ActorRef = {
-    val config = configuration.clone()
-
-    val jobManagerName = getJobManagerName(index)
-    val archiveName = getArchiveName(index)
-
-    val jobManagerPort = config.getInteger(
-      ConfigConstants.JOB_MANAGER_IPC_PORT_KEY,
-      ConfigConstants.DEFAULT_JOB_MANAGER_IPC_PORT)
-
-    if (jobManagerPort > 0) {
-      config.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, jobManagerPort + index)
-    }
-
-    val (jobManager, _) = JobManager.startJobManagerActors(
-      config,
-      actorSystem,
-      Some(jobManagerName),
-      Some(archiveName),
-      classOf[TestingJobManager],
-      classOf[TestingMemoryArchivist])
-
-    jobManager
-  }
-
-  override def startResourceManager(index: Int, system: ActorSystem): ActorRef = {
-    val config = configuration.clone()
-
-    val resourceManagerName = getResourceManagerName(index)
-
-    val resourceManagerPort = config.getInteger(
-      ConfigConstants.RESOURCE_MANAGER_IPC_PORT_KEY,
-      ConfigConstants.DEFAULT_RESOURCE_MANAGER_IPC_PORT)
-
-    if (resourceManagerPort > 0) {
-      config.setInteger(ConfigConstants.RESOURCE_MANAGER_IPC_PORT_KEY, resourceManagerPort + index)
-    }
-
-    val resourceManager = FlinkResourceManager.startResourceManagerActors(
-      config,
-      system,
-      createLeaderRetrievalService(),
-      classOf[TestingResourceManager],
-      resourceManagerName)
-
-    resourceManager
-  }
-
-  override def startTaskManager(index: Int, system: ActorSystem): ActorRef = {
-    val config = configuration.clone()
-
-    val rpcPort = config.getInteger(
-      ConfigConstants.TASK_MANAGER_IPC_PORT_KEY,
-      ConfigConstants.DEFAULT_TASK_MANAGER_IPC_PORT)
-
-    val dataPort = config.getInteger(
-      ConfigConstants.TASK_MANAGER_DATA_PORT_KEY,
-      ConfigConstants.DEFAULT_TASK_MANAGER_DATA_PORT)
-
-    if (rpcPort > 0) {
-      config.setInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, rpcPort + index)
-    }
-    if (dataPort > 0) {
-      config.setInteger(ConfigConstants.TASK_MANAGER_DATA_PORT_KEY, dataPort + index)
-    }
-
-    val localExecution = numTaskManagers == 1
-
-    TaskManager.startTaskManagerComponentsAndActor(
-      config,
-      ResourceID.generate(),
-      system,
-      hostname,
-      Some(TaskManager.TASK_MANAGER_NAME + index),
-      Some(createLeaderRetrievalService()),
-      localExecution,
-      classOf[TestingTaskManager])
-  }
-
-  def addTaskManager(): Unit = {
-    if (useSingleActorSystem) {
-      (jobManagerActorSystems, taskManagerActors) match {
-        case (Some(jmSystems), Some(tmActors)) =>
-          val index = numTaskManagers
-          taskManagerActors = Some(tmActors :+ startTaskManager(index, jmSystems(0)))
-          numTaskManagers += 1
-        case _ => throw new IllegalStateException("Cluster has not been started properly.")
-      }
-    } else {
-      (taskManagerActorSystems, taskManagerActors) match {
-        case (Some(tmSystems), Some(tmActors)) =>
-          val index = numTaskManagers
-          val newTmSystem = startTaskManagerActorSystem(index)
-          val newTmActor = startTaskManager(index, newTmSystem)
-
-          taskManagerActorSystems = Some(tmSystems :+ newTmSystem)
-          taskManagerActors = Some(tmActors :+ newTmActor)
-
-          numTaskManagers += 1
-        case _ => throw new IllegalStateException("Cluster has not been started properly.")
-      }
-    }
-  }
-
-  def restartLeadingJobManager(): Unit = {
-    this.synchronized {
-      (jobManagerActorSystems, jobManagerActors) match {
-        case (Some(jmActorSystems), Some(jmActors)) =>
-          val leader = getLeaderGateway(AkkaUtils.getTimeout(configuration))
-          val index = getLeaderIndex(AkkaUtils.getTimeout(configuration))
-
-          clearLeader()
-
-          val stopped = gracefulStop(leader.actor(), ForkableFlinkMiniCluster.MAX_RESTART_DURATION)
-          Await.result(stopped, ForkableFlinkMiniCluster.MAX_RESTART_DURATION)
-
-          if(!singleActorSystem) {
-            jmActorSystems(index).shutdown()
-            jmActorSystems(index).awaitTermination()
-          }
-
-          val newJobManagerActorSystem = if(!singleActorSystem) {
-            startJobManagerActorSystem(index)
-          } else {
-            jmActorSystems.head
-          }
-
-          val newJobManagerActor = startJobManager(index, newJobManagerActorSystem)
-
-          jobManagerActors = Some(jmActors.patch(index, Seq(newJobManagerActor), 1))
-          jobManagerActorSystems = Some(jmActorSystems.patch(
-            index,
-            Seq(newJobManagerActorSystem),
-            1))
-
-          val lrs = createLeaderRetrievalService()
-
-          jobManagerLeaderRetrievalService = Some(lrs)
-          lrs.start(this)
-
-        case _ => throw new Exception("The JobManager of the ForkableFlinkMiniCluster have not " +
-          "been started properly.")
-      }
-    }
-  }
-
-
-  def restartTaskManager(index: Int): Unit = {
-    (taskManagerActorSystems, taskManagerActors) match {
-      case (Some(tmActorSystems), Some(tmActors)) =>
-        val stopped = gracefulStop(tmActors(index), ForkableFlinkMiniCluster.MAX_RESTART_DURATION)
-        Await.result(stopped, ForkableFlinkMiniCluster.MAX_RESTART_DURATION)
-
-        if(!singleActorSystem) {
-          tmActorSystems(index).shutdown()
-          tmActorSystems(index).awaitTermination()
-        }
-
-        val taskManagerActorSystem  = if(!singleActorSystem) {
-          startTaskManagerActorSystem(index)
-        } else {
-          tmActorSystems.head
-        }
-
-        val taskManagerActor = startTaskManager(index, taskManagerActorSystem)
-
-        taskManagerActors = Some(tmActors.patch(index, Seq(taskManagerActor), 1))
-        taskManagerActorSystems = Some(tmActorSystems.patch(index, Seq(taskManagerActorSystem), 1))
-
-      case _ => throw new Exception("The TaskManager of the ForkableFlinkMiniCluster have not " +
-        "been started properly.")
-    }
-  }
-
-  override def start(): Unit = {
-    val zookeeperURL = configuration.getString(ConfigConstants.HA_ZOOKEEPER_QUORUM_KEY, "")
-
-    zookeeperCluster = if (haMode == HighAvailabilityMode.ZOOKEEPER &&
-      zookeeperURL.equals("")) {
-      LOG.info("Starting ZooKeeper cluster.")
-
-      val testingCluster = new TestingCluster(1)
-
-      configuration.setString(ConfigConstants.HA_ZOOKEEPER_QUORUM_KEY,
-        testingCluster.getConnectString)
-
-      testingCluster.start()
-
-      Some(testingCluster)
-    } else {
-      None
-    }
-
-    super.start()
-  }
-
-  override def stop(): Unit = {
-    super.stop()
-
-    zookeeperCluster.foreach{
-      LOG.info("Stopping ZooKeeper cluster.")
-      _.close()
-    }
-  }
-
-  def waitForTaskManagersToBeRegisteredAtJobManager(jobManager: ActorRef): Unit = {
-    val futures = taskManagerActors.map {
-      _.map {
-        tm => (tm ? NotifyWhenRegisteredAtJobManager(jobManager))(timeout)
-      }
-    }.getOrElse(Seq())
-
-    try {
-      Await.ready(Future.sequence(futures), timeout)
-    } catch {
-      case t: TimeoutException =>
-        throw new Exception("Timeout while waiting for TaskManagers to register at " +
-          s"${jobManager.path}")
-    }
-
-  }
-}
-
-object ForkableFlinkMiniCluster {
-
-  val MAX_RESTART_DURATION = 2 minute
-
-  val DEFAULT_MINICLUSTER_AKKA_ASK_TIMEOUT = "200 s"
-
-  def startCluster(
-                    numSlots: Int,
-                    numTaskManagers: Int,
-                    timeout: String = DEFAULT_MINICLUSTER_AKKA_ASK_TIMEOUT)
-  : ForkableFlinkMiniCluster = {
-
-    val config = new Configuration()
-    config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, numSlots)
-    config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, numTaskManagers)
-    config.setString(ConfigConstants.AKKA_ASK_TIMEOUT, timeout)
-
-    val cluster = new ForkableFlinkMiniCluster(config)
-
-    cluster.start()
-
-    cluster
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorErrorITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorErrorITCase.java b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorErrorITCase.java
index cac8451..cc70fee 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorErrorITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorErrorITCase.java
@@ -28,7 +28,7 @@ import org.apache.flink.client.program.ProgramInvocationException;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.client.JobExecutionException;
-import org.apache.flink.test.util.ForkableFlinkMiniCluster;
+import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
@@ -44,7 +44,7 @@ import static org.junit.Assert.fail;
  */
 public class AccumulatorErrorITCase {
 
-	private static ForkableFlinkMiniCluster cluster;
+	private static LocalFlinkMiniCluster cluster;
 
 	@BeforeClass
 	public static void startCluster() {
@@ -53,7 +53,7 @@ public class AccumulatorErrorITCase {
 			config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 2);
 			config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 3);
 			config.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 12);
-			cluster = new ForkableFlinkMiniCluster(config, false);
+			cluster = new LocalFlinkMiniCluster(config, false);
 
 			cluster.start();
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorLiveITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorLiveITCase.java b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorLiveITCase.java
index 49e18e0..624bfff 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorLiveITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorLiveITCase.java
@@ -234,7 +234,6 @@ public class AccumulatorLiveITCase {
 				fail("Wrong accumulator results when map task begins execution.");
 			}
 
-
 			int expectedAccVal = 0;
 
 			/* for mapper task */

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/flink-tests/src/test/java/org/apache/flink/test/cancelling/CancelingTestBase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/cancelling/CancelingTestBase.java b/flink-tests/src/test/java/org/apache/flink/test/cancelling/CancelingTestBase.java
index 9671fce..8a08f15 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/cancelling/CancelingTestBase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/cancelling/CancelingTestBase.java
@@ -22,8 +22,8 @@ package org.apache.flink.test.cancelling;
 import java.util.concurrent.TimeUnit;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
 import org.apache.flink.runtime.testingUtils.TestingUtils;
-import org.apache.flink.test.util.ForkableFlinkMiniCluster;
 import org.apache.flink.util.TestLogger;
 import org.junit.Assert;
 
@@ -39,7 +39,6 @@ import org.apache.flink.runtime.jobgraph.JobStatus;
 import static org.apache.flink.runtime.taskmanager.TaskCancelTest.awaitRunning;
 import static org.apache.flink.runtime.taskmanager.TaskCancelTest.cancelJob;
 import org.apache.flink.runtime.testutils.JobManagerActorTestUtils;
-import org.apache.flink.util.StringUtils;
 import org.apache.hadoop.fs.FileSystem;
 
 import org.junit.After;
@@ -65,7 +64,7 @@ public abstract class CancelingTestBase extends TestLogger {
 
 	// --------------------------------------------------------------------------------------------
 	
-	protected ForkableFlinkMiniCluster executor;
+	protected LocalFlinkMiniCluster executor;
 
 	protected int taskManagerNumSlots = DEFAULT_TASK_MANAGER_NUM_SLOTS;
 	
@@ -88,7 +87,7 @@ public abstract class CancelingTestBase extends TestLogger {
 		config.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY, 4096);
 		config.setInteger(ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY, 2048);
 
-		this.executor = new ForkableFlinkMiniCluster(config, false);
+		this.executor = new LocalFlinkMiniCluster(config, false);
 		this.executor.start();
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeAllWindowCheckpointingITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeAllWindowCheckpointingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeAllWindowCheckpointingITCase.java
index 163fb42..94ff66f 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeAllWindowCheckpointingITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeAllWindowCheckpointingITCase.java
@@ -25,6 +25,7 @@ import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.tuple.Tuple4;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
 import org.apache.flink.streaming.api.TimeCharacteristic;
 import org.apache.flink.runtime.state.CheckpointListener;
 import org.apache.flink.streaming.api.checkpoint.Checkpointed;
@@ -35,7 +36,6 @@ import org.apache.flink.streaming.api.functions.windowing.RichAllWindowFunction;
 import org.apache.flink.streaming.api.watermark.Watermark;
 import org.apache.flink.streaming.api.windowing.time.Time;
 import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
-import org.apache.flink.test.util.ForkableFlinkMiniCluster;
 import org.apache.flink.test.util.SuccessException;
 import org.apache.flink.util.Collector;
 import org.apache.flink.util.TestLogger;
@@ -60,7 +60,7 @@ public class EventTimeAllWindowCheckpointingITCase extends TestLogger {
 
 	private static final int PARALLELISM = 4;
 
-	private static ForkableFlinkMiniCluster cluster;
+	private static LocalFlinkMiniCluster cluster;
 
 
 	@BeforeClass
@@ -71,7 +71,7 @@ public class EventTimeAllWindowCheckpointingITCase extends TestLogger {
 		config.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 48);
 		config.setString(ConfigConstants.DEFAULT_AKKA_LOOKUP_TIMEOUT, "60 s");
 		config.setString(ConfigConstants.DEFAULT_AKKA_ASK_TIMEOUT, "60 s");
-		cluster = new ForkableFlinkMiniCluster(config, false);
+		cluster = new LocalFlinkMiniCluster(config, false);
 		cluster.start();
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeWindowCheckpointingITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeWindowCheckpointingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeWindowCheckpointingITCase.java
index fa5339d..0aee128 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeWindowCheckpointingITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeWindowCheckpointingITCase.java
@@ -28,6 +28,7 @@ import org.apache.flink.api.java.tuple.Tuple4;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.contrib.streaming.state.RocksDBStateBackend;
+import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
 import org.apache.flink.runtime.state.AbstractStateBackend;
 import org.apache.flink.runtime.state.filesystem.FsStateBackend;
 import org.apache.flink.runtime.state.memory.MemoryStateBackend;
@@ -41,7 +42,6 @@ import org.apache.flink.streaming.api.functions.windowing.RichWindowFunction;
 import org.apache.flink.streaming.api.watermark.Watermark;
 import org.apache.flink.streaming.api.windowing.time.Time;
 import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
-import org.apache.flink.test.util.ForkableFlinkMiniCluster;
 import org.apache.flink.test.util.SuccessException;
 import org.apache.flink.util.Collector;
 import org.apache.flink.util.TestLogger;
@@ -76,7 +76,7 @@ public class EventTimeWindowCheckpointingITCase extends TestLogger {
 	private static final int MAX_MEM_STATE_SIZE = 10 * 1024 * 1024;
 	private static final int PARALLELISM = 4;
 
-	private static ForkableFlinkMiniCluster cluster;
+	private static LocalFlinkMiniCluster cluster;
 
 	@Rule
 	public TemporaryFolder tempFolder = new TemporaryFolder();
@@ -95,7 +95,7 @@ public class EventTimeWindowCheckpointingITCase extends TestLogger {
 		config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, PARALLELISM / 2);
 		config.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 48);
 
-		cluster = new ForkableFlinkMiniCluster(config, false);
+		cluster = new LocalFlinkMiniCluster(config, false);
 		cluster.start();
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RescalingITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RescalingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RescalingITCase.java
index 8915bff..7f1d7f3 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RescalingITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RescalingITCase.java
@@ -36,6 +36,7 @@ import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.messages.JobManagerMessages;
 import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
 import org.apache.flink.runtime.state.filesystem.FsStateBackendFactory;
+import org.apache.flink.runtime.testingUtils.TestingCluster;
 import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages;
 import org.apache.flink.streaming.api.checkpoint.Checkpointed;
 import org.apache.flink.streaming.api.datastream.DataStream;
@@ -43,7 +44,6 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
 import org.apache.flink.streaming.api.functions.sink.SinkFunction;
 import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
-import org.apache.flink.test.util.ForkableFlinkMiniCluster;
 import org.apache.flink.util.Collector;
 import org.apache.flink.util.TestLogger;
 import org.junit.AfterClass;
@@ -73,7 +73,7 @@ public class RescalingITCase extends TestLogger {
 	private static int slotsPerTaskManager = 2;
 	private static int numSlots = numTaskManagers * slotsPerTaskManager;
 
-	private static ForkableFlinkMiniCluster cluster;
+	private static TestingCluster cluster;
 
 	@ClassRule
 	public static TemporaryFolder temporaryFolder = new TemporaryFolder();
@@ -92,7 +92,7 @@ public class RescalingITCase extends TestLogger {
 		config.setString(SavepointStoreFactory.SAVEPOINT_BACKEND_KEY, "filesystem");
 		config.setString(SavepointStoreFactory.SAVEPOINT_DIRECTORY_KEY, savepointDir.toURI().toString());
 
-		cluster = new ForkableFlinkMiniCluster(config);
+		cluster = new TestingCluster(config);
 		cluster.start();
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java
index 550ba75..7409fe7 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java
@@ -43,7 +43,6 @@ import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.messages.JobManagerMessages.CancelJob;
 import org.apache.flink.runtime.messages.JobManagerMessages.DisposeSavepoint;
 import org.apache.flink.runtime.messages.JobManagerMessages.TriggerSavepoint;
-import org.apache.flink.runtime.messages.JobManagerMessages.TriggerSavepointFailure;
 import org.apache.flink.runtime.messages.JobManagerMessages.TriggerSavepointSuccess;
 import org.apache.flink.runtime.state.ChainedStateHandle;
 import org.apache.flink.runtime.state.CheckpointListener;
@@ -51,6 +50,7 @@ import org.apache.flink.runtime.state.StreamStateHandle;
 import org.apache.flink.runtime.state.filesystem.FileStateHandle;
 import org.apache.flink.runtime.state.filesystem.FsStateBackend;
 import org.apache.flink.runtime.state.filesystem.FsStateBackendFactory;
+import org.apache.flink.runtime.testingUtils.TestingCluster;
 import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages.NotifyWhenJobRemoved;
 import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages.RequestSavepoint;
 import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages.ResponseSavepoint;
@@ -62,8 +62,6 @@ import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
 import org.apache.flink.streaming.api.functions.source.SourceFunction;
-import org.apache.flink.test.util.ForkableFlinkMiniCluster;
-import org.apache.flink.testutils.junit.RetryOnFailure;
 import org.apache.flink.testutils.junit.RetryRule;
 import org.apache.flink.util.TestLogger;
 import org.junit.Rule;
@@ -76,7 +74,6 @@ import scala.concurrent.duration.Deadline;
 import scala.concurrent.duration.FiniteDuration;
 
 import java.io.File;
-import java.io.FileNotFoundException;
 import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -137,7 +134,7 @@ public class SavepointITCase extends TestLogger {
 
 		LOG.info("Created temporary directory: " + tmpDir + ".");
 
-		ForkableFlinkMiniCluster flink = null;
+		TestingCluster flink = null;
 
 		try {
 			// Create a test actor system
@@ -168,7 +165,7 @@ public class SavepointITCase extends TestLogger {
 			LOG.info("Flink configuration: " + config + ".");
 
 			// Start Flink
-			flink = new ForkableFlinkMiniCluster(config);
+			flink = new TestingCluster(config);
 			LOG.info("Starting Flink cluster.");
 			flink.start();
 
@@ -261,7 +258,7 @@ public class SavepointITCase extends TestLogger {
 			LOG.info("JobManager: " + jobManager + ".");
 
 			final Throwable[] error = new Throwable[1];
-			final ForkableFlinkMiniCluster finalFlink = flink;
+			final TestingCluster finalFlink = flink;
 			final Multimap<JobVertexID, TaskDeploymentDescriptor> tdds = HashMultimap.create();
 			new JavaTestKit(testActorSystem) {{
 
@@ -422,7 +419,7 @@ public class SavepointITCase extends TestLogger {
 
 		LOG.info("Created temporary directory: " + tmpDir + ".");
 
-		ForkableFlinkMiniCluster flink = null;
+		TestingCluster flink = null;
 		List<File> checkpointFiles = new ArrayList<>();
 
 		try {
@@ -447,7 +444,7 @@ public class SavepointITCase extends TestLogger {
 			LOG.info("Flink configuration: " + config + ".");
 
 			// Start Flink
-			flink = new ForkableFlinkMiniCluster(config);
+			flink = new TestingCluster(config);
 			LOG.info("Starting Flink cluster.");
 			flink.start();
 
@@ -559,7 +556,7 @@ public class SavepointITCase extends TestLogger {
 		// Test deadline
 		final Deadline deadline = new FiniteDuration(5, TimeUnit.MINUTES).fromNow();
 
-		ForkableFlinkMiniCluster flink = null;
+		TestingCluster flink = null;
 
 		try {
 			// Flink configuration
@@ -570,7 +567,7 @@ public class SavepointITCase extends TestLogger {
 			LOG.info("Flink configuration: " + config + ".");
 
 			// Start Flink
-			flink = new ForkableFlinkMiniCluster(config);
+			flink = new TestingCluster(config);
 			LOG.info("Starting Flink cluster.");
 			flink.start();
 

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointNotifierITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointNotifierITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointNotifierITCase.java
index cf15052..6bf511f 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointNotifierITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointNotifierITCase.java
@@ -24,6 +24,7 @@ import org.apache.flink.api.common.functions.RichReduceFunction;
 import org.apache.flink.api.java.tuple.Tuple1;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
 import org.apache.flink.runtime.state.CheckpointListener;
 import org.apache.flink.streaming.api.checkpoint.Checkpointed;
 import org.apache.flink.streaming.api.datastream.DataStream;
@@ -34,7 +35,6 @@ import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction;
 import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
 import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
 import org.apache.flink.streaming.api.operators.TwoInputStreamOperator;
-import org.apache.flink.test.util.ForkableFlinkMiniCluster;
 import org.apache.flink.util.Collector;
 
 import org.apache.flink.util.TestLogger;
@@ -80,7 +80,7 @@ public class StreamCheckpointNotifierITCase extends TestLogger {
 	private static final int NUM_TASK_SLOTS = 3;
 	private static final int PARALLELISM = NUM_TASK_MANAGERS * NUM_TASK_SLOTS;
 
-	private static ForkableFlinkMiniCluster cluster;
+	private static LocalFlinkMiniCluster cluster;
 
 	@BeforeClass
 	public static void startCluster() {
@@ -91,7 +91,7 @@ public class StreamCheckpointNotifierITCase extends TestLogger {
 			config.setString(ConfigConstants.RESTART_STRATEGY_FIXED_DELAY_DELAY, "0 ms");
 			config.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 12);
 
-			cluster = new ForkableFlinkMiniCluster(config, false);
+			cluster = new LocalFlinkMiniCluster(config, false);
 			cluster.start();
 		}
 		catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamFaultToleranceTestBase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamFaultToleranceTestBase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamFaultToleranceTestBase.java
index 67c05e5..5f6cd4a 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamFaultToleranceTestBase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamFaultToleranceTestBase.java
@@ -20,8 +20,8 @@ package org.apache.flink.test.checkpointing;
 
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.test.util.ForkableFlinkMiniCluster;
 
 import org.apache.flink.test.util.TestUtils;
 import org.apache.flink.util.TestLogger;
@@ -43,7 +43,7 @@ public abstract class StreamFaultToleranceTestBase extends TestLogger {
 	protected static final int NUM_TASK_SLOTS = 4;
 	protected static final int PARALLELISM = NUM_TASK_MANAGERS * NUM_TASK_SLOTS;
 
-	private static ForkableFlinkMiniCluster cluster;
+	private static LocalFlinkMiniCluster cluster;
 
 	@BeforeClass
 	public static void startCluster() {
@@ -53,7 +53,7 @@ public abstract class StreamFaultToleranceTestBase extends TestLogger {
 			config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, NUM_TASK_SLOTS);
 			config.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 12);
 			
-			cluster = new ForkableFlinkMiniCluster(config, false);
+			cluster = new LocalFlinkMiniCluster(config, false);
 
 			cluster.start();
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/flink-tests/src/test/java/org/apache/flink/test/checkpointing/WindowCheckpointingITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/WindowCheckpointingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/WindowCheckpointingITCase.java
index 2e6ce78..e424a8d 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/WindowCheckpointingITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/WindowCheckpointingITCase.java
@@ -25,6 +25,7 @@ import org.apache.flink.api.java.tuple.Tuple;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
 import org.apache.flink.streaming.api.TimeCharacteristic;
 import org.apache.flink.runtime.state.CheckpointListener;
 import org.apache.flink.streaming.api.checkpoint.Checkpointed;
@@ -34,7 +35,6 @@ import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
 import org.apache.flink.streaming.api.functions.windowing.RichWindowFunction;
 import org.apache.flink.streaming.api.windowing.time.Time;
 import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
-import org.apache.flink.test.util.ForkableFlinkMiniCluster;
 import org.apache.flink.test.util.SuccessException;
 import org.apache.flink.util.Collector;
 import org.apache.flink.util.TestLogger;
@@ -71,7 +71,7 @@ public class WindowCheckpointingITCase extends TestLogger {
 
 	private static final int PARALLELISM = 4;
 
-	private static ForkableFlinkMiniCluster cluster;
+	private static LocalFlinkMiniCluster cluster;
 
 
 	@BeforeClass
@@ -81,7 +81,7 @@ public class WindowCheckpointingITCase extends TestLogger {
 		config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, PARALLELISM / 2);
 		config.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 48);
 
-		cluster = new ForkableFlinkMiniCluster(config, false);
+		cluster = new LocalFlinkMiniCluster(config, false);
 		cluster.start();
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/flink-tests/src/test/java/org/apache/flink/test/classloading/ClassLoaderITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/classloading/ClassLoaderITCase.java b/flink-tests/src/test/java/org/apache/flink/test/classloading/ClassLoaderITCase.java
index 8b56d3d..7afafe4 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/classloading/ClassLoaderITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/classloading/ClassLoaderITCase.java
@@ -18,7 +18,6 @@
 
 package org.apache.flink.test.classloading;
 
-import akka.pattern.AskTimeoutException;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.client.program.PackagedProgram;
 import org.apache.flink.client.program.ProgramInvocationException;
@@ -37,9 +36,9 @@ import org.apache.flink.runtime.messages.JobManagerMessages.RunningJobsStatus;
 import org.apache.flink.runtime.messages.JobManagerMessages.TriggerSavepoint;
 import org.apache.flink.runtime.messages.JobManagerMessages.TriggerSavepointSuccess;
 import org.apache.flink.runtime.state.filesystem.FsStateBackendFactory;
+import org.apache.flink.runtime.testingUtils.TestingCluster;
 import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages.WaitForAllVerticesToBeRunning;
 import org.apache.flink.test.testdata.KMeansData;
-import org.apache.flink.test.util.ForkableFlinkMiniCluster;
 import org.apache.flink.util.TestLogger;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -82,7 +81,7 @@ public class ClassLoaderITCase extends TestLogger {
 
 	public static final TemporaryFolder FOLDER = new TemporaryFolder();
 
-	private static ForkableFlinkMiniCluster testCluster;
+	private static TestingCluster testCluster;
 
 	private static int parallelism;
 
@@ -105,7 +104,7 @@ public class ClassLoaderITCase extends TestLogger {
 		config.setString(SavepointStoreFactory.SAVEPOINT_DIRECTORY_KEY,
 				FOLDER.newFolder().getAbsoluteFile().toURI().toString());
 
-		testCluster = new ForkableFlinkMiniCluster(config, false);
+		testCluster = new TestingCluster(config, false);
 		testCluster.start();
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/flink-tests/src/test/java/org/apache/flink/test/clients/examples/JobRetrievalITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/clients/examples/JobRetrievalITCase.java b/flink-tests/src/test/java/org/apache/flink/test/clients/examples/JobRetrievalITCase.java
index c9059f1..a74ed34 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/clients/examples/JobRetrievalITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/clients/examples/JobRetrievalITCase.java
@@ -29,8 +29,8 @@ import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.jobgraph.JobVertex;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
 import org.apache.flink.runtime.minicluster.FlinkMiniCluster;
+import org.apache.flink.runtime.testingUtils.TestingCluster;
 import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages;
-import org.apache.flink.test.util.ForkableFlinkMiniCluster;
 import org.apache.flink.util.TestLogger;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -42,7 +42,6 @@ import java.util.concurrent.Semaphore;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.fail;
 
-
 /**
  * Tests retrieval of a job from a running Flink cluster
  */
@@ -54,7 +53,7 @@ public class JobRetrievalITCase extends TestLogger {
 
 	@BeforeClass
 	public static void before() {
-		cluster = new ForkableFlinkMiniCluster(new Configuration(), false);
+		cluster = new TestingCluster(new Configuration(), false);
 		cluster.start();
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/flink-tests/src/test/java/org/apache/flink/test/failingPrograms/JobSubmissionFailsITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/failingPrograms/JobSubmissionFailsITCase.java b/flink-tests/src/test/java/org/apache/flink/test/failingPrograms/JobSubmissionFailsITCase.java
index 28c2e58..178656d 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/failingPrograms/JobSubmissionFailsITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/failingPrograms/JobSubmissionFailsITCase.java
@@ -26,8 +26,8 @@ import org.apache.flink.runtime.client.JobSubmissionException;
 import org.apache.flink.runtime.jobgraph.JobVertex;
 import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.jobmanager.Tasks;
+import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
 import org.apache.flink.runtime.testingUtils.TestingUtils;
-import org.apache.flink.test.util.ForkableFlinkMiniCluster;
 
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -47,7 +47,7 @@ public class JobSubmissionFailsITCase {
 	
 	private static final int NUM_SLOTS = 20;
 	
-	private static ForkableFlinkMiniCluster cluster;
+	private static LocalFlinkMiniCluster cluster;
 	private static JobGraph workingJobGraph;
 
 	@BeforeClass
@@ -58,7 +58,7 @@ public class JobSubmissionFailsITCase {
 			config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 2);
 			config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, NUM_SLOTS / 2);
 			
-			cluster = new ForkableFlinkMiniCluster(config);
+			cluster = new LocalFlinkMiniCluster(config);
 
 			cluster.start();
 			

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/CustomDistributionITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/CustomDistributionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/CustomDistributionITCase.java
index ca2c156..133ebd0 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/CustomDistributionITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/CustomDistributionITCase.java
@@ -29,8 +29,8 @@ import org.apache.flink.api.java.tuple.Tuple3;
 import org.apache.flink.api.java.utils.DataSetUtils;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
 import org.apache.flink.test.javaApiOperators.util.CollectionDataSets;
-import org.apache.flink.test.util.ForkableFlinkMiniCluster;
 import org.apache.flink.test.util.TestBaseUtils;
 import org.apache.flink.test.util.TestEnvironment;
 import org.apache.flink.util.Collector;
@@ -52,7 +52,7 @@ public class CustomDistributionITCase extends TestLogger {
 	//  The mini cluster that is shared across tests
 	// ------------------------------------------------------------------------
 
-	private static ForkableFlinkMiniCluster cluster;
+	private static LocalFlinkMiniCluster cluster;
 
 	@BeforeClass
 	public static void setup() throws Exception {

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/RemoteEnvironmentITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/RemoteEnvironmentITCase.java b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/RemoteEnvironmentITCase.java
index 34a7eed..e18e82a 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/RemoteEnvironmentITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/javaApiOperators/RemoteEnvironmentITCase.java
@@ -23,11 +23,10 @@ import org.apache.flink.api.common.operators.util.TestNonRichInputFormat;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.io.LocalCollectionOutputFormat;
-import org.apache.flink.client.program.ProgramInvocationException;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.core.io.GenericInputSplit;
-import org.apache.flink.test.util.ForkableFlinkMiniCluster;
+import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
 import org.apache.flink.util.Collector;
 import org.junit.AfterClass;
 import org.junit.Assert;
@@ -54,7 +53,7 @@ public class RemoteEnvironmentITCase {
 
 	private static final String VALID_STARTUP_TIMEOUT = "100 s";
 
-	private static ForkableFlinkMiniCluster cluster;
+	private static LocalFlinkMiniCluster cluster;
 
 	@BeforeClass
 	public static void setupCluster() {
@@ -62,7 +61,7 @@ public class RemoteEnvironmentITCase {
 			Configuration config = new Configuration();
 			config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, NUM_TM);
 			config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, TM_SLOTS);
-			cluster = new ForkableFlinkMiniCluster(config, false);
+			cluster = new LocalFlinkMiniCluster(config, false);
 			cluster.start();
 		}
 		catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/flink-tests/src/test/java/org/apache/flink/test/misc/AutoParallelismITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/misc/AutoParallelismITCase.java b/flink-tests/src/test/java/org/apache/flink/test/misc/AutoParallelismITCase.java
index 09b5e7e..a67e6ef 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/misc/AutoParallelismITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/misc/AutoParallelismITCase.java
@@ -29,7 +29,7 @@ import org.apache.flink.api.java.io.LocalCollectionOutputFormat;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.core.io.GenericInputSplit;
-import org.apache.flink.test.util.ForkableFlinkMiniCluster;
+import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
 import org.apache.flink.util.Collector;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -49,14 +49,14 @@ public class AutoParallelismITCase {
 	private static final int SLOTS_PER_TM = 7;
 	private static final int PARALLELISM = NUM_TM * SLOTS_PER_TM;
 
-	private static ForkableFlinkMiniCluster cluster;
+	private static LocalFlinkMiniCluster cluster;
 
 	@BeforeClass
 	public static void setupCluster() {
 		Configuration config = new Configuration();
 		config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, NUM_TM);
 		config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, SLOTS_PER_TM);
-		cluster = new ForkableFlinkMiniCluster(config, false);
+		cluster = new LocalFlinkMiniCluster(config, false);
 
 		cluster.start();
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/flink-tests/src/test/java/org/apache/flink/test/misc/CustomSerializationITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/misc/CustomSerializationITCase.java b/flink-tests/src/test/java/org/apache/flink/test/misc/CustomSerializationITCase.java
index f30f61f..51f3534 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/misc/CustomSerializationITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/misc/CustomSerializationITCase.java
@@ -26,7 +26,7 @@ import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
-import org.apache.flink.test.util.ForkableFlinkMiniCluster;
+import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
 import org.apache.flink.types.Value;
 
 import org.junit.AfterClass;
@@ -43,7 +43,7 @@ public class CustomSerializationITCase {
 
 	private static final int PARLLELISM = 5;
 	
-	private static ForkableFlinkMiniCluster cluster;
+	private static LocalFlinkMiniCluster cluster;
 
 	@BeforeClass
 	public static void startCluster() {
@@ -51,7 +51,7 @@ public class CustomSerializationITCase {
 			Configuration config = new Configuration();
 			config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, PARLLELISM);
 			config.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 30);
-			cluster = new ForkableFlinkMiniCluster(config, false);
+			cluster = new LocalFlinkMiniCluster(config, false);
 			cluster.start();
 		}
 		catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/flink-tests/src/test/java/org/apache/flink/test/misc/MiscellaneousIssuesITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/misc/MiscellaneousIssuesITCase.java b/flink-tests/src/test/java/org/apache/flink/test/misc/MiscellaneousIssuesITCase.java
index 42419fb..06b93ea 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/misc/MiscellaneousIssuesITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/misc/MiscellaneousIssuesITCase.java
@@ -29,7 +29,7 @@ import org.apache.flink.client.program.ProgramInvocationException;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.core.fs.FileSystem;
-import org.apache.flink.test.util.ForkableFlinkMiniCluster;
+import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
 
 import org.apache.flink.util.Collector;
 
@@ -52,7 +52,7 @@ import static org.junit.Assert.*;
 @SuppressWarnings("serial")
 public class MiscellaneousIssuesITCase {
 
-	private static ForkableFlinkMiniCluster cluster;
+	private static LocalFlinkMiniCluster cluster;
 	
 	@BeforeClass
 	public static void startCluster() {
@@ -61,7 +61,7 @@ public class MiscellaneousIssuesITCase {
 			config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 2);
 			config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 3);
 			config.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 12);
-			cluster = new ForkableFlinkMiniCluster(config, false);
+			cluster = new LocalFlinkMiniCluster(config, false);
 
 			cluster.start();
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/flink-tests/src/test/java/org/apache/flink/test/misc/SuccessAfterNetworkBuffersFailureITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/misc/SuccessAfterNetworkBuffersFailureITCase.java b/flink-tests/src/test/java/org/apache/flink/test/misc/SuccessAfterNetworkBuffersFailureITCase.java
index 12b7a68..a43bab6 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/misc/SuccessAfterNetworkBuffersFailureITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/misc/SuccessAfterNetworkBuffersFailureITCase.java
@@ -32,8 +32,8 @@ import org.apache.flink.examples.java.clustering.KMeans;
 import org.apache.flink.examples.java.clustering.util.KMeansData;
 import org.apache.flink.examples.java.graph.ConnectedComponents;
 import org.apache.flink.examples.java.graph.util.ConnectedComponentsData;
-import org.apache.flink.test.util.ForkableFlinkMiniCluster;
 
+import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
 import org.junit.Test;
 
 import static org.junit.Assert.*;
@@ -43,7 +43,7 @@ public class SuccessAfterNetworkBuffersFailureITCase {
 	
 	@Test
 	public void testSuccessfulProgramAfterFailure() {
-		ForkableFlinkMiniCluster cluster = null;
+		LocalFlinkMiniCluster cluster = null;
 		
 		try {
 			Configuration config = new Configuration();
@@ -52,7 +52,7 @@ public class SuccessAfterNetworkBuffersFailureITCase {
 			config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 8);
 			config.setInteger(ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY, 840);
 			
-			cluster = new ForkableFlinkMiniCluster(config, false);
+			cluster = new LocalFlinkMiniCluster(config, false);
 
 			cluster.start();
 			

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/flink-tests/src/test/java/org/apache/flink/test/query/QueryableStateITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/query/QueryableStateITCase.java b/flink-tests/src/test/java/org/apache/flink/test/query/QueryableStateITCase.java
index 40732df..b99858a 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/query/QueryableStateITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/query/QueryableStateITCase.java
@@ -55,6 +55,7 @@ import org.apache.flink.runtime.query.netty.message.KvStateRequestSerializer;
 import org.apache.flink.runtime.state.CheckpointListener;
 import org.apache.flink.runtime.state.VoidNamespace;
 import org.apache.flink.runtime.state.VoidNamespaceSerializer;
+import org.apache.flink.runtime.testingUtils.TestingCluster;
 import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages;
 import org.apache.flink.runtime.testingUtils.TestingTaskManagerMessages;
 import org.apache.flink.runtime.testingUtils.TestingTaskManagerMessages.ResponseRunningTasks;
@@ -62,7 +63,6 @@ import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.datastream.QueryableStateStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
-import org.apache.flink.test.util.ForkableFlinkMiniCluster;
 import org.apache.flink.util.MathUtils;
 import org.apache.flink.util.Preconditions;
 import org.apache.flink.util.TestLogger;
@@ -108,7 +108,7 @@ public class QueryableStateITCase extends TestLogger {
 	 * Shared between all the test. Make sure to have at least NUM_SLOTS
 	 * available after your test finishes, e.g. cancel the job you submitted.
 	 */
-	private static ForkableFlinkMiniCluster cluster;
+	private static TestingCluster cluster;
 
 	@BeforeClass
 	public static void setup() {
@@ -120,7 +120,7 @@ public class QueryableStateITCase extends TestLogger {
 			config.setInteger(ConfigConstants.QUERYABLE_STATE_CLIENT_NETWORK_THREADS, 1);
 			config.setInteger(ConfigConstants.QUERYABLE_STATE_SERVER_NETWORK_THREADS, 1);
 
-			cluster = new ForkableFlinkMiniCluster(config, false);
+			cluster = new TestingCluster(config, false);
 			cluster.start(true);
 		} catch (Exception e) {
 			e.printStackTrace();

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/flink-tests/src/test/java/org/apache/flink/test/recovery/FastFailuresITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/FastFailuresITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/FastFailuresITCase.java
index 8a45d62..8a43ee4 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recovery/FastFailuresITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/FastFailuresITCase.java
@@ -24,11 +24,11 @@ import org.apache.flink.api.common.restartstrategy.RestartStrategies;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.functions.sink.SinkFunction;
 import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
-import org.apache.flink.test.util.ForkableFlinkMiniCluster;
 
 import org.apache.flink.util.TestLogger;
 import org.junit.Test;
@@ -49,7 +49,7 @@ public class FastFailuresITCase extends TestLogger {
 		config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 2);
 		config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 2);
 		
-		ForkableFlinkMiniCluster cluster = new ForkableFlinkMiniCluster(config, false);
+		LocalFlinkMiniCluster cluster = new LocalFlinkMiniCluster(config, false);
 		cluster.start();
 		
 		StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment(

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryFailureRateStrategyITBase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryFailureRateStrategyITBase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryFailureRateStrategyITBase.java
index 0c5d14b..a0d6b58 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryFailureRateStrategyITBase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryFailureRateStrategyITBase.java
@@ -20,7 +20,7 @@ package org.apache.flink.test.recovery;
 
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.test.util.ForkableFlinkMiniCluster;
+import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
 import org.junit.BeforeClass;
 
 public class SimpleRecoveryFailureRateStrategyITBase extends SimpleRecoveryITCaseBase {
@@ -34,8 +34,8 @@ public class SimpleRecoveryFailureRateStrategyITBase extends SimpleRecoveryITCas
 		config.setString(ConfigConstants.RESTART_STRATEGY_FAILURE_RATE_FAILURE_RATE_INTERVAL, "1 second");
 		config.setString(ConfigConstants.RESTART_STRATEGY_FAILURE_RATE_DELAY, "100 ms");
 
-		cluster = new ForkableFlinkMiniCluster(config, false);
+		cluster = new LocalFlinkMiniCluster(config, false);
 
 		cluster.start();
 	}
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryFixedDelayRestartStrategyITBase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryFixedDelayRestartStrategyITBase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryFixedDelayRestartStrategyITBase.java
index 6355a8f..f09efc5 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryFixedDelayRestartStrategyITBase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryFixedDelayRestartStrategyITBase.java
@@ -20,7 +20,7 @@ package org.apache.flink.test.recovery;
 
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.test.util.ForkableFlinkMiniCluster;
+import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
 import org.junit.BeforeClass;
 
 public class SimpleRecoveryFixedDelayRestartStrategyITBase extends SimpleRecoveryITCaseBase {
@@ -33,8 +33,8 @@ public class SimpleRecoveryFixedDelayRestartStrategyITBase extends SimpleRecover
 		config.setInteger(ConfigConstants.RESTART_STRATEGY_FIXED_DELAY_ATTEMPTS, 1);
 		config.setString(ConfigConstants.RESTART_STRATEGY_FIXED_DELAY_DELAY, "100 ms");
 
-		cluster = new ForkableFlinkMiniCluster(config, false);
+		cluster = new LocalFlinkMiniCluster(config, false);
 
 		cluster.start();
 	}
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryITCaseBase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryITCaseBase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryITCaseBase.java
index 004340c..bf7c524 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryITCaseBase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/SimpleRecoveryITCaseBase.java
@@ -26,7 +26,7 @@ import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.api.java.io.LocalCollectionOutputFormat;
 import org.apache.flink.client.program.ProgramInvocationException;
 import org.apache.flink.runtime.client.JobExecutionException;
-import org.apache.flink.test.util.ForkableFlinkMiniCluster;
+import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
 import org.junit.AfterClass;
 import org.junit.Test;
 
@@ -42,7 +42,7 @@ import static org.junit.Assert.*;
 @SuppressWarnings("serial")
 public abstract class SimpleRecoveryITCaseBase {
 
-	protected static ForkableFlinkMiniCluster cluster;
+	protected static LocalFlinkMiniCluster cluster;
 
 	@AfterClass
 	public static void teardownCluster() {

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerFailureRecoveryITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerFailureRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerFailureRecoveryITCase.java
index 6c621ac..5d29905 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerFailureRecoveryITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerFailureRecoveryITCase.java
@@ -30,7 +30,7 @@ import org.apache.flink.api.java.io.LocalCollectionOutputFormat;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.messages.TaskManagerMessages;
-import org.apache.flink.test.util.ForkableFlinkMiniCluster;
+import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
 import org.junit.Test;
 import scala.concurrent.Await;
 import scala.concurrent.Future;
@@ -65,7 +65,7 @@ public class TaskManagerFailureRecoveryITCase {
 
 		final int PARALLELISM = 4;
 
-		ForkableFlinkMiniCluster cluster = null;
+		LocalFlinkMiniCluster cluster = null;
 		ActorSystem additionalSystem = null;
 
 		try {
@@ -78,7 +78,7 @@ public class TaskManagerFailureRecoveryITCase {
 			config.setString(ConfigConstants.AKKA_WATCH_HEARTBEAT_PAUSE, "20 s");
 			config.setInteger(ConfigConstants.AKKA_WATCH_THRESHOLD, 20);
 
-			cluster = new ForkableFlinkMiniCluster(config, false);
+			cluster = new LocalFlinkMiniCluster(config, false);
 
 			cluster.start();
 

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/flink-tests/src/test/java/org/apache/flink/test/runtime/IPv6HostnamesITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/runtime/IPv6HostnamesITCase.java b/flink-tests/src/test/java/org/apache/flink/test/runtime/IPv6HostnamesITCase.java
index 7710f06..0b008eb 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/runtime/IPv6HostnamesITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/runtime/IPv6HostnamesITCase.java
@@ -26,8 +26,8 @@ import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
 import org.apache.flink.test.testdata.WordCountData;
-import org.apache.flink.test.util.ForkableFlinkMiniCluster;
 
 import org.apache.flink.test.util.TestBaseUtils;
 import org.apache.flink.util.Collector;
@@ -63,7 +63,7 @@ public class IPv6HostnamesITCase extends TestLogger {
 
 		
 		
-		ForkableFlinkMiniCluster flink = null;
+		LocalFlinkMiniCluster flink = null;
 		try {
 			final String addressString = ipv6address.getHostAddress();
 			log.info("Test will use IPv6 address " + addressString + " for connection tests");
@@ -75,7 +75,7 @@ public class IPv6HostnamesITCase extends TestLogger {
 			conf.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 2);
 			conf.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 16);
 			
-			flink = new ForkableFlinkMiniCluster(conf, false);
+			flink = new LocalFlinkMiniCluster(conf, false);
 			flink.start();
 
 			ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(addressString, flink.getLeaderRPCPort());


[03/50] [abbrv] flink git commit: [FLINK-3580] [table] Add current time point functions

Posted by tr...@apache.org.
[FLINK-3580] [table] Add current time point functions

This closes #2441.


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

Branch: refs/heads/flip-6
Commit: e376c003c0fac456136e63e9d4cac4bfc07c02d4
Parents: fb8f2c9
Author: twalthr <tw...@apache.org>
Authored: Tue Aug 30 17:21:13 2016 +0200
Committer: twalthr <tw...@apache.org>
Committed: Fri Sep 2 17:44:30 2016 +0200

----------------------------------------------------------------------
 docs/dev/table_api.md                           | 169 ++++++++++++++++++-
 .../flink/api/scala/table/expressionDsl.scala   |  72 ++++++++
 .../flink/api/table/codegen/CodeGenerator.scala | 102 +++++++++++
 .../codegen/calls/CurrentTimePointCallGen.scala |  58 +++++++
 .../table/codegen/calls/ScalarFunctions.scala   |  25 +++
 .../flink/api/table/expressions/time.scala      |  47 ++++++
 .../api/table/validate/FunctionCatalog.scala    |   8 +-
 .../expressions/NonDeterministicTests.scala     |  89 ++++++++++
 .../table/expressions/ScalarFunctionsTest.scala |  45 +++++
 9 files changed, 613 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/e376c003/docs/dev/table_api.md
----------------------------------------------------------------------
diff --git a/docs/dev/table_api.md b/docs/dev/table_api.md
index 68a2b95..9272ea3 100644
--- a/docs/dev/table_api.md
+++ b/docs/dev/table_api.md
@@ -981,6 +981,8 @@ functionCall = composite , "." , functionIdentifier , [ "(" , [ expression , { "
 
 atom = ( "(" , expression , ")" ) | literal | nullLiteral | fieldReference ;
 
+fieldReference = "*" | identifier ;
+
 nullLiteral = "Null(" , dataType , ")" ;
 
 timeIntervalUnit = "YEAR" | "YEAR_TO_MONTH" | "MONTH" | "DAY" | "DAY_TO_HOUR" | "DAY_TO_MINUTE" | "DAY_TO_SECOND" | "HOUR" | "HOUR_TO_MINUTE" | "HOUR_TO_SECOND" | "MINUTE" | "MINUTE_TO_SECOND" | "SECOND" ;
@@ -989,7 +991,7 @@ timePointUnit = "YEAR" | "MONTH" | "DAY" | "HOUR" | "MINUTE" | "SECOND" | "QUART
 
 {% endhighlight %}
 
-Here, `literal` is a valid Java literal, `fieldReference` specifies a column in the data, and `functionIdentifier` specifies a supported scalar function. The
+Here, `literal` is a valid Java literal, `fieldReference` specifies a column in the data (or all columns if `*` is used), and `functionIdentifier` specifies a supported scalar function. The
 column names and function names follow Java identifier syntax. Expressions specified as Strings can also use prefix notation instead of suffix notation to call operators and functions.
 
 If working with exact numeric values or large decimals is required, the Table API also supports Java's BigDecimal type. In the Scala Table API decimals can be defined by `BigDecimal("123456")` and in Java by appending a "p" for precise e.g. `123456p`.
@@ -1521,6 +1523,61 @@ TIMEPOINT.ceil(TIMEINTERVALUNIT)
       </td>
     </tr>
 
+    <tr>
+      <td>
+        {% highlight java %}
+currentDate()
+{% endhighlight %}
+      </td>
+      <td>
+        <p>Returns the current SQL date in UTC time zone.</p>
+      </td>
+    </tr>
+
+    <tr>
+      <td>
+        {% highlight java %}
+currentTime()
+{% endhighlight %}
+      </td>
+      <td>
+        <p>Returns the current SQL time in UTC time zone.</p>
+      </td>
+    </tr>
+
+    <tr>
+      <td>
+        {% highlight java %}
+currentTimestamp()
+{% endhighlight %}
+      </td>
+      <td>
+        <p>Returns the current SQL timestamp in UTC time zone.</p>
+      </td>
+    </tr>
+
+    <tr>
+      <td>
+        {% highlight java %}
+localTime()
+{% endhighlight %}
+      </td>
+      <td>
+        <p>Returns the current SQL time in local time zone.</p>
+      </td>
+    </tr>
+
+    <tr>
+      <td>
+        {% highlight java %}
+localTimestamp()
+{% endhighlight %}
+      </td>
+      <td>
+        <p>Returns the current SQL timestamp in local time zone.</p>
+      </td>
+    </tr>
+
   </tbody>
 </table>
 
@@ -1828,6 +1885,61 @@ TIMEPOINT.ceil(TimeIntervalUnit)
       </td>
     </tr>
 
+    <tr>
+      <td>
+        {% highlight scala %}
+currentDate()
+{% endhighlight %}
+      </td>
+      <td>
+        <p>Returns the current SQL date in UTC time zone.</p>
+      </td>
+    </tr>
+
+    <tr>
+      <td>
+        {% highlight scala %}
+currentTime()
+{% endhighlight %}
+      </td>
+      <td>
+        <p>Returns the current SQL time in UTC time zone.</p>
+      </td>
+    </tr>
+
+    <tr>
+      <td>
+        {% highlight scala %}
+currentTimestamp()
+{% endhighlight %}
+      </td>
+      <td>
+        <p>Returns the current SQL timestamp in UTC time zone.</p>
+      </td>
+    </tr>
+
+    <tr>
+      <td>
+        {% highlight scala %}
+localTime()
+{% endhighlight %}
+      </td>
+      <td>
+        <p>Returns the current SQL time in local time zone.</p>
+      </td>
+    </tr>
+
+    <tr>
+      <td>
+        {% highlight scala %}
+localTimestamp()
+{% endhighlight %}
+      </td>
+      <td>
+        <p>Returns the current SQL timestamp in local time zone.</p>
+      </td>
+    </tr>
+
   </tbody>
 </table>
 </div>
@@ -2093,6 +2205,61 @@ CEIL(TIMEPOINT TO TIMEINTERVALUNIT)
       </td>
     </tr>
 
+    <tr>
+      <td>
+        {% highlight sql %}
+CURRENT_DATE
+{% endhighlight %}
+      </td>
+      <td>
+        <p>Returns the current SQL date in UTC time zone.</p>
+      </td>
+    </tr>
+
+    <tr>
+      <td>
+        {% highlight sql %}
+CURRENT_TIME
+{% endhighlight %}
+      </td>
+      <td>
+        <p>Returns the current SQL time in UTC time zone.</p>
+      </td>
+    </tr>
+
+    <tr>
+      <td>
+        {% highlight sql %}
+CURRENT_TIMESTAMP
+{% endhighlight %}
+      </td>
+      <td>
+        <p>Returns the current SQL timestamp in UTC time zone.</p>
+      </td>
+    </tr>
+
+    <tr>
+      <td>
+        {% highlight sql %}
+LOCALTIME
+{% endhighlight %}
+      </td>
+      <td>
+        <p>Returns the current SQL time in local time zone.</p>
+      </td>
+    </tr>
+
+    <tr>
+      <td>
+        {% highlight sql %}
+LOCALTIMESTAMP
+{% endhighlight %}
+      </td>
+      <td>
+        <p>Returns the current SQL timestamp in local time zone.</p>
+      </td>
+    </tr>
+
   </tbody>
 </table>
 </div>

http://git-wip-us.apache.org/repos/asf/flink/blob/e376c003/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/scala/table/expressionDsl.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/scala/table/expressionDsl.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/scala/table/expressionDsl.scala
index 9bfe6c3..942b07e 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/scala/table/expressionDsl.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/scala/table/expressionDsl.scala
@@ -402,3 +402,75 @@ trait ImplicitExpressionConversions {
   implicit def sqlTime2Literal(sqlTime: Time): Expression = Literal(sqlTime)
   implicit def sqlTimestamp2Literal(sqlTimestamp: Timestamp): Expression = Literal(sqlTimestamp)
 }
+
+// ------------------------------------------------------------------------------------------------
+// Expressions with no parameters
+// ------------------------------------------------------------------------------------------------
+
+/**
+  * Returns the current SQL date in UTC time zone.
+  */
+object currentDate {
+
+  /**
+    * Returns the current SQL date in UTC time zone.
+    */
+  def apply(): Expression = {
+    CurrentDate()
+  }
+}
+
+/**
+  * Returns the current SQL time in UTC time zone.
+  */
+object currentTime {
+
+  /**
+    * Returns the current SQL time in UTC time zone.
+    */
+  def apply(): Expression = {
+    CurrentTime()
+  }
+}
+
+/**
+  * Returns the current SQL timestamp in UTC time zone.
+  */
+object currentTimestamp {
+
+  /**
+    * Returns the current SQL timestamp in UTC time zone.
+    */
+  def apply(): Expression = {
+    CurrentTimestamp()
+  }
+}
+
+/**
+  * Returns the current SQL time in local time zone.
+  */
+object localTime {
+
+  /**
+    * Returns the current SQL time in local time zone.
+    */
+  def apply(): Expression = {
+    LocalTime()
+  }
+}
+
+/**
+  * Returns the current SQL timestamp in local time zone.
+  */
+object localTimestamp {
+
+  /**
+    * Returns the current SQL timestamp in local time zone.
+    */
+  def apply(): Expression = {
+    LocalTimestamp()
+  }
+}
+
+
+

http://git-wip-us.apache.org/repos/asf/flink/blob/e376c003/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/CodeGenerator.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/CodeGenerator.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/CodeGenerator.scala
index 6463ff9..39ee26c 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/CodeGenerator.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/CodeGenerator.scala
@@ -20,6 +20,7 @@ package org.apache.flink.api.table.codegen
 
 import java.math.{BigDecimal => JBigDecimal}
 
+import org.apache.calcite.avatica.util.DateTimeUtils
 import org.apache.calcite.rex._
 import org.apache.calcite.sql.SqlOperator
 import org.apache.calcite.sql.`type`.SqlTypeName._
@@ -106,6 +107,10 @@ class CodeGenerator(
   // we use a LinkedHashSet to keep the insertion order
   private val reusableInitStatements = mutable.LinkedHashSet[String]()
 
+  // set of statements that will be added only once per record
+  // we use a LinkedHashSet to keep the insertion order
+  private val reusablePerRecordStatements = mutable.LinkedHashSet[String]()
+
   // map of initial input unboxing expressions that will be added only once
   // (inputTerm, index) -> expr
   private val reusableInputUnboxingExprs = mutable.Map[(String, Int), GeneratedExpression]()
@@ -126,6 +131,13 @@ class CodeGenerator(
   }
 
   /**
+    * @return code block of statements that need to be placed in the SAM of the Function
+    */
+  def reusePerRecordCode(): String = {
+    reusablePerRecordStatements.mkString("", "\n", "\n")
+  }
+
+  /**
     * @return code block of statements that unbox input variables to a primitive variable
     *         and a corresponding null flag variable
     */
@@ -234,6 +246,7 @@ class CodeGenerator(
         @Override
         public ${samHeader._1} throws Exception {
           ${samHeader._2.mkString("\n")}
+          ${reusePerRecordCode()}
           ${reuseInputUnboxingCode()}
           $bodyCode
         }
@@ -1243,4 +1256,93 @@ class CodeGenerator(
     reusableInitStatements.add(constructorAccessibility)
     fieldTerm
   }
+
+  /**
+    * Adds a reusable timestamp to the beginning of the SAM of the generated [[Function]].
+    */
+  def addReusableTimestamp(): String = {
+    val fieldTerm = s"timestamp"
+
+    val field =
+      s"""
+        |final long $fieldTerm = java.lang.System.currentTimeMillis();
+        |""".stripMargin
+    reusablePerRecordStatements.add(field)
+    fieldTerm
+  }
+
+    /**
+    * Adds a reusable local timestamp to the beginning of the SAM of the generated [[Function]].
+    */
+  def addReusableLocalTimestamp(): String = {
+    val fieldTerm = s"localtimestamp"
+
+    val timestamp = addReusableTimestamp()
+
+    val field =
+      s"""
+        |final long $fieldTerm = $timestamp + java.util.TimeZone.getDefault().getOffset(timestamp);
+        |""".stripMargin
+    reusablePerRecordStatements.add(field)
+    fieldTerm
+  }
+
+  /**
+    * Adds a reusable time to the beginning of the SAM of the generated [[Function]].
+    */
+  def addReusableTime(): String = {
+    val fieldTerm = s"time"
+
+    val timestamp = addReusableTimestamp()
+
+    // adopted from org.apache.calcite.runtime.SqlFunctions.currentTime()
+    val field =
+      s"""
+        |final int $fieldTerm = (int) ($timestamp % ${DateTimeUtils.MILLIS_PER_DAY});
+        |if (time < 0) {
+        |  time += ${DateTimeUtils.MILLIS_PER_DAY};
+        |}
+        |""".stripMargin
+    reusablePerRecordStatements.add(field)
+    fieldTerm
+  }
+
+  /**
+    * Adds a reusable local time to the beginning of the SAM of the generated [[Function]].
+    */
+  def addReusableLocalTime(): String = {
+    val fieldTerm = s"localtime"
+
+    val localtimestamp = addReusableLocalTimestamp()
+
+    // adopted from org.apache.calcite.runtime.SqlFunctions.localTime()
+    val field =
+      s"""
+        |final int $fieldTerm = (int) ($localtimestamp % ${DateTimeUtils.MILLIS_PER_DAY});
+        |""".stripMargin
+    reusablePerRecordStatements.add(field)
+    fieldTerm
+  }
+
+
+  /**
+    * Adds a reusable date to the beginning of the SAM of the generated [[Function]].
+    */
+  def addReusableDate(): String = {
+    val fieldTerm = s"date"
+
+    val timestamp = addReusableTimestamp()
+    val time = addReusableTime()
+
+    // adopted from org.apache.calcite.runtime.SqlFunctions.currentDate()
+    val field =
+      s"""
+        |final int $fieldTerm = (int) ($timestamp / ${DateTimeUtils.MILLIS_PER_DAY});
+        |if ($time < 0) {
+        |  $fieldTerm -= 1;
+        |}
+        |""".stripMargin
+    reusablePerRecordStatements.add(field)
+    fieldTerm
+  }
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/e376c003/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/calls/CurrentTimePointCallGen.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/calls/CurrentTimePointCallGen.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/calls/CurrentTimePointCallGen.scala
new file mode 100644
index 0000000..4aaa209
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/calls/CurrentTimePointCallGen.scala
@@ -0,0 +1,58 @@
+/*
+ * 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.table.codegen.calls
+
+import org.apache.flink.api.common.typeinfo.{SqlTimeTypeInfo, TypeInformation}
+import org.apache.flink.api.table.codegen.{CodeGenerator, GeneratedExpression}
+
+/**
+  * Generates function call to determine current time point (as date/time/timestamp) in
+  * local timezone or not.
+  */
+class CurrentTimePointCallGen(
+    targetType: TypeInformation[_],
+    local: Boolean)
+  extends CallGenerator {
+
+  override def generate(
+      codeGenerator: CodeGenerator,
+      operands: Seq[GeneratedExpression])
+    : GeneratedExpression = targetType match {
+    case SqlTimeTypeInfo.TIME if local =>
+      val time = codeGenerator.addReusableLocalTime()
+      codeGenerator.generateNonNullLiteral(targetType, time)
+
+    case SqlTimeTypeInfo.TIMESTAMP if local =>
+      val timestamp = codeGenerator.addReusableLocalTimestamp()
+      codeGenerator.generateNonNullLiteral(targetType, timestamp)
+
+    case SqlTimeTypeInfo.DATE =>
+      val date = codeGenerator.addReusableDate()
+      codeGenerator.generateNonNullLiteral(targetType, date)
+
+    case SqlTimeTypeInfo.TIME =>
+      val time = codeGenerator.addReusableTime()
+      codeGenerator.generateNonNullLiteral(targetType, time)
+
+    case SqlTimeTypeInfo.TIMESTAMP =>
+      val timestamp = codeGenerator.addReusableTimestamp()
+      codeGenerator.generateNonNullLiteral(targetType, timestamp)
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/e376c003/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/calls/ScalarFunctions.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/calls/ScalarFunctions.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/calls/ScalarFunctions.scala
index 8aa632f..24e8290 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/calls/ScalarFunctions.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/calls/ScalarFunctions.scala
@@ -223,6 +223,31 @@ object ScalarFunctions {
       BuiltInMethod.CEIL.method,
       Some(BuiltInMethod.UNIX_TIMESTAMP_CEIL.method)))
 
+  addSqlFunction(
+    CURRENT_DATE,
+    Seq(),
+    new CurrentTimePointCallGen(SqlTimeTypeInfo.DATE, local = false))
+
+  addSqlFunction(
+    CURRENT_TIME,
+    Seq(),
+    new CurrentTimePointCallGen(SqlTimeTypeInfo.TIME, local = false))
+
+  addSqlFunction(
+    CURRENT_TIMESTAMP,
+    Seq(),
+    new CurrentTimePointCallGen(SqlTimeTypeInfo.TIMESTAMP, local = false))
+
+  addSqlFunction(
+    LOCALTIME,
+    Seq(),
+    new CurrentTimePointCallGen(SqlTimeTypeInfo.TIME, local = true))
+
+  addSqlFunction(
+    LOCALTIMESTAMP,
+    Seq(),
+    new CurrentTimePointCallGen(SqlTimeTypeInfo.TIMESTAMP, local = true))
+
   // ----------------------------------------------------------------------------------------------
 
   /**

http://git-wip-us.apache.org/repos/asf/flink/blob/e376c003/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/time.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/time.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/time.scala
index 48b512c..385b3d5 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/time.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/time.scala
@@ -201,3 +201,50 @@ case class TemporalCeil(
   }
 }
 
+abstract class CurrentTimePoint(
+    targetType: TypeInformation[_],
+    local: Boolean)
+  extends LeafExpression {
+
+  override private[flink] def resultType: TypeInformation[_] = targetType
+
+  override private[flink] def validateInput(): ExprValidationResult = {
+    if (!TypeCheckUtils.isTimePoint(targetType)) {
+      ValidationFailure(s"CurrentTimePoint operator requires Time Point target type, " +
+        s"but get $targetType.")
+    } else if (local && targetType == SqlTimeTypeInfo.DATE) {
+      ValidationFailure(s"Localized CurrentTimePoint operator requires Time or Timestamp target " +
+        s"type, but get $targetType.")
+    } else {
+      ValidationSuccess
+    }
+  }
+
+  override def toString: String = if (local) {
+    s"local$targetType()"
+  } else {
+    s"current$targetType()"
+  }
+
+  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
+    val operator = targetType match {
+      case SqlTimeTypeInfo.TIME if local => SqlStdOperatorTable.LOCALTIME
+      case SqlTimeTypeInfo.TIMESTAMP if local => SqlStdOperatorTable.LOCALTIMESTAMP
+      case SqlTimeTypeInfo.DATE => SqlStdOperatorTable.CURRENT_DATE
+      case SqlTimeTypeInfo.TIME => SqlStdOperatorTable.CURRENT_TIME
+      case SqlTimeTypeInfo.TIMESTAMP => SqlStdOperatorTable.CURRENT_TIMESTAMP
+    }
+    relBuilder.call(operator)
+  }
+}
+
+case class CurrentDate() extends CurrentTimePoint(SqlTimeTypeInfo.DATE, local = false)
+
+case class CurrentTime() extends CurrentTimePoint(SqlTimeTypeInfo.TIME, local = false)
+
+case class CurrentTimestamp() extends CurrentTimePoint(SqlTimeTypeInfo.TIMESTAMP, local = false)
+
+case class LocalTime() extends CurrentTimePoint(SqlTimeTypeInfo.TIME, local = true)
+
+case class LocalTimestamp() extends CurrentTimePoint(SqlTimeTypeInfo.TIMESTAMP, local = true)
+

http://git-wip-us.apache.org/repos/asf/flink/blob/e376c003/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/validate/FunctionCatalog.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/validate/FunctionCatalog.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/validate/FunctionCatalog.scala
index b9a3f71..9808672 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/validate/FunctionCatalog.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/validate/FunctionCatalog.scala
@@ -151,7 +151,13 @@ object FunctionCatalog {
     "mod" -> classOf[Mod],
 
     // temporal functions
-    "extract" -> classOf[Extract]
+    "extract" -> classOf[Extract],
+    "currentDate" -> classOf[CurrentDate],
+    "currentTime" -> classOf[CurrentTime],
+    "currentTimestamp" -> classOf[CurrentTimestamp],
+    "localTime" -> classOf[LocalTime],
+    "localTimestamp" -> classOf[LocalTimestamp]
+
     // TODO implement function overloading here
     // "floor" -> classOf[TemporalFloor]
     // "ceil" -> classOf[TemporalCeil]

http://git-wip-us.apache.org/repos/asf/flink/blob/e376c003/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/NonDeterministicTests.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/NonDeterministicTests.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/NonDeterministicTests.scala
new file mode 100644
index 0000000..de48849
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/NonDeterministicTests.scala
@@ -0,0 +1,89 @@
+/*
+ * 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.table.expressions
+
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.scala.table._
+import org.apache.flink.api.table.Row
+import org.apache.flink.api.table.expressions.utils.ExpressionTestBase
+import org.apache.flink.api.table.typeutils.RowTypeInfo
+import org.junit.{Ignore, Test}
+
+/**
+  * Tests that can only be checked manually as they are non-deterministic.
+  */
+class NonDeterministicTests extends ExpressionTestBase {
+
+  @Ignore
+  @Test
+  def testCurrentDate(): Unit = {
+    testAllApis(
+      currentDate(),
+      "currentDate()",
+      "CURRENT_DATE",
+      "PLEASE CHECK MANUALLY")
+  }
+
+  @Ignore
+  @Test
+  def testCurrentTime(): Unit = {
+    testAllApis(
+      currentTime(),
+      "currentTime()",
+      "CURRENT_TIME",
+      "PLEASE CHECK MANUALLY")
+  }
+
+  @Ignore
+  @Test
+  def testCurrentTimestamp(): Unit = {
+    testAllApis(
+      currentTimestamp(),
+      "currentTimestamp()",
+      "CURRENT_TIMESTAMP",
+      "PLEASE CHECK MANUALLY")
+  }
+
+  @Ignore
+  @Test
+  def testLocalTimestamp(): Unit = {
+    testAllApis(
+      localTimestamp(),
+      "localTimestamp()",
+      "LOCALTIMESTAMP",
+      "PLEASE CHECK MANUALLY")
+  }
+
+  @Ignore
+  @Test
+  def testLocalTime(): Unit = {
+    testAllApis(
+      localTime(),
+      "localTime()",
+      "LOCALTIME",
+      "PLEASE CHECK MANUALLY")
+  }
+
+  // ----------------------------------------------------------------------------------------------
+
+  override def testData: Any = new Row(0)
+
+  override def typeInfo: TypeInformation[Any] =
+    new RowTypeInfo(Seq[TypeInformation[_]]()).asInstanceOf[TypeInformation[Any]]
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/e376c003/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/ScalarFunctionsTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/ScalarFunctionsTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/ScalarFunctionsTest.scala
index 7ab0c7d..516bfca 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/ScalarFunctionsTest.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/ScalarFunctionsTest.scala
@@ -753,6 +753,51 @@ class ScalarFunctionsTest extends ExpressionTestBase {
       "true")
   }
 
+  @Test
+  def testCurrentTimePoint(): Unit = {
+
+    // current time points are non-deterministic
+    // we just test the format of the output
+    // manual test can be found in NonDeterministicTests
+
+    testAllApis(
+      currentDate().cast(Types.STRING).charLength(),
+      "currentDate().cast(STRING).charLength()",
+      "CHAR_LENGTH(CAST(CURRENT_DATE AS VARCHAR))",
+      "10")
+
+    testAllApis(
+      currentTime().cast(Types.STRING).charLength(),
+      "currentTime().cast(STRING).charLength()",
+      "CHAR_LENGTH(CAST(CURRENT_TIME AS VARCHAR))",
+      "8")
+
+    testAllApis(
+      currentTimestamp().cast(Types.STRING).charLength() >= 22,
+      "currentTimestamp().cast(STRING).charLength() >= 22",
+      "CHAR_LENGTH(CAST(CURRENT_TIMESTAMP AS VARCHAR)) >= 22",
+      "true")
+
+    testAllApis(
+      localTimestamp().cast(Types.STRING).charLength() >= 22,
+      "localTimestamp().cast(STRING).charLength() >= 22",
+      "CHAR_LENGTH(CAST(LOCALTIMESTAMP AS VARCHAR)) >= 22",
+      "true")
+
+    testAllApis(
+      localTime().cast(Types.STRING).charLength(),
+      "localTime().cast(STRING).charLength()",
+      "CHAR_LENGTH(CAST(LOCALTIME AS VARCHAR))",
+      "8")
+
+    // comparisons are deterministic
+    testAllApis(
+      localTimestamp() === localTimestamp(),
+      "localTimestamp() === localTimestamp()",
+      "LOCALTIMESTAMP = LOCALTIMESTAMP",
+      "true")
+  }
+
   // ----------------------------------------------------------------------------------------------
 
   def testData = {


[17/50] [abbrv] flink git commit: [FLINK-4458] Replace ForkableFlinkMiniCluster by LocalFlinkMiniCluster

Posted by tr...@apache.org.
[FLINK-4458] Replace ForkableFlinkMiniCluster by LocalFlinkMiniCluster

Rename _configuration to originalConfiguration

Remove testing classes from main scope in flink-runtime

Previously, the ForkableFlinkMiniCluster which resided in flink-test-utils required
these files to be in the main scope of flink-runtime. With the removal of the
ForkableFlinkMiniCluster, these classes are now no longer needed and can be moved
back to the test scope.

This closes #2450.


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

Branch: refs/heads/flip-6
Commit: 02b852e3571e46f25fdfc79f43ceb726ddff9ba7
Parents: 920cda4
Author: Till Rohrmann <tr...@apache.org>
Authored: Wed Aug 31 17:58:09 2016 +0200
Committer: Till Rohrmann <tr...@apache.org>
Committed: Thu Sep 8 17:17:28 2016 +0200

----------------------------------------------------------------------
 .../api/avro/AvroExternalJarProgramITCase.java  |   7 +-
 .../flink/contrib/streaming/CollectITCase.java  |   4 +-
 .../operations/DegreesWithExceptionITCase.java  |   6 +-
 .../ReduceOnEdgesWithExceptionITCase.java       |   6 +-
 .../ReduceOnNeighborsWithExceptionITCase.java   |   6 +-
 .../apache/flink/ml/util/FlinkTestBase.scala    |  11 +-
 .../clusterframework/FlinkResourceManager.java  |  13 +-
 .../testutils/TestingResourceManager.java       | 137 ------
 .../flink/runtime/jobmanager/JobManager.scala   |  45 +-
 .../runtime/messages/TaskManagerMessages.scala  |  26 ++
 .../runtime/minicluster/FlinkMiniCluster.scala  |  73 +++-
 .../minicluster/LocalFlinkMiniCluster.scala     | 235 ++++++++---
 .../flink/runtime/taskmanager/TaskManager.scala | 130 ++++--
 .../testingUtils/TestingJobManager.scala        |  72 ----
 .../testingUtils/TestingJobManagerLike.scala    | 417 -------------------
 .../TestingJobManagerMessages.scala             | 133 ------
 .../testingUtils/TestingMemoryArchivist.scala   |  43 --
 .../runtime/testingUtils/TestingMessages.scala  |  40 --
 .../testingUtils/TestingTaskManager.scala       |  70 ----
 .../testingUtils/TestingTaskManagerLike.scala   | 248 -----------
 .../TestingTaskManagerMessages.scala            |  94 -----
 .../LeaderElectionRetrievalTestingCluster.java  |   3 +-
 .../testutils/TestingResourceManager.java       | 137 ++++++
 .../runtime/testingUtils/TestingCluster.scala   | 322 ++++++++------
 .../testingUtils/TestingJobManager.scala        |  71 ++++
 .../testingUtils/TestingJobManagerLike.scala    | 417 +++++++++++++++++++
 .../TestingJobManagerMessages.scala             | 132 ++++++
 .../testingUtils/TestingMemoryArchivist.scala   |  43 ++
 .../runtime/testingUtils/TestingMessages.scala  |  40 ++
 .../testingUtils/TestingTaskManager.scala       |  70 ++++
 .../testingUtils/TestingTaskManagerLike.scala   | 234 +++++++++++
 .../TestingTaskManagerMessages.scala            |  82 ++++
 .../flink/api/scala/ScalaShellITCase.scala      |   7 +-
 .../cassandra/CassandraConnectorITCase.java     |   6 +-
 .../kafka/KafkaShortRetentionTestBase.java      |   6 +-
 .../connectors/kafka/KafkaTestBase.java         |   6 +-
 .../manualtests/ManualExactlyOnceTest.java      |   4 +-
 ...nualExactlyOnceWithStreamReshardingTest.java |   4 +-
 ...ScalaStreamingMultipleProgramsTestBase.scala |   5 +-
 .../flink-test-utils/pom.xml                    | 149 -------
 .../util/StreamingMultipleProgramsTestBase.java |   4 +-
 .../streaming/util/TestStreamEnvironment.java   |   8 +-
 .../flink/test/util/AbstractTestBase.java       |   3 +-
 .../test/util/MultipleProgramsTestBase.java     |   3 +-
 .../apache/flink/test/util/TestBaseUtils.java   |  31 +-
 .../apache/flink/test/util/TestEnvironment.java |   7 +-
 .../test/util/ForkableFlinkMiniCluster.scala    | 335 ---------------
 .../accumulators/AccumulatorErrorITCase.java    |   6 +-
 .../accumulators/AccumulatorLiveITCase.java     |   1 -
 .../test/cancelling/CancelingTestBase.java      |   7 +-
 .../EventTimeAllWindowCheckpointingITCase.java  |   6 +-
 .../EventTimeWindowCheckpointingITCase.java     |   6 +-
 .../test/checkpointing/RescalingITCase.java     |   6 +-
 .../test/checkpointing/SavepointITCase.java     |  19 +-
 .../StreamCheckpointNotifierITCase.java         |   6 +-
 .../StreamFaultToleranceTestBase.java           |   6 +-
 .../WindowCheckpointingITCase.java              |   6 +-
 .../test/classloading/ClassLoaderITCase.java    |   7 +-
 .../clients/examples/JobRetrievalITCase.java    |   5 +-
 .../JobSubmissionFailsITCase.java               |   6 +-
 .../CustomDistributionITCase.java               |   4 +-
 .../RemoteEnvironmentITCase.java                |   7 +-
 .../flink/test/misc/AutoParallelismITCase.java  |   6 +-
 .../test/misc/CustomSerializationITCase.java    |   6 +-
 .../test/misc/MiscellaneousIssuesITCase.java    |   6 +-
 ...SuccessAfterNetworkBuffersFailureITCase.java |   6 +-
 .../flink/test/query/QueryableStateITCase.java  |   6 +-
 .../flink/test/recovery/FastFailuresITCase.java |   4 +-
 ...SimpleRecoveryFailureRateStrategyITBase.java |   6 +-
 ...RecoveryFixedDelayRestartStrategyITBase.java |   6 +-
 .../test/recovery/SimpleRecoveryITCaseBase.java |   4 +-
 .../TaskManagerFailureRecoveryITCase.java       |   6 +-
 .../flink/test/runtime/IPv6HostnamesITCase.java |   6 +-
 .../ZooKeeperLeaderElectionITCase.java          |  56 +--
 .../test/streaming/runtime/TimestampITCase.java |   6 +-
 .../flink/test/web/WebFrontendITCase.java       |   6 +-
 .../jobmanager/JobManagerFailsITCase.scala      |   8 +-
 .../taskmanager/TaskManagerFailsITCase.scala    |  12 +-
 flink-yarn-tests/pom.xml                        |   8 +
 .../org/apache/flink/yarn/YarnTestBase.java     |   1 -
 tools/maven/scalastyle-config.xml               |   2 +-
 81 files changed, 2037 insertions(+), 2167 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/flink-batch-connectors/flink-avro/src/test/java/org/apache/flink/api/avro/AvroExternalJarProgramITCase.java
----------------------------------------------------------------------
diff --git a/flink-batch-connectors/flink-avro/src/test/java/org/apache/flink/api/avro/AvroExternalJarProgramITCase.java b/flink-batch-connectors/flink-avro/src/test/java/org/apache/flink/api/avro/AvroExternalJarProgramITCase.java
index 29a7e58..1030ff8 100644
--- a/flink-batch-connectors/flink-avro/src/test/java/org/apache/flink/api/avro/AvroExternalJarProgramITCase.java
+++ b/flink-batch-connectors/flink-avro/src/test/java/org/apache/flink/api/avro/AvroExternalJarProgramITCase.java
@@ -25,12 +25,11 @@ import org.apache.flink.client.program.PackagedProgram;
 import org.apache.flink.client.program.StandaloneClusterClient;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.test.util.ForkableFlinkMiniCluster;
+import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
 
 import org.junit.Assert;
 import org.junit.Test;
 
-
 public class AvroExternalJarProgramITCase {
 
 	private static final String JAR_FILE = "maven-test-jar.jar";
@@ -40,12 +39,12 @@ public class AvroExternalJarProgramITCase {
 	@Test
 	public void testExternalProgram() {
 
-		ForkableFlinkMiniCluster testMiniCluster = null;
+		LocalFlinkMiniCluster testMiniCluster = null;
 
 		try {
 			Configuration config = new Configuration();
 			config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 4);
-			testMiniCluster = new ForkableFlinkMiniCluster(config, false);
+			testMiniCluster = new LocalFlinkMiniCluster(config, false);
 			testMiniCluster.start();
 
 			String jarFile = JAR_FILE;

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/flink-contrib/flink-streaming-contrib/src/test/java/org/apache/flink/contrib/streaming/CollectITCase.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-streaming-contrib/src/test/java/org/apache/flink/contrib/streaming/CollectITCase.java b/flink-contrib/flink-streaming-contrib/src/test/java/org/apache/flink/contrib/streaming/CollectITCase.java
index 10ea85c..d691621 100644
--- a/flink-contrib/flink-streaming-contrib/src/test/java/org/apache/flink/contrib/streaming/CollectITCase.java
+++ b/flink-contrib/flink-streaming-contrib/src/test/java/org/apache/flink/contrib/streaming/CollectITCase.java
@@ -19,9 +19,9 @@
 package org.apache.flink.contrib.streaming;
 
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.test.util.ForkableFlinkMiniCluster;
 
 import org.junit.Test;
 
@@ -36,7 +36,7 @@ public class CollectITCase {
 
 	@Test
 	public void testCollect() throws Exception {
-		final ForkableFlinkMiniCluster cluster = new ForkableFlinkMiniCluster(new Configuration(), false);
+		final LocalFlinkMiniCluster cluster = new LocalFlinkMiniCluster(new Configuration(), false);
 		try {
 			cluster.start();
 

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/DegreesWithExceptionITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/DegreesWithExceptionITCase.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/DegreesWithExceptionITCase.java
index 551a97b..02eea07 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/DegreesWithExceptionITCase.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/DegreesWithExceptionITCase.java
@@ -26,7 +26,7 @@ import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.graph.Graph;
 import org.apache.flink.graph.test.TestGraphUtils;
-import org.apache.flink.test.util.ForkableFlinkMiniCluster;
+import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
 
 import org.apache.flink.types.LongValue;
 import org.junit.AfterClass;
@@ -39,7 +39,7 @@ public class DegreesWithExceptionITCase {
 
 	private static final int PARALLELISM = 4;
 
-	private static ForkableFlinkMiniCluster cluster;
+	private static LocalFlinkMiniCluster cluster;
 	
 
 	@BeforeClass
@@ -47,7 +47,7 @@ public class DegreesWithExceptionITCase {
 		try {
 			Configuration config = new Configuration();
 			config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, PARALLELISM);
-			cluster = new ForkableFlinkMiniCluster(config, false);
+			cluster = new LocalFlinkMiniCluster(config, false);
 			cluster.start();
 		}
 		catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnEdgesWithExceptionITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnEdgesWithExceptionITCase.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnEdgesWithExceptionITCase.java
index 56a0a59..666f7ef 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnEdgesWithExceptionITCase.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnEdgesWithExceptionITCase.java
@@ -30,7 +30,7 @@ import org.apache.flink.graph.EdgesFunctionWithVertexValue;
 import org.apache.flink.graph.Graph;
 import org.apache.flink.graph.Vertex;
 import org.apache.flink.graph.test.TestGraphUtils;
-import org.apache.flink.test.util.ForkableFlinkMiniCluster;
+import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
 import org.apache.flink.util.Collector;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -42,7 +42,7 @@ public class ReduceOnEdgesWithExceptionITCase {
 
 	private static final int PARALLELISM = 4;
 
-	private static ForkableFlinkMiniCluster cluster;
+	private static LocalFlinkMiniCluster cluster;
 
 
 	@BeforeClass
@@ -50,7 +50,7 @@ public class ReduceOnEdgesWithExceptionITCase {
 		try {
 			Configuration config = new Configuration();
 			config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, PARALLELISM);
-			cluster = new ForkableFlinkMiniCluster(config, false);
+			cluster = new LocalFlinkMiniCluster(config, false);
 			cluster.start();
 		}
 		catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnNeighborsWithExceptionITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnNeighborsWithExceptionITCase.java b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnNeighborsWithExceptionITCase.java
index 7458e08..0bbdc84 100644
--- a/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnNeighborsWithExceptionITCase.java
+++ b/flink-libraries/flink-gelly/src/test/java/org/apache/flink/graph/test/operations/ReduceOnNeighborsWithExceptionITCase.java
@@ -31,7 +31,7 @@ import org.apache.flink.graph.NeighborsFunctionWithVertexValue;
 import org.apache.flink.graph.ReduceNeighborsFunction;
 import org.apache.flink.graph.Vertex;
 import org.apache.flink.graph.test.TestGraphUtils;
-import org.apache.flink.test.util.ForkableFlinkMiniCluster;
+import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
 import org.apache.flink.util.Collector;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -43,7 +43,7 @@ public class ReduceOnNeighborsWithExceptionITCase {
 
 	private static final int PARALLELISM = 4;
 
-	private static ForkableFlinkMiniCluster cluster;
+	private static LocalFlinkMiniCluster cluster;
 
 
 	@BeforeClass
@@ -51,7 +51,7 @@ public class ReduceOnNeighborsWithExceptionITCase {
 		try {
 			Configuration config = new Configuration();
 			config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, PARALLELISM);
-			cluster = new ForkableFlinkMiniCluster(config, false);
+			cluster = new LocalFlinkMiniCluster(config, false);
 			cluster.start();
 		}
 		catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/flink-libraries/flink-ml/src/test/scala/org/apache/flink/ml/util/FlinkTestBase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-ml/src/test/scala/org/apache/flink/ml/util/FlinkTestBase.scala b/flink-libraries/flink-ml/src/test/scala/org/apache/flink/ml/util/FlinkTestBase.scala
index fb98f24..6353d6a 100644
--- a/flink-libraries/flink-ml/src/test/scala/org/apache/flink/ml/util/FlinkTestBase.scala
+++ b/flink-libraries/flink-ml/src/test/scala/org/apache/flink/ml/util/FlinkTestBase.scala
@@ -18,14 +18,15 @@
 
 package org.apache.flink.ml.util
 
-import org.apache.flink.test.util.{ForkableFlinkMiniCluster, TestBaseUtils, TestEnvironment}
+import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster
+import org.apache.flink.test.util.{TestBaseUtils, TestEnvironment}
 import org.scalatest.{BeforeAndAfter, Suite}
 
-/** Mixin to start and stop a ForkableFlinkMiniCluster automatically for Scala based tests.
+/** Mixin to start and stop a LocalFlinkMiniCluster automatically for Scala based tests.
   * Additionally a TestEnvironment with the started cluster is created and set as the default
   * [[org.apache.flink.api.java.ExecutionEnvironment]].
   *
-  * This mixin starts a ForkableFlinkMiniCluster with one TaskManager and a number of slots given
+  * This mixin starts a LocalFlinkMiniCluster with one TaskManager and a number of slots given
   * by parallelism. This value can be overridden in a sub class in order to start the cluster
   * with a different number of slots.
   *
@@ -37,7 +38,7 @@ import org.scalatest.{BeforeAndAfter, Suite}
   * @example
   *          {{{
   *            def testSomething: Unit = {
-  *             // Obtain TestEnvironment with started ForkableFlinkMiniCluster
+  *             // Obtain TestEnvironment with started LocalFlinkMiniCluster
   *             val env = ExecutionEnvironment.getExecutionEnvironment
   *
   *             env.fromCollection(...)
@@ -50,7 +51,7 @@ import org.scalatest.{BeforeAndAfter, Suite}
 trait FlinkTestBase extends BeforeAndAfter {
   that: Suite =>
 
-  var cluster: Option[ForkableFlinkMiniCluster] = None
+  var cluster: Option[LocalFlinkMiniCluster] = None
   val parallelism = 4
 
   before {

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/FlinkResourceManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/FlinkResourceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/FlinkResourceManager.java
index 95be084..7ea286d 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/FlinkResourceManager.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/FlinkResourceManager.java
@@ -767,8 +767,19 @@ public abstract class FlinkResourceManager<WorkerType extends ResourceIDRetrieva
 			Class<? extends FlinkResourceManager<?>> resourceManagerClass,
 			String resourceManagerActorName) {
 
-		Props resourceMasterProps = Props.create(resourceManagerClass, configuration, leaderRetriever);
+		Props resourceMasterProps = getResourceManagerProps(
+			resourceManagerClass,
+			configuration,
+			leaderRetriever);
 
 		return actorSystem.actorOf(resourceMasterProps, resourceManagerActorName);
 	}
+
+	public static Props getResourceManagerProps(
+		Class<? extends FlinkResourceManager> resourceManagerClass,
+		Configuration configuration,
+		LeaderRetrievalService leaderRetrievalService) {
+
+		return Props.create(resourceManagerClass, configuration, leaderRetrievalService);
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/flink-runtime/src/main/java/org/apache/flink/runtime/testutils/TestingResourceManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/testutils/TestingResourceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/testutils/TestingResourceManager.java
deleted file mode 100644
index 495cacd..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/testutils/TestingResourceManager.java
+++ /dev/null
@@ -1,137 +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.testutils;
-
-import akka.actor.ActorRef;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.clusterframework.ApplicationStatus;
-import org.apache.flink.runtime.clusterframework.messages.RegisterResourceManagerSuccessful;
-import org.apache.flink.runtime.clusterframework.standalone.StandaloneResourceManager;
-import org.apache.flink.runtime.clusterframework.types.ResourceID;
-import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
-import org.apache.flink.runtime.messages.Messages;
-import org.apache.flink.runtime.testingUtils.TestingMessages;
-
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.Set;
-
-
-/**
- * A testing resource manager which may alter the default standalone resource master's behavior.
- */
-public class TestingResourceManager extends StandaloneResourceManager {
-
-	/** Set of Actors which want to be informed of a connection to the job manager */
-	private Set<ActorRef> waitForResourceManagerConnected = new HashSet<>();
-
-	/** Set of Actors which want to be informed of a shutdown */
-	private Set<ActorRef> waitForShutdown = new HashSet<>();
-
-	/** Flag to signal a connection to the JobManager */
-	private boolean isConnected = false;
-
-	public TestingResourceManager(Configuration flinkConfig, LeaderRetrievalService leaderRetriever) {
-		super(flinkConfig, leaderRetriever);
-	}
-
-	/**
-	 * Overwrite messages here if desired
-	 */
-	@Override
-	protected void handleMessage(Object message) {
-
-		if (message instanceof GetRegisteredResources) {
-			sender().tell(new GetRegisteredResourcesReply(getStartedTaskManagers()), self());
-		} else if (message instanceof FailResource) {
-			ResourceID resourceID = ((FailResource) message).resourceID;
-			notifyWorkerFailed(resourceID, "Failed for test case.");
-
-		} else if (message instanceof NotifyWhenResourceManagerConnected) {
-			if (isConnected) {
-				sender().tell(
-					Messages.getAcknowledge(),
-					self());
-			} else {
-				waitForResourceManagerConnected.add(sender());
-			}
-		} else if (message instanceof RegisterResourceManagerSuccessful) {
-			super.handleMessage(message);
-
-			isConnected = true;
-
-			for (ActorRef ref : waitForResourceManagerConnected) {
-				ref.tell(
-					Messages.getAcknowledge(),
-					self());
-			}
-			waitForResourceManagerConnected.clear();
-
-		} else if (message instanceof TestingMessages.NotifyOfComponentShutdown$) {
-			waitForShutdown.add(sender());
-		} else if (message instanceof TestingMessages.Alive$) {
-			sender().tell(Messages.getAcknowledge(), self());
-		} else {
-			super.handleMessage(message);
-		}
-	}
-
-	/**
-	 * Testing messages
-	 */
-	public static class GetRegisteredResources {}
-
-	public static class GetRegisteredResourcesReply {
-
-		public Collection<ResourceID> resources;
-
-		public GetRegisteredResourcesReply(Collection<ResourceID> resources) {
-			this.resources = resources;
-		}
-
-	}
-
-	/**
-	 * Fails all resources that the resource manager has registered
-	 */
-	public static class FailResource {
-
-		public ResourceID resourceID;
-
-		public FailResource(ResourceID resourceID) {
-			this.resourceID = resourceID;
-		}
-	}
-
-	/**
-	 * The sender of this message will be informed of a connection to the Job Manager
-	 */
-	public static class NotifyWhenResourceManagerConnected {}
-
-	/**
-	 * Inform registered listeners about a shutdown of the application.
-     */
-	@Override
-	protected void shutdownApplication(ApplicationStatus finalStatus, String optionalDiagnostics) {
-		for (ActorRef listener : waitForShutdown) {
-			listener.tell(new TestingMessages.ComponentShutdown(self()), self());
-		}
-		waitForShutdown.clear();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/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 88af604..f67be0e 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
@@ -49,7 +49,7 @@ import org.apache.flink.runtime.clusterframework.standalone.StandaloneResourceMa
 import org.apache.flink.runtime.clusterframework.types.ResourceID
 import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager
 import org.apache.flink.runtime.executiongraph.restart.RestartStrategyFactory
-import org.apache.flink.runtime.executiongraph.{StatusListenerMessenger, ExecutionGraph, ExecutionJobVertex}
+import org.apache.flink.runtime.executiongraph.{ExecutionGraph, ExecutionJobVertex, StatusListenerMessenger}
 import org.apache.flink.runtime.instance.{AkkaActorGateway, InstanceManager}
 import org.apache.flink.runtime.jobgraph.jsonplan.JsonPlanGenerator
 import org.apache.flink.runtime.jobgraph.{JobGraph, JobStatus, JobVertexID}
@@ -2721,7 +2721,7 @@ object JobManager {
       configuration,
       None)
 
-    val archiveProps = Props(archiveClass, archiveCount)
+    val archiveProps = getArchiveProps(archiveClass, archiveCount)
 
     // start the archiver with the given name, or without (avoid name conflicts)
     val archive: ActorRef = archiveActorName match {
@@ -2729,7 +2729,7 @@ object JobManager {
       case None => actorSystem.actorOf(archiveProps)
     }
 
-    val jobManagerProps = Props(
+    val jobManagerProps = getJobManagerProps(
       jobManagerClass,
       configuration,
       executorService,
@@ -2754,6 +2754,45 @@ object JobManager {
     (jobManager, archive)
   }
 
+  def getArchiveProps(archiveClass: Class[_ <: MemoryArchivist], archiveCount: Int): Props = {
+    Props(archiveClass, archiveCount)
+  }
+
+  def getJobManagerProps(
+    jobManagerClass: Class[_ <: JobManager],
+    configuration: Configuration,
+    executorService: ExecutorService,
+    instanceManager: InstanceManager,
+    scheduler: FlinkScheduler,
+    libraryCacheManager: BlobLibraryCacheManager,
+    archive: ActorRef,
+    restartStrategyFactory: RestartStrategyFactory,
+    timeout: FiniteDuration,
+    leaderElectionService: LeaderElectionService,
+    submittedJobGraphStore: SubmittedJobGraphStore,
+    checkpointRecoveryFactory: CheckpointRecoveryFactory,
+    savepointStore: SavepointStore,
+    jobRecoveryTimeout: FiniteDuration,
+    metricsRegistry: Option[FlinkMetricRegistry]): Props = {
+
+    Props(
+      jobManagerClass,
+      configuration,
+      executorService,
+      instanceManager,
+      scheduler,
+      libraryCacheManager,
+      archive,
+      restartStrategyFactory,
+      timeout,
+      leaderElectionService,
+      submittedJobGraphStore,
+      checkpointRecoveryFactory,
+      savepointStore,
+      jobRecoveryTimeout,
+      metricsRegistry)
+  }
+
   // --------------------------------------------------------------------------
   //  Resolving the JobManager endpoint
   // --------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/TaskManagerMessages.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/TaskManagerMessages.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/TaskManagerMessages.scala
index 2d99245..b433015 100644
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/TaskManagerMessages.scala
+++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/messages/TaskManagerMessages.scala
@@ -130,6 +130,16 @@ object TaskManagerMessages {
     */
   case class RequestTaskManagerLog(requestType : LogTypeRequest)
 
+  /** Requests the number of active connections at the ConnectionManager */
+  case object RequestNumActiveConnections
+
+  case class ResponseNumActiveConnections(number: Int)
+
+  /** Requests the number of broadcast variables with references */
+  case object RequestBroadcastVariablesWithReferences
+
+  case class ResponseBroadcastVariablesWithReferences(number: Int)
+
 
   // --------------------------------------------------------------------------
   //  Utility getters for case objects to simplify access from Java
@@ -166,4 +176,20 @@ object TaskManagerMessages {
   def getRequestTaskManagerStdout(): AnyRef = {
     RequestTaskManagerLog(StdOutFileRequest)
   }
+
+  /**
+    * Accessor for the case object instance, to simplify Java interoperability.
+    * @return The RequestBroadcastVariablesWithReferences case object instance.
+    */
+  def getRequestBroadcastVariablesWithReferences(): RequestBroadcastVariablesWithReferences.type = {
+    RequestBroadcastVariablesWithReferences
+  }
+
+  /**
+    * Accessor for the case object instance, to simplify Java interoperability.
+    * @return The RequestNumActiveConnections case object instance.
+    */
+  def getRequestNumActiveConnections(): RequestNumActiveConnections.type  = {
+    RequestNumActiveConnections
+  }
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/FlinkMiniCluster.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/FlinkMiniCluster.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/FlinkMiniCluster.scala
index a547d25..0178bd3 100644
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/FlinkMiniCluster.scala
+++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/minicluster/FlinkMiniCluster.scala
@@ -69,7 +69,7 @@ abstract class FlinkMiniCluster(
     ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY,
     InetAddress.getByName("localhost").getHostAddress())
 
-  val configuration = generateConfiguration(userConfiguration)
+  protected val originalConfiguration = generateConfiguration(userConfiguration)
 
   /** Future to the [[ActorGateway]] of the current leader */
   var leaderGateway: Promise[ActorGateway] = Promise()
@@ -79,16 +79,16 @@ abstract class FlinkMiniCluster(
 
   /** Future lock */
   val futureLock = new Object()
-  
+
   implicit val executionContext = ExecutionContext.global
 
-  implicit val timeout = AkkaUtils.getTimeout(configuration)
+  implicit val timeout = AkkaUtils.getTimeout(originalConfiguration)
 
-  val haMode = HighAvailabilityMode.fromConfig(configuration)
+  val haMode = HighAvailabilityMode.fromConfig(originalConfiguration)
 
   val numJobManagers = getNumberOfJobManagers
 
-  var numTaskManagers = configuration.getInteger(
+  var numTaskManagers = originalConfiguration.getInteger(
     ConfigConstants.LOCAL_NUMBER_TASK_MANAGER,
     ConfigConstants.DEFAULT_LOCAL_NUMBER_TASK_MANAGER)
 
@@ -105,6 +105,22 @@ abstract class FlinkMiniCluster(
 
   private var isRunning = false
 
+  def configuration: Configuration = {
+    if (originalConfiguration.getInteger(
+      ConfigConstants.JOB_MANAGER_IPC_PORT_KEY,
+      ConfigConstants.DEFAULT_JOB_MANAGER_IPC_PORT) == 0) {
+      val leaderConfiguration = new Configuration(originalConfiguration)
+
+      val leaderPort = getLeaderRPCPort
+
+      leaderConfiguration.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, leaderPort)
+
+      leaderConfiguration
+    } else {
+      originalConfiguration
+    }
+  }
+
   // --------------------------------------------------------------------------
   //                           Abstract Methods
   // --------------------------------------------------------------------------
@@ -125,7 +141,7 @@ abstract class FlinkMiniCluster(
     if(haMode == HighAvailabilityMode.NONE) {
       1
     } else {
-      configuration.getInteger(
+      originalConfiguration.getInteger(
         ConfigConstants.LOCAL_NUMBER_JOB_MANAGER,
         ConfigConstants.DEFAULT_LOCAL_NUMBER_JOB_MANAGER
       )
@@ -136,7 +152,7 @@ abstract class FlinkMiniCluster(
     if(haMode == HighAvailabilityMode.NONE) {
       1
     } else {
-      configuration.getInteger(
+      originalConfiguration.getInteger(
         ConfigConstants.LOCAL_NUMBER_RESOURCE_MANAGER,
         ConfigConstants.DEFAULT_LOCAL_NUMBER_RESOURCE_MANAGER
       )
@@ -177,40 +193,55 @@ abstract class FlinkMiniCluster(
     Await.result(indexFuture, timeout)
   }
 
+  def getLeaderRPCPort: Int = {
+    val index = getLeaderIndex(timeout)
+
+    jobManagerActorSystems match {
+      case Some(jmActorSystems) =>
+        AkkaUtils.getAddress(jmActorSystems(index)).port match {
+          case Some(p) => p
+          case None => -1
+        }
+
+      case None => throw new Exception("The JobManager of the LocalFlinkMiniCluster has not been " +
+                                         "started properly.")
+    }
+  }
+
   def getResourceManagerAkkaConfig(index: Int): Config = {
     if (useSingleActorSystem) {
-      AkkaUtils.getAkkaConfig(configuration, None)
+      AkkaUtils.getAkkaConfig(originalConfiguration, None)
     } else {
-      val port = configuration.getInteger(ConfigConstants.RESOURCE_MANAGER_IPC_PORT_KEY,
-        ConfigConstants.DEFAULT_RESOURCE_MANAGER_IPC_PORT)
+      val port = originalConfiguration.getInteger(ConfigConstants.RESOURCE_MANAGER_IPC_PORT_KEY,
+                                                  ConfigConstants.DEFAULT_RESOURCE_MANAGER_IPC_PORT)
 
       val resolvedPort = if(port != 0) port + index else port
 
-      AkkaUtils.getAkkaConfig(configuration, Some((hostname, resolvedPort)))
+      AkkaUtils.getAkkaConfig(originalConfiguration, Some((hostname, resolvedPort)))
     }
   }
 
   def getJobManagerAkkaConfig(index: Int): Config = {
     if (useSingleActorSystem) {
-      AkkaUtils.getAkkaConfig(configuration, None)
+      AkkaUtils.getAkkaConfig(originalConfiguration, None)
     }
     else {
-      val port = configuration.getInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY,
-        ConfigConstants.DEFAULT_JOB_MANAGER_IPC_PORT)
+      val port = originalConfiguration.getInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY,
+                                                  ConfigConstants.DEFAULT_JOB_MANAGER_IPC_PORT)
 
       val resolvedPort = if(port != 0) port + index else port
 
-      AkkaUtils.getAkkaConfig(configuration, Some((hostname, resolvedPort)))
+      AkkaUtils.getAkkaConfig(originalConfiguration, Some((hostname, resolvedPort)))
     }
   }
 
   def getTaskManagerAkkaConfig(index: Int): Config = {
-    val port = configuration.getInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY,
-      ConfigConstants.DEFAULT_TASK_MANAGER_IPC_PORT)
+    val port = originalConfiguration.getInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY,
+                                                ConfigConstants.DEFAULT_TASK_MANAGER_IPC_PORT)
 
     val resolvedPort = if(port != 0) port + index else port
 
-    AkkaUtils.getAkkaConfig(configuration, Some((hostname, resolvedPort)))
+    AkkaUtils.getAkkaConfig(originalConfiguration, Some((hostname, resolvedPort)))
   }
 
   /**
@@ -257,7 +288,7 @@ abstract class FlinkMiniCluster(
           "The FlinkMiniCluster has not been started yet.")
       }
     } else {
-      JobClient.startJobClientActorSystem(configuration)
+      JobClient.startJobClientActorSystem(originalConfiguration)
     }
   }
 
@@ -320,7 +351,7 @@ abstract class FlinkMiniCluster(
 
     val jobManagerAkkaURL = AkkaUtils.getAkkaURL(jmActorSystems(0), jmActors(0))
 
-    webMonitor = startWebServer(configuration, jmActorSystems(0), jobManagerAkkaURL)
+    webMonitor = startWebServer(originalConfiguration, jmActorSystems(0), jobManagerAkkaURL)
 
     if(waitForTaskManagerRegistration) {
       waitForTaskManagersToBeRegistered()
@@ -528,7 +559,7 @@ abstract class FlinkMiniCluster(
           new StandaloneLeaderRetrievalService(
             AkkaUtils.getAkkaURL(jmActorSystems(0), jmActors(0)))
         } else {
-          ZooKeeperUtils.createLeaderRetrievalService(configuration)
+          ZooKeeperUtils.createLeaderRetrievalService(originalConfiguration)
         }
 
       case _ => throw new Exception("The FlinkMiniCluster has not been started properly.")

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/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 d30c047..cac5d91 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
@@ -18,23 +18,36 @@
 
 package org.apache.flink.runtime.minicluster
 
-import akka.actor.{ActorRef, ActorSystem}
-import org.apache.flink.api.common.JobID
+import java.util.concurrent.ExecutorService
 
+import akka.actor.{ActorRef, ActorSystem, Props}
+import org.apache.flink.api.common.JobID
 import org.apache.flink.api.common.io.FileOutputFormat
 import org.apache.flink.configuration.{ConfigConstants, Configuration}
-import org.apache.flink.runtime.akka.AkkaUtils
+import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory
+import org.apache.flink.runtime.checkpoint.savepoint.SavepointStore
 import org.apache.flink.runtime.clusterframework.FlinkResourceManager
 import org.apache.flink.runtime.clusterframework.standalone.StandaloneResourceManager
-import org.apache.flink.runtime.clusterframework.types.ResourceID
+import org.apache.flink.runtime.clusterframework.types.{ResourceID, ResourceIDRetrievable}
+import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager
+import org.apache.flink.runtime.executiongraph.restart.RestartStrategyFactory
+import org.apache.flink.runtime.instance.InstanceManager
+import org.apache.flink.runtime.io.disk.iomanager.IOManager
+import org.apache.flink.runtime.io.network.NetworkEnvironment
 import org.apache.flink.runtime.io.network.netty.NettyConfig
-import org.apache.flink.runtime.jobmanager.{MemoryArchivist, JobManager}
+import org.apache.flink.runtime.jobmanager.scheduler.Scheduler
+import org.apache.flink.runtime.jobmanager.{JobManager, MemoryArchivist, SubmittedJobGraphStore}
+import org.apache.flink.runtime.leaderelection.LeaderElectionService
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService
+import org.apache.flink.runtime.memory.MemoryManager
 import org.apache.flink.runtime.messages.JobManagerMessages
-import org.apache.flink.runtime.messages.JobManagerMessages.{CancellationFailure, CancellationResponse, StoppingFailure, StoppingResponse, RunningJobsStatus, RunningJobs}
-import org.apache.flink.runtime.taskmanager.TaskManager
+import org.apache.flink.runtime.messages.JobManagerMessages.{RunningJobsStatus, StoppingFailure, StoppingResponse}
+import org.apache.flink.runtime.metrics.MetricRegistry
+import org.apache.flink.runtime.taskmanager.{TaskManager, TaskManagerConfiguration, TaskManagerLocation}
 import org.apache.flink.runtime.util.EnvironmentInformation
 
 import scala.concurrent.Await
+import scala.concurrent.duration.FiniteDuration
 
 /**
  * Local Flink mini cluster which executes all [[TaskManager]]s and the [[JobManager]] in the same
@@ -65,8 +78,25 @@ class LocalFlinkMiniCluster(
     config
   }
 
+  //------------------------------------------------------------------------------------------------
+  // Actor classes
+  //------------------------------------------------------------------------------------------------
+
+  val jobManagerClass: Class[_ <: JobManager] = classOf[JobManager]
+
+  val taskManagerClass: Class[_ <: TaskManager] = classOf[TaskManager]
+
+  val memoryArchivistClass: Class[_ <: MemoryArchivist] = classOf[MemoryArchivist]
+
+  val resourceManagerClass: Class[_ <: FlinkResourceManager[_ <: ResourceIDRetrievable]] =
+    classOf[StandaloneResourceManager]
+
+  //------------------------------------------------------------------------------------------------
+  // Start methods for the distributed components
+  //------------------------------------------------------------------------------------------------
+
   override def startJobManager(index: Int, system: ActorSystem): ActorRef = {
-    val config = configuration.clone()
+    val config = originalConfiguration.clone()
 
     val jobManagerName = getJobManagerName(index)
     val archiveName = getArchiveName(index)
@@ -79,19 +109,48 @@ class LocalFlinkMiniCluster(
       config.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, jobManagerPort + index)
     }
 
-    val (jobManager, _) = JobManager.startJobManagerActors(
-      config,
-      system,
-      Some(jobManagerName),
-      Some(archiveName),
-      classOf[JobManager],
-      classOf[MemoryArchivist])
-
-    jobManager
+    val (executorService,
+    instanceManager,
+    scheduler,
+    libraryCacheManager,
+    restartStrategyFactory,
+    timeout,
+    archiveCount,
+    leaderElectionService,
+    submittedJobGraphStore,
+    checkpointRecoveryFactory,
+    savepointStore,
+    jobRecoveryTimeout,
+    metricsRegistry) = JobManager.createJobManagerComponents(config, createLeaderElectionService())
+
+    val archive = system.actorOf(
+      getArchiveProps(
+        memoryArchivistClass,
+        archiveCount),
+      archiveName)
+
+    system.actorOf(
+      getJobManagerProps(
+        jobManagerClass,
+        config,
+        executorService,
+        instanceManager,
+        scheduler,
+        libraryCacheManager,
+        archive,
+        restartStrategyFactory,
+        timeout,
+        leaderElectionService,
+        submittedJobGraphStore,
+        checkpointRecoveryFactory,
+        savepointStore,
+        jobRecoveryTimeout,
+        metricsRegistry),
+      jobManagerName)
   }
 
   override def startResourceManager(index: Int, system: ActorSystem): ActorRef = {
-    val config = configuration.clone()
+    val config = originalConfiguration.clone()
 
     val resourceManagerName = getResourceManagerName(index)
 
@@ -103,18 +162,16 @@ class LocalFlinkMiniCluster(
       config.setInteger(ConfigConstants.RESOURCE_MANAGER_IPC_PORT_KEY, resourceManagerPort + index)
     }
 
-    val resourceManager = FlinkResourceManager.startResourceManagerActors(
+    val resourceManagerProps = getResourceManagerProps(
+      resourceManagerClass,
       config,
-      system,
-      createLeaderRetrievalService(),
-      classOf[StandaloneResourceManager],
-      resourceManagerName)
+      createLeaderRetrievalService())
 
-    resourceManager
+    system.actorOf(resourceManagerProps, resourceManagerName)
   }
 
   override def startTaskManager(index: Int, system: ActorSystem): ActorRef = {
-    val config = configuration.clone()
+    val config = originalConfiguration.clone()
 
     val rpcPort = config.getInteger(
       ConfigConstants.TASK_MANAGER_IPC_PORT_KEY,
@@ -138,32 +195,115 @@ class LocalFlinkMiniCluster(
     } else {
       TaskManager.TASK_MANAGER_NAME
     }
-    
-    TaskManager.startTaskManagerComponentsAndActor(
+
+    val resourceID = ResourceID.generate() // generate random resource id
+
+    val (taskManagerConfig,
+    taskManagerLocation,
+    memoryManager,
+    ioManager,
+    network,
+    leaderRetrievalService) = TaskManager.createTaskManagerComponents(
       config,
-      ResourceID.generate(), // generate random resource id
-      system,
+      resourceID,
       hostname, // network interface to bind to
-      Some(taskManagerActorName), // actor name
-      Some(createLeaderRetrievalService()), // job manager leader retrieval service
       localExecution, // start network stack?
-      classOf[TaskManager])
+      Some(createLeaderRetrievalService()))
+
+    val props = getTaskManagerProps(
+      taskManagerClass,
+      taskManagerConfig,
+      resourceID,
+      taskManagerLocation,
+      memoryManager,
+      ioManager,
+      network,
+      leaderRetrievalService)
+
+    system.actorOf(props, taskManagerActorName)
   }
 
-  def getLeaderRPCPort: Int = {
-    val index = getLeaderIndex(timeout)
+  //------------------------------------------------------------------------------------------------
+  // Props for the distributed components
+  //------------------------------------------------------------------------------------------------
 
-    jobManagerActorSystems match {
-      case Some(jmActorSystems) =>
-        AkkaUtils.getAddress(jmActorSystems(index)).port match {
-          case Some(p) => p
-          case None => -1
-        }
+  def getArchiveProps(archiveClass: Class[_ <: MemoryArchivist], archiveCount: Int): Props = {
+    JobManager.getArchiveProps(archiveClass, archiveCount)
+  }
 
-      case None => throw new Exception("The JobManager of the LocalFlinkMiniCluster has not been " +
-        "started properly.")
-    }
+  def getJobManagerProps(
+    jobManagerClass: Class[_ <: JobManager],
+    configuration: Configuration,
+    executorService: ExecutorService,
+    instanceManager: InstanceManager,
+    scheduler: Scheduler,
+    libraryCacheManager: BlobLibraryCacheManager,
+    archive: ActorRef,
+    restartStrategyFactory: RestartStrategyFactory,
+    timeout: FiniteDuration,
+    leaderElectionService: LeaderElectionService,
+    submittedJobGraphStore: SubmittedJobGraphStore,
+    checkpointRecoveryFactory: CheckpointRecoveryFactory,
+    savepointStore: SavepointStore,
+    jobRecoveryTimeout: FiniteDuration,
+    metricsRegistry: Option[MetricRegistry]): Props = {
+
+    JobManager.getJobManagerProps(
+      jobManagerClass,
+      configuration,
+      executorService,
+      instanceManager,
+      scheduler,
+      libraryCacheManager,
+      archive,
+      restartStrategyFactory,
+      timeout,
+      leaderElectionService,
+      submittedJobGraphStore,
+      checkpointRecoveryFactory,
+      savepointStore,
+      jobRecoveryTimeout,
+      metricsRegistry)
+  }
+
+  def getTaskManagerProps(
+    taskManagerClass: Class[_ <: TaskManager],
+    taskManagerConfig: TaskManagerConfiguration,
+    resourceID: ResourceID,
+    taskManagerLocation: TaskManagerLocation,
+    memoryManager: MemoryManager,
+    ioManager: IOManager,
+    networkEnvironment: NetworkEnvironment,
+    leaderRetrievalService: LeaderRetrievalService): Props = {
+
+    TaskManager.getTaskManagerProps(
+      taskManagerClass,
+      taskManagerConfig,
+      resourceID,
+      taskManagerLocation,
+      memoryManager,
+      ioManager,
+      networkEnvironment,
+      leaderRetrievalService)
+  }
+
+  def getResourceManagerProps(
+    resourceManagerClass: Class[_ <: FlinkResourceManager[_ <: ResourceIDRetrievable]],
+    configuration: Configuration,
+    leaderRetrievalService: LeaderRetrievalService): Props = {
+
+    FlinkResourceManager.getResourceManagerProps(
+      resourceManagerClass,
+      configuration,
+      leaderRetrievalService)
+  }
+
+  //------------------------------------------------------------------------------------------------
+  // Helper methods
+  //------------------------------------------------------------------------------------------------
 
+  def createLeaderElectionService(): Option[LeaderElectionService] = {
+    None
   }
 
   def initializeIOFormatClasses(configuration: Configuration): Unit = {
@@ -186,7 +326,7 @@ class LocalFlinkMiniCluster(
       val bufferSize: Int = config.getInteger(
         ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
         ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_SEGMENT_SIZE)
-      
+
       val bufferMem: Long = config.getLong(
         ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY,
         ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_NUM_BUFFERS) * bufferSize.toLong
@@ -218,6 +358,7 @@ class LocalFlinkMiniCluster(
     val config: Configuration = new Configuration()
 
     config.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, hostname)
+    config.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, 0)
 
     config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER,
       ConfigConstants.DEFAULT_LOCAL_NUMBER_TASK_MANAGER)
@@ -252,11 +393,11 @@ class LocalFlinkMiniCluster(
       JobManager.ARCHIVE_NAME
     }
   }
-  
+
   // --------------------------------------------------------------------------
   //  Actions on running jobs
   // --------------------------------------------------------------------------
-  
+
   def currentlyRunningJobs: Iterable[JobID] = {
     val leader = getLeaderGateway(timeout)
     val future = leader.ask(JobManagerMessages.RequestRunningJobsStatus, timeout)
@@ -269,7 +410,7 @@ class LocalFlinkMiniCluster(
     currentlyRunningJobs.foreach(list.add)
     list
   }
-  
+
   def stopJob(id: JobID) : Unit = {
     val leader = getLeaderGateway(timeout)
     val response = leader.ask(new JobManagerMessages.StopJob(id), timeout)

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala
index 84750a3..de85f30 100644
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala
+++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala
@@ -354,6 +354,21 @@ class TaskManager(
         case None =>
           sender() ! new IOException("BlobService not available. Cannot upload TaskManager logs.")
       }
+
+    case RequestBroadcastVariablesWithReferences =>
+      sender ! decorateMessage(
+        ResponseBroadcastVariablesWithReferences(
+          bcVarManager.getNumberOfVariablesWithReferences)
+      )
+
+    case RequestNumActiveConnections =>
+      val numActive = if (!network.isShutdown) {
+        network.getConnectionManager.getNumberOfActiveConnections
+      } else {
+        0
+      }
+
+      sender ! decorateMessage(ResponseNumActiveConnections(numActive))
   }
 
   /**
@@ -1781,6 +1796,7 @@ object TaskManager {
   }
 
   /**
+   * Starts the task manager actor.
    *
    * @param configuration The configuration for the TaskManager.
    * @param resourceID The id of the resource which the task manager will run on.
@@ -1817,11 +1833,75 @@ object TaskManager {
       taskManagerClass: Class[_ <: TaskManager])
     : ActorRef = {
 
-    val (taskManagerConfig : TaskManagerConfiguration,      
-      netConfig: NetworkEnvironmentConfiguration,
-      taskManagerAddress: InetSocketAddress,
-      memType: MemoryType
-    ) = parseTaskManagerConfiguration(
+    val (taskManagerConfig,
+      connectionInfo,
+      memoryManager,
+      ioManager,
+      network,
+      leaderRetrievalService) = createTaskManagerComponents(
+      configuration,
+      resourceID,
+      taskManagerHostname,
+      localTaskManagerCommunication,
+      leaderRetrievalServiceOption)
+
+    // create the actor properties (which define the actor constructor parameters)
+    val tmProps = getTaskManagerProps(
+      taskManagerClass,
+      taskManagerConfig,
+      resourceID,
+      connectionInfo,
+      memoryManager,
+      ioManager,
+      network,
+      leaderRetrievalService)
+
+    taskManagerActorName match {
+      case Some(actorName) => actorSystem.actorOf(tmProps, actorName)
+      case None => actorSystem.actorOf(tmProps)
+    }
+  }
+
+  def getTaskManagerProps(
+    taskManagerClass: Class[_ <: TaskManager],
+    taskManagerConfig: TaskManagerConfiguration,
+    resourceID: ResourceID,
+    taskManagerLocation: TaskManagerLocation,
+    memoryManager: MemoryManager,
+    ioManager: IOManager,
+    networkEnvironment: NetworkEnvironment,
+    leaderRetrievalService: LeaderRetrievalService
+  ): Props = {
+    Props(
+      taskManagerClass,
+      taskManagerConfig,
+      resourceID,
+      taskManagerLocation,
+      memoryManager,
+      ioManager,
+      networkEnvironment,
+      taskManagerConfig.numberOfSlots,
+      leaderRetrievalService)
+  }
+
+  def createTaskManagerComponents(
+    configuration: Configuration,
+    resourceID: ResourceID,
+    taskManagerHostname: String,
+    localTaskManagerCommunication: Boolean,
+    leaderRetrievalServiceOption: Option[LeaderRetrievalService]):
+      (TaskManagerConfiguration,
+      TaskManagerLocation,
+      MemoryManager,
+      IOManager,
+      NetworkEnvironment,
+      LeaderRetrievalService) = {
+
+    val (taskManagerConfig : TaskManagerConfiguration,
+    netConfig: NetworkEnvironmentConfiguration,
+    taskManagerAddress: InetSocketAddress,
+    memType: MemoryType
+      ) = parseTaskManagerConfiguration(
       configuration,
       taskManagerHostname,
       localTaskManagerCommunication)
@@ -1895,10 +1975,10 @@ object TaskManager {
     // check if a value has been configured
     val configuredMemory = configuration.getLong(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, -1L)
     checkConfigParameter(configuredMemory == -1 || configuredMemory > 0, configuredMemory,
-      ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY,
-      "MemoryManager needs at least one MB of memory. " +
-        "If you leave this config parameter empty, the system automatically " +
-        "pick a fraction of the available memory.")
+                         ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY,
+                         "MemoryManager needs at least one MB of memory. " +
+                           "If you leave this config parameter empty, the system automatically " +
+                           "pick a fraction of the available memory.")
 
 
     val preAllocateMemory = configuration.getBoolean(
@@ -1910,7 +1990,7 @@ object TaskManager {
         LOG.info(s"Using $configuredMemory MB for managed memory.")
       } else {
         LOG.info(s"Limiting managed memory to $configuredMemory MB, " +
-          s"memory will be allocated lazily.")
+                   s"memory will be allocated lazily.")
       }
       configuredMemory << 20 // megabytes to bytes
     }
@@ -1928,10 +2008,10 @@ object TaskManager {
 
         if (preAllocateMemory) {
           LOG.info(s"Using $fraction of the currently free heap space for managed " +
-            s"heap memory (${relativeMemSize >> 20} MB).")
+                     s"heap memory (${relativeMemSize >> 20} MB).")
         } else {
           LOG.info(s"Limiting managed memory to $fraction of the currently free heap space " +
-            s"(${relativeMemSize >> 20} MB), memory will be allocated lazily.")
+                     s"(${relativeMemSize >> 20} MB), memory will be allocated lazily.")
         }
 
         relativeMemSize
@@ -1944,10 +2024,10 @@ object TaskManager {
 
         if (preAllocateMemory) {
           LOG.info(s"Using $fraction of the maximum memory size for " +
-            s"managed off-heap memory (${directMemorySize >> 20} MB).")
+                     s"managed off-heap memory (${directMemorySize >> 20} MB).")
         } else {
           LOG.info(s"Limiting managed memory to $fraction of the maximum memory size " +
-            s"(${directMemorySize >> 20} MB), memory will be allocated lazily.")
+                     s"(${directMemorySize >> 20} MB), memory will be allocated lazily.")
         }
 
         directMemorySize
@@ -1971,12 +2051,12 @@ object TaskManager {
         memType match {
           case MemoryType.HEAP =>
             throw new Exception(s"OutOfMemory error (${e.getMessage()})" +
-              s" while allocating the TaskManager heap memory ($memorySize bytes).", e)
+                      s" while allocating the TaskManager heap memory ($memorySize bytes).", e)
 
           case MemoryType.OFF_HEAP =>
             throw new Exception(s"OutOfMemory error (${e.getMessage()})" +
-              s" while allocating the TaskManager off-heap memory ($memorySize bytes). " +
-              s"Try increasing the maximum direct memory (-XX:MaxDirectMemorySize)", e)
+                      s" while allocating the TaskManager off-heap memory ($memorySize bytes). " +
+                      s"Try increasing the maximum direct memory (-XX:MaxDirectMemorySize)", e)
 
           case _ => throw e
         }
@@ -1990,22 +2070,12 @@ object TaskManager {
       case None => LeaderRetrievalUtils.createLeaderRetrievalService(configuration)
     }
 
-    // create the actor properties (which define the actor constructor parameters)
-    val tmProps = Props(
-      taskManagerClass,
-      taskManagerConfig,
-      resourceID,
+    (taskManagerConfig,
       taskManagerLocation,
       memoryManager,
       ioManager,
       network,
-      taskManagerConfig.numberOfSlots,
       leaderRetrievalService)
-
-    taskManagerActorName match {
-      case Some(actorName) => actorSystem.actorOf(tmProps, actorName)
-      case None => actorSystem.actorOf(tmProps)
-    }
   }
 
 
@@ -2055,8 +2125,8 @@ object TaskManager {
    * @param taskManagerHostname The host name under which the TaskManager communicates.
    * @param localTaskManagerCommunication True, to skip initializing the network stack.
    *                                      Use only in cases where only one task manager runs.
-   * @return A tuple (TaskManagerConfiguration, network configuration,
-   *                  InstanceConnectionInfo, JobManager actor Akka URL).
+   * @return A tuple (TaskManagerConfiguration, network configuration, inet socket address,
+    *         memory tyep).
    */
   @throws(classOf[IllegalArgumentException])
   def parseTaskManagerConfiguration(

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingJobManager.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingJobManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingJobManager.scala
deleted file mode 100644
index 16331ac..0000000
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingJobManager.scala
+++ /dev/null
@@ -1,72 +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.testingUtils
-
-import akka.actor.ActorRef
-
-import org.apache.flink.configuration.Configuration
-import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory
-import org.apache.flink.runtime.checkpoint.savepoint.SavepointStore
-import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager
-import org.apache.flink.runtime.executiongraph.restart.RestartStrategyFactory
-import org.apache.flink.runtime.instance.InstanceManager
-import org.apache.flink.runtime.jobmanager.scheduler.Scheduler
-import org.apache.flink.runtime.jobmanager.{JobManager, SubmittedJobGraphStore}
-import org.apache.flink.runtime.leaderelection.LeaderElectionService
-import org.apache.flink.runtime.metrics.MetricRegistry
-
-import scala.concurrent.duration._
-import scala.language.postfixOps
-
-import java.util.concurrent.ExecutorService
-
-/** JobManager implementation extended by testing messages
-  *
-  */
-class TestingJobManager(
-    flinkConfiguration: Configuration,
-    executorService: ExecutorService,
-    instanceManager: InstanceManager,
-    scheduler: Scheduler,
-    libraryCacheManager: BlobLibraryCacheManager,
-    archive: ActorRef,
-    restartStrategyFactory: RestartStrategyFactory,
-    timeout: FiniteDuration,
-    leaderElectionService: LeaderElectionService,
-    submittedJobGraphs : SubmittedJobGraphStore,
-    checkpointRecoveryFactory : CheckpointRecoveryFactory,
-    savepointStore : SavepointStore,
-    jobRecoveryTimeout : FiniteDuration,
-    metricRegistry : Option[MetricRegistry])
-  extends JobManager(
-    flinkConfiguration,
-      executorService,
-    instanceManager,
-    scheduler,
-    libraryCacheManager,
-    archive,
-    restartStrategyFactory,
-    timeout,
-    leaderElectionService,
-    submittedJobGraphs,
-    checkpointRecoveryFactory,
-    savepointStore,
-    jobRecoveryTimeout,
-    metricRegistry)
-  with TestingJobManagerLike {}

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingJobManagerLike.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingJobManagerLike.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingJobManagerLike.scala
deleted file mode 100644
index 3947b17..0000000
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingJobManagerLike.scala
+++ /dev/null
@@ -1,417 +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.testingUtils
-
-import akka.actor.{ActorRef, Cancellable, Terminated}
-import akka.pattern.{ask, pipe}
-import org.apache.flink.api.common.JobID
-import org.apache.flink.runtime.FlinkActor
-import org.apache.flink.runtime.execution.ExecutionState
-import org.apache.flink.runtime.jobgraph.JobStatus
-import org.apache.flink.runtime.jobmanager.JobManager
-import org.apache.flink.runtime.messages.ExecutionGraphMessages.JobStatusChanged
-import org.apache.flink.runtime.messages.JobManagerMessages.{GrantLeadership, RegisterJobClient, RequestClassloadingProps}
-import org.apache.flink.runtime.messages.Messages.{Acknowledge, Disconnect}
-import org.apache.flink.runtime.messages.RegistrationMessages.RegisterTaskManager
-import org.apache.flink.runtime.messages.TaskManagerMessages.Heartbeat
-import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages._
-import org.apache.flink.runtime.testingUtils.TestingMessages._
-import org.apache.flink.runtime.testingUtils.TestingTaskManagerMessages.AccumulatorsChanged
-
-import scala.collection.mutable
-import scala.concurrent.Future
-import scala.concurrent.duration._
-import scala.language.postfixOps
-
-/** This mixin can be used to decorate a JobManager with messages for testing purpose.  */
-trait TestingJobManagerLike extends FlinkActor {
-  that: JobManager =>
-
-  import context._
-
-  import scala.collection.JavaConverters._
-
-  val waitForAllVerticesToBeRunning = scala.collection.mutable.HashMap[JobID, Set[ActorRef]]()
-  val waitForTaskManagerToBeTerminated = scala.collection.mutable.HashMap[String, Set[ActorRef]]()
-
-  val waitForAllVerticesToBeRunningOrFinished =
-    scala.collection.mutable.HashMap[JobID, Set[ActorRef]]()
-
-  var periodicCheck: Option[Cancellable] = None
-
-  val waitForJobStatus = scala.collection.mutable.HashMap[JobID,
-    collection.mutable.HashMap[JobStatus, Set[ActorRef]]]()
-
-  val waitForAccumulatorUpdate = scala.collection.mutable.HashMap[JobID, (Boolean, Set[ActorRef])]()
-
-  val waitForLeader = scala.collection.mutable.HashSet[ActorRef]()
-
-  val waitForNumRegisteredTaskManagers = mutable.PriorityQueue.newBuilder(
-    new Ordering[(Int, ActorRef)] {
-      override def compare(x: (Int, ActorRef), y: (Int, ActorRef)): Int = y._1 - x._1
-    })
-
-  val waitForClient = scala.collection.mutable.HashSet[ActorRef]()
-
-  val waitForShutdown = scala.collection.mutable.HashSet[ActorRef]()
-
-  var disconnectDisabled = false
-
-  var postStopEnabled = true
-
-  abstract override def postStop(): Unit = {
-    if (postStopEnabled) {
-      super.postStop()
-    } else {
-      // only stop leader election service to revoke the leadership of this JM so that a new JM
-      // can be elected leader
-      leaderElectionService.stop()
-    }
-  }
-
-  abstract override def handleMessage: Receive = {
-    handleTestingMessage orElse super.handleMessage
-  }
-
-  def handleTestingMessage: Receive = {
-    case Alive => sender() ! Acknowledge
-
-    case RequestExecutionGraph(jobID) =>
-      currentJobs.get(jobID) match {
-        case Some((executionGraph, jobInfo)) => sender() ! decorateMessage(
-          ExecutionGraphFound(
-            jobID,
-            executionGraph)
-        )
-
-        case None => archive.tell(decorateMessage(RequestExecutionGraph(jobID)), sender())
-      }
-
-    case WaitForAllVerticesToBeRunning(jobID) =>
-      if(checkIfAllVerticesRunning(jobID)){
-        sender() ! decorateMessage(AllVerticesRunning(jobID))
-      }else{
-        val waiting = waitForAllVerticesToBeRunning.getOrElse(jobID, Set[ActorRef]())
-        waitForAllVerticesToBeRunning += jobID -> (waiting + sender())
-
-        if(periodicCheck.isEmpty){
-          periodicCheck =
-            Some(
-              context.system.scheduler.schedule(
-                0 seconds,
-                200 millis,
-                self,
-                decorateMessage(NotifyListeners)
-              )
-            )
-        }
-      }
-    case WaitForAllVerticesToBeRunningOrFinished(jobID) =>
-      if(checkIfAllVerticesRunningOrFinished(jobID)){
-        sender() ! decorateMessage(AllVerticesRunning(jobID))
-      }else{
-        val waiting = waitForAllVerticesToBeRunningOrFinished.getOrElse(jobID, Set[ActorRef]())
-        waitForAllVerticesToBeRunningOrFinished += jobID -> (waiting + sender())
-
-        if(periodicCheck.isEmpty){
-          periodicCheck =
-            Some(
-              context.system.scheduler.schedule(
-                0 seconds,
-                200 millis,
-                self,
-                decorateMessage(NotifyListeners)
-              )
-            )
-        }
-      }
-
-    case NotifyListeners =>
-      for(jobID <- currentJobs.keySet){
-        notifyListeners(jobID)
-      }
-
-      if(waitForAllVerticesToBeRunning.isEmpty && waitForAllVerticesToBeRunningOrFinished.isEmpty) {
-        periodicCheck foreach { _.cancel() }
-        periodicCheck = None
-      }
-
-
-    case NotifyWhenJobRemoved(jobID) =>
-      val gateways = instanceManager.getAllRegisteredInstances.asScala.map(_.getActorGateway)
-
-      val responses = gateways.map{
-        gateway => gateway.ask(NotifyWhenJobRemoved(jobID), timeout).mapTo[Boolean]
-      }
-
-      val jobRemovedOnJobManager = (self ? CheckIfJobRemoved(jobID))(timeout).mapTo[Boolean]
-
-      val allFutures = responses ++ Seq(jobRemovedOnJobManager)
-
-      import context.dispatcher
-      Future.fold(allFutures)(true)(_ & _) map(decorateMessage(_)) pipeTo sender()
-
-    case CheckIfJobRemoved(jobID) =>
-      if(currentJobs.contains(jobID)) {
-        context.system.scheduler.scheduleOnce(
-          200 milliseconds,
-          self,
-          decorateMessage(CheckIfJobRemoved(jobID))
-        )(context.dispatcher, sender())
-      } else {
-        sender() ! decorateMessage(true)
-      }
-
-    case NotifyWhenTaskManagerTerminated(taskManager) =>
-      val waiting = waitForTaskManagerToBeTerminated.getOrElse(taskManager.path.name, Set())
-      waitForTaskManagerToBeTerminated += taskManager.path.name -> (waiting + sender)
-
-    case msg@Terminated(taskManager) =>
-      super.handleMessage(msg)
-
-      waitForTaskManagerToBeTerminated.remove(taskManager.path.name) foreach {
-        _ foreach {
-          listener =>
-            listener ! decorateMessage(TaskManagerTerminated(taskManager))
-        }
-      }
-
-    // see shutdown method for reply
-    case NotifyOfComponentShutdown =>
-      waitForShutdown += sender()
-
-    case NotifyWhenAccumulatorChange(jobID) =>
-
-      val (updated, registered) = waitForAccumulatorUpdate.
-        getOrElse(jobID, (false, Set[ActorRef]()))
-      waitForAccumulatorUpdate += jobID -> (updated, registered + sender)
-      sender ! true
-
-    /**
-     * Notification from the task manager that changed accumulator are transferred on next
-     * Hearbeat. We need to keep this state to notify the listeners on next Heartbeat report.
-     */
-    case AccumulatorsChanged(jobID: JobID) =>
-      waitForAccumulatorUpdate.get(jobID) match {
-        case Some((updated, registered)) =>
-          waitForAccumulatorUpdate.put(jobID, (true, registered))
-        case None =>
-      }
-
-    /**
-     * Disabled async processing of accumulator values and send accumulators to the listeners if
-     * we previously received an [[AccumulatorsChanged]] message.
-     */
-    case msg : Heartbeat =>
-      super.handleMessage(msg)
-
-      waitForAccumulatorUpdate foreach {
-        case (jobID, (updated, actors)) if updated =>
-          currentJobs.get(jobID) match {
-            case Some((graph, jobInfo)) =>
-              val flinkAccumulators = graph.getFlinkAccumulators
-              val userAccumulators = graph.aggregateUserAccumulators
-              actors foreach {
-                actor => actor ! UpdatedAccumulators(jobID, flinkAccumulators, userAccumulators)
-              }
-            case None =>
-          }
-          waitForAccumulatorUpdate.put(jobID, (false, actors))
-        case _ =>
-      }
-
-    case RequestWorkingTaskManager(jobID) =>
-      currentJobs.get(jobID) match {
-        case Some((eg, _)) =>
-          if(eg.getAllExecutionVertices.asScala.isEmpty){
-            sender ! decorateMessage(WorkingTaskManager(None))
-          } else {
-            val resource = eg.getAllExecutionVertices.asScala.head.getCurrentAssignedResource
-
-            if(resource == null){
-              sender ! decorateMessage(WorkingTaskManager(None))
-            } else {
-              sender ! decorateMessage(
-                WorkingTaskManager(
-                  Some(resource.getTaskManagerActorGateway())
-                )
-              )
-            }
-          }
-        case None => sender ! decorateMessage(WorkingTaskManager(None))
-      }
-
-    case NotifyWhenJobStatus(jobID, state) =>
-      val jobStatusListener = waitForJobStatus.getOrElseUpdate(jobID,
-        scala.collection.mutable.HashMap[JobStatus, Set[ActorRef]]())
-
-      val listener = jobStatusListener.getOrElse(state, Set[ActorRef]())
-
-      jobStatusListener += state -> (listener + sender)
-
-    case msg@JobStatusChanged(jobID, newJobStatus, _, _) =>
-      super.handleMessage(msg)
-
-      val cleanup = waitForJobStatus.get(jobID) match {
-        case Some(stateListener) =>
-          stateListener.remove(newJobStatus) match {
-            case Some(listeners) =>
-              listeners foreach {
-                _ ! decorateMessage(JobStatusIs(jobID, newJobStatus))
-              }
-            case _ =>
-          }
-          stateListener.isEmpty
-
-        case _ => false
-      }
-
-      if (cleanup) {
-        waitForJobStatus.remove(jobID)
-      }
-
-    case DisableDisconnect =>
-      disconnectDisabled = true
-
-    case DisablePostStop =>
-      postStopEnabled = false
-
-    case RequestSavepoint(savepointPath) =>
-      try {
-        val savepoint = savepointStore.loadSavepoint(savepointPath)
-        sender ! ResponseSavepoint(savepoint)
-      }
-      catch {
-        case e: Exception =>
-          sender ! ResponseSavepoint(null)
-      }
-
-    case msg: Disconnect =>
-      if (!disconnectDisabled) {
-        super.handleMessage(msg)
-
-        val taskManager = sender()
-
-        waitForTaskManagerToBeTerminated.remove(taskManager.path.name) foreach {
-          _ foreach {
-            listener =>
-              listener ! decorateMessage(TaskManagerTerminated(taskManager))
-          }
-        }
-      }
-
-    case NotifyWhenLeader =>
-      if (leaderElectionService.hasLeadership) {
-        sender() ! true
-      } else {
-        waitForLeader += sender()
-      }
-
-    case msg: GrantLeadership =>
-      super.handleMessage(msg)
-
-      waitForLeader.foreach(_ ! true)
-
-      waitForLeader.clear()
-
-    case NotifyWhenClientConnects =>
-      waitForClient += sender()
-      sender() ! true
-
-    case msg: RegisterJobClient =>
-      super.handleMessage(msg)
-      waitForClient.foreach(_ ! ClientConnected)
-    case msg: RequestClassloadingProps =>
-      super.handleMessage(msg)
-      waitForClient.foreach(_ ! ClassLoadingPropsDelivered)
-
-    case NotifyWhenAtLeastNumTaskManagerAreRegistered(numRegisteredTaskManager) =>
-      if (that.instanceManager.getNumberOfRegisteredTaskManagers >= numRegisteredTaskManager) {
-        // there are already at least numRegisteredTaskManager registered --> send Acknowledge
-        sender() ! Acknowledge
-      } else {
-        // wait until we see at least numRegisteredTaskManager being registered at the JobManager
-        waitForNumRegisteredTaskManagers += ((numRegisteredTaskManager, sender()))
-      }
-
-    // TaskManager may be registered on these two messages
-    case msg @ (_: RegisterTaskManager) =>
-      super.handleMessage(msg)
-
-      // dequeue all senders which wait for instanceManager.getNumberOfStartedTaskManagers or
-      // fewer registered TaskManagers
-      while (waitForNumRegisteredTaskManagers.nonEmpty &&
-        waitForNumRegisteredTaskManagers.head._1 <=
-          instanceManager.getNumberOfRegisteredTaskManagers) {
-        val receiver = waitForNumRegisteredTaskManagers.dequeue()._2
-        receiver ! Acknowledge
-      }
-  }
-
-  def checkIfAllVerticesRunning(jobID: JobID): Boolean = {
-    currentJobs.get(jobID) match {
-      case Some((eg, _)) =>
-        eg.getAllExecutionVertices.asScala.forall( _.getExecutionState == ExecutionState.RUNNING)
-      case None => false
-    }
-  }
-
-  def checkIfAllVerticesRunningOrFinished(jobID: JobID): Boolean = {
-    currentJobs.get(jobID) match {
-      case Some((eg, _)) =>
-        eg.getAllExecutionVertices.asScala.forall {
-          case vertex =>
-            (vertex.getExecutionState == ExecutionState.RUNNING
-              || vertex.getExecutionState == ExecutionState.FINISHED)
-        }
-      case None => false
-    }
-  }
-
-  def notifyListeners(jobID: JobID): Unit = {
-    if(checkIfAllVerticesRunning(jobID)) {
-      waitForAllVerticesToBeRunning.remove(jobID) match {
-        case Some(listeners) =>
-          for (listener <- listeners) {
-            listener ! decorateMessage(AllVerticesRunning(jobID))
-          }
-        case _ =>
-      }
-    }
-
-    if(checkIfAllVerticesRunningOrFinished(jobID)) {
-      waitForAllVerticesToBeRunningOrFinished.remove(jobID) match {
-        case Some(listeners) =>
-          for (listener <- listeners) {
-            listener ! decorateMessage(AllVerticesRunning(jobID))
-          }
-        case _ =>
-      }
-    }
-  }
-
-  /**
-    * No killing of the VM for testing.
-    */
-  override protected def shutdown(): Unit = {
-    log.info("Shutting down TestingJobManager.")
-    waitForShutdown.foreach(_ ! ComponentShutdown(self))
-    waitForShutdown.clear()
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingJobManagerMessages.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingJobManagerMessages.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingJobManagerMessages.scala
deleted file mode 100644
index f121305..0000000
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingJobManagerMessages.scala
+++ /dev/null
@@ -1,133 +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.testingUtils
-
-import java.util.Map
-
-import akka.actor.ActorRef
-import org.apache.flink.api.common.JobID
-import org.apache.flink.api.common.accumulators.Accumulator
-import org.apache.flink.runtime.accumulators.AccumulatorRegistry
-import org.apache.flink.runtime.checkpoint.CompletedCheckpoint
-import org.apache.flink.runtime.executiongraph.{ExecutionAttemptID, ExecutionGraph}
-import org.apache.flink.runtime.instance.ActorGateway
-import org.apache.flink.runtime.jobgraph.JobStatus
-import org.apache.flink.runtime.checkpoint.savepoint.Savepoint
-
-object TestingJobManagerMessages {
-
-  case class RequestExecutionGraph(jobID: JobID)
-
-  sealed trait ResponseExecutionGraph {
-    def jobID: JobID
-  }
-
-  case class ExecutionGraphFound(jobID: JobID, executionGraph: ExecutionGraph) extends
-  ResponseExecutionGraph
-
-  case class ExecutionGraphNotFound(jobID: JobID) extends ResponseExecutionGraph
-
-  case class WaitForAllVerticesToBeRunning(jobID: JobID)
-  case class WaitForAllVerticesToBeRunningOrFinished(jobID: JobID)
-  case class AllVerticesRunning(jobID: JobID)
-
-  case class NotifyWhenJobRemoved(jobID: JobID)
-
-  case class RequestWorkingTaskManager(jobID: JobID)
-  case class WorkingTaskManager(gatewayOption: Option[ActorGateway])
-
-  case class NotifyWhenJobStatus(jobID: JobID, state: JobStatus)
-  case class JobStatusIs(jobID: JobID, state: JobStatus)
-
-  case object NotifyListeners
-
-  case class NotifyWhenTaskManagerTerminated(taskManager: ActorRef)
-  case class TaskManagerTerminated(taskManager: ActorRef)
-
-  /**
-   * Registers a listener to receive a message when accumulators changed.
-   * The change must be explicitly triggered by the TestingTaskManager which can receive an
-   * [[org.apache.flink.runtime.testingUtils.TestingTaskManagerMessages.AccumulatorsChanged]]
-   * message by a task that changed the accumulators. This message is then
-   * forwarded to the JobManager which will send the accumulators in the [[UpdatedAccumulators]]
-   * message when the next Heartbeat occurs.
-   */
-  case class NotifyWhenAccumulatorChange(jobID: JobID)
-
-  /**
-   * Reports updated accumulators back to the listener.
-   */
-  case class UpdatedAccumulators(jobID: JobID,
-    flinkAccumulators: Map[ExecutionAttemptID, Map[AccumulatorRegistry.Metric, Accumulator[_,_]]],
-    userAccumulators: Map[String, Accumulator[_,_]])
-
-  /** Notifies the sender when the [[TestingJobManager]] has been elected as the leader
-   *
-   */
-  case object NotifyWhenLeader
-
-  /**
-    * Notifies the sender when the [[TestingJobManager]] receives new clients for jobs
-    */
-  case object NotifyWhenClientConnects
-  /**
-    * Notifes of client connect
-    */
-  case object ClientConnected
-  /**
-    * Notifies when the client has requested class loading information
-    */
-  case object ClassLoadingPropsDelivered
-
-  /**
-   * Registers to be notified by an [[org.apache.flink.runtime.messages.Messages.Acknowledge]]
-   * message when at least numRegisteredTaskManager have registered at the JobManager.
-   *
-   * @param numRegisteredTaskManager minimum number of registered TMs before the sender is notified
-   */
-  case class NotifyWhenAtLeastNumTaskManagerAreRegistered(numRegisteredTaskManager: Int)
-
-  /** Disables the post stop method of the [[TestingJobManager]].
-    *
-    * Only the leaderElectionService is stopped in the postStop method call to revoke the leadership
-    */
-  case object DisablePostStop
-
-  /**
-    * Requests a savepoint from the job manager.
-    *
-    * @param savepointPath The path of the savepoint to request.
-    */
-  case class RequestSavepoint(savepointPath: String)
-
-  /**
-    * Response to a savepoint request.
-    *
-    * @param savepoint The requested savepoint or null if none available.
-    */
-  case class ResponseSavepoint(savepoint: Savepoint)
-
-  def getNotifyWhenLeader(): AnyRef = NotifyWhenLeader
-  def getNotifyWhenClientConnects(): AnyRef = NotifyWhenClientConnects
-  def getDisablePostStop(): AnyRef = DisablePostStop
-
-  def getClientConnected(): AnyRef = ClientConnected
-  def getClassLoadingPropsDelivered(): AnyRef = ClassLoadingPropsDelivered
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingMemoryArchivist.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingMemoryArchivist.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingMemoryArchivist.scala
deleted file mode 100644
index 48a1ddd..0000000
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingMemoryArchivist.scala
+++ /dev/null
@@ -1,43 +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.testingUtils
-
-import org.apache.flink.runtime.jobmanager.MemoryArchivist
-import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages.{ExecutionGraphFound, ExecutionGraphNotFound, RequestExecutionGraph}
-
-/** Memory archivist extended by testing messages
-  *
-  * @param maxEntries number of maximum number of archived jobs
-  */
-class TestingMemoryArchivist(maxEntries: Int) extends MemoryArchivist(maxEntries) {
-
-  override def handleMessage: Receive = {
-    handleTestingMessage orElse super.handleMessage
-  }
-
-  def handleTestingMessage: Receive = {
-    case RequestExecutionGraph(jobID) =>
-      val executionGraph = graphs.get(jobID)
-      
-      executionGraph match {
-        case Some(graph) => sender ! decorateMessage(ExecutionGraphFound(jobID, graph))
-        case None => sender ! decorateMessage(ExecutionGraphNotFound(jobID))
-      }
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingMessages.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingMessages.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingMessages.scala
deleted file mode 100644
index 91d169a..0000000
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingMessages.scala
+++ /dev/null
@@ -1,40 +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.testingUtils
-
-import akka.actor.ActorRef
-import org.apache.flink.api.common.JobID
-
-object TestingMessages {
-
-  case class CheckIfJobRemoved(jobID: JobID)
-
-  case object DisableDisconnect
-
-  case object Alive
-
-  def getAlive: AnyRef = Alive
-
-  def getDisableDisconnect: AnyRef = DisableDisconnect
-
-  case object NotifyOfComponentShutdown
-  case class ComponentShutdown(ref: ActorRef)
-
-  def getNotifyOfComponentShutdown(): AnyRef = NotifyOfComponentShutdown
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingTaskManager.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingTaskManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingTaskManager.scala
deleted file mode 100644
index 9b5a147..0000000
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingTaskManager.scala
+++ /dev/null
@@ -1,70 +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.testingUtils
-
-import org.apache.flink.runtime.clusterframework.types.ResourceID
-import org.apache.flink.runtime.io.disk.iomanager.IOManager
-import org.apache.flink.runtime.io.network.NetworkEnvironment
-import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService
-import org.apache.flink.runtime.memory.MemoryManager
-import org.apache.flink.runtime.taskmanager.{TaskManagerLocation, TaskManager, TaskManagerConfiguration}
-
-import scala.language.postfixOps
-
-/** Subclass of the [[TaskManager]] to support testing messages
- */
-class TestingTaskManager(
-                          config: TaskManagerConfiguration,
-                          resourceID: ResourceID,
-                          connectionInfo: TaskManagerLocation,
-                          memoryManager: MemoryManager,
-                          ioManager: IOManager,
-                          network: NetworkEnvironment,
-                          numberOfSlots: Int,
-                          leaderRetrievalService: LeaderRetrievalService)
-  extends TaskManager(
-    config,
-    resourceID,
-    connectionInfo,
-    memoryManager,
-    ioManager,
-    network,
-    numberOfSlots,
-    leaderRetrievalService)
-  with TestingTaskManagerLike {
-
-  def this(
-            config: TaskManagerConfiguration,
-            connectionInfo: TaskManagerLocation,
-            memoryManager: MemoryManager,
-            ioManager: IOManager,
-            network: NetworkEnvironment,
-            numberOfSlots: Int,
-            leaderRetrievalService: LeaderRetrievalService) {
-    this(
-      config,
-      ResourceID.generate(),
-      connectionInfo,
-      memoryManager,
-      ioManager,
-      network,
-      numberOfSlots,
-      leaderRetrievalService)
-  }
-}


[49/50] [abbrv] flink git commit: [hotfix] Add self rpc gateway registration to TestingSerialRpcService

Posted by tr...@apache.org.
[hotfix] Add self rpc gateway registration to TestingSerialRpcService


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

Branch: refs/heads/flip-6
Commit: 562b29375d6b70a321e93bd77a3f3f995eb13937
Parents: 002f650
Author: Till Rohrmann <tr...@apache.org>
Authored: Fri Sep 2 14:51:16 2016 +0200
Committer: Till Rohrmann <tr...@apache.org>
Committed: Thu Sep 8 17:27:01 2016 +0200

----------------------------------------------------------------------
 .../flink/runtime/rpc/TestingSerialRpcService.java  | 16 ++++++++++------
 1 file changed, 10 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/562b2937/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingSerialRpcService.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingSerialRpcService.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingSerialRpcService.java
index 7bdbb99..955edcc 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingSerialRpcService.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingSerialRpcService.java
@@ -43,7 +43,8 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
 
 
 /**
- * An RPC Service implementation for testing. This RPC service directly executes all asynchronous calls one by one in the main thread.
+ * An RPC Service implementation for testing. This RPC service directly executes all asynchronous
+ * calls one by one in the calling thread.
  */
 public class TestingSerialRpcService implements RpcService {
 
@@ -52,7 +53,7 @@ public class TestingSerialRpcService implements RpcService {
 
 	public TestingSerialRpcService() {
 		executorService = new DirectExecutorService();
-		this.registeredConnections = new ConcurrentHashMap<>();
+		this.registeredConnections = new ConcurrentHashMap<>(16);
 	}
 
 	@Override
@@ -78,14 +79,14 @@ public class TestingSerialRpcService implements RpcService {
 
 	@Override
 	public void stopServer(RpcGateway selfGateway) {
-
+		registeredConnections.remove(selfGateway.getAddress());
 	}
 
 	@Override
 	public <C extends RpcGateway, S extends RpcEndpoint<C>> C startServer(S rpcEndpoint) {
 		final String address = UUID.randomUUID().toString();
 
-		InvocationHandler akkaInvocationHandler = new TestingSerialInvocationHandler(address, rpcEndpoint);
+		InvocationHandler akkaInvocationHandler = new TestingSerialRpcService.TestingSerialInvocationHandler<>(address, rpcEndpoint);
 		ClassLoader classLoader = getClass().getClassLoader();
 
 		@SuppressWarnings("unchecked")
@@ -99,6 +100,9 @@ public class TestingSerialRpcService implements RpcService {
 			},
 			akkaInvocationHandler);
 
+		// register self
+		registeredConnections.putIfAbsent(self.getAddress(), self);
+
 		return self;
 	}
 
@@ -133,7 +137,7 @@ public class TestingSerialRpcService implements RpcService {
 		}
 	}
 
-	private static class TestingSerialInvocationHandler<C extends RpcGateway, T extends RpcEndpoint<C>> implements InvocationHandler, RpcGateway, MainThreadExecutor, StartStoppable {
+	private static final class TestingSerialInvocationHandler<C extends RpcGateway, T extends RpcEndpoint<C>> implements InvocationHandler, RpcGateway, MainThreadExecutor, StartStoppable {
 
 		private final T rpcEndpoint;
 
@@ -197,7 +201,7 @@ public class TestingSerialRpcService implements RpcService {
 			final Method rpcMethod = lookupRpcMethod(methodName, parameterTypes);
 			Object result = rpcMethod.invoke(rpcEndpoint, args);
 
-			if (result != null && result instanceof Future) {
+			if (result instanceof Future) {
 				Future<?> future = (Future<?>) result;
 				return Await.result(future, futureTimeout.duration());
 			} else {


[04/50] [abbrv] flink git commit: [FLINK-4570] remove conflicting Unicode character

Posted by tr...@apache.org.
[FLINK-4570] remove conflicting Unicode character

This caused Scalastyle to fail, presumably depending on the locale
used. After a bit of debugging on the Scalastyle plugin I found out that
the number in the error is the byte position.

"Expected identifier, but got Token(COMMA,,,1772,,)"

head -c 1772 flink-mesos/src/test/scala/org/apache/flink/mesos/Utils.scala

pointed to the Unicode character '\u21d2' which causes Scalastyle to fail in
certain environments.

This closes #2466


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

Branch: refs/heads/flip-6
Commit: 502a79d3945584b579db96adcaf916328cba0b76
Parents: e376c00
Author: Maximilian Michels <mx...@apache.org>
Authored: Fri Sep 2 17:52:51 2016 +0200
Committer: Maximilian Michels <mx...@apache.org>
Committed: Fri Sep 2 18:05:50 2016 +0200

----------------------------------------------------------------------
 flink-mesos/src/test/scala/org/apache/flink/mesos/Utils.scala | 7 ++-----
 1 file changed, 2 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/502a79d3/flink-mesos/src/test/scala/org/apache/flink/mesos/Utils.scala
----------------------------------------------------------------------
diff --git a/flink-mesos/src/test/scala/org/apache/flink/mesos/Utils.scala b/flink-mesos/src/test/scala/org/apache/flink/mesos/Utils.scala
index 6f27795..bd6235a 100644
--- a/flink-mesos/src/test/scala/org/apache/flink/mesos/Utils.scala
+++ b/flink-mesos/src/test/scala/org/apache/flink/mesos/Utils.scala
@@ -16,14 +16,12 @@
  * limitations under the License.
  */
 
-// disable Scalastyle for now to prevent random failures reported in FLINK-4570
-// scalastyle:off
 package org.apache.flink.mesos
 
 import java.util.concurrent.atomic.AtomicLong
 
 import akka.actor._
-import akka.testkit.{TestActorRef, TestFSMRef}
+import akka.testkit.TestFSMRef
 import org.mockito.ArgumentMatcher
 
 import scala.collection.JavaConverters._
@@ -49,9 +47,8 @@ object TestFSMUtils {
     "$" + akka.util.Helpers.base64(l)
   }
 
-  def testFSMRef[S, D, T <: Actor: ClassTag](factory: \u21d2 T, supervisor: ActorRef)
+  def testFSMRef[S, D, T <: Actor: ClassTag](factory: => T, supervisor: ActorRef)
       (implicit ev: T <:< FSM[S, D], system: ActorSystem): TestFSMRef[S, D, T] = {
     new TestFSMRef(system, Props(factory), supervisor, TestFSMUtils.randomName)
   }
 }
-// scalastyle:on


[46/50] [abbrv] flink git commit: [FLINK-4529] [flip-6] Move TaskExecutor, JobMaster and ResourceManager out of the rpc package

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/72468d14/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java
deleted file mode 100644
index 36d6310..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java
+++ /dev/null
@@ -1,827 +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.rpc.taskexecutor;
-
-import akka.actor.ActorSystem;
-import akka.dispatch.ExecutionContexts$;
-import akka.util.Timeout;
-import com.typesafe.config.Config;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.configuration.ConfigConstants;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.configuration.IllegalConfigurationException;
-import org.apache.flink.core.memory.HeapMemorySegment;
-import org.apache.flink.core.memory.HybridMemorySegment;
-import org.apache.flink.core.memory.MemorySegmentFactory;
-import org.apache.flink.core.memory.MemoryType;
-import org.apache.flink.runtime.akka.AkkaUtils;
-import org.apache.flink.annotation.VisibleForTesting;
-import org.apache.flink.runtime.clusterframework.types.ResourceID;
-import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
-import org.apache.flink.runtime.instance.InstanceConnectionInfo;
-import org.apache.flink.runtime.io.disk.iomanager.IOManager;
-import org.apache.flink.runtime.io.disk.iomanager.IOManager.IOMode;
-import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
-import org.apache.flink.runtime.io.network.NetworkEnvironment;
-import org.apache.flink.runtime.io.network.netty.NettyConfig;
-import org.apache.flink.runtime.leaderelection.LeaderElectionService;
-import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
-import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
-import org.apache.flink.runtime.memory.MemoryManager;
-import org.apache.flink.runtime.rpc.RpcEndpoint;
-import org.apache.flink.runtime.rpc.RpcMethod;
-import org.apache.flink.runtime.rpc.RpcService;
-import org.apache.flink.runtime.rpc.akka.AkkaRpcService;
-import org.apache.flink.runtime.taskmanager.MemoryLogger;
-import org.apache.flink.runtime.util.EnvironmentInformation;
-import org.apache.flink.runtime.util.LeaderRetrievalUtils;
-import org.apache.flink.runtime.taskmanager.NetworkEnvironmentConfiguration;
-import org.apache.flink.util.MathUtils;
-import org.apache.flink.util.NetUtils;
-
-import scala.Tuple2;
-import scala.Option;
-import scala.Some;
-import scala.concurrent.ExecutionContext;
-import scala.concurrent.duration.Duration;
-import scala.concurrent.duration.FiniteDuration;
-
-import java.io.File;
-import java.io.IOException;
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
-import java.util.UUID;
-import java.util.concurrent.ForkJoinPool;
-import java.util.concurrent.TimeUnit;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/**
- * TaskExecutor implementation. The task executor is responsible for the execution of multiple
- * {@link org.apache.flink.runtime.taskmanager.Task}.
- */
-public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
-
-	private static final Logger LOG = LoggerFactory.getLogger(TaskExecutor.class);
-
-	/** The unique resource ID of this TaskExecutor */
-	private final ResourceID resourceID;
-
-	/** The access to the leader election and metadata storage services */
-	private final HighAvailabilityServices haServices;
-
-	/** The task manager configuration */
-	private final TaskExecutorConfiguration taskExecutorConfig;
-
-	/** The I/O manager component in the task manager */
-	private final IOManager ioManager;
-
-	/** The memory manager component in the task manager */
-	private final MemoryManager memoryManager;
-
-	/** The network component in the task manager */
-	private final NetworkEnvironment networkEnvironment;
-
-	/** The number of slots in the task manager, should be 1 for YARN */
-	private final int numberOfSlots;
-
-	// --------- resource manager --------
-
-	private TaskExecutorToResourceManagerConnection resourceManagerConnection;
-
-	// ------------------------------------------------------------------------
-
-	public TaskExecutor(
-			TaskExecutorConfiguration taskExecutorConfig,
-			ResourceID resourceID,
-			MemoryManager memoryManager,
-			IOManager ioManager,
-			NetworkEnvironment networkEnvironment,
-			int numberOfSlots,
-			RpcService rpcService,
-			HighAvailabilityServices haServices) {
-
-		super(rpcService);
-
-		this.taskExecutorConfig = checkNotNull(taskExecutorConfig);
-		this.resourceID = checkNotNull(resourceID);
-		this.memoryManager = checkNotNull(memoryManager);
-		this.ioManager = checkNotNull(ioManager);
-		this.networkEnvironment = checkNotNull(networkEnvironment);
-		this.numberOfSlots = checkNotNull(numberOfSlots);
-		this.haServices = checkNotNull(haServices);
-	}
-
-	// ------------------------------------------------------------------------
-	//  Life cycle
-	// ------------------------------------------------------------------------
-
-	@Override
-	public void start() {
-		super.start();
-
-		// start by connecting to the ResourceManager
-		try {
-			haServices.getResourceManagerLeaderRetriever().start(new ResourceManagerLeaderListener());
-		} catch (Exception e) {
-			onFatalErrorAsync(e);
-		}
-	}
-
-	// ------------------------------------------------------------------------
-	//  RPC methods - ResourceManager related
-	// ------------------------------------------------------------------------
-
-	@RpcMethod
-	public void notifyOfNewResourceManagerLeader(String newLeaderAddress, UUID newLeaderId) {
-		if (resourceManagerConnection != null) {
-			if (newLeaderAddress != null) {
-				// the resource manager switched to a new leader
-				log.info("ResourceManager leader changed from {} to {}. Registering at new leader.",
-					resourceManagerConnection.getResourceManagerAddress(), newLeaderAddress);
-			}
-			else {
-				// address null means that the current leader is lost without a new leader being there, yet
-				log.info("Current ResourceManager {} lost leader status. Waiting for new ResourceManager leader.",
-					resourceManagerConnection.getResourceManagerAddress());
-			}
-
-			// drop the current connection or connection attempt
-			if (resourceManagerConnection != null) {
-				resourceManagerConnection.close();
-				resourceManagerConnection = null;
-			}
-		}
-
-		// establish a connection to the new leader
-		if (newLeaderAddress != null) {
-			log.info("Attempting to register at ResourceManager {}", newLeaderAddress);
-			resourceManagerConnection =
-				new TaskExecutorToResourceManagerConnection(log, this, newLeaderAddress, newLeaderId);
-			resourceManagerConnection.start();
-		}
-	}
-
-	/**
-	 * Starts and runs the TaskManager.
-	 * <p/>
-	 * This method first tries to select the network interface to use for the TaskManager
-	 * communication. The network interface is used both for the actor communication
-	 * (coordination) as well as for the data exchange between task managers. Unless
-	 * the hostname/interface is explicitly configured in the configuration, this
-	 * method will try out various interfaces and methods to connect to the JobManager
-	 * and select the one where the connection attempt is successful.
-	 * <p/>
-	 * After selecting the network interface, this method brings up an actor system
-	 * for the TaskManager and its actors, starts the TaskManager's services
-	 * (library cache, shuffle network stack, ...), and starts the TaskManager itself.
-	 *
-	 * @param configuration    The configuration for the TaskManager.
-	 * @param resourceID       The id of the resource which the task manager will run on.
-	 */
-	public static void selectNetworkInterfaceAndRunTaskManager(
-		Configuration configuration,
-		ResourceID resourceID) throws Exception {
-
-		final InetSocketAddress taskManagerAddress = selectNetworkInterfaceAndPort(configuration);
-
-		runTaskManager(taskManagerAddress.getHostName(), resourceID, taskManagerAddress.getPort(), configuration);
-	}
-
-	private static InetSocketAddress selectNetworkInterfaceAndPort(Configuration configuration)
-		throws Exception {
-		String taskManagerHostname = configuration.getString(ConfigConstants.TASK_MANAGER_HOSTNAME_KEY, null);
-		if (taskManagerHostname != null) {
-			LOG.info("Using configured hostname/address for TaskManager: " + taskManagerHostname);
-		} else {
-			LeaderRetrievalService leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
-			FiniteDuration lookupTimeout = AkkaUtils.getLookupTimeout(configuration);
-
-			InetAddress taskManagerAddress = LeaderRetrievalUtils.findConnectingAddress(leaderRetrievalService, lookupTimeout);
-			taskManagerHostname = taskManagerAddress.getHostName();
-			LOG.info("TaskManager will use hostname/address '{}' ({}) for communication.",
-				taskManagerHostname, taskManagerAddress.getHostAddress());
-		}
-
-		// if no task manager port has been configured, use 0 (system will pick any free port)
-		final int actorSystemPort = configuration.getInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, 0);
-		if (actorSystemPort < 0 || actorSystemPort > 65535) {
-			throw new IllegalConfigurationException("Invalid value for '" +
-				ConfigConstants.TASK_MANAGER_IPC_PORT_KEY +
-				"' (port for the TaskManager actor system) : " + actorSystemPort +
-				" - Leave config parameter empty or use 0 to let the system choose a port automatically.");
-		}
-
-		return new InetSocketAddress(taskManagerHostname, actorSystemPort);
-	}
-
-	/**
-	 * Starts and runs the TaskManager. Brings up an actor system for the TaskManager and its
-	 * actors, starts the TaskManager's services (library cache, shuffle network stack, ...),
-	 * and starts the TaskManager itself.
-	 * <p/>
-	 * This method will also spawn a process reaper for the TaskManager (kill the process if
-	 * the actor fails) and optionally start the JVM memory logging thread.
-	 *
-	 * @param taskManagerHostname The hostname/address of the interface where the actor system
-	 *                            will communicate.
-	 * @param resourceID          The id of the resource which the task manager will run on.
-	 * @param actorSystemPort   The port at which the actor system will communicate.
-	 * @param configuration       The configuration for the TaskManager.
-	 */
-	private static void runTaskManager(
-		String taskManagerHostname,
-		ResourceID resourceID,
-		int actorSystemPort,
-		final Configuration configuration) throws Exception {
-
-		LOG.info("Starting TaskManager");
-
-		// Bring up the TaskManager actor system first, bind it to the given address.
-
-		LOG.info("Starting TaskManager actor system at " +
-			NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort));
-
-		final ActorSystem taskManagerSystem;
-		try {
-			Tuple2<String, Object> address = new Tuple2<String, Object>(taskManagerHostname, actorSystemPort);
-			Config akkaConfig = AkkaUtils.getAkkaConfig(configuration, new Some<>(address));
-			LOG.debug("Using akka configuration\n " + akkaConfig);
-			taskManagerSystem = AkkaUtils.createActorSystem(akkaConfig);
-		} catch (Throwable t) {
-			if (t instanceof org.jboss.netty.channel.ChannelException) {
-				Throwable cause = t.getCause();
-				if (cause != null && t.getCause() instanceof java.net.BindException) {
-					String address = NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort);
-					throw new IOException("Unable to bind TaskManager actor system to address " +
-						address + " - " + cause.getMessage(), t);
-				}
-			}
-			throw new Exception("Could not create TaskManager actor system", t);
-		}
-
-		// start akka rpc service based on actor system
-		final Timeout timeout = new Timeout(AkkaUtils.getTimeout(configuration).toMillis(), TimeUnit.MILLISECONDS);
-		final AkkaRpcService akkaRpcService = new AkkaRpcService(taskManagerSystem, timeout);
-
-		// start high availability service to implement getResourceManagerLeaderRetriever method only
-		final HighAvailabilityServices haServices = new HighAvailabilityServices() {
-			@Override
-			public LeaderRetrievalService getResourceManagerLeaderRetriever() throws Exception {
-				return LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
-			}
-
-			@Override
-			public LeaderElectionService getResourceManagerLeaderElectionService() throws Exception {
-				return null;
-			}
-
-			@Override
-			public LeaderElectionService getJobMasterLeaderElectionService(JobID jobID) throws Exception {
-				return null;
-			}
-		};
-
-		// start all the TaskManager services (network stack,  library cache, ...)
-		// and the TaskManager actor
-		try {
-			LOG.info("Starting TaskManager actor");
-			TaskExecutor taskExecutor = startTaskManagerComponentsAndActor(
-				configuration,
-				resourceID,
-				akkaRpcService,
-				taskManagerHostname,
-				haServices,
-				false);
-
-			taskExecutor.start();
-
-			// if desired, start the logging daemon that periodically logs the memory usage information
-			if (LOG.isInfoEnabled() && configuration.getBoolean(
-				ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD,
-				ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD)) {
-				LOG.info("Starting periodic memory usage logger");
-
-				long interval = configuration.getLong(
-					ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS,
-					ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS);
-
-				MemoryLogger logger = new MemoryLogger(LOG, interval, taskManagerSystem);
-				logger.start();
-			}
-
-			// block until everything is done
-			taskManagerSystem.awaitTermination();
-		} catch (Throwable t) {
-			LOG.error("Error while starting up taskManager", t);
-			try {
-				taskManagerSystem.shutdown();
-			} catch (Throwable tt) {
-				LOG.warn("Could not cleanly shut down actor system", tt);
-			}
-			throw t;
-		}
-	}
-
-	// --------------------------------------------------------------------------
-	//  Starting and running the TaskManager
-	// --------------------------------------------------------------------------
-
-	/**
-	 * @param configuration                 The configuration for the TaskManager.
-	 * @param resourceID                    The id of the resource which the task manager will run on.
-	 * @param rpcService                  The rpc service which is used to start and connect to the TaskManager RpcEndpoint .
-	 * @param taskManagerHostname       The hostname/address that describes the TaskManager's data location.
-	 * @param haServices        Optionally, a high availability service can be provided. If none is given,
-	 *                                      then a HighAvailabilityServices is constructed from the configuration.
-	 * @param localTaskManagerCommunication     If true, the TaskManager will not initiate the TCP network stack.
-	 * @return An ActorRef to the TaskManager actor.
-	 * @throws org.apache.flink.configuration.IllegalConfigurationException     Thrown, if the given config contains illegal values.
-	 * @throws java.io.IOException      Thrown, if any of the I/O components (such as buffer pools,
-	 *                                       I/O manager, ...) cannot be properly started.
-	 * @throws java.lang.Exception      Thrown is some other error occurs while parsing the configuration
-	 *                                      or starting the TaskManager components.
-	 */
-	public static TaskExecutor startTaskManagerComponentsAndActor(
-		Configuration configuration,
-		ResourceID resourceID,
-		RpcService rpcService,
-		String taskManagerHostname,
-		HighAvailabilityServices haServices,
-		boolean localTaskManagerCommunication) throws Exception {
-
-		final TaskExecutorConfiguration taskExecutorConfig = parseTaskManagerConfiguration(
-			configuration, taskManagerHostname, localTaskManagerCommunication);
-
-		MemoryType memType = taskExecutorConfig.getNetworkConfig().memoryType();
-
-		// pre-start checks
-		checkTempDirs(taskExecutorConfig.getTmpDirPaths());
-
-		ExecutionContext executionContext = ExecutionContexts$.MODULE$.fromExecutor(new ForkJoinPool());
-
-		// we start the network first, to make sure it can allocate its buffers first
-		final NetworkEnvironment network = new NetworkEnvironment(
-			executionContext,
-			taskExecutorConfig.getTimeout(),
-			taskExecutorConfig.getNetworkConfig(),
-			taskExecutorConfig.getConnectionInfo());
-
-		// computing the amount of memory to use depends on how much memory is available
-		// it strictly needs to happen AFTER the network stack has been initialized
-
-		// check if a value has been configured
-		long configuredMemory = configuration.getLong(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, -1L);
-		checkConfigParameter(configuredMemory == -1 || configuredMemory > 0, configuredMemory,
-			ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY,
-			"MemoryManager needs at least one MB of memory. " +
-				"If you leave this config parameter empty, the system automatically " +
-				"pick a fraction of the available memory.");
-
-		final long memorySize;
-		boolean preAllocateMemory = configuration.getBoolean(
-			ConfigConstants.TASK_MANAGER_MEMORY_PRE_ALLOCATE_KEY,
-			ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_PRE_ALLOCATE);
-		if (configuredMemory > 0) {
-			if (preAllocateMemory) {
-				LOG.info("Using {} MB for managed memory." , configuredMemory);
-			} else {
-				LOG.info("Limiting managed memory to {} MB, memory will be allocated lazily." , configuredMemory);
-			}
-			memorySize = configuredMemory << 20; // megabytes to bytes
-		} else {
-			float fraction = configuration.getFloat(
-				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
-				ConfigConstants.DEFAULT_MEMORY_MANAGER_MEMORY_FRACTION);
-			checkConfigParameter(fraction > 0.0f && fraction < 1.0f, fraction,
-				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
-				"MemoryManager fraction of the free memory must be between 0.0 and 1.0");
-
-			if (memType == MemoryType.HEAP) {
-				long relativeMemSize = (long) (EnvironmentInformation.getSizeOfFreeHeapMemoryWithDefrag() * fraction);
-				if (preAllocateMemory) {
-					LOG.info("Using {} of the currently free heap space for managed heap memory ({} MB)." ,
-						fraction , relativeMemSize >> 20);
-				} else {
-					LOG.info("Limiting managed memory to {} of the currently free heap space ({} MB), " +
-						"memory will be allocated lazily." , fraction , relativeMemSize >> 20);
-				}
-				memorySize = relativeMemSize;
-			} else if (memType == MemoryType.OFF_HEAP) {
-				// The maximum heap memory has been adjusted according to the fraction
-				long maxMemory = EnvironmentInformation.getMaxJvmHeapMemory();
-				long directMemorySize = (long) (maxMemory / (1.0 - fraction) * fraction);
-				if (preAllocateMemory) {
-					LOG.info("Using {} of the maximum memory size for managed off-heap memory ({} MB)." ,
-						fraction, directMemorySize >> 20);
-				} else {
-					LOG.info("Limiting managed memory to {} of the maximum memory size ({} MB)," +
-						" memory will be allocated lazily.", fraction, directMemorySize >> 20);
-				}
-				memorySize = directMemorySize;
-			} else {
-				throw new RuntimeException("No supported memory type detected.");
-			}
-		}
-
-		// now start the memory manager
-		final MemoryManager memoryManager;
-		try {
-			memoryManager = new MemoryManager(
-				memorySize,
-				taskExecutorConfig.getNumberOfSlots(),
-				taskExecutorConfig.getNetworkConfig().networkBufferSize(),
-				memType,
-				preAllocateMemory);
-		} catch (OutOfMemoryError e) {
-			if (memType == MemoryType.HEAP) {
-				throw new Exception("OutOfMemory error (" + e.getMessage() +
-					") while allocating the TaskManager heap memory (" + memorySize + " bytes).", e);
-			} else if (memType == MemoryType.OFF_HEAP) {
-				throw new Exception("OutOfMemory error (" + e.getMessage() +
-					") while allocating the TaskManager off-heap memory (" + memorySize +
-					" bytes).Try increasing the maximum direct memory (-XX:MaxDirectMemorySize)", e);
-			} else {
-				throw e;
-			}
-		}
-
-		// start the I/O manager, it will create some temp directories.
-		final IOManager ioManager = new IOManagerAsync(taskExecutorConfig.getTmpDirPaths());
-
-		final TaskExecutor taskExecutor = new TaskExecutor(
-			taskExecutorConfig,
-			resourceID,
-			memoryManager,
-			ioManager,
-			network,
-			taskExecutorConfig.getNumberOfSlots(),
-			rpcService,
-			haServices);
-
-		return taskExecutor;
-	}
-
-	// --------------------------------------------------------------------------
-	//  Parsing and checking the TaskManager Configuration
-	// --------------------------------------------------------------------------
-
-	/**
-	 * Utility method to extract TaskManager config parameters from the configuration and to
-	 * sanity check them.
-	 *
-	 * @param configuration                 The configuration.
-	 * @param taskManagerHostname           The host name under which the TaskManager communicates.
-	 * @param localTaskManagerCommunication             True, to skip initializing the network stack.
-	 *                                      Use only in cases where only one task manager runs.
-	 * @return TaskExecutorConfiguration that wrappers InstanceConnectionInfo, NetworkEnvironmentConfiguration, etc.
-	 */
-	private static TaskExecutorConfiguration parseTaskManagerConfiguration(
-		Configuration configuration,
-		String taskManagerHostname,
-		boolean localTaskManagerCommunication) throws Exception {
-
-		// ------- read values from the config and check them ---------
-		//                      (a lot of them)
-
-		// ----> hosts / ports for communication and data exchange
-
-		int dataport = configuration.getInteger(ConfigConstants.TASK_MANAGER_DATA_PORT_KEY,
-			ConfigConstants.DEFAULT_TASK_MANAGER_DATA_PORT);
-		if (dataport == 0) {
-			dataport = NetUtils.getAvailablePort();
-		}
-		checkConfigParameter(dataport > 0, dataport, ConfigConstants.TASK_MANAGER_DATA_PORT_KEY,
-			"Leave config parameter empty or use 0 to let the system choose a port automatically.");
-
-		InetAddress taskManagerAddress = InetAddress.getByName(taskManagerHostname);
-		final InstanceConnectionInfo connectionInfo = new InstanceConnectionInfo(taskManagerAddress, dataport);
-
-		// ----> memory / network stack (shuffles/broadcasts), task slots, temp directories
-
-		// we need this because many configs have been written with a "-1" entry
-		int slots = configuration.getInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 1);
-		if (slots == -1) {
-			slots = 1;
-		}
-		checkConfigParameter(slots >= 1, slots, ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS,
-			"Number of task slots must be at least one.");
-
-		final int numNetworkBuffers = configuration.getInteger(
-			ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY,
-			ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_NUM_BUFFERS);
-		checkConfigParameter(numNetworkBuffers > 0, numNetworkBuffers,
-			ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY, "");
-
-		final int pageSize = configuration.getInteger(
-			ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
-			ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_SEGMENT_SIZE);
-		// check page size of for minimum size
-		checkConfigParameter(pageSize >= MemoryManager.MIN_PAGE_SIZE, pageSize,
-			ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
-			"Minimum memory segment size is " + MemoryManager.MIN_PAGE_SIZE);
-		// check page size for power of two
-		checkConfigParameter(MathUtils.isPowerOf2(pageSize), pageSize,
-			ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
-			"Memory segment size must be a power of 2.");
-
-		// check whether we use heap or off-heap memory
-		final MemoryType memType;
-		if (configuration.getBoolean(ConfigConstants.TASK_MANAGER_MEMORY_OFF_HEAP_KEY, false)) {
-			memType = MemoryType.OFF_HEAP;
-		} else {
-			memType = MemoryType.HEAP;
-		}
-
-		// initialize the memory segment factory accordingly
-		if (memType == MemoryType.HEAP) {
-			if (!MemorySegmentFactory.initializeIfNotInitialized(HeapMemorySegment.FACTORY)) {
-				throw new Exception("Memory type is set to heap memory, but memory segment " +
-					"factory has been initialized for off-heap memory segments");
-			}
-		} else {
-			if (!MemorySegmentFactory.initializeIfNotInitialized(HybridMemorySegment.FACTORY)) {
-				throw new Exception("Memory type is set to off-heap memory, but memory segment " +
-					"factory has been initialized for heap memory segments");
-			}
-		}
-
-		final String[] tmpDirs = configuration.getString(
-			ConfigConstants.TASK_MANAGER_TMP_DIR_KEY,
-			ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH).split(",|" + File.pathSeparator);
-
-		final NettyConfig nettyConfig;
-		if (!localTaskManagerCommunication) {
-			nettyConfig = new NettyConfig(connectionInfo.address(), connectionInfo.dataPort(), pageSize, slots, configuration);
-		} else {
-			nettyConfig = null;
-		}
-
-		// Default spill I/O mode for intermediate results
-		final String syncOrAsync = configuration.getString(
-			ConfigConstants.TASK_MANAGER_NETWORK_DEFAULT_IO_MODE,
-			ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_DEFAULT_IO_MODE);
-
-		final IOMode ioMode;
-		if (syncOrAsync.equals("async")) {
-			ioMode = IOManager.IOMode.ASYNC;
-		} else {
-			ioMode = IOManager.IOMode.SYNC;
-		}
-
-		final int queryServerPort =  configuration.getInteger(
-			ConfigConstants.QUERYABLE_STATE_SERVER_PORT,
-			ConfigConstants.DEFAULT_QUERYABLE_STATE_SERVER_PORT);
-
-		final int queryServerNetworkThreads =  configuration.getInteger(
-			ConfigConstants.QUERYABLE_STATE_SERVER_NETWORK_THREADS,
-			ConfigConstants.DEFAULT_QUERYABLE_STATE_SERVER_NETWORK_THREADS);
-
-		final int queryServerQueryThreads =  configuration.getInteger(
-			ConfigConstants.QUERYABLE_STATE_SERVER_QUERY_THREADS,
-			ConfigConstants.DEFAULT_QUERYABLE_STATE_SERVER_QUERY_THREADS);
-
-		final NetworkEnvironmentConfiguration networkConfig = new NetworkEnvironmentConfiguration(
-			numNetworkBuffers,
-			pageSize,
-			memType,
-			ioMode,
-			queryServerPort,
-			queryServerNetworkThreads,
-			queryServerQueryThreads,
-			localTaskManagerCommunication ? Option.<NettyConfig>empty() : new Some<>(nettyConfig),
-			new Tuple2<>(500, 3000));
-
-		// ----> timeouts, library caching, profiling
-
-		final FiniteDuration timeout;
-		try {
-			timeout = AkkaUtils.getTimeout(configuration);
-		} catch (Exception e) {
-			throw new IllegalArgumentException(
-				"Invalid format for '" + ConfigConstants.AKKA_ASK_TIMEOUT +
-					"'.Use formats like '50 s' or '1 min' to specify the timeout.");
-		}
-		LOG.info("Messages between TaskManager and JobManager have a max timeout of " + timeout);
-
-		final long cleanupInterval = configuration.getLong(
-			ConfigConstants.LIBRARY_CACHE_MANAGER_CLEANUP_INTERVAL,
-			ConfigConstants.DEFAULT_LIBRARY_CACHE_MANAGER_CLEANUP_INTERVAL) * 1000;
-
-		final FiniteDuration finiteRegistrationDuration;
-		try {
-			Duration maxRegistrationDuration = Duration.create(configuration.getString(
-				ConfigConstants.TASK_MANAGER_MAX_REGISTRATION_DURATION,
-				ConfigConstants.DEFAULT_TASK_MANAGER_MAX_REGISTRATION_DURATION));
-			if (maxRegistrationDuration.isFinite()) {
-				finiteRegistrationDuration = new FiniteDuration(maxRegistrationDuration.toSeconds(), TimeUnit.SECONDS);
-			} else {
-				finiteRegistrationDuration = null;
-			}
-		} catch (NumberFormatException e) {
-			throw new IllegalArgumentException("Invalid format for parameter " +
-				ConfigConstants.TASK_MANAGER_MAX_REGISTRATION_DURATION, e);
-		}
-
-		final FiniteDuration initialRegistrationPause;
-		try {
-			Duration pause = Duration.create(configuration.getString(
-				ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE,
-				ConfigConstants.DEFAULT_TASK_MANAGER_INITIAL_REGISTRATION_PAUSE));
-			if (pause.isFinite()) {
-				initialRegistrationPause = new FiniteDuration(pause.toSeconds(), TimeUnit.SECONDS);
-			} else {
-				throw new IllegalArgumentException("The initial registration pause must be finite: " + pause);
-			}
-		} catch (NumberFormatException e) {
-			throw new IllegalArgumentException("Invalid format for parameter " +
-				ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE, e);
-		}
-
-		final FiniteDuration maxRegistrationPause;
-		try {
-			Duration pause = Duration.create(configuration.getString(
-				ConfigConstants.TASK_MANAGER_MAX_REGISTARTION_PAUSE,
-				ConfigConstants.DEFAULT_TASK_MANAGER_MAX_REGISTRATION_PAUSE));
-			if (pause.isFinite()) {
-				maxRegistrationPause = new FiniteDuration(pause.toSeconds(), TimeUnit.SECONDS);
-			} else {
-				throw new IllegalArgumentException("The maximum registration pause must be finite: " + pause);
-			}
-		} catch (NumberFormatException e) {
-			throw new IllegalArgumentException("Invalid format for parameter " +
-				ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE, e);
-		}
-
-		final FiniteDuration refusedRegistrationPause;
-		try {
-			Duration pause = Duration.create(configuration.getString(
-				ConfigConstants.TASK_MANAGER_REFUSED_REGISTRATION_PAUSE,
-				ConfigConstants.DEFAULT_TASK_MANAGER_REFUSED_REGISTRATION_PAUSE));
-			if (pause.isFinite()) {
-				refusedRegistrationPause = new FiniteDuration(pause.toSeconds(), TimeUnit.SECONDS);
-			} else {
-				throw new IllegalArgumentException("The refused registration pause must be finite: " + pause);
-			}
-		} catch (NumberFormatException e) {
-			throw new IllegalArgumentException("Invalid format for parameter " +
-				ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE, e);
-		}
-
-		return new TaskExecutorConfiguration(
-			tmpDirs,
-			cleanupInterval,
-			connectionInfo,
-			networkConfig,
-			timeout,
-			finiteRegistrationDuration,
-			slots,
-			configuration,
-			initialRegistrationPause,
-			maxRegistrationPause,
-			refusedRegistrationPause);
-	}
-
-	/**
-	 * Validates a condition for a config parameter and displays a standard exception, if the
-	 * the condition does not hold.
-	 *
-	 * @param condition    The condition that must hold. If the condition is false, an exception is thrown.
-	 * @param parameter    The parameter value. Will be shown in the exception message.
-	 * @param name         The name of the config parameter. Will be shown in the exception message.
-	 * @param errorMessage The optional custom error message to append to the exception message.
-	 */
-	private static void checkConfigParameter(
-		boolean condition,
-		Object parameter,
-		String name,
-		String errorMessage) {
-		if (!condition) {
-			throw new IllegalConfigurationException("Invalid configuration value for " + name + " : " + parameter + " - " + errorMessage);
-		}
-	}
-
-	/**
-	 * Validates that all the directories denoted by the strings do actually exist, are proper
-	 * directories (not files), and are writable.
-	 *
-	 * @param tmpDirs The array of directory paths to check.
-	 * @throws Exception Thrown if any of the directories does not exist or is not writable
-	 *                   or is a file, rather than a directory.
-	 */
-	private static void checkTempDirs(String[] tmpDirs) throws IOException {
-		for (String dir : tmpDirs) {
-			if (dir != null && !dir.equals("")) {
-				File file = new File(dir);
-				if (!file.exists()) {
-					throw new IOException("Temporary file directory " + file.getAbsolutePath() + " does not exist.");
-				}
-				if (!file.isDirectory()) {
-					throw new IOException("Temporary file directory " + file.getAbsolutePath() + " is not a directory.");
-				}
-				if (!file.canWrite()) {
-					throw new IOException("Temporary file directory " + file.getAbsolutePath() + " is not writable.");
-				}
-
-				if (LOG.isInfoEnabled()) {
-					long totalSpaceGb = file.getTotalSpace() >> 30;
-					long usableSpaceGb = file.getUsableSpace() >> 30;
-					double usablePercentage = (double)usableSpaceGb / totalSpaceGb * 100;
-					String path = file.getAbsolutePath();
-					LOG.info(String.format("Temporary file directory '%s': total %d GB, " + "usable %d GB (%.2f%% usable)",
-						path, totalSpaceGb, usableSpaceGb, usablePercentage));
-				}
-			} else {
-				throw new IllegalArgumentException("Temporary file directory #$id is null.");
-			}
-		}
-	}
-
-	// ------------------------------------------------------------------------
-	//  Properties
-	// ------------------------------------------------------------------------
-
-	public ResourceID getResourceID() {
-		return resourceID;
-	}
-
-	// ------------------------------------------------------------------------
-	//  Error Handling
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Notifies the TaskExecutor that a fatal error has occurred and it cannot proceed.
-	 * This method should be used when asynchronous threads want to notify the
-	 * TaskExecutor of a fatal error.
-	 *
-	 * @param t The exception describing the fatal error
-	 */
-	void onFatalErrorAsync(final Throwable t) {
-		runAsync(new Runnable() {
-			@Override
-			public void run() {
-				onFatalError(t);
-			}
-		});
-	}
-
-	/**
-	 * Notifies the TaskExecutor that a fatal error has occurred and it cannot proceed.
-	 * This method must only be called from within the TaskExecutor's main thread.
-	 *
-	 * @param t The exception describing the fatal error
-	 */
-	void onFatalError(Throwable t) {
-		// to be determined, probably delegate to a fatal error handler that 
-		// would either log (mini cluster) ot kill the process (yarn, mesos, ...)
-		log.error("FATAL ERROR", t);
-	}
-
-	// ------------------------------------------------------------------------
-	//  Access to fields for testing
-	// ------------------------------------------------------------------------
-
-	@VisibleForTesting
-	TaskExecutorToResourceManagerConnection getResourceManagerConnection() {
-		return resourceManagerConnection;
-	}
-
-	// ------------------------------------------------------------------------
-	//  Utility classes
-	// ------------------------------------------------------------------------
-
-	/**
-	 * The listener for leader changes of the resource manager
-	 */
-	private class ResourceManagerLeaderListener implements LeaderRetrievalListener {
-
-		@Override
-		public void notifyLeaderAddress(String leaderAddress, UUID leaderSessionID) {
-			getSelf().notifyOfNewResourceManagerLeader(leaderAddress, leaderSessionID);
-		}
-
-		@Override
-		public void handleError(Exception exception) {
-			onFatalErrorAsync(exception);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/72468d14/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorConfiguration.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorConfiguration.java
deleted file mode 100644
index 32484e1..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorConfiguration.java
+++ /dev/null
@@ -1,151 +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.rpc.taskexecutor;
-
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.instance.InstanceConnectionInfo;
-import org.apache.flink.runtime.taskmanager.NetworkEnvironmentConfiguration;
-
-import scala.concurrent.duration.FiniteDuration;
-
-import java.io.Serializable;
-import java.util.concurrent.TimeUnit;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/**
- * {@link TaskExecutor} Configuration
- */
-public class TaskExecutorConfiguration implements Serializable {
-
-	private static final long serialVersionUID = 1L;
-
-	private final String[] tmpDirPaths;
-
-	private final long cleanupInterval;
-
-	private final int numberOfSlots;
-
-	private final Configuration configuration;
-
-	private final FiniteDuration timeout;
-	private final FiniteDuration maxRegistrationDuration;
-	private final FiniteDuration initialRegistrationPause;
-	private final FiniteDuration maxRegistrationPause;
-	private final FiniteDuration refusedRegistrationPause;
-
-	private final NetworkEnvironmentConfiguration networkConfig;
-
-	private final InstanceConnectionInfo connectionInfo;
-
-	public TaskExecutorConfiguration(
-			String[] tmpDirPaths,
-			long cleanupInterval,
-			InstanceConnectionInfo connectionInfo,
-			NetworkEnvironmentConfiguration networkConfig,
-			FiniteDuration timeout,
-			FiniteDuration maxRegistrationDuration,
-			int numberOfSlots,
-			Configuration configuration) {
-
-		this (tmpDirPaths,
-			cleanupInterval,
-			connectionInfo,
-			networkConfig,
-			timeout,
-			maxRegistrationDuration,
-			numberOfSlots,
-			configuration,
-			new FiniteDuration(500, TimeUnit.MILLISECONDS),
-			new FiniteDuration(30, TimeUnit.SECONDS),
-			new FiniteDuration(10, TimeUnit.SECONDS));
-	}
-
-	public TaskExecutorConfiguration(
-			String[] tmpDirPaths,
-			long cleanupInterval,
-			InstanceConnectionInfo connectionInfo,
-			NetworkEnvironmentConfiguration networkConfig,
-			FiniteDuration timeout,
-			FiniteDuration maxRegistrationDuration,
-			int numberOfSlots,
-			Configuration configuration,
-			FiniteDuration initialRegistrationPause,
-			FiniteDuration maxRegistrationPause,
-			FiniteDuration refusedRegistrationPause) {
-
-		this.tmpDirPaths = checkNotNull(tmpDirPaths);
-		this.cleanupInterval = checkNotNull(cleanupInterval);
-		this.connectionInfo = checkNotNull(connectionInfo);
-		this.networkConfig = checkNotNull(networkConfig);
-		this.timeout = checkNotNull(timeout);
-		this.maxRegistrationDuration = maxRegistrationDuration;
-		this.numberOfSlots = checkNotNull(numberOfSlots);
-		this.configuration = checkNotNull(configuration);
-		this.initialRegistrationPause = checkNotNull(initialRegistrationPause);
-		this.maxRegistrationPause = checkNotNull(maxRegistrationPause);
-		this.refusedRegistrationPause = checkNotNull(refusedRegistrationPause);
-	}
-
-	// --------------------------------------------------------------------------------------------
-	//  Properties
-	// --------------------------------------------------------------------------------------------
-
-	public String[] getTmpDirPaths() {
-		return tmpDirPaths;
-	}
-
-	public long getCleanupInterval() {
-		return cleanupInterval;
-	}
-
-	public InstanceConnectionInfo getConnectionInfo() { return connectionInfo; }
-
-	public NetworkEnvironmentConfiguration getNetworkConfig() { return networkConfig; }
-
-	public FiniteDuration getTimeout() {
-		return timeout;
-	}
-
-	public FiniteDuration getMaxRegistrationDuration() {
-		return maxRegistrationDuration;
-	}
-
-	public int getNumberOfSlots() {
-		return numberOfSlots;
-	}
-
-	public Configuration getConfiguration() {
-		return configuration;
-	}
-
-	public FiniteDuration getInitialRegistrationPause() {
-		return initialRegistrationPause;
-	}
-
-	public FiniteDuration getMaxRegistrationPause() {
-		return maxRegistrationPause;
-	}
-
-	public FiniteDuration getRefusedRegistrationPause() {
-		return refusedRegistrationPause;
-	}
-
-}
-

http://git-wip-us.apache.org/repos/asf/flink/blob/72468d14/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorGateway.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorGateway.java
deleted file mode 100644
index b0b21b0..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorGateway.java
+++ /dev/null
@@ -1,35 +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.rpc.taskexecutor;
-
-import org.apache.flink.runtime.rpc.RpcGateway;
-
-import java.util.UUID;
-
-/**
- * {@link TaskExecutor} RPC gateway interface
- */
-public interface TaskExecutorGateway extends RpcGateway {
-
-	// ------------------------------------------------------------------------
-	//  ResourceManager handlers
-	// ------------------------------------------------------------------------
-
-	void notifyOfNewResourceManagerLeader(String address, UUID resourceManagerLeaderId);
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/72468d14/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorRegistrationSuccess.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorRegistrationSuccess.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorRegistrationSuccess.java
deleted file mode 100644
index 641102d..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorRegistrationSuccess.java
+++ /dev/null
@@ -1,75 +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.rpc.taskexecutor;
-
-import org.apache.flink.runtime.instance.InstanceID;
-import org.apache.flink.runtime.rpc.registration.RegistrationResponse;
-
-import java.io.Serializable;
-
-/**
- * Base class for responses from the ResourceManager to a registration attempt by a
- * TaskExecutor.
- */
-public final class TaskExecutorRegistrationSuccess extends RegistrationResponse.Success implements Serializable {
-
-	private static final long serialVersionUID = 1L;
-
-	private final InstanceID registrationId;
-
-	private final long heartbeatInterval;
-
-	/**
-	 * Create a new {@code TaskExecutorRegistrationSuccess} message.
-	 * 
-	 * @param registrationId     The ID that the ResourceManager assigned the registration.
-	 * @param heartbeatInterval  The interval in which the ResourceManager will heartbeat the TaskExecutor.
-	 */
-	public TaskExecutorRegistrationSuccess(InstanceID registrationId, long heartbeatInterval) {
-		this.registrationId = registrationId;
-		this.heartbeatInterval = heartbeatInterval;
-	}
-
-	/**
-	 * Gets the ID that the ResourceManager assigned the registration.
-	 */
-	public InstanceID getRegistrationId() {
-		return registrationId;
-	}
-
-	/**
-	 * Gets the interval in which the ResourceManager will heartbeat the TaskExecutor.
-	 */
-	public long getHeartbeatInterval() {
-		return heartbeatInterval;
-	}
-
-	@Override
-	public String toString() {
-		return "TaskExecutorRegistrationSuccess (" + registrationId + " / " + heartbeatInterval + ')';
-	}
-
-}
-
-
-
-
-
-
-

http://git-wip-us.apache.org/repos/asf/flink/blob/72468d14/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorToResourceManagerConnection.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorToResourceManagerConnection.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorToResourceManagerConnection.java
deleted file mode 100644
index 7ccc879..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorToResourceManagerConnection.java
+++ /dev/null
@@ -1,198 +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.rpc.taskexecutor;
-
-import akka.dispatch.OnFailure;
-import akka.dispatch.OnSuccess;
-
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.runtime.clusterframework.types.ResourceID;
-import org.apache.flink.runtime.instance.InstanceID;
-import org.apache.flink.runtime.rpc.RpcService;
-import org.apache.flink.runtime.rpc.registration.RegistrationResponse;
-import org.apache.flink.runtime.rpc.registration.RetryingRegistration;
-import org.apache.flink.runtime.rpc.resourcemanager.ResourceManagerGateway;
-
-import org.slf4j.Logger;
-
-import scala.concurrent.Future;
-import scala.concurrent.duration.FiniteDuration;
-
-import java.util.UUID;
-import java.util.concurrent.TimeUnit;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
-import static org.apache.flink.util.Preconditions.checkState;
-
-/**
- * The connection between a TaskExecutor and the ResourceManager.
- */
-public class TaskExecutorToResourceManagerConnection {
-
-	/** the logger for all log messages of this class */
-	private final Logger log;
-
-	/** the TaskExecutor whose connection to the ResourceManager this represents */
-	private final TaskExecutor taskExecutor;
-
-	private final UUID resourceManagerLeaderId;
-
-	private final String resourceManagerAddress;
-
-	private TaskExecutorToResourceManagerConnection.ResourceManagerRegistration pendingRegistration;
-
-	private ResourceManagerGateway registeredResourceManager;
-
-	private InstanceID registrationId;
-
-	/** flag indicating that the connection is closed */
-	private volatile boolean closed;
-
-
-	public TaskExecutorToResourceManagerConnection(
-			Logger log,
-			TaskExecutor taskExecutor,
-			String resourceManagerAddress,
-			UUID resourceManagerLeaderId) {
-
-		this.log = checkNotNull(log);
-		this.taskExecutor = checkNotNull(taskExecutor);
-		this.resourceManagerAddress = checkNotNull(resourceManagerAddress);
-		this.resourceManagerLeaderId = checkNotNull(resourceManagerLeaderId);
-	}
-
-	// ------------------------------------------------------------------------
-	//  Life cycle
-	// ------------------------------------------------------------------------
-
-	@SuppressWarnings("unchecked")
-	public void start() {
-		checkState(!closed, "The connection is already closed");
-		checkState(!isRegistered() && pendingRegistration == null, "The connection is already started");
-
-		pendingRegistration = new TaskExecutorToResourceManagerConnection.ResourceManagerRegistration(
-				log, taskExecutor.getRpcService(),
-				resourceManagerAddress, resourceManagerLeaderId,
-				taskExecutor.getAddress(), taskExecutor.getResourceID());
-		pendingRegistration.startRegistration();
-
-		Future<Tuple2<ResourceManagerGateway, TaskExecutorRegistrationSuccess>> future = pendingRegistration.getFuture();
-		
-		future.onSuccess(new OnSuccess<Tuple2<ResourceManagerGateway, TaskExecutorRegistrationSuccess>>() {
-			@Override
-			public void onSuccess(Tuple2<ResourceManagerGateway, TaskExecutorRegistrationSuccess> result) {
-				registeredResourceManager = result.f0;
-				registrationId = result.f1.getRegistrationId();
-			}
-		}, taskExecutor.getMainThreadExecutionContext());
-		
-		// this future should only ever fail if there is a bug, not if the registration is declined
-		future.onFailure(new OnFailure() {
-			@Override
-			public void onFailure(Throwable failure) {
-				taskExecutor.onFatalError(failure);
-			}
-		}, taskExecutor.getMainThreadExecutionContext());
-	}
-
-	public void close() {
-		closed = true;
-
-		// make sure we do not keep re-trying forever
-		if (pendingRegistration != null) {
-			pendingRegistration.cancel();
-		}
-	}
-
-	public boolean isClosed() {
-		return closed;
-	}
-
-	// ------------------------------------------------------------------------
-	//  Properties
-	// ------------------------------------------------------------------------
-
-	public UUID getResourceManagerLeaderId() {
-		return resourceManagerLeaderId;
-	}
-
-	public String getResourceManagerAddress() {
-		return resourceManagerAddress;
-	}
-
-	/**
-	 * Gets the ResourceManagerGateway. This returns null until the registration is completed.
-	 */
-	public ResourceManagerGateway getResourceManager() {
-		return registeredResourceManager;
-	}
-
-	/**
-	 * Gets the ID under which the TaskExecutor is registered at the ResourceManager.
-	 * This returns null until the registration is completed.
-	 */
-	public InstanceID getRegistrationId() {
-		return registrationId;
-	}
-
-	public boolean isRegistered() {
-		return registeredResourceManager != null;
-	}
-
-	// ------------------------------------------------------------------------
-
-	@Override
-	public String toString() {
-		return String.format("Connection to ResourceManager %s (leaderId=%s)",
-				resourceManagerAddress, resourceManagerLeaderId); 
-	}
-
-	// ------------------------------------------------------------------------
-	//  Utilities
-	// ------------------------------------------------------------------------
-
-	private static class ResourceManagerRegistration
-			extends RetryingRegistration<ResourceManagerGateway, TaskExecutorRegistrationSuccess> {
-
-		private final String taskExecutorAddress;
-		
-		private final ResourceID resourceID;
-
-		ResourceManagerRegistration(
-				Logger log,
-				RpcService rpcService,
-				String targetAddress,
-				UUID leaderId,
-				String taskExecutorAddress,
-				ResourceID resourceID) {
-
-			super(log, rpcService, "ResourceManager", ResourceManagerGateway.class, targetAddress, leaderId);
-			this.taskExecutorAddress = checkNotNull(taskExecutorAddress);
-			this.resourceID = checkNotNull(resourceID);
-		}
-
-		@Override
-		protected Future<RegistrationResponse> invokeRegistration(
-				ResourceManagerGateway resourceManager, UUID leaderId, long timeoutMillis) throws Exception {
-
-			FiniteDuration timeout = new FiniteDuration(timeoutMillis, TimeUnit.MILLISECONDS);
-			return resourceManager.registerTaskExecutor(leaderId, taskExecutorAddress, resourceID, timeout);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/72468d14/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/SlotReport.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/SlotReport.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/SlotReport.java
new file mode 100644
index 0000000..a5de2d5
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/SlotReport.java
@@ -0,0 +1,56 @@
+/*
+ * 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.taskexecutor;
+
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+
+import java.io.Serializable;
+import java.util.List;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A report about the current status of all slots of the TaskExecutor, describing
+ * which slots are available and allocated, and what jobs (JobManagers) the allocated slots
+ * have been allocated to.
+ */
+public class SlotReport implements Serializable {
+
+	private static final long serialVersionUID = -3150175198722481689L;
+
+	/** The slots status of the TaskManager */
+	private final List<SlotStatus> slotsStatus;
+
+	/** The resource id which identifies the TaskManager */
+	private final ResourceID resourceID;
+
+	public SlotReport(final List<SlotStatus> slotsStatus, final ResourceID resourceID) {
+		this.slotsStatus = checkNotNull(slotsStatus);
+		this.resourceID = checkNotNull(resourceID);
+	}
+
+	public List<SlotStatus> getSlotsStatus() {
+		return slotsStatus;
+	}
+
+	public ResourceID getResourceID() {
+		return resourceID;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/72468d14/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/SlotStatus.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/SlotStatus.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/SlotStatus.java
new file mode 100644
index 0000000..744b674
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/SlotStatus.java
@@ -0,0 +1,129 @@
+/*
+ * 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.taskexecutor;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+
+import java.io.Serializable;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * This describes the slot current status which located in TaskManager.
+ */
+public class SlotStatus implements Serializable {
+
+	private static final long serialVersionUID = 5099191707339664493L;
+
+	/** slotID to identify a slot */
+	private final SlotID slotID;
+
+	/** the resource profile of the slot */
+	private final ResourceProfile profiler;
+
+	/** if the slot is allocated, allocationId identify its allocation; else, allocationId is null */
+	private final AllocationID allocationID;
+
+	/** if the slot is allocated, jobId identify which job this slot is allocated to; else, jobId is null */
+	private final JobID jobID;
+
+	public SlotStatus(SlotID slotID, ResourceProfile profiler) {
+		this(slotID, profiler, null, null);
+	}
+
+	public SlotStatus(SlotID slotID, ResourceProfile profiler, AllocationID allocationID, JobID jobID) {
+		this.slotID = checkNotNull(slotID, "slotID cannot be null");
+		this.profiler = checkNotNull(profiler, "profile cannot be null");
+		this.allocationID = allocationID;
+		this.jobID = jobID;
+	}
+
+	/**
+	 * Get the unique identification of this slot
+	 *
+	 * @return The slot id
+	 */
+	public SlotID getSlotID() {
+		return slotID;
+	}
+
+	/**
+	 * Get the resource profile of this slot
+	 *
+	 * @return The resource profile
+	 */
+	public ResourceProfile getProfiler() {
+		return profiler;
+	}
+
+	/**
+	 * Get the allocation id of this slot
+	 *
+	 * @return The allocation id if this slot is allocated, otherwise null
+	 */
+	public AllocationID getAllocationID() {
+		return allocationID;
+	}
+
+	/**
+	 * Get the job id of the slot allocated for
+	 *
+	 * @return The job id if this slot is allocated, otherwise null
+	 */
+	public JobID getJobID() {
+		return jobID;
+	}
+
+	@Override
+	public boolean equals(Object o) {
+		if (this == o) {
+			return true;
+		}
+		if (o == null || getClass() != o.getClass()) {
+			return false;
+		}
+
+		SlotStatus that = (SlotStatus) o;
+
+		if (!slotID.equals(that.slotID)) {
+			return false;
+		}
+		if (!profiler.equals(that.profiler)) {
+			return false;
+		}
+		if (allocationID != null ? !allocationID.equals(that.allocationID) : that.allocationID != null) {
+			return false;
+		}
+		return jobID != null ? jobID.equals(that.jobID) : that.jobID == null;
+
+	}
+
+	@Override
+	public int hashCode() {
+		int result = slotID.hashCode();
+		result = 31 * result + profiler.hashCode();
+		result = 31 * result + (allocationID != null ? allocationID.hashCode() : 0);
+		result = 31 * result + (jobID != null ? jobID.hashCode() : 0);
+		return result;
+	}
+
+}


[13/50] [abbrv] flink git commit: [FLINK-4595] Close FileOutputStream in ParameterTool

Posted by tr...@apache.org.
[FLINK-4595] Close FileOutputStream in ParameterTool

This closes #2478


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

Branch: refs/heads/flip-6
Commit: 920cda408f61a27f0b1731325b62bf141dd4b530
Parents: 7e07bde
Author: Alexander Pivovarov <ap...@gmail.com>
Authored: Wed Sep 7 14:11:06 2016 -0700
Committer: zentol <ch...@apache.org>
Committed: Thu Sep 8 12:04:26 2016 +0200

----------------------------------------------------------------------
 .../java/org/apache/flink/api/java/utils/ParameterTool.java     | 5 ++++-
 .../java/org/apache/flink/api/java/utils/ParameterToolTest.java | 5 ++++-
 2 files changed, 8 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/920cda40/flink-java/src/main/java/org/apache/flink/api/java/utils/ParameterTool.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/utils/ParameterTool.java b/flink-java/src/main/java/org/apache/flink/api/java/utils/ParameterTool.java
index 46c7387..38ae6df 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/utils/ParameterTool.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/utils/ParameterTool.java
@@ -31,6 +31,7 @@ import java.io.FileInputStream;
 import java.io.FileNotFoundException;
 import java.io.FileOutputStream;
 import java.io.IOException;
+import java.io.OutputStream;
 import java.io.Serializable;
 import java.util.Arrays;
 import java.util.HashMap;
@@ -523,7 +524,9 @@ public class ParameterTool extends ExecutionConfig.GlobalJobParameters implement
 		}
 		Properties defaultProps = new Properties();
 		defaultProps.putAll(this.defaultData);
-		defaultProps.store(new FileOutputStream(file), "Default file created by Flink's ParameterUtil.createPropertiesFile()");
+		try (final OutputStream out = new FileOutputStream(file)) {
+			defaultProps.store(out, "Default file created by Flink's ParameterUtil.createPropertiesFile()");
+		}
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/920cda40/flink-java/src/test/java/org/apache/flink/api/java/utils/ParameterToolTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/utils/ParameterToolTest.java b/flink-java/src/test/java/org/apache/flink/api/java/utils/ParameterToolTest.java
index a070473..09a7781 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/utils/ParameterToolTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/utils/ParameterToolTest.java
@@ -29,6 +29,7 @@ import java.io.File;
 import java.io.FileInputStream;
 import java.io.FileOutputStream;
 import java.io.IOException;
+import java.io.OutputStream;
 import java.util.Map;
 import java.util.Properties;
 
@@ -118,7 +119,9 @@ public class ParameterToolTest {
 		Properties props = new Properties();
 		props.setProperty("input", "myInput");
 		props.setProperty("expectedCount", "15");
-		props.store(new FileOutputStream(propertiesFile), "Test properties");
+		try (final OutputStream out = new FileOutputStream(propertiesFile)) {
+			props.store(out, "Test properties");
+		}
 		ParameterTool parameter = ParameterTool.fromPropertiesFile(propertiesFile.getAbsolutePath());
 		Assert.assertEquals(2, parameter.getNumberOfParameters());
 		validate(parameter);


[05/50] [abbrv] flink git commit: [FLINK-4459] [distributed runtime] Introduce SlotProvider for Scheduler

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/6e40f590/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerSlotSharingTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerSlotSharingTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerSlotSharingTest.java
index fd0523b..c4121f6 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerSlotSharingTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerSlotSharingTest.java
@@ -62,10 +62,10 @@ public class SchedulerSlotSharingTest {
 			scheduler.newInstanceAvailable(i2);
 			
 			// schedule 4 tasks from the first vertex group
-			SimpleSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 0, 8), sharingGroup));
-			SimpleSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 1, 8), sharingGroup));
-			SimpleSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 2, 8), sharingGroup));
-			SimpleSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 3, 8), sharingGroup));
+			SimpleSlot s1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 0, 8), sharingGroup), false).get();
+			SimpleSlot s2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 1, 8), sharingGroup), false).get();
+			SimpleSlot s3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 2, 8), sharingGroup), false).get();
+			SimpleSlot s4 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 3, 8), sharingGroup), false).get();
 			
 			assertNotNull(s1);
 			assertNotNull(s2);
@@ -76,7 +76,7 @@ public class SchedulerSlotSharingTest {
 			
 			// we cannot schedule another task from the first vertex group
 			try {
-				scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 4, 8), sharingGroup));
+				scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 4, 8), sharingGroup), false).get();
 				fail("Scheduler accepted too many tasks at the same time");
 			}
 			catch (NoResourceAvailableException e) {
@@ -90,7 +90,7 @@ public class SchedulerSlotSharingTest {
 			s3.releaseSlot();
 			
 			// allocate another slot from that group
-			SimpleSlot s5 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 4, 8), sharingGroup));
+			SimpleSlot s5 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 4, 8), sharingGroup), false).get();
 			assertNotNull(s5);
 			
 			// release all old slots
@@ -98,9 +98,9 @@ public class SchedulerSlotSharingTest {
 			s2.releaseSlot();
 			s4.releaseSlot();
 			
-			SimpleSlot s6 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 5, 8), sharingGroup));
-			SimpleSlot s7 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 6, 8), sharingGroup));
-			SimpleSlot s8 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 7, 8), sharingGroup));
+			SimpleSlot s6 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 5, 8), sharingGroup), false).get();
+			SimpleSlot s7 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 6, 8), sharingGroup), false).get();
+			SimpleSlot s8 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 7, 8), sharingGroup), false).get();
 			
 			assertNotNull(s6);
 			assertNotNull(s7);
@@ -135,7 +135,7 @@ public class SchedulerSlotSharingTest {
 	}
 	
 	@Test
-	public void scheduleImmediatelyWithSharing() {
+	public void allocateSlotWithSharing() {
 		try {
 			JobVertexID jid1 = new JobVertexID();
 			JobVertexID jid2 = new JobVertexID();
@@ -147,10 +147,10 @@ public class SchedulerSlotSharingTest {
 			scheduler.newInstanceAvailable(getRandomInstance(2));
 			
 			// schedule 4 tasks from the first vertex group
-			SimpleSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 0, 5), sharingGroup));
-			SimpleSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 1, 5), sharingGroup));
-			SimpleSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 2, 5), sharingGroup));
-			SimpleSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 3, 5), sharingGroup));
+			SimpleSlot s1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 0, 5), sharingGroup), false).get();
+			SimpleSlot s2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 1, 5), sharingGroup), false).get();
+			SimpleSlot s3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 2, 5), sharingGroup), false).get();
+			SimpleSlot s4 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 3, 5), sharingGroup), false).get();
 			
 			assertNotNull(s1);
 			assertNotNull(s2);
@@ -161,7 +161,7 @@ public class SchedulerSlotSharingTest {
 			
 			// we cannot schedule another task from the first vertex group
 			try {
-				scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 4, 5), sharingGroup));
+				scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 4, 5), sharingGroup), false).get();
 				fail("Scheduler accepted too many tasks at the same time");
 			}
 			catch (NoResourceAvailableException e) {
@@ -172,10 +172,10 @@ public class SchedulerSlotSharingTest {
 			}
 			
 			// schedule some tasks from the second ID group
-			SimpleSlot s1_2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 0, 5), sharingGroup));
-			SimpleSlot s2_2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 1, 5), sharingGroup));
-			SimpleSlot s3_2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 2, 5), sharingGroup));
-			SimpleSlot s4_2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 3, 5), sharingGroup));
+			SimpleSlot s1_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 0, 5), sharingGroup), false).get();
+			SimpleSlot s2_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 1, 5), sharingGroup), false).get();
+			SimpleSlot s3_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 2, 5), sharingGroup), false).get();
+			SimpleSlot s4_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 3, 5), sharingGroup), false).get();
 			
 			assertNotNull(s1_2);
 			assertNotNull(s2_2);
@@ -184,7 +184,7 @@ public class SchedulerSlotSharingTest {
 			
 			// we cannot schedule another task from the second vertex group
 			try {
-				scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 4, 5), sharingGroup));
+				scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 4, 5), sharingGroup), false).get();
 				fail("Scheduler accepted too many tasks at the same time");
 			}
 			catch (NoResourceAvailableException e) {
@@ -205,7 +205,7 @@ public class SchedulerSlotSharingTest {
 			
 			// we can still not schedule anything from the second group of vertices
 			try {
-				scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 4, 5), sharingGroup));
+				scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 4, 5), sharingGroup), false).get();
 				fail("Scheduler accepted too many tasks at the same time");
 			}
 			catch (NoResourceAvailableException e) {
@@ -216,7 +216,7 @@ public class SchedulerSlotSharingTest {
 			}
 			
 			// we can schedule something from the first vertex group
-			SimpleSlot s5 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 4, 5), sharingGroup));
+			SimpleSlot s5 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 4, 5), sharingGroup), false).get();
 			assertNotNull(s5);
 			
 			assertEquals(4, sharingGroup.getTaskAssignment().getNumberOfSlots());
@@ -226,7 +226,7 @@ public class SchedulerSlotSharingTest {
 			
 			// now we release a slot from the second vertex group and schedule another task from that group
 			s2_2.releaseSlot();
-			SimpleSlot s5_2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 4, 5), sharingGroup));
+			SimpleSlot s5_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 4, 5), sharingGroup), false).get();
 			assertNotNull(s5_2);
 			
 			// release all slots
@@ -255,7 +255,7 @@ public class SchedulerSlotSharingTest {
 	}
 	
 	@Test
-	public void scheduleImmediatelyWithIntermediateTotallyEmptySharingGroup() {
+	public void allocateSlotWithIntermediateTotallyEmptySharingGroup() {
 		try {
 			JobVertexID jid1 = new JobVertexID();
 			JobVertexID jid2 = new JobVertexID();
@@ -267,10 +267,10 @@ public class SchedulerSlotSharingTest {
 			scheduler.newInstanceAvailable(getRandomInstance(2));
 			
 			// schedule 4 tasks from the first vertex group
-			SimpleSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 0, 4), sharingGroup));
-			SimpleSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 1, 4), sharingGroup));
-			SimpleSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 2, 4), sharingGroup));
-			SimpleSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 3, 4), sharingGroup));
+			SimpleSlot s1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 0, 4), sharingGroup), false).get();
+			SimpleSlot s2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 1, 4), sharingGroup), false).get();
+			SimpleSlot s3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 2, 4), sharingGroup), false).get();
+			SimpleSlot s4 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 3, 4), sharingGroup), false).get();
 			
 			assertEquals(4, sharingGroup.getTaskAssignment().getNumberOfSlots());
 			assertEquals(0, sharingGroup.getTaskAssignment().getNumberOfAvailableSlotsForGroup(jid1));
@@ -286,10 +286,10 @@ public class SchedulerSlotSharingTest {
 			assertEquals(0, sharingGroup.getTaskAssignment().getNumberOfAvailableSlotsForGroup(jid2));
 			
 			// schedule some tasks from the second ID group
-			SimpleSlot s1_2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 0, 4), sharingGroup));
-			SimpleSlot s2_2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 1, 4), sharingGroup));
-			SimpleSlot s3_2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 2, 4), sharingGroup));
-			SimpleSlot s4_2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 3, 4), sharingGroup));
+			SimpleSlot s1_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 0, 4), sharingGroup), false).get();
+			SimpleSlot s2_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 1, 4), sharingGroup), false).get();
+			SimpleSlot s3_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 2, 4), sharingGroup), false).get();
+			SimpleSlot s4_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 3, 4), sharingGroup), false).get();
 
 			assertEquals(4, sharingGroup.getTaskAssignment().getNumberOfSlots());
 			assertEquals(4, sharingGroup.getTaskAssignment().getNumberOfAvailableSlotsForGroup(jid1));
@@ -319,7 +319,7 @@ public class SchedulerSlotSharingTest {
 	}
 	
 	@Test
-	public void scheduleImmediatelyWithTemprarilyEmptyVertexGroup() {
+	public void allocateSlotWithTemprarilyEmptyVertexGroup() {
 		try {
 			JobVertexID jid1 = new JobVertexID();
 			JobVertexID jid2 = new JobVertexID();
@@ -332,10 +332,10 @@ public class SchedulerSlotSharingTest {
 			scheduler.newInstanceAvailable(getRandomInstance(2));
 			
 			// schedule 4 tasks from the first vertex group
-			SimpleSlot s1_1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 0, 4), sharingGroup));
-			SimpleSlot s2_1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 1, 4), sharingGroup));
-			SimpleSlot s3_1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 2, 4), sharingGroup));
-			SimpleSlot s4_1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 3, 4), sharingGroup));
+			SimpleSlot s1_1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 0, 4), sharingGroup), false).get();
+			SimpleSlot s2_1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 1, 4), sharingGroup), false).get();
+			SimpleSlot s3_1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 2, 4), sharingGroup), false).get();
+			SimpleSlot s4_1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 3, 4), sharingGroup), false).get();
 			
 			assertNotNull(s1_1);
 			assertNotNull(s2_1);
@@ -345,10 +345,10 @@ public class SchedulerSlotSharingTest {
 			assertTrue(areAllDistinct(s1_1, s2_1, s3_1, s4_1));
 			
 			// schedule 4 tasks from the second vertex group
-			SimpleSlot s1_2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 0, 7), sharingGroup));
-			SimpleSlot s2_2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 1, 7), sharingGroup));
-			SimpleSlot s3_2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 2, 7), sharingGroup));
-			SimpleSlot s4_2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 3, 7), sharingGroup));
+			SimpleSlot s1_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 0, 7), sharingGroup), false).get();
+			SimpleSlot s2_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 1, 7), sharingGroup), false).get();
+			SimpleSlot s3_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 2, 7), sharingGroup), false).get();
+			SimpleSlot s4_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 3, 7), sharingGroup), false).get();
 			
 			assertNotNull(s1_2);
 			assertNotNull(s2_2);
@@ -358,10 +358,10 @@ public class SchedulerSlotSharingTest {
 			assertTrue(areAllDistinct(s1_2, s2_2, s3_2, s4_2));
 			
 			// schedule 4 tasks from the third vertex group
-			SimpleSlot s1_3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid3, 0, 4), sharingGroup));
-			SimpleSlot s2_3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid3, 1, 4), sharingGroup));
-			SimpleSlot s3_3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid3, 2, 4), sharingGroup));
-			SimpleSlot s4_3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid3, 3, 4), sharingGroup));
+			SimpleSlot s1_3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid3, 0, 4), sharingGroup), false).get();
+			SimpleSlot s2_3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid3, 1, 4), sharingGroup), false).get();
+			SimpleSlot s3_3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid3, 2, 4), sharingGroup), false).get();
+			SimpleSlot s4_3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid3, 3, 4), sharingGroup), false).get();
 			
 			assertNotNull(s1_3);
 			assertNotNull(s2_3);
@@ -373,7 +373,7 @@ public class SchedulerSlotSharingTest {
 			
 			// we cannot schedule another task from the second vertex group
 			try {
-				scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 4, 5), sharingGroup));
+				scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 4, 5), sharingGroup), false).get();
 				fail("Scheduler accepted too many tasks at the same time");
 			}
 			catch (NoResourceAvailableException e) {
@@ -389,9 +389,9 @@ public class SchedulerSlotSharingTest {
 			s3_2.releaseSlot();
 			s4_2.releaseSlot();
 			
-			SimpleSlot s5_2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 5, 7), sharingGroup));
-			SimpleSlot s6_2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 6, 7), sharingGroup));
-			SimpleSlot s7_2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 7, 7), sharingGroup));
+			SimpleSlot s5_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 5, 7), sharingGroup), false).get();
+			SimpleSlot s6_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 6, 7), sharingGroup), false).get();
+			SimpleSlot s7_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 7, 7), sharingGroup), false).get();
 			
 			assertNotNull(s5_2);
 			assertNotNull(s6_2);
@@ -430,7 +430,7 @@ public class SchedulerSlotSharingTest {
 	}
 	
 	@Test
-	public void scheduleImmediatelyWithTemprarilyEmptyVertexGroup2() {
+	public void allocateSlotWithTemporarilyEmptyVertexGroup2() {
 		try {
 			JobVertexID jid1 = new JobVertexID();
 			JobVertexID jid2 = new JobVertexID();
@@ -442,9 +442,9 @@ public class SchedulerSlotSharingTest {
 			scheduler.newInstanceAvailable(getRandomInstance(2));
 			
 			// schedule 1 tasks from the first vertex group and 2 from the second
-			SimpleSlot s1_1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 0, 2), sharingGroup));
-			SimpleSlot s2_1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 0, 2), sharingGroup));
-			SimpleSlot s2_2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 1, 2), sharingGroup));
+			SimpleSlot s1_1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 0, 2), sharingGroup), false).get();
+			SimpleSlot s2_1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 0, 2), sharingGroup), false).get();
+			SimpleSlot s2_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 1, 2), sharingGroup), false).get();
 			
 			assertNotNull(s1_1);
 			assertNotNull(s2_1);
@@ -460,7 +460,7 @@ public class SchedulerSlotSharingTest {
 			
 			
 			// this should free one slot so we can allocate one non-shared
-			SimpleSlot sx = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid3, 0, 1)));
+			SimpleSlot sx = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid3, 0, 1)), false).get();
 			assertNotNull(sx);
 			
 			assertEquals(1, sharingGroup.getTaskAssignment().getNumberOfSlots());
@@ -495,28 +495,28 @@ public class SchedulerSlotSharingTest {
 			scheduler.newInstanceAvailable(getRandomInstance(2));
 			
 			// schedule some individual vertices
-			SimpleSlot sA1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jidA, 0, 2)));
-			SimpleSlot sA2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jidA, 1, 2)));
+			SimpleSlot sA1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jidA, 0, 2)), false).get();
+			SimpleSlot sA2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jidA, 1, 2)), false).get();
 			assertNotNull(sA1);
 			assertNotNull(sA2);
 			
 			// schedule some vertices in the sharing group
-			SimpleSlot s1_0 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 0, 4), sharingGroup));
-			SimpleSlot s1_1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 1, 4), sharingGroup));
-			SimpleSlot s2_0 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 0, 4), sharingGroup));
-			SimpleSlot s2_1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 1, 4), sharingGroup));
+			SimpleSlot s1_0 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 0, 4), sharingGroup), false).get();
+			SimpleSlot s1_1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 1, 4), sharingGroup), false).get();
+			SimpleSlot s2_0 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 0, 4), sharingGroup), false).get();
+			SimpleSlot s2_1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 1, 4), sharingGroup), false).get();
 			assertNotNull(s1_0);
 			assertNotNull(s1_1);
 			assertNotNull(s2_0);
 			assertNotNull(s2_1);
 			
 			// schedule another isolated vertex
-			SimpleSlot sB1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jidB, 1, 3)));
+			SimpleSlot sB1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jidB, 1, 3)), false).get();
 			assertNotNull(sB1);
 			
 			// should not be able to schedule more vertices
 			try {
-				scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 2, 4), sharingGroup));
+				scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 2, 4), sharingGroup), false).get();
 				fail("Scheduler accepted too many tasks at the same time");
 			}
 			catch (NoResourceAvailableException e) {
@@ -527,7 +527,7 @@ public class SchedulerSlotSharingTest {
 			}
 			
 			try {
-				scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 2, 4), sharingGroup));
+				scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 2, 4), sharingGroup), false).get();
 				fail("Scheduler accepted too many tasks at the same time");
 			}
 			catch (NoResourceAvailableException e) {
@@ -538,7 +538,7 @@ public class SchedulerSlotSharingTest {
 			}
 			
 			try {
-				scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jidB, 0, 3)));
+				scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jidB, 0, 3)), false);
 				fail("Scheduler accepted too many tasks at the same time");
 			}
 			catch (NoResourceAvailableException e) {
@@ -549,7 +549,7 @@ public class SchedulerSlotSharingTest {
 			}
 			
 			try {
-				scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jidC, 0, 1)));
+				scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jidC, 0, 1)), false);
 				fail("Scheduler accepted too many tasks at the same time");
 			}
 			catch (NoResourceAvailableException e) {
@@ -562,8 +562,8 @@ public class SchedulerSlotSharingTest {
 			// release some isolated task and check that the sharing group may grow
 			sA1.releaseSlot();
 			
-			SimpleSlot s1_2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 2, 4), sharingGroup));
-			SimpleSlot s2_2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 3, 4), sharingGroup));
+			SimpleSlot s1_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 2, 4), sharingGroup), false).get();
+			SimpleSlot s2_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 3, 4), sharingGroup), false).get();
 			assertNotNull(s1_2);
 			assertNotNull(s2_2);
 			
@@ -575,19 +575,19 @@ public class SchedulerSlotSharingTest {
 			assertEquals(1, scheduler.getNumberOfAvailableSlots());
 			
 			// schedule one more no-shared task
-			SimpleSlot sB0 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jidB, 0, 3)));
+			SimpleSlot sB0 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jidB, 0, 3)), false).get();
 			assertNotNull(sB0);
 			
 			// release the last of the original shared slots and allocate one more non-shared slot
 			s2_1.releaseSlot();
-			SimpleSlot sB2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jidB, 2, 3)));
+			SimpleSlot sB2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jidB, 2, 3)), false).get();
 			assertNotNull(sB2);
 			
 			
 			// release on non-shared and add some shared slots
 			sA2.releaseSlot();
-			SimpleSlot s1_3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 3, 4), sharingGroup));
-			SimpleSlot s2_3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 2, 4), sharingGroup));
+			SimpleSlot s1_3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 3, 4), sharingGroup), false).get();
+			SimpleSlot s2_3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 2, 4), sharingGroup), false).get();
 			assertNotNull(s1_3);
 			assertNotNull(s2_3);
 			
@@ -597,8 +597,8 @@ public class SchedulerSlotSharingTest {
 			s1_3.releaseSlot();
 			s2_3.releaseSlot();
 			
-			SimpleSlot sC0 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jidC, 1, 2)));
-			SimpleSlot sC1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jidC, 0, 2)));
+			SimpleSlot sC0 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jidC, 1, 2)), false).get();
+			SimpleSlot sC1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jidC, 0, 2)), false).get();
 			assertNotNull(sC0);
 			assertNotNull(sC1);
 			
@@ -646,8 +646,8 @@ public class SchedulerSlotSharingTest {
 			
 			
 			// schedule one to each instance
-			SimpleSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, loc1), sharingGroup));
-			SimpleSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, loc2), sharingGroup));
+			SimpleSlot s1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, loc1), sharingGroup), false).get();
+			SimpleSlot s2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, loc2), sharingGroup), false).get();
 			assertNotNull(s1);
 			assertNotNull(s2);
 			
@@ -656,8 +656,8 @@ public class SchedulerSlotSharingTest {
 			assertEquals(1, i2.getNumberOfAvailableSlots());
 			
 			// schedule one from the other group to each instance
-			SimpleSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, loc1), sharingGroup));
-			SimpleSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, loc2), sharingGroup));
+			SimpleSlot s3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, loc1), sharingGroup), false).get();
+			SimpleSlot s4 = scheduler.allocateSlot(new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, loc2), sharingGroup), false).get();
 			assertNotNull(s3);
 			assertNotNull(s4);
 			
@@ -699,8 +699,8 @@ public class SchedulerSlotSharingTest {
 			
 			
 			// schedule one to each instance
-			SimpleSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, loc1), sharingGroup));
-			SimpleSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, loc1), sharingGroup));
+			SimpleSlot s1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, loc1), sharingGroup), false).get();
+			SimpleSlot s2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, loc1), sharingGroup), false).get();
 			assertNotNull(s1);
 			assertNotNull(s2);
 			
@@ -709,8 +709,8 @@ public class SchedulerSlotSharingTest {
 			assertEquals(2, i2.getNumberOfAvailableSlots());
 			
 			// schedule one from the other group to each instance
-			SimpleSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, loc2), sharingGroup));
-			SimpleSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, loc2), sharingGroup));
+			SimpleSlot s3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, loc2), sharingGroup), false).get();
+			SimpleSlot s4 = scheduler.allocateSlot(new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, loc2), sharingGroup), false).get();
 			assertNotNull(s3);
 			assertNotNull(s4);
 			
@@ -750,14 +750,14 @@ public class SchedulerSlotSharingTest {
 			scheduler.newInstanceAvailable(i2);
 			
 			// schedule until the one instance is full
-			SimpleSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, loc1), sharingGroup));
-			SimpleSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, loc1), sharingGroup));
-			SimpleSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 4, loc1), sharingGroup));
-			SimpleSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 4, loc1), sharingGroup));
+			SimpleSlot s1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, loc1), sharingGroup), false).get();
+			SimpleSlot s2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, loc1), sharingGroup), false).get();
+			SimpleSlot s3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 4, loc1), sharingGroup), false).get();
+			SimpleSlot s4 = scheduler.allocateSlot(new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 4, loc1), sharingGroup), false).get();
 
 			// schedule two more with preference of same instance --> need to go to other instance
-			SimpleSlot s5 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 3, 4, loc1), sharingGroup));
-			SimpleSlot s6 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 4, 4, loc1), sharingGroup));
+			SimpleSlot s5 = scheduler.allocateSlot(new ScheduledUnit(getTestVertexWithLocation(jid2, 3, 4, loc1), sharingGroup), false).get();
+			SimpleSlot s6 = scheduler.allocateSlot(new ScheduledUnit(getTestVertexWithLocation(jid2, 4, 4, loc1), sharingGroup), false).get();
 			
 			assertNotNull(s1);
 			assertNotNull(s2);
@@ -803,19 +803,19 @@ public class SchedulerSlotSharingTest {
 			scheduler.newInstanceAvailable(getRandomInstance(4));
 			
 			// allocate something from group 1 and 2 interleaved with schedule for group 3
-			SimpleSlot slot_1_1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 0, 4), sharingGroup));
-			SimpleSlot slot_1_2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 1, 4), sharingGroup));
+			SimpleSlot slot_1_1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 0, 4), sharingGroup), false).get();
+			SimpleSlot slot_1_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 1, 4), sharingGroup), false).get();
 
-			SimpleSlot slot_2_1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 0, 4), sharingGroup));
-			SimpleSlot slot_2_2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 1, 4), sharingGroup));
+			SimpleSlot slot_2_1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 0, 4), sharingGroup), false).get();
+			SimpleSlot slot_2_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 1, 4), sharingGroup), false).get();
 			
-			SimpleSlot slot_3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid3, 0, 1), sharingGroup));
+			SimpleSlot slot_3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid3, 0, 1), sharingGroup), false).get();
 			
-			SimpleSlot slot_1_3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 2, 4), sharingGroup));
-			SimpleSlot slot_1_4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 3, 4), sharingGroup));
+			SimpleSlot slot_1_3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 2, 4), sharingGroup), false).get();
+			SimpleSlot slot_1_4 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 3, 4), sharingGroup), false).get();
 			
-			SimpleSlot slot_2_3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 2, 4), sharingGroup));
-			SimpleSlot slot_2_4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 3, 4), sharingGroup));
+			SimpleSlot slot_2_3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 2, 4), sharingGroup), false).get();
+			SimpleSlot slot_2_4 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 3, 4), sharingGroup), false).get();
 			
 			// release groups 1 and 2
 			
@@ -831,10 +831,10 @@ public class SchedulerSlotSharingTest {
 			
 			// allocate group 4
 			
-			SimpleSlot slot_4_1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid4, 0, 4), sharingGroup));
-			SimpleSlot slot_4_2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid4, 1, 4), sharingGroup));
-			SimpleSlot slot_4_3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid4, 2, 4), sharingGroup));
-			SimpleSlot slot_4_4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid4, 3, 4), sharingGroup));
+			SimpleSlot slot_4_1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid4, 0, 4), sharingGroup), false).get();
+			SimpleSlot slot_4_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid4, 1, 4), sharingGroup), false).get();
+			SimpleSlot slot_4_3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid4, 2, 4), sharingGroup), false).get();
+			SimpleSlot slot_4_4 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid4, 3, 4), sharingGroup), false).get();
 			
 			// release groups 3 and 4
 			
@@ -885,7 +885,7 @@ public class SchedulerSlotSharingTest {
 					@Override
 					public void run() {
 						try {
-							SimpleSlot slot = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid4, enumerator4.getAndIncrement(), 4), sharingGroup));
+							SimpleSlot slot = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid4, enumerator4.getAndIncrement(), 4), sharingGroup), false).get();
 
 							sleepUninterruptibly(rnd.nextInt(5));
 							slot.releaseSlot();
@@ -908,7 +908,7 @@ public class SchedulerSlotSharingTest {
 					public void run() {
 						try {
 							if (flag3.compareAndSet(false, true)) {
-								SimpleSlot slot = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid3, 0, 1), sharingGroup));
+								SimpleSlot slot = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid3, 0, 1), sharingGroup), false).get();
 								
 								sleepUninterruptibly(5);
 								
@@ -937,7 +937,7 @@ public class SchedulerSlotSharingTest {
 					@Override
 					public void run() {
 						try {
-							SimpleSlot slot = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, enumerator2.getAndIncrement(), 4), sharingGroup));
+							SimpleSlot slot = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, enumerator2.getAndIncrement(), 4), sharingGroup), false).get();
 							
 							// wait a bit till scheduling the successor
 							sleepUninterruptibly(rnd.nextInt(5));
@@ -964,7 +964,7 @@ public class SchedulerSlotSharingTest {
 					@Override
 					public void run() {
 						try {
-							SimpleSlot slot = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, enumerator1.getAndIncrement(), 4), sharingGroup));
+							SimpleSlot slot = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, enumerator1.getAndIncrement(), 4), sharingGroup), false).get();
 							
 							// wait a bit till scheduling the successor
 							sleepUninterruptibly(rnd.nextInt(5));
@@ -1039,27 +1039,27 @@ public class SchedulerSlotSharingTest {
 			scheduler.newInstanceAvailable(getRandomInstance(4));
 			
 			// schedule one task for the first and second vertex
-			SimpleSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 0, 1), sharingGroup));
-			SimpleSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 0, 1), sharingGroup));
+			SimpleSlot s1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 0, 1), sharingGroup), false).get();
+			SimpleSlot s2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 0, 1), sharingGroup), false).get();
 			
 			assertTrue(  s1.getParent() == s2.getParent() );
 			assertEquals(3, scheduler.getNumberOfAvailableSlots());
 			
-			SimpleSlot s3_0 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid3, 0, 5), sharingGroup));
-			SimpleSlot s3_1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid3, 1, 5), sharingGroup));
-			SimpleSlot s4_0 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid4, 0, 4), sharingGroup));
-			SimpleSlot s4_1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid4, 1, 4), sharingGroup));
+			SimpleSlot s3_0 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid3, 0, 5), sharingGroup), false).get();
+			SimpleSlot s3_1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid3, 1, 5), sharingGroup), false).get();
+			SimpleSlot s4_0 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid4, 0, 4), sharingGroup), false).get();
+			SimpleSlot s4_1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid4, 1, 4), sharingGroup), false).get();
 			
 			s1.releaseSlot();
 			s2.releaseSlot();
 			
-			SimpleSlot s3_2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid3, 2, 5), sharingGroup));
-			SimpleSlot s3_3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid3, 3, 5), sharingGroup));
-			SimpleSlot s4_2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid4, 2, 4), sharingGroup));
-			SimpleSlot s4_3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid4, 3, 4), sharingGroup));
+			SimpleSlot s3_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid3, 2, 5), sharingGroup), false).get();
+			SimpleSlot s3_3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid3, 3, 5), sharingGroup), false).get();
+			SimpleSlot s4_2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid4, 2, 4), sharingGroup), false).get();
+			SimpleSlot s4_3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid4, 3, 4), sharingGroup), false).get();
 			
 			try {
-				scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid3, 4, 5), sharingGroup));
+				scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid3, 4, 5), sharingGroup), false);
 				fail("should throw an exception");
 			}
 			catch (NoResourceAvailableException e) {


[14/50] [abbrv] flink git commit: [FLINK-4458] Replace ForkableFlinkMiniCluster by LocalFlinkMiniCluster

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java
index 9bd8cc3..2738d22 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java
@@ -21,7 +21,7 @@ package org.apache.flink.test.runtime.leaderelection;
 import akka.actor.ActorSystem;
 import akka.actor.Kill;
 import akka.actor.PoisonPill;
-import org.apache.commons.io.FileUtils;
+import org.apache.curator.test.TestingServer;
 import org.apache.flink.api.common.JobExecutionResult;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
@@ -35,14 +35,19 @@ import org.apache.flink.runtime.jobmanager.Tasks;
 import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup;
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
 import org.apache.flink.runtime.messages.JobManagerMessages;
+import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
+import org.apache.flink.runtime.testingUtils.TestingCluster;
 import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages;
 import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages.WaitForAllVerticesToBeRunningOrFinished;
 import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.runtime.testutils.ZooKeeperTestUtils;
 import org.apache.flink.runtime.util.LeaderRetrievalUtils;
-import org.apache.flink.test.util.ForkableFlinkMiniCluster;
 import org.apache.flink.util.TestLogger;
 import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import scala.concurrent.Await;
@@ -52,7 +57,6 @@ import scala.concurrent.duration.FiniteDuration;
 import scala.concurrent.impl.Promise;
 
 import java.io.File;
-import java.io.IOException;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
@@ -61,22 +65,20 @@ public class ZooKeeperLeaderElectionITCase extends TestLogger {
 
 	private static final FiniteDuration timeout = TestingUtils.TESTING_DURATION();
 
-	private static final File tempDirectory;
+	private static TestingServer zkServer;
 
-	static {
-		try {
-			tempDirectory = org.apache.flink.runtime.testutils
-					.CommonTestUtils.createTempDirectory();
-		}
-		catch (IOException e) {
-			throw new RuntimeException("Test setup failed", e);
-		}
+	@Rule
+	public TemporaryFolder tempFolder = new TemporaryFolder();
+
+	@BeforeClass
+	public static void setup() throws Exception {
+		zkServer = new TestingServer(true);
 	}
 
 	@AfterClass
 	public static void tearDown() throws Exception {
-		if (tempDirectory != null) {
-			FileUtils.deleteDirectory(tempDirectory);
+		if (zkServer != null) {
+			zkServer.close();
 		}
 	}
 
@@ -86,18 +88,19 @@ public class ZooKeeperLeaderElectionITCase extends TestLogger {
 	 */
 	@Test
 	public void testTaskManagerRegistrationAtReelectedLeader() throws Exception {
-		Configuration configuration = new Configuration();
+		File rootFolder = tempFolder.getRoot();
+
+		Configuration configuration = ZooKeeperTestUtils.createZooKeeperHAConfig(
+			zkServer.getConnectString(),
+			rootFolder.getPath());
 
 		int numJMs = 10;
 		int numTMs = 3;
 
-		configuration.setString(ConfigConstants.HA_MODE, "zookeeper");
 		configuration.setInteger(ConfigConstants.LOCAL_NUMBER_JOB_MANAGER, numJMs);
 		configuration.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, numTMs);
-		configuration.setString(ConfigConstants.STATE_BACKEND, "filesystem");
-		configuration.setString(ConfigConstants.HA_ZOOKEEPER_STORAGE_PATH, tempDirectory.getAbsoluteFile().toURI().toString());
 
-		ForkableFlinkMiniCluster cluster = new ForkableFlinkMiniCluster(configuration);
+		TestingCluster cluster = new TestingCluster(configuration);
 
 		try {
 			cluster.start();
@@ -137,14 +140,15 @@ public class ZooKeeperLeaderElectionITCase extends TestLogger {
 		int numSlotsPerTM = 3;
 		int parallelism = numTMs * numSlotsPerTM;
 
-		Configuration configuration = new Configuration();
+		File rootFolder = tempFolder.getRoot();
+
+		Configuration configuration = ZooKeeperTestUtils.createZooKeeperHAConfig(
+			zkServer.getConnectString(),
+			rootFolder.getPath());
 
-		configuration.setString(ConfigConstants.HA_MODE, "zookeeper");
 		configuration.setInteger(ConfigConstants.LOCAL_NUMBER_JOB_MANAGER, numJMs);
 		configuration.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, numTMs);
 		configuration.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, numSlotsPerTM);
-		configuration.setString(ConfigConstants.STATE_BACKEND, "filesystem");
-		configuration.setString(ConfigConstants.HA_ZOOKEEPER_STORAGE_PATH, tempDirectory.getAbsoluteFile().toURI().toString());
 
 		// we "effectively" disable the automatic RecoverAllJobs message and sent it manually to make
 		// sure that all TMs have registered to the JM prior to issueing the RecoverAllJobs message
@@ -169,7 +173,7 @@ public class ZooKeeperLeaderElectionITCase extends TestLogger {
 
 		final JobGraph graph = new JobGraph("Blocking test job", sender, receiver);
 
-		final ForkableFlinkMiniCluster cluster = new ForkableFlinkMiniCluster(configuration);
+		final TestingCluster cluster = new TestingCluster(configuration);
 
 		ActorSystem clientActorSystem = null;
 
@@ -250,14 +254,14 @@ public class ZooKeeperLeaderElectionITCase extends TestLogger {
 		boolean finished = false;
 
 		final ActorSystem clientActorSystem;
-		final ForkableFlinkMiniCluster cluster;
+		final LocalFlinkMiniCluster cluster;
 		final JobGraph graph;
 
 		final Promise<JobExecutionResult> resultPromise = new Promise.DefaultPromise<>();
 
 		public JobSubmitterRunnable(
 				ActorSystem actorSystem,
-				ForkableFlinkMiniCluster cluster,
+				LocalFlinkMiniCluster cluster,
 				JobGraph graph) {
 			this.clientActorSystem = actorSystem;
 			this.cluster = cluster;

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/TimestampITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/TimestampITCase.java b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/TimestampITCase.java
index d693aaa..2ed759d 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/TimestampITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/TimestampITCase.java
@@ -27,6 +27,7 @@ import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.core.testutils.MultiShotLatch;
+import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
 import org.apache.flink.streaming.api.TimeCharacteristic;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
@@ -43,7 +44,6 @@ import org.apache.flink.streaming.api.watermark.Watermark;
 import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows;
 import org.apache.flink.streaming.api.windowing.time.Time;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.test.util.ForkableFlinkMiniCluster;
 import org.apache.flink.util.TestLogger;
 
 import org.apache.flink.util.TestLogger;
@@ -75,7 +75,7 @@ public class TimestampITCase extends TestLogger {
 	static MultiShotLatch latch;
 
 
-	private static ForkableFlinkMiniCluster cluster;
+	private static LocalFlinkMiniCluster cluster;
 
 	@Before
 	public void setupLatch() {
@@ -92,7 +92,7 @@ public class TimestampITCase extends TestLogger {
 			config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, NUM_TASK_SLOTS);
 			config.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 12);
 
-			cluster = new ForkableFlinkMiniCluster(config, false);
+			cluster = new LocalFlinkMiniCluster(config, false);
 
 			cluster.start();
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/flink-tests/src/test/java/org/apache/flink/test/web/WebFrontendITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/web/WebFrontendITCase.java b/flink-tests/src/test/java/org/apache/flink/test/web/WebFrontendITCase.java
index fc90994..a8482ac 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/web/WebFrontendITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/web/WebFrontendITCase.java
@@ -29,11 +29,11 @@ import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
 import org.apache.flink.runtime.testutils.StoppableInvokable;
 import org.apache.flink.runtime.webmonitor.WebMonitorUtils;
 import org.apache.flink.runtime.webmonitor.files.MimeTypes;
 import org.apache.flink.runtime.webmonitor.testutils.HttpTestClient;
-import org.apache.flink.test.util.ForkableFlinkMiniCluster;
 import org.apache.flink.test.util.TestBaseUtils;
 
 import org.apache.flink.util.TestLogger;
@@ -62,7 +62,7 @@ public class WebFrontendITCase extends TestLogger {
 	private static final int NUM_TASK_MANAGERS = 2;
 	private static final int NUM_SLOTS = 4;
 	
-	private static ForkableFlinkMiniCluster cluster;
+	private static LocalFlinkMiniCluster cluster;
 
 	private static int port = -1;
 	
@@ -86,7 +86,7 @@ public class WebFrontendITCase extends TestLogger {
 		config.setString(ConfigConstants.JOB_MANAGER_WEB_LOG_PATH_KEY, logFile.getAbsolutePath());
 		config.setString(ConfigConstants.TASK_MANAGER_LOG_PATH_KEY, logFile.getAbsolutePath());
 
-		cluster = new ForkableFlinkMiniCluster(config, false);
+		cluster = new LocalFlinkMiniCluster(config, false);
 		cluster.start();
 		
 		port = cluster.webMonitor().get().getServerPort();

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/jobmanager/JobManagerFailsITCase.scala
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/jobmanager/JobManagerFailsITCase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/jobmanager/JobManagerFailsITCase.scala
index ac661f3..1b2838d 100644
--- a/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/jobmanager/JobManagerFailsITCase.scala
+++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/jobmanager/JobManagerFailsITCase.scala
@@ -21,7 +21,6 @@ package org.apache.flink.api.scala.runtime.jobmanager
 import akka.actor.{ActorSystem, PoisonPill}
 import akka.testkit.{ImplicitSender, TestKit}
 import org.apache.flink.configuration.{ConfigConstants, Configuration}
-import org.apache.flink.api.common.{ExecutionConfig, ExecutionConfigTest}
 import org.apache.flink.runtime.akka.{AkkaUtils, ListeningBehaviour}
 import org.apache.flink.runtime.jobgraph.{JobGraph, JobVertex}
 import org.apache.flink.runtime.jobmanager.Tasks.{BlockingNoOpInvokable, NoOpInvokable}
@@ -30,8 +29,7 @@ import org.apache.flink.runtime.messages.Messages.Acknowledge
 import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages.NotifyWhenAtLeastNumTaskManagerAreRegistered
 import org.apache.flink.runtime.testingUtils.TestingMessages.DisableDisconnect
 import org.apache.flink.runtime.testingUtils.TestingTaskManagerMessages.{JobManagerTerminated, NotifyWhenJobManagerTerminated}
-import org.apache.flink.runtime.testingUtils.{ScalaTestingUtils, TestingUtils}
-import org.apache.flink.test.util.ForkableFlinkMiniCluster
+import org.apache.flink.runtime.testingUtils.{ScalaTestingUtils, TestingCluster, TestingUtils}
 import org.junit.runner.RunWith
 import org.scalatest.junit.JUnitRunner
 import org.scalatest.{BeforeAndAfterAll, Matchers, WordSpecLike}
@@ -140,12 +138,12 @@ class JobManagerFailsITCase(_system: ActorSystem)
     }
   }
 
-  def startDeathwatchCluster(numSlots: Int, numTaskmanagers: Int): ForkableFlinkMiniCluster = {
+  def startDeathwatchCluster(numSlots: Int, numTaskmanagers: Int): TestingCluster = {
     val config = new Configuration()
     config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, numSlots)
     config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, numTaskmanagers)
 
-    val cluster = new ForkableFlinkMiniCluster(config, singleActorSystem = false)
+    val cluster = new TestingCluster(config, singleActorSystem = false)
 
     cluster.start()
 

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/taskmanager/TaskManagerFailsITCase.scala
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/taskmanager/TaskManagerFailsITCase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/taskmanager/TaskManagerFailsITCase.scala
index 258f6df..3b39b3f 100644
--- a/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/taskmanager/TaskManagerFailsITCase.scala
+++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/taskmanager/TaskManagerFailsITCase.scala
@@ -20,7 +20,6 @@ package org.apache.flink.api.scala.runtime.taskmanager
 
 import akka.actor.{ActorSystem, Kill, PoisonPill}
 import akka.testkit.{ImplicitSender, TestKit}
-import org.apache.flink.api.common.{ExecutionConfig, ExecutionConfigTest}
 import org.apache.flink.configuration.ConfigConstants
 import org.apache.flink.configuration.Configuration
 import org.apache.flink.runtime.akka.{AkkaUtils, ListeningBehaviour}
@@ -31,8 +30,7 @@ import org.apache.flink.runtime.messages.JobManagerMessages._
 import org.apache.flink.runtime.messages.TaskManagerMessages.{NotifyWhenRegisteredAtJobManager, RegisteredAtJobManager}
 import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages._
 import org.apache.flink.runtime.testingUtils.TestingMessages.DisableDisconnect
-import org.apache.flink.runtime.testingUtils.{ScalaTestingUtils, TestingUtils}
-import org.apache.flink.test.util.ForkableFlinkMiniCluster
+import org.apache.flink.runtime.testingUtils.{ScalaTestingUtils, TestingCluster, TestingUtils}
 import org.junit.runner.RunWith
 import org.scalatest.junit.JUnitRunner
 import org.scalatest.{BeforeAndAfterAll, Matchers, WordSpecLike}
@@ -100,7 +98,7 @@ class TaskManagerFailsITCase(_system: ActorSystem)
       val jobGraph = new JobGraph("Pointwise Job", sender, receiver)
       val jobID = jobGraph.getJobID
 
-      val cluster = ForkableFlinkMiniCluster.startCluster(num_tasks, 2)
+      val cluster = TestingUtils.startTestingCluster(num_tasks, 2)
 
       val jmGateway = cluster.getLeaderGateway(TestingUtils.TESTING_DURATION)
 
@@ -152,7 +150,7 @@ class TaskManagerFailsITCase(_system: ActorSystem)
       val jobGraph = new JobGraph("Pointwise Job", sender, receiver)
       val jobID = jobGraph.getJobID
 
-      val cluster = ForkableFlinkMiniCluster.startCluster(num_tasks, 2)
+      val cluster = TestingUtils.startTestingCluster(num_tasks, 2)
 
       val taskManagers = cluster.getTaskManagers
       val jmGateway = cluster.getLeaderGateway(TestingUtils.TESTING_DURATION)
@@ -239,11 +237,11 @@ class TaskManagerFailsITCase(_system: ActorSystem)
     }
   }
 
-  def createDeathwatchCluster(numSlots: Int, numTaskmanagers: Int): ForkableFlinkMiniCluster = {
+  def createDeathwatchCluster(numSlots: Int, numTaskmanagers: Int): TestingCluster = {
     val config = new Configuration()
     config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, numSlots)
     config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, numTaskmanagers)
 
-    new ForkableFlinkMiniCluster(config, singleActorSystem = false)
+    new TestingCluster(config, singleActorSystem = false)
   }
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/flink-yarn-tests/pom.xml
----------------------------------------------------------------------
diff --git a/flink-yarn-tests/pom.xml b/flink-yarn-tests/pom.xml
index 8c211ef..ffdca36 100644
--- a/flink-yarn-tests/pom.xml
+++ b/flink-yarn-tests/pom.xml
@@ -48,6 +48,14 @@ under the License.
 			<scope>test</scope>
 		</dependency>
 
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-runtime_2.10</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
+			<type>test-jar</type>
+		</dependency>
+
 		<!-- Needed for the streaming wordcount example -->
 		<dependency>
 			<groupId>org.apache.flink</groupId>

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java
----------------------------------------------------------------------
diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java
index 0243012..31a3d98 100644
--- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java
+++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java
@@ -48,7 +48,6 @@ import org.junit.BeforeClass;
 import org.junit.ClassRule;
 import org.junit.rules.TemporaryFolder;
 import org.mockito.Mockito;
-import org.mockito.verification.VerificationMode;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.Marker;

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/tools/maven/scalastyle-config.xml
----------------------------------------------------------------------
diff --git a/tools/maven/scalastyle-config.xml b/tools/maven/scalastyle-config.xml
index f7bb0d4..0f7f6bb 100644
--- a/tools/maven/scalastyle-config.xml
+++ b/tools/maven/scalastyle-config.xml
@@ -86,7 +86,7 @@
  <!-- </check> -->
  <check level="error" class="org.scalastyle.scalariform.ParameterNumberChecker" enabled="true">
   <parameters>
-   <parameter name="maxParameters"><![CDATA[10]]></parameter>
+   <parameter name="maxParameters"><![CDATA[15]]></parameter>
   </parameters>
  </check>
  <!-- <check level="error" class="org.scalastyle.scalariform.MagicNumberChecker" enabled="true"> -->


[30/50] [abbrv] flink git commit: [FLINK-4414] [cluster] Add getAddress method to RpcGateway

Posted by tr...@apache.org.
[FLINK-4414] [cluster] Add getAddress method to RpcGateway

The RpcGateway.getAddress method allows to retrieve the fully qualified address of the
associated RpcEndpoint.

This closes #2392.


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

Branch: refs/heads/flip-6
Commit: 6c9f8844952b91f9164aee4ebc75481c8f2eafef
Parents: a7e3579
Author: Till Rohrmann <tr...@apache.org>
Authored: Thu Aug 18 16:34:47 2016 +0200
Committer: Till Rohrmann <tr...@apache.org>
Committed: Thu Sep 8 17:26:58 2016 +0200

----------------------------------------------------------------------
 .../apache/flink/runtime/rpc/RpcEndpoint.java   |  6 +-----
 .../apache/flink/runtime/rpc/RpcGateway.java    |  7 +++++++
 .../apache/flink/runtime/rpc/RpcService.java    | 11 ----------
 .../runtime/rpc/akka/AkkaInvocationHandler.java | 14 +++++++++++--
 .../flink/runtime/rpc/akka/AkkaRpcService.java  | 21 ++++++--------------
 .../runtime/rpc/akka/AkkaRpcActorTest.java      | 16 +++++++++++++++
 6 files changed, 42 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/6c9f8844/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java
index a28bc14..7b3f8a1 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java
@@ -69,9 +69,6 @@ public abstract class RpcEndpoint<C extends RpcGateway> {
 	/** Self gateway which can be used to schedule asynchronous calls on yourself */
 	private final C self;
 
-	/** the fully qualified address of the this RPC endpoint */
-	private final String selfAddress;
-
 	/** The main thread execution context to be used to execute future callbacks in the main thread
 	 * of the executing rpc server. */
 	private final ExecutionContext mainThreadExecutionContext;
@@ -92,7 +89,6 @@ public abstract class RpcEndpoint<C extends RpcGateway> {
 		this.selfGatewayType = ReflectionUtil.getTemplateType1(getClass());
 		this.self = rpcService.startServer(this);
 		
-		this.selfAddress = rpcService.getAddress(self);
 		this.mainThreadExecutionContext = new MainThreadExecutionContext((MainThreadExecutor) self);
 	}
 
@@ -156,7 +152,7 @@ public abstract class RpcEndpoint<C extends RpcGateway> {
 	 * @return Fully qualified address of the underlying RPC endpoint
 	 */
 	public String getAddress() {
-		return selfAddress;
+		return self.getAddress();
 	}
 
 	/**

http://git-wip-us.apache.org/repos/asf/flink/blob/6c9f8844/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcGateway.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcGateway.java
index e3a16b4..81075ee 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcGateway.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcGateway.java
@@ -22,4 +22,11 @@ package org.apache.flink.runtime.rpc;
  * Rpc gateway interface which has to be implemented by Rpc gateways.
  */
 public interface RpcGateway {
+
+	/**
+	 * Returns the fully qualified address under which the associated rpc endpoint is reachable.
+	 *
+	 * @return Fully qualified address under which the associated rpc endpoint is reachable
+	 */
+	String getAddress();
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/6c9f8844/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java
index fabdb05..bc0f5cb 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java
@@ -65,17 +65,6 @@ public interface RpcService {
 	void stopService();
 
 	/**
-	 * Get the fully qualified address of the underlying rpc server represented by the self gateway.
-	 * It must be possible to connect from a remote host to the rpc server via the returned fully
-	 * qualified address.
-	 *
-	 * @param selfGateway Self gateway associated with the underlying rpc server
-	 * @param <C> Type of the rpc gateway
-	 * @return Fully qualified address
-	 */
-	<C extends RpcGateway> String getAddress(C selfGateway);
-
-	/**
 	 * Gets the execution context, provided by this RPC service. This execution
 	 * context can be used for example for the {@code onComplete(...)} or {@code onSuccess(...)}
 	 * methods of Futures.

http://git-wip-us.apache.org/repos/asf/flink/blob/6c9f8844/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaInvocationHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaInvocationHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaInvocationHandler.java
index 524bf74..bfa04f6 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaInvocationHandler.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaInvocationHandler.java
@@ -23,6 +23,7 @@ import akka.pattern.Patterns;
 import akka.util.Timeout;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.runtime.rpc.MainThreadExecutor;
+import org.apache.flink.runtime.rpc.RpcGateway;
 import org.apache.flink.runtime.rpc.RpcTimeout;
 import org.apache.flink.runtime.rpc.StartStoppable;
 import org.apache.flink.runtime.rpc.akka.messages.CallAsync;
@@ -55,6 +56,8 @@ import static org.apache.flink.util.Preconditions.checkArgument;
 class AkkaInvocationHandler implements InvocationHandler, AkkaGateway, MainThreadExecutor, StartStoppable {
 	private static final Logger LOG = Logger.getLogger(AkkaInvocationHandler.class);
 
+	private final String address;
+
 	private final ActorRef rpcEndpoint;
 
 	// whether the actor ref is local and thus no message serialization is needed
@@ -65,7 +68,8 @@ class AkkaInvocationHandler implements InvocationHandler, AkkaGateway, MainThrea
 
 	private final long maximumFramesize;
 
-	AkkaInvocationHandler(ActorRef rpcEndpoint, Timeout timeout, long maximumFramesize) {
+	AkkaInvocationHandler(String address, ActorRef rpcEndpoint, Timeout timeout, long maximumFramesize) {
+		this.address = Preconditions.checkNotNull(address);
 		this.rpcEndpoint = Preconditions.checkNotNull(rpcEndpoint);
 		this.isLocal = this.rpcEndpoint.path().address().hasLocalScope();
 		this.timeout = Preconditions.checkNotNull(timeout);
@@ -79,7 +83,8 @@ class AkkaInvocationHandler implements InvocationHandler, AkkaGateway, MainThrea
 		Object result;
 
 		if (declaringClass.equals(AkkaGateway.class) || declaringClass.equals(MainThreadExecutor.class) ||
-			declaringClass.equals(Object.class) || declaringClass.equals(StartStoppable.class)) {
+			declaringClass.equals(Object.class) || declaringClass.equals(StartStoppable.class) ||
+			declaringClass.equals(RpcGateway.class)) {
 			result = method.invoke(this, args);
 		} else {
 			String methodName = method.getName();
@@ -290,4 +295,9 @@ class AkkaInvocationHandler implements InvocationHandler, AkkaGateway, MainThrea
 
 		return false;
 	}
+
+	@Override
+	public String getAddress() {
+		return address;
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/6c9f8844/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java
index d987c2f..00a6932 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java
@@ -102,7 +102,9 @@ public class AkkaRpcService implements RpcService {
 			public C apply(Object obj) {
 				ActorRef actorRef = ((ActorIdentity) obj).getRef();
 
-				InvocationHandler akkaInvocationHandler = new AkkaInvocationHandler(actorRef, timeout, maximumFramesize);
+				final String address = AkkaUtils.getAkkaURL(actorSystem, actorRef);
+
+				InvocationHandler akkaInvocationHandler = new AkkaInvocationHandler(address, actorRef, timeout, maximumFramesize);
 
 				// Rather than using the System ClassLoader directly, we derive the ClassLoader
 				// from this class . That works better in cases where Flink runs embedded and all Flink
@@ -135,7 +137,9 @@ public class AkkaRpcService implements RpcService {
 
 		LOG.info("Starting RPC endpoint for {} at {} .", rpcEndpoint.getClass().getName(), actorRef.path());
 
-		InvocationHandler akkaInvocationHandler = new AkkaInvocationHandler(actorRef, timeout, maximumFramesize);
+		final String address = AkkaUtils.getAkkaURL(actorSystem, actorRef);
+
+		InvocationHandler akkaInvocationHandler = new AkkaInvocationHandler(address, actorRef, timeout, maximumFramesize);
 
 		// Rather than using the System ClassLoader directly, we derive the ClassLoader
 		// from this class . That works better in cases where Flink runs embedded and all Flink
@@ -197,19 +201,6 @@ public class AkkaRpcService implements RpcService {
 	}
 
 	@Override
-	public String getAddress(RpcGateway selfGateway) {
-		checkState(!stopped, "RpcService is stopped");
-
-		if (selfGateway instanceof AkkaGateway) {
-			ActorRef actorRef = ((AkkaGateway) selfGateway).getRpcEndpoint();
-			return AkkaUtils.getAkkaURL(actorSystem, actorRef);
-		} else {
-			String className = AkkaGateway.class.getName();
-			throw new IllegalArgumentException("Cannot get address for non " + className + '.');
-		}
-	}
-
-	@Override
 	public ExecutionContext getExecutionContext() {
 		return actorSystem.dispatcher();
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/6c9f8844/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java
index 1653fac..82d13f0 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java
@@ -34,6 +34,7 @@ import scala.concurrent.Future;
 
 import java.util.concurrent.TimeUnit;
 
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
 
 public class AkkaRpcActorTest extends TestLogger {
@@ -57,6 +58,21 @@ public class AkkaRpcActorTest extends TestLogger {
 	}
 
 	/**
+	 * Tests that the rpc endpoint and the associated rpc gateway have the same addresses.
+	 * @throws Exception
+	 */
+	@Test
+	public void testAddressResolution() throws Exception {
+		DummyRpcEndpoint rpcEndpoint = new DummyRpcEndpoint(akkaRpcService);
+
+		Future<DummyRpcGateway> futureRpcGateway = akkaRpcService.connect(rpcEndpoint.getAddress(), DummyRpcGateway.class);
+
+		DummyRpcGateway rpcGateway = Await.result(futureRpcGateway, timeout.duration());
+
+		assertEquals(rpcEndpoint.getAddress(), rpcGateway.getAddress());
+	}
+
+	/**
 	 * Tests that the {@link AkkaRpcActor} stashes messages until the corresponding
 	 * {@link RpcEndpoint} has been started.
 	 */


[44/50] [abbrv] flink git commit: [FLINK-4529] [flip-6] Move TaskExecutor, JobMaster and ResourceManager out of the rpc package

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/72468d14/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/SlotManagerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/SlotManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/SlotManagerTest.java
deleted file mode 100644
index 2ee280f..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/SlotManagerTest.java
+++ /dev/null
@@ -1,540 +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.clusterframework;
-
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.runtime.clusterframework.types.AllocationID;
-import org.apache.flink.runtime.clusterframework.types.ResourceID;
-import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
-import org.apache.flink.runtime.clusterframework.types.ResourceSlot;
-import org.apache.flink.runtime.clusterframework.types.SlotID;
-import org.apache.flink.runtime.rpc.resourcemanager.ResourceManagerGateway;
-import org.apache.flink.runtime.rpc.resourcemanager.SlotRequest;
-import org.apache.flink.runtime.rpc.taskexecutor.SlotStatus;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Mockito.mock;
-
-public class SlotManagerTest {
-
-	private static final double DEFAULT_TESTING_CPU_CORES = 1.0;
-
-	private static final long DEFAULT_TESTING_MEMORY = 512;
-
-	private static final ResourceProfile DEFAULT_TESTING_PROFILE =
-		new ResourceProfile(DEFAULT_TESTING_CPU_CORES, DEFAULT_TESTING_MEMORY);
-
-	private static final ResourceProfile DEFAULT_TESTING_BIG_PROFILE =
-		new ResourceProfile(DEFAULT_TESTING_CPU_CORES * 2, DEFAULT_TESTING_MEMORY * 2);
-
-	private ResourceManagerGateway resourceManagerGateway;
-
-	@Before
-	public void setUp() {
-		resourceManagerGateway = mock(ResourceManagerGateway.class);
-	}
-
-	/**
-	 * Tests that there are no free slots when we request, need to allocate from cluster manager master
-	 */
-	@Test
-	public void testRequestSlotWithoutFreeSlot() {
-		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
-		slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE));
-
-		assertEquals(0, slotManager.getAllocatedSlotCount());
-		assertEquals(0, slotManager.getFreeSlotCount());
-		assertEquals(1, slotManager.getPendingRequestCount());
-		assertEquals(1, slotManager.getAllocatedContainers().size());
-		assertEquals(DEFAULT_TESTING_PROFILE, slotManager.getAllocatedContainers().get(0));
-	}
-
-	/**
-	 * Tests that there are some free slots when we request, and the request is fulfilled immediately
-	 */
-	@Test
-	public void testRequestSlotWithFreeSlot() {
-		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
-
-		directlyProvideFreeSlots(slotManager, DEFAULT_TESTING_PROFILE, 1);
-		assertEquals(1, slotManager.getFreeSlotCount());
-
-		slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE));
-		assertEquals(1, slotManager.getAllocatedSlotCount());
-		assertEquals(0, slotManager.getFreeSlotCount());
-		assertEquals(0, slotManager.getPendingRequestCount());
-		assertEquals(0, slotManager.getAllocatedContainers().size());
-	}
-
-	/**
-	 * Tests that there are some free slots when we request, but none of them are suitable
-	 */
-	@Test
-	public void testRequestSlotWithoutSuitableSlot() {
-		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
-
-		directlyProvideFreeSlots(slotManager, DEFAULT_TESTING_PROFILE, 2);
-		assertEquals(2, slotManager.getFreeSlotCount());
-
-		slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_BIG_PROFILE));
-		assertEquals(0, slotManager.getAllocatedSlotCount());
-		assertEquals(2, slotManager.getFreeSlotCount());
-		assertEquals(1, slotManager.getPendingRequestCount());
-		assertEquals(1, slotManager.getAllocatedContainers().size());
-		assertEquals(DEFAULT_TESTING_BIG_PROFILE, slotManager.getAllocatedContainers().get(0));
-	}
-
-	/**
-	 * Tests that we send duplicated slot request
-	 */
-	@Test
-	public void testDuplicatedSlotRequest() {
-		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
-		directlyProvideFreeSlots(slotManager, DEFAULT_TESTING_PROFILE, 1);
-
-		SlotRequest request1 = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE);
-		SlotRequest request2 = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_BIG_PROFILE);
-
-		slotManager.requestSlot(request1);
-		slotManager.requestSlot(request2);
-		slotManager.requestSlot(request2);
-		slotManager.requestSlot(request1);
-
-		assertEquals(1, slotManager.getAllocatedSlotCount());
-		assertEquals(0, slotManager.getFreeSlotCount());
-		assertEquals(1, slotManager.getPendingRequestCount());
-		assertEquals(1, slotManager.getAllocatedContainers().size());
-		assertEquals(DEFAULT_TESTING_BIG_PROFILE, slotManager.getAllocatedContainers().get(0));
-	}
-
-	/**
-	 * Tests that we send multiple slot requests
-	 */
-	@Test
-	public void testRequestMultipleSlots() {
-		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
-		directlyProvideFreeSlots(slotManager, DEFAULT_TESTING_PROFILE, 5);
-
-		// request 3 normal slots
-		for (int i = 0; i < 3; ++i) {
-			slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE));
-		}
-
-		// request 2 big slots
-		for (int i = 0; i < 2; ++i) {
-			slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_BIG_PROFILE));
-		}
-
-		// request 1 normal slot again
-		slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE));
-
-		assertEquals(4, slotManager.getAllocatedSlotCount());
-		assertEquals(1, slotManager.getFreeSlotCount());
-		assertEquals(2, slotManager.getPendingRequestCount());
-		assertEquals(2, slotManager.getAllocatedContainers().size());
-		assertEquals(DEFAULT_TESTING_BIG_PROFILE, slotManager.getAllocatedContainers().get(0));
-		assertEquals(DEFAULT_TESTING_BIG_PROFILE, slotManager.getAllocatedContainers().get(1));
-	}
-
-	/**
-	 * Tests that a new slot appeared in SlotReport, and we used it to fulfill a pending request
-	 */
-	@Test
-	public void testNewlyAppearedFreeSlotFulfillPendingRequest() {
-		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
-		slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE));
-		assertEquals(1, slotManager.getPendingRequestCount());
-
-		SlotID slotId = SlotID.generate();
-		SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE);
-		slotManager.updateSlotStatus(slotStatus);
-
-		assertEquals(1, slotManager.getAllocatedSlotCount());
-		assertEquals(0, slotManager.getFreeSlotCount());
-		assertEquals(0, slotManager.getPendingRequestCount());
-		assertTrue(slotManager.isAllocated(slotId));
-	}
-
-	/**
-	 * Tests that a new slot appeared in SlotReport, but we have no pending request
-	 */
-	@Test
-	public void testNewlyAppearedFreeSlot() {
-		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
-		SlotID slotId = SlotID.generate();
-		SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE);
-		slotManager.updateSlotStatus(slotStatus);
-
-		assertEquals(0, slotManager.getAllocatedSlotCount());
-		assertEquals(1, slotManager.getFreeSlotCount());
-	}
-
-	/**
-	 * Tests that a new slot appeared in SlotReport, but it't not suitable for all the pending requests
-	 */
-	@Test
-	public void testNewlyAppearedFreeSlotNotMatchPendingRequests() {
-		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
-		slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_BIG_PROFILE));
-		assertEquals(1, slotManager.getPendingRequestCount());
-
-		SlotID slotId = SlotID.generate();
-		SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE);
-		slotManager.updateSlotStatus(slotStatus);
-
-		assertEquals(0, slotManager.getAllocatedSlotCount());
-		assertEquals(1, slotManager.getFreeSlotCount());
-		assertEquals(1, slotManager.getPendingRequestCount());
-		assertFalse(slotManager.isAllocated(slotId));
-	}
-
-	/**
-	 * Tests that a new slot appeared in SlotReport, and it's been reported using by some job
-	 */
-	@Test
-	public void testNewlyAppearedInUseSlot() {
-		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
-
-		SlotID slotId = SlotID.generate();
-		SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE, new AllocationID(), new JobID());
-		slotManager.updateSlotStatus(slotStatus);
-
-		assertEquals(1, slotManager.getAllocatedSlotCount());
-		assertEquals(0, slotManager.getFreeSlotCount());
-		assertTrue(slotManager.isAllocated(slotId));
-	}
-
-	/**
-	 * Tests that we had a slot in-use, and it's confirmed by SlotReport
-	 */
-	@Test
-	public void testExistingInUseSlotUpdateStatus() {
-		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
-		SlotRequest request = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE);
-		slotManager.requestSlot(request);
-
-		// make this slot in use
-		SlotID slotId = SlotID.generate();
-		SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE);
-		slotManager.updateSlotStatus(slotStatus);
-
-		assertEquals(1, slotManager.getAllocatedSlotCount());
-		assertEquals(0, slotManager.getFreeSlotCount());
-		assertTrue(slotManager.isAllocated(slotId));
-
-		// slot status is confirmed
-		SlotStatus slotStatus2 = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE,
-			request.getAllocationId(), request.getJobId());
-		slotManager.updateSlotStatus(slotStatus2);
-
-		assertEquals(1, slotManager.getAllocatedSlotCount());
-		assertEquals(0, slotManager.getFreeSlotCount());
-		assertTrue(slotManager.isAllocated(slotId));
-	}
-
-	/**
-	 * Tests that we had a slot in-use, but it's empty according to the SlotReport
-	 */
-	@Test
-	public void testExistingInUseSlotAdjustedToEmpty() {
-		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
-		SlotRequest request1 = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE);
-		slotManager.requestSlot(request1);
-
-		// make this slot in use
-		SlotID slotId = SlotID.generate();
-		SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE);
-		slotManager.updateSlotStatus(slotStatus);
-
-		// another request pending
-		SlotRequest request2 = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE);
-		slotManager.requestSlot(request2);
-
-		assertEquals(1, slotManager.getAllocatedSlotCount());
-		assertEquals(0, slotManager.getFreeSlotCount());
-		assertEquals(1, slotManager.getPendingRequestCount());
-		assertTrue(slotManager.isAllocated(slotId));
-		assertTrue(slotManager.isAllocated(request1.getAllocationId()));
-
-
-		// but slot is reported empty again, request2 will be fulfilled, request1 will be missing
-		slotManager.updateSlotStatus(slotStatus);
-
-		assertEquals(1, slotManager.getAllocatedSlotCount());
-		assertEquals(0, slotManager.getFreeSlotCount());
-		assertEquals(0, slotManager.getPendingRequestCount());
-		assertTrue(slotManager.isAllocated(slotId));
-		assertTrue(slotManager.isAllocated(request2.getAllocationId()));
-	}
-
-	/**
-	 * Tests that we had a slot in use, and it's also reported in use by TaskManager, but the allocation
-	 * information didn't match.
-	 */
-	@Test
-	public void testExistingInUseSlotWithDifferentAllocationInfo() {
-		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
-		SlotRequest request = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE);
-		slotManager.requestSlot(request);
-
-		// make this slot in use
-		SlotID slotId = SlotID.generate();
-		SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE);
-		slotManager.updateSlotStatus(slotStatus);
-
-		assertEquals(1, slotManager.getAllocatedSlotCount());
-		assertEquals(0, slotManager.getFreeSlotCount());
-		assertEquals(0, slotManager.getPendingRequestCount());
-		assertTrue(slotManager.isAllocated(slotId));
-		assertTrue(slotManager.isAllocated(request.getAllocationId()));
-
-		SlotStatus slotStatus2 = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE, new AllocationID(), new JobID());
-		// update slot status with different allocation info
-		slotManager.updateSlotStatus(slotStatus2);
-
-		// original request is missing and won't be allocated
-		assertEquals(1, slotManager.getAllocatedSlotCount());
-		assertEquals(0, slotManager.getFreeSlotCount());
-		assertEquals(0, slotManager.getPendingRequestCount());
-		assertTrue(slotManager.isAllocated(slotId));
-		assertFalse(slotManager.isAllocated(request.getAllocationId()));
-		assertTrue(slotManager.isAllocated(slotStatus2.getAllocationID()));
-	}
-
-	/**
-	 * Tests that we had a free slot, and it's confirmed by SlotReport
-	 */
-	@Test
-	public void testExistingEmptySlotUpdateStatus() {
-		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
-		ResourceSlot slot = new ResourceSlot(SlotID.generate(), DEFAULT_TESTING_PROFILE);
-		slotManager.addFreeSlot(slot);
-
-		SlotStatus slotStatus = new SlotStatus(slot.getSlotId(), DEFAULT_TESTING_PROFILE);
-		slotManager.updateSlotStatus(slotStatus);
-
-		assertEquals(0, slotManager.getAllocatedSlotCount());
-		assertEquals(1, slotManager.getFreeSlotCount());
-		assertEquals(0, slotManager.getPendingRequestCount());
-	}
-
-	/**
-	 * Tests that we had a free slot, and it's reported in-use by TaskManager
-	 */
-	@Test
-	public void testExistingEmptySlotAdjustedToInUse() {
-		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
-		ResourceSlot slot = new ResourceSlot(SlotID.generate(), DEFAULT_TESTING_PROFILE);
-		slotManager.addFreeSlot(slot);
-
-		SlotStatus slotStatus = new SlotStatus(slot.getSlotId(), DEFAULT_TESTING_PROFILE,
-			new AllocationID(), new JobID());
-		slotManager.updateSlotStatus(slotStatus);
-
-		assertEquals(1, slotManager.getAllocatedSlotCount());
-		assertEquals(0, slotManager.getFreeSlotCount());
-		assertEquals(0, slotManager.getPendingRequestCount());
-		assertTrue(slotManager.isAllocated(slot.getSlotId()));
-	}
-
-	/**
-	 * Tests that we did some allocation but failed / rejected by TaskManager, request will retry
-	 */
-	@Test
-	public void testSlotAllocationFailedAtTaskManager() {
-		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
-		ResourceSlot slot = new ResourceSlot(SlotID.generate(), DEFAULT_TESTING_PROFILE);
-		slotManager.addFreeSlot(slot);
-
-		SlotRequest request = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE);
-		slotManager.requestSlot(request);
-
-		assertEquals(1, slotManager.getAllocatedSlotCount());
-		assertEquals(0, slotManager.getFreeSlotCount());
-		assertEquals(0, slotManager.getPendingRequestCount());
-		assertTrue(slotManager.isAllocated(slot.getSlotId()));
-
-		slotManager.handleSlotRequestFailedAtTaskManager(request, slot.getSlotId());
-
-		assertEquals(1, slotManager.getAllocatedSlotCount());
-		assertEquals(0, slotManager.getFreeSlotCount());
-		assertEquals(0, slotManager.getPendingRequestCount());
-	}
-
-
-	/**
-	 * Tests that we did some allocation but failed / rejected by TaskManager, and slot is occupied by another request
-	 */
-	@Test
-	public void testSlotAllocationFailedAtTaskManagerOccupiedByOther() {
-		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
-		ResourceSlot slot = new ResourceSlot(SlotID.generate(), DEFAULT_TESTING_PROFILE);
-		slotManager.addFreeSlot(slot);
-
-		SlotRequest request = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE);
-		slotManager.requestSlot(request);
-
-		// slot is set empty by heartbeat
-		SlotStatus slotStatus = new SlotStatus(slot.getSlotId(), slot.getResourceProfile());
-		slotManager.updateSlotStatus(slotStatus);
-
-		// another request took this slot
-		SlotRequest request2 = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE);
-		slotManager.requestSlot(request2);
-
-		assertEquals(1, slotManager.getAllocatedSlotCount());
-		assertEquals(0, slotManager.getFreeSlotCount());
-		assertEquals(0, slotManager.getPendingRequestCount());
-		assertFalse(slotManager.isAllocated(request.getAllocationId()));
-		assertTrue(slotManager.isAllocated(request2.getAllocationId()));
-
-		// original request should be pended
-		slotManager.handleSlotRequestFailedAtTaskManager(request, slot.getSlotId());
-
-		assertEquals(1, slotManager.getAllocatedSlotCount());
-		assertEquals(0, slotManager.getFreeSlotCount());
-		assertEquals(1, slotManager.getPendingRequestCount());
-		assertFalse(slotManager.isAllocated(request.getAllocationId()));
-		assertTrue(slotManager.isAllocated(request2.getAllocationId()));
-	}
-
-	@Test
-	public void testNotifyTaskManagerFailure() {
-		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
-
-		ResourceID resource1 = ResourceID.generate();
-		ResourceID resource2 = ResourceID.generate();
-
-		ResourceSlot slot11 = new ResourceSlot(new SlotID(resource1, 1), DEFAULT_TESTING_PROFILE);
-		ResourceSlot slot12 = new ResourceSlot(new SlotID(resource1, 2), DEFAULT_TESTING_PROFILE);
-		ResourceSlot slot21 = new ResourceSlot(new SlotID(resource2, 1), DEFAULT_TESTING_PROFILE);
-		ResourceSlot slot22 = new ResourceSlot(new SlotID(resource2, 2), DEFAULT_TESTING_PROFILE);
-
-		slotManager.addFreeSlot(slot11);
-		slotManager.addFreeSlot(slot21);
-
-		slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE));
-		slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE));
-
-		assertEquals(2, slotManager.getAllocatedSlotCount());
-		assertEquals(0, slotManager.getFreeSlotCount());
-		assertEquals(0, slotManager.getPendingRequestCount());
-
-		slotManager.addFreeSlot(slot12);
-		slotManager.addFreeSlot(slot22);
-
-		assertEquals(2, slotManager.getAllocatedSlotCount());
-		assertEquals(2, slotManager.getFreeSlotCount());
-		assertEquals(0, slotManager.getPendingRequestCount());
-
-		slotManager.notifyTaskManagerFailure(resource2);
-
-		assertEquals(1, slotManager.getAllocatedSlotCount());
-		assertEquals(1, slotManager.getFreeSlotCount());
-		assertEquals(0, slotManager.getPendingRequestCount());
-
-		// notify an not exist resource failure
-		slotManager.notifyTaskManagerFailure(ResourceID.generate());
-
-		assertEquals(1, slotManager.getAllocatedSlotCount());
-		assertEquals(1, slotManager.getFreeSlotCount());
-		assertEquals(0, slotManager.getPendingRequestCount());
-	}
-
-	// ------------------------------------------------------------------------
-	//  testing utilities
-	// ------------------------------------------------------------------------
-
-	private void directlyProvideFreeSlots(
-		final SlotManager slotManager,
-		final ResourceProfile resourceProfile,
-		final int freeSlotNum)
-	{
-		for (int i = 0; i < freeSlotNum; ++i) {
-			slotManager.addFreeSlot(new ResourceSlot(SlotID.generate(), new ResourceProfile(resourceProfile)));
-		}
-	}
-
-	// ------------------------------------------------------------------------
-	//  testing classes
-	// ------------------------------------------------------------------------
-
-	private static class TestingSlotManager extends SlotManager {
-
-		private final List<ResourceProfile> allocatedContainers;
-
-		TestingSlotManager(ResourceManagerGateway resourceManagerGateway) {
-			super(resourceManagerGateway);
-			this.allocatedContainers = new LinkedList<>();
-		}
-
-		/**
-		 * Choose slot randomly if it matches requirement
-		 *
-		 * @param request   The slot request
-		 * @param freeSlots All slots which can be used
-		 * @return The chosen slot or null if cannot find a match
-		 */
-		@Override
-		protected ResourceSlot chooseSlotToUse(SlotRequest request, Map<SlotID, ResourceSlot> freeSlots) {
-			for (ResourceSlot slot : freeSlots.values()) {
-				if (slot.isMatchingRequirement(request.getResourceProfile())) {
-					return slot;
-				}
-			}
-			return null;
-		}
-
-		/**
-		 * Choose request randomly if offered slot can match its requirement
-		 *
-		 * @param offeredSlot     The free slot
-		 * @param pendingRequests All the pending slot requests
-		 * @return The chosen request's AllocationID or null if cannot find a match
-		 */
-		@Override
-		protected SlotRequest chooseRequestToFulfill(ResourceSlot offeredSlot,
-			Map<AllocationID, SlotRequest> pendingRequests)
-		{
-			for (Map.Entry<AllocationID, SlotRequest> pendingRequest : pendingRequests.entrySet()) {
-				if (offeredSlot.isMatchingRequirement(pendingRequest.getValue().getResourceProfile())) {
-					return pendingRequest.getValue();
-				}
-			}
-			return null;
-		}
-
-		@Override
-		protected void allocateContainer(ResourceProfile resourceProfile) {
-			allocatedContainers.add(resourceProfile);
-		}
-
-		List<ResourceProfile> getAllocatedContainers() {
-			return allocatedContainers;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/72468d14/flink-runtime/src/test/java/org/apache/flink/runtime/registration/RetryingRegistrationTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/registration/RetryingRegistrationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/registration/RetryingRegistrationTest.java
new file mode 100644
index 0000000..80fa19c
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/registration/RetryingRegistrationTest.java
@@ -0,0 +1,336 @@
+/*
+ * 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.registration;
+
+import akka.dispatch.Futures;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.runtime.rpc.RpcService;
+import org.apache.flink.runtime.rpc.TestingRpcService;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Test;
+
+import org.slf4j.LoggerFactory;
+
+import scala.concurrent.Await;
+import scala.concurrent.ExecutionContext$;
+import scala.concurrent.Future;
+import scala.concurrent.Promise;
+import scala.concurrent.duration.FiniteDuration;
+
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeoutException;
+
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
+
+/**
+ * Tests for the generic retrying registration class, validating the failure, retry, and back-off behavior.
+ */
+public class RetryingRegistrationTest extends TestLogger {
+
+	@Test
+	public void testSimpleSuccessfulRegistration() throws Exception {
+		final String testId = "laissez les bon temps roulez";
+		final String testEndpointAddress = "<test-address>";
+		final UUID leaderId = UUID.randomUUID();
+
+		// an endpoint that immediately returns success
+		TestRegistrationGateway testGateway = new TestRegistrationGateway(new TestRegistrationSuccess(testId));
+		TestingRpcService rpc = new TestingRpcService();
+
+		try {
+			rpc.registerGateway(testEndpointAddress, testGateway);
+
+			TestRetryingRegistration registration = new TestRetryingRegistration(rpc, testEndpointAddress, leaderId);
+			registration.startRegistration();
+
+			Future<Tuple2<TestRegistrationGateway, TestRegistrationSuccess>> future = registration.getFuture();
+			assertNotNull(future);
+
+			// multiple accesses return the same future
+			assertEquals(future, registration.getFuture());
+
+			Tuple2<TestRegistrationGateway, TestRegistrationSuccess> success = 
+					Await.result(future, new FiniteDuration(10, SECONDS));
+
+			// validate correct invocation and result
+			assertEquals(testId, success.f1.getCorrelationId());
+			assertEquals(leaderId, testGateway.getInvocations().take().leaderId());
+		}
+		finally {
+			testGateway.stop();
+			rpc.stopService();
+		}
+	}
+	
+	@Test
+	public void testPropagateFailures() throws Exception {
+		final String testExceptionMessage = "testExceptionMessage";
+
+		// RPC service that fails with exception upon the connection
+		RpcService rpc = mock(RpcService.class);
+		when(rpc.connect(anyString(), any(Class.class))).thenThrow(new RuntimeException(testExceptionMessage));
+
+		TestRetryingRegistration registration = new TestRetryingRegistration(rpc, "testaddress", UUID.randomUUID());
+		registration.startRegistration();
+
+		Future<?> future = registration.getFuture();
+		assertTrue(future.failed().isCompleted());
+
+		assertEquals(testExceptionMessage, future.failed().value().get().get().getMessage());
+	}
+
+	@Test
+	public void testRetryConnectOnFailure() throws Exception {
+		final String testId = "laissez les bon temps roulez";
+		final UUID leaderId = UUID.randomUUID();
+
+		ExecutorService executor = Executors.newCachedThreadPool();
+		TestRegistrationGateway testGateway = new TestRegistrationGateway(new TestRegistrationSuccess(testId));
+
+		try {
+			// RPC service that fails upon the first connection, but succeeds on the second
+			RpcService rpc = mock(RpcService.class);
+			when(rpc.connect(anyString(), any(Class.class))).thenReturn(
+					Futures.failed(new Exception("test connect failure")),  // first connection attempt fails
+					Futures.successful(testGateway)                         // second connection attempt succeeds
+			);
+			when(rpc.getExecutionContext()).thenReturn(ExecutionContext$.MODULE$.fromExecutor(executor));
+
+			TestRetryingRegistration registration = new TestRetryingRegistration(rpc, "foobar address", leaderId);
+			registration.startRegistration();
+
+			Tuple2<TestRegistrationGateway, TestRegistrationSuccess> success =
+					Await.result(registration.getFuture(), new FiniteDuration(10, SECONDS));
+
+			// validate correct invocation and result
+			assertEquals(testId, success.f1.getCorrelationId());
+			assertEquals(leaderId, testGateway.getInvocations().take().leaderId());
+		}
+		finally {
+			testGateway.stop();
+			executor.shutdown();
+		}
+	}
+
+	@Test
+	public void testRetriesOnTimeouts() throws Exception {
+		final String testId = "rien ne va plus";
+		final String testEndpointAddress = "<test-address>";
+		final UUID leaderId = UUID.randomUUID();
+
+		// an endpoint that immediately returns futures with timeouts before returning a successful future
+		TestRegistrationGateway testGateway = new TestRegistrationGateway(
+				null, // timeout
+				null, // timeout
+				new TestRegistrationSuccess(testId) // success
+		);
+
+		TestingRpcService rpc = new TestingRpcService();
+
+		try {
+			rpc.registerGateway(testEndpointAddress, testGateway);
+	
+			TestRetryingRegistration registration = new TestRetryingRegistration(rpc, testEndpointAddress, leaderId);
+	
+			long started = System.nanoTime();
+			registration.startRegistration();
+	
+			Future<Tuple2<TestRegistrationGateway, TestRegistrationSuccess>> future = registration.getFuture();
+			Tuple2<TestRegistrationGateway, TestRegistrationSuccess> success =
+					Await.result(future, new FiniteDuration(10, SECONDS));
+	
+			long finished = System.nanoTime();
+			long elapsedMillis = (finished - started) / 1000000;
+	
+			// validate correct invocation and result
+			assertEquals(testId, success.f1.getCorrelationId());
+			assertEquals(leaderId, testGateway.getInvocations().take().leaderId());
+	
+			// validate that some retry-delay / back-off behavior happened
+			assertTrue("retries did not properly back off", elapsedMillis >= 3 * TestRetryingRegistration.INITIAL_TIMEOUT);
+		}
+		finally {
+			rpc.stopService();
+			testGateway.stop();
+		}
+	}
+
+	@Test
+	public void testDecline() throws Exception {
+		final String testId = "qui a coupe le fromage";
+		final String testEndpointAddress = "<test-address>";
+		final UUID leaderId = UUID.randomUUID();
+
+		TestingRpcService rpc = new TestingRpcService();
+
+		TestRegistrationGateway testGateway = new TestRegistrationGateway(
+				null, // timeout
+				new RegistrationResponse.Decline("no reason "),
+				null, // timeout
+				new TestRegistrationSuccess(testId) // success
+		);
+
+		try {
+			rpc.registerGateway(testEndpointAddress, testGateway);
+
+			TestRetryingRegistration registration = new TestRetryingRegistration(rpc, testEndpointAddress, leaderId);
+
+			long started = System.nanoTime();
+			registration.startRegistration();
+	
+			Future<Tuple2<TestRegistrationGateway, TestRegistrationSuccess>> future = registration.getFuture();
+			Tuple2<TestRegistrationGateway, TestRegistrationSuccess> success =
+					Await.result(future, new FiniteDuration(10, SECONDS));
+
+			long finished = System.nanoTime();
+			long elapsedMillis = (finished - started) / 1000000;
+
+			// validate correct invocation and result
+			assertEquals(testId, success.f1.getCorrelationId());
+			assertEquals(leaderId, testGateway.getInvocations().take().leaderId());
+
+			// validate that some retry-delay / back-off behavior happened
+			assertTrue("retries did not properly back off", elapsedMillis >= 
+					2 * TestRetryingRegistration.INITIAL_TIMEOUT + TestRetryingRegistration.DELAY_ON_DECLINE);
+		}
+		finally {
+			testGateway.stop();
+			rpc.stopService();
+		}
+	}
+	
+	@Test
+	@SuppressWarnings("unchecked")
+	public void testRetryOnError() throws Exception {
+		final String testId = "Petit a petit, l'oiseau fait son nid";
+		final String testEndpointAddress = "<test-address>";
+		final UUID leaderId = UUID.randomUUID();
+
+		TestingRpcService rpc = new TestingRpcService();
+
+		try {
+			// gateway that upon calls first responds with a failure, then with a success
+			TestRegistrationGateway testGateway = mock(TestRegistrationGateway.class);
+
+			when(testGateway.registrationCall(any(UUID.class), anyLong())).thenReturn(
+					Futures.<RegistrationResponse>failed(new Exception("test exception")),
+					Futures.<RegistrationResponse>successful(new TestRegistrationSuccess(testId)));
+			
+			rpc.registerGateway(testEndpointAddress, testGateway);
+
+			TestRetryingRegistration registration = new TestRetryingRegistration(rpc, testEndpointAddress, leaderId);
+
+			long started = System.nanoTime();
+			registration.startRegistration();
+
+			Future<Tuple2<TestRegistrationGateway, TestRegistrationSuccess>> future = registration.getFuture();
+			Tuple2<TestRegistrationGateway, TestRegistrationSuccess> success =
+					Await.result(future, new FiniteDuration(10, SECONDS));
+
+			long finished = System.nanoTime();
+			long elapsedMillis = (finished - started) / 1000000;
+			
+			assertEquals(testId, success.f1.getCorrelationId());
+
+			// validate that some retry-delay / back-off behavior happened
+			assertTrue("retries did not properly back off",
+					elapsedMillis >= TestRetryingRegistration.DELAY_ON_ERROR);
+		}
+		finally {
+			rpc.stopService();
+		}
+	}
+
+	@Test
+	public void testCancellation() throws Exception {
+		final String testEndpointAddress = "my-test-address";
+		final UUID leaderId = UUID.randomUUID();
+
+		TestingRpcService rpc = new TestingRpcService();
+
+		try {
+			Promise<RegistrationResponse> result = Futures.promise();
+
+			TestRegistrationGateway testGateway = mock(TestRegistrationGateway.class);
+			when(testGateway.registrationCall(any(UUID.class), anyLong())).thenReturn(result.future());
+
+			rpc.registerGateway(testEndpointAddress, testGateway);
+
+			TestRetryingRegistration registration = new TestRetryingRegistration(rpc, testEndpointAddress, leaderId);
+			registration.startRegistration();
+
+			// cancel and fail the current registration attempt
+			registration.cancel();
+			result.failure(new TimeoutException());
+
+			// there should not be a second registration attempt
+			verify(testGateway, atMost(1)).registrationCall(any(UUID.class), anyLong());
+		}
+		finally {
+			rpc.stopService();
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  test registration
+	// ------------------------------------------------------------------------
+
+	private static class TestRegistrationSuccess extends RegistrationResponse.Success {
+		private static final long serialVersionUID = 5542698790917150604L;
+
+		private final String correlationId;
+
+		private TestRegistrationSuccess(String correlationId) {
+			this.correlationId = correlationId;
+		}
+
+		public String getCorrelationId() {
+			return correlationId;
+		}
+	}
+
+	private static class TestRetryingRegistration extends RetryingRegistration<TestRegistrationGateway, TestRegistrationSuccess> {
+
+		// we use shorter timeouts here to speed up the tests
+		static final long INITIAL_TIMEOUT = 20;
+		static final long MAX_TIMEOUT = 200;
+		static final long DELAY_ON_ERROR = 200;
+		static final long DELAY_ON_DECLINE = 200;
+
+		public TestRetryingRegistration(RpcService rpc, String targetAddress, UUID leaderId) {
+			super(LoggerFactory.getLogger(RetryingRegistrationTest.class),
+					rpc, "TestEndpoint",
+					TestRegistrationGateway.class,
+					targetAddress, leaderId,
+					INITIAL_TIMEOUT, MAX_TIMEOUT, DELAY_ON_ERROR, DELAY_ON_DECLINE);
+		}
+
+		@Override
+		protected Future<RegistrationResponse> invokeRegistration(
+				TestRegistrationGateway gateway, UUID leaderId, long timeoutMillis) {
+			return gateway.registrationCall(leaderId, timeoutMillis);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/72468d14/flink-runtime/src/test/java/org/apache/flink/runtime/registration/TestRegistrationGateway.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/registration/TestRegistrationGateway.java b/flink-runtime/src/test/java/org/apache/flink/runtime/registration/TestRegistrationGateway.java
new file mode 100644
index 0000000..431fbe8
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/registration/TestRegistrationGateway.java
@@ -0,0 +1,85 @@
+/*
+ * 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.registration;
+
+import akka.dispatch.Futures;
+
+import org.apache.flink.runtime.rpc.TestingGatewayBase;
+import org.apache.flink.util.Preconditions;
+
+import scala.concurrent.Future;
+
+import java.util.UUID;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+
+public class TestRegistrationGateway extends TestingGatewayBase {
+
+	private final BlockingQueue<RegistrationCall> invocations;
+
+	private final RegistrationResponse[] responses;
+
+	private int pos;
+
+	public TestRegistrationGateway(RegistrationResponse... responses) {
+		Preconditions.checkArgument(responses != null && responses.length > 0);
+
+		this.invocations = new LinkedBlockingQueue<>();
+		this.responses = responses;
+		
+	}
+
+	// ------------------------------------------------------------------------
+
+	public Future<RegistrationResponse> registrationCall(UUID leaderId, long timeout) {
+		invocations.add(new RegistrationCall(leaderId, timeout));
+
+		RegistrationResponse response = responses[pos];
+		if (pos < responses.length - 1) {
+			pos++;
+		}
+
+		// return a completed future (for a proper value), or one that never completes and will time out (for null)
+		return response != null ? Futures.successful(response) : this.<RegistrationResponse>futureWithTimeout(timeout);
+	}
+
+	public BlockingQueue<RegistrationCall> getInvocations() {
+		return invocations;
+	}
+
+	// ------------------------------------------------------------------------
+
+	public static class RegistrationCall {
+		private final UUID leaderId;
+		private final long timeout;
+
+		public RegistrationCall(UUID leaderId, long timeout) {
+			this.leaderId = leaderId;
+			this.timeout = timeout;
+		}
+
+		public UUID leaderId() {
+			return leaderId;
+		}
+
+		public long timeout() {
+			return timeout;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/72468d14/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ClusterShutdownITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ClusterShutdownITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ClusterShutdownITCase.java
deleted file mode 100644
index 32c6cac..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ClusterShutdownITCase.java
+++ /dev/null
@@ -1,156 +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.resourcemanager;
-
-import akka.actor.ActorSystem;
-import akka.testkit.JavaTestKit;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.akka.AkkaUtils;
-import org.apache.flink.runtime.clusterframework.ApplicationStatus;
-import org.apache.flink.runtime.clusterframework.messages.StopCluster;
-import org.apache.flink.runtime.clusterframework.messages.StopClusterSuccessful;
-import org.apache.flink.runtime.instance.ActorGateway;
-import org.apache.flink.runtime.messages.Messages;
-import org.apache.flink.runtime.testingUtils.TestingMessages;
-import org.apache.flink.runtime.testingUtils.TestingUtils;
-import org.apache.flink.runtime.testutils.TestingResourceManager;
-import org.apache.flink.util.TestLogger;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import scala.Option;
-
-
-/**
- * Runs tests to ensure that a cluster is shutdown properly.
- */
-public class ClusterShutdownITCase extends TestLogger {
-
-	private static ActorSystem system;
-
-	private static Configuration config = new Configuration();
-
-	@BeforeClass
-	public static void setup() {
-		system = AkkaUtils.createActorSystem(AkkaUtils.getDefaultAkkaConfig());
-	}
-
-	@AfterClass
-	public static void teardown() {
-		JavaTestKit.shutdownActorSystem(system);
-	}
-
-	/**
-	 * Tests a faked cluster shutdown procedure without the ResourceManager.
-	 */
-	@Test
-	public void testClusterShutdownWithoutResourceManager() {
-
-		new JavaTestKit(system){{
-		new Within(duration("30 seconds")) {
-		@Override
-		protected void run() {
-
-			ActorGateway me =
-				TestingUtils.createForwardingActor(system, getTestActor(), Option.<String>empty());
-
-			// start job manager which doesn't shutdown the actor system
-			ActorGateway jobManager =
-				TestingUtils.createJobManager(system, config, "jobmanager1");
-
-			// Tell the JobManager to inform us of shutdown actions
-			jobManager.tell(TestingMessages.getNotifyOfComponentShutdown(), me);
-
-			// Register a TaskManager
-			ActorGateway taskManager =
-				TestingUtils.createTaskManager(system, jobManager, config, true, true);
-
-			// Tell the TaskManager to inform us of TaskManager shutdowns
-			taskManager.tell(TestingMessages.getNotifyOfComponentShutdown(), me);
-
-
-			// No resource manager connected
-			jobManager.tell(new StopCluster(ApplicationStatus.SUCCEEDED, "Shutting down."), me);
-
-			expectMsgAllOf(
-				new TestingMessages.ComponentShutdown(taskManager.actor()),
-				new TestingMessages.ComponentShutdown(jobManager.actor()),
-				StopClusterSuccessful.getInstance()
-			);
-
-		}};
-		}};
-	}
-
-	/**
-	 * Tests a faked cluster shutdown procedure with the ResourceManager.
-	 */
-	@Test
-	public void testClusterShutdownWithResourceManager() {
-
-		new JavaTestKit(system){{
-		new Within(duration("30 seconds")) {
-		@Override
-		protected void run() {
-
-			ActorGateway me =
-				TestingUtils.createForwardingActor(system, getTestActor(), Option.<String>empty());
-
-			// start job manager which doesn't shutdown the actor system
-			ActorGateway jobManager =
-				TestingUtils.createJobManager(system, config, "jobmanager2");
-
-			// Tell the JobManager to inform us of shutdown actions
-			jobManager.tell(TestingMessages.getNotifyOfComponentShutdown(), me);
-
-			// Register a TaskManager
-			ActorGateway taskManager =
-				TestingUtils.createTaskManager(system, jobManager, config, true, true);
-
-			// Tell the TaskManager to inform us of TaskManager shutdowns
-			taskManager.tell(TestingMessages.getNotifyOfComponentShutdown(), me);
-
-			// Start resource manager and let it register
-			ActorGateway resourceManager =
-				TestingUtils.createResourceManager(system, jobManager.actor(), config);
-
-			// Tell the ResourceManager to inform us of ResourceManager shutdowns
-			resourceManager.tell(TestingMessages.getNotifyOfComponentShutdown(), me);
-
-			// notify about a resource manager registration at the job manager
-			resourceManager.tell(new TestingResourceManager.NotifyWhenResourceManagerConnected(), me);
-
-			// Wait for resource manager
-			expectMsgEquals(Messages.getAcknowledge());
-
-
-			// Shutdown cluster with resource manager connected
-			jobManager.tell(new StopCluster(ApplicationStatus.SUCCEEDED, "Shutting down."), me);
-
-			expectMsgAllOf(
-				new TestingMessages.ComponentShutdown(taskManager.actor()),
-				new TestingMessages.ComponentShutdown(jobManager.actor()),
-				new TestingMessages.ComponentShutdown(resourceManager.actor()),
-				StopClusterSuccessful.getInstance()
-			);
-
-		}};
-		}};
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/72468d14/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerHATest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerHATest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerHATest.java
new file mode 100644
index 0000000..5799e62
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerHATest.java
@@ -0,0 +1,76 @@
+/*
+ * 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.resourcemanager;
+
+import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices;
+import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService;
+import org.apache.flink.runtime.rpc.MainThreadExecutor;
+import org.apache.flink.runtime.rpc.RpcEndpoint;
+import org.apache.flink.runtime.rpc.RpcGateway;
+import org.apache.flink.runtime.rpc.RpcService;
+import org.apache.flink.runtime.rpc.StartStoppable;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.UUID;
+
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.doCallRealMethod;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+/**
+ * resourceManager HA test, including grant leadership and revoke leadership
+ */
+public class ResourceManagerHATest {
+
+	@Test
+	public void testGrantAndRevokeLeadership() throws Exception {
+		// mock a RpcService which will return a special RpcGateway when call its startServer method, the returned RpcGateway directly execute runAsync call
+		TestingResourceManagerGatewayProxy gateway = mock(TestingResourceManagerGatewayProxy.class);
+		doCallRealMethod().when(gateway).runAsync(any(Runnable.class));
+
+		RpcService rpcService = mock(RpcService.class);
+		when(rpcService.startServer(any(RpcEndpoint.class))).thenReturn(gateway);
+
+		TestingLeaderElectionService leaderElectionService = new TestingLeaderElectionService();
+		TestingHighAvailabilityServices highAvailabilityServices = new TestingHighAvailabilityServices();
+		highAvailabilityServices.setResourceManagerLeaderElectionService(leaderElectionService);
+
+		final ResourceManager resourceManager = new ResourceManager(rpcService, highAvailabilityServices);
+		resourceManager.start();
+		// before grant leadership, resourceManager's leaderId is null
+		Assert.assertNull(resourceManager.getLeaderSessionID());
+		final UUID leaderId = UUID.randomUUID();
+		leaderElectionService.isLeader(leaderId);
+		// after grant leadership, resourceManager's leaderId has value
+		Assert.assertEquals(leaderId, resourceManager.getLeaderSessionID());
+		// then revoke leadership, resourceManager's leaderId is null again
+		leaderElectionService.notLeader();
+		Assert.assertNull(resourceManager.getLeaderSessionID());
+	}
+
+	private static abstract class TestingResourceManagerGatewayProxy implements MainThreadExecutor, StartStoppable, RpcGateway {
+		@Override
+		public void runAsync(Runnable runnable) {
+			runnable.run();
+		}
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/72468d14/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerITCase.java
deleted file mode 100644
index 0c2ca1a..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerITCase.java
+++ /dev/null
@@ -1,162 +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.resourcemanager;
-
-import akka.actor.ActorSystem;
-import akka.testkit.JavaTestKit;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.akka.AkkaUtils;
-import org.apache.flink.runtime.clusterframework.types.ResourceID;
-import org.apache.flink.runtime.instance.ActorGateway;
-import org.apache.flink.runtime.instance.HardwareDescription;
-import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
-import org.apache.flink.runtime.messages.Messages;
-import org.apache.flink.runtime.messages.RegistrationMessages;
-import org.apache.flink.runtime.testingUtils.TestingUtils;
-import org.apache.flink.runtime.testutils.TestingResourceManager;
-import org.apache.flink.util.TestLogger;
-
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import scala.Option;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-/**
- * It cases which test the interaction of the resource manager with job manager and task managers.
- * Runs all tests in one Actor system.
- */
-public class ResourceManagerITCase extends TestLogger {
-
-	private static ActorSystem system;
-
-	private static Configuration config = new Configuration();
-
-	@BeforeClass
-	public static void setup() {
-		system = AkkaUtils.createActorSystem(AkkaUtils.getDefaultAkkaConfig());
-	}
-
-	@AfterClass
-	public static void teardown() {
-		JavaTestKit.shutdownActorSystem(system);
-	}
-
-	/**
-	 * Tests whether the resource manager connects and reconciles existing task managers.
-	 */
-	@Test
-	public void testResourceManagerReconciliation() {
-
-		new JavaTestKit(system){{
-		new Within(duration("10 seconds")) {
-		@Override
-		protected void run() {
-
-			ActorGateway jobManager =
-				TestingUtils.createJobManager(system, config, "ReconciliationTest");
-			ActorGateway me =
-				TestingUtils.createForwardingActor(system, getTestActor(), Option.<String>empty());
-
-			// !! no resource manager started !!
-
-			ResourceID resourceID = ResourceID.generate();
-
-			TaskManagerLocation location = mock(TaskManagerLocation.class);
-			when(location.getResourceID()).thenReturn(resourceID);
-
-			HardwareDescription resourceProfile = HardwareDescription.extractFromSystem(1_000_000);
-
-			jobManager.tell(
-				new RegistrationMessages.RegisterTaskManager(resourceID, location, resourceProfile, 1),
-				me);
-
-			expectMsgClass(RegistrationMessages.AcknowledgeRegistration.class);
-
-			// now start the resource manager
-			ActorGateway resourceManager =
-				TestingUtils.createResourceManager(system, jobManager.actor(), config);
-
-			// register at testing job manager to receive a message once a resource manager registers
-			resourceManager.tell(new TestingResourceManager.NotifyWhenResourceManagerConnected(), me);
-
-			// Wait for resource manager
-			expectMsgEquals(Messages.getAcknowledge());
-
-			// check if we registered the task manager resource
-			resourceManager.tell(new TestingResourceManager.GetRegisteredResources(), me);
-
-			TestingResourceManager.GetRegisteredResourcesReply reply =
-				expectMsgClass(TestingResourceManager.GetRegisteredResourcesReply.class);
-
-			assertEquals(1, reply.resources.size());
-			assertTrue(reply.resources.contains(resourceID));
-
-		}};
-		}};
-	}
-
-	/**
-	 * Tests whether the resource manager gets informed upon TaskManager registration.
-	 */
-	@Test
-	public void testResourceManagerTaskManagerRegistration() {
-
-		new JavaTestKit(system){{
-		new Within(duration("30 seconds")) {
-		@Override
-		protected void run() {
-
-			ActorGateway jobManager =
-				TestingUtils.createJobManager(system, config, "RegTest");
-			ActorGateway me =
-				TestingUtils.createForwardingActor(system, getTestActor(), Option.<String>empty());
-
-			// start the resource manager
-			ActorGateway resourceManager =
-				TestingUtils.createResourceManager(system, jobManager.actor(), config);
-
-			// notify about a resource manager registration at the job manager
-			resourceManager.tell(new TestingResourceManager.NotifyWhenResourceManagerConnected(), me);
-
-			// Wait for resource manager
-			expectMsgEquals(Messages.getAcknowledge());
-
-			// start task manager and wait for registration
-			ActorGateway taskManager =
-				TestingUtils.createTaskManager(system, jobManager.actor(), config, true, true);
-
-			// check if we registered the task manager resource
-			resourceManager.tell(new TestingResourceManager.GetRegisteredResources(), me);
-
-			TestingResourceManager.GetRegisteredResourcesReply reply =
-				expectMsgClass(TestingResourceManager.GetRegisteredResourcesReply.class);
-
-			assertEquals(1, reply.resources.size());
-
-		}};
-		}};
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/72468d14/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTest.java
deleted file mode 100644
index 043c81c..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/ResourceManagerTest.java
+++ /dev/null
@@ -1,338 +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.resourcemanager;
-
-import akka.actor.ActorSystem;
-import akka.testkit.JavaTestKit;
-import org.apache.flink.configuration.ConfigConstants;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.akka.AkkaUtils;
-import org.apache.flink.runtime.clusterframework.messages.NotifyResourceStarted;
-import org.apache.flink.runtime.clusterframework.messages.RegisterResourceManager;
-import org.apache.flink.runtime.clusterframework.messages.RegisterResourceManagerSuccessful;
-import org.apache.flink.runtime.clusterframework.messages.RemoveResource;
-import org.apache.flink.runtime.clusterframework.messages.ResourceRemoved;
-import org.apache.flink.runtime.clusterframework.messages.TriggerRegistrationAtJobManager;
-import org.apache.flink.runtime.clusterframework.types.ResourceID;
-import org.apache.flink.runtime.instance.ActorGateway;
-import org.apache.flink.runtime.messages.Acknowledge;
-import org.apache.flink.runtime.messages.JobManagerMessages;
-import org.apache.flink.runtime.testingUtils.TestingUtils;
-import org.apache.flink.runtime.testutils.TestingResourceManager;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import scala.Option;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
-import static org.junit.Assert.*;
-
-/**
- * General tests for the resource manager component.
- */
-public class ResourceManagerTest {
-
-	private static ActorSystem system;
-
-	private static ActorGateway fakeJobManager;
-	private static ActorGateway resourceManager;
-
-	private static Configuration config = new Configuration();
-
-	@BeforeClass
-	public static void setup() {
-		system = AkkaUtils.createLocalActorSystem(config);
-	}
-
-	@AfterClass
-	public static void teardown() {
-		JavaTestKit.shutdownActorSystem(system);
-	}
-
-	/**
-	 * Tests the registration and reconciliation of the ResourceManager with the JobManager
-	 */
-	@Test
-	public void testJobManagerRegistrationAndReconciliation() {
-		new JavaTestKit(system){{
-		new Within(duration("10 seconds")) {
-		@Override
-		protected void run() {
-			fakeJobManager = TestingUtils.createForwardingActor(system, getTestActor(), Option.<String>empty());
-			resourceManager = TestingUtils.createResourceManager(system, fakeJobManager.actor(), config);
-
-			expectMsgClass(RegisterResourceManager.class);
-
-			List<ResourceID> resourceList = new ArrayList<>();
-			resourceList.add(ResourceID.generate());
-			resourceList.add(ResourceID.generate());
-			resourceList.add(ResourceID.generate());
-
-			resourceManager.tell(
-				new RegisterResourceManagerSuccessful(fakeJobManager.actor(), resourceList),
-				fakeJobManager);
-
-			resourceManager.tell(new TestingResourceManager.GetRegisteredResources(), fakeJobManager);
-			TestingResourceManager.GetRegisteredResourcesReply reply =
-				expectMsgClass(TestingResourceManager.GetRegisteredResourcesReply.class);
-
-			for (ResourceID id : resourceList) {
-				if (!reply.resources.contains(id)) {
-					fail("Expected to find all resources that were provided during registration.");
-				}
-			}
-		}};
-		}};
-	}
-
-	/**
-	 * Tests delayed or erroneous registration of the ResourceManager with the JobManager
-	 */
-	@Test
-	public void testDelayedJobManagerRegistration() {
-		new JavaTestKit(system){{
-		new Within(duration("10 seconds")) {
-		@Override
-		protected void run() {
-
-			// set a short timeout for lookups
-			Configuration shortTimeoutConfig = config.clone();
-			shortTimeoutConfig.setString(ConfigConstants.AKKA_LOOKUP_TIMEOUT, "1 s");
-
-			fakeJobManager = TestingUtils.createForwardingActor(system, getTestActor(), Option.<String>empty());
-			resourceManager = TestingUtils.createResourceManager(system, fakeJobManager.actor(), shortTimeoutConfig);
-
-			// wait for registration message
-			RegisterResourceManager msg = expectMsgClass(RegisterResourceManager.class);
-			// give wrong response
-			getLastSender().tell(new JobManagerMessages.LeaderSessionMessage(null, new Object()),
-				fakeJobManager.actor());
-
-			// expect another retry and let it time out
-			expectMsgClass(RegisterResourceManager.class);
-
-			// wait for next try after timeout
-			expectMsgClass(RegisterResourceManager.class);
-
-		}};
-		}};
-	}
-
-	@Test
-	public void testTriggerReconnect() {
-		new JavaTestKit(system){{
-		new Within(duration("10 seconds")) {
-		@Override
-		protected void run() {
-
-			// set a long timeout for lookups such that the test fails in case of timeouts
-			Configuration shortTimeoutConfig = config.clone();
-			shortTimeoutConfig.setString(ConfigConstants.AKKA_LOOKUP_TIMEOUT, "99999 s");
-
-			fakeJobManager = TestingUtils.createForwardingActor(system, getTestActor(), Option.<String>empty());
-			resourceManager = TestingUtils.createResourceManager(system, fakeJobManager.actor(), shortTimeoutConfig);
-
-			// wait for registration message
-			RegisterResourceManager msg = expectMsgClass(RegisterResourceManager.class);
-			// all went well
-			resourceManager.tell(
-				new RegisterResourceManagerSuccessful(fakeJobManager.actor(), Collections.<ResourceID>emptyList()),
-				fakeJobManager);
-
-			// force a reconnect
-			resourceManager.tell(
-				new TriggerRegistrationAtJobManager(fakeJobManager.actor()),
-				fakeJobManager);
-
-			// new registration attempt should come in
-			expectMsgClass(RegisterResourceManager.class);
-
-		}};
-		}};
-	}
-
-	/**
-	 * Tests the registration and accounting of resources at the ResourceManager.
-	 */
-	@Test
-	public void testTaskManagerRegistration() {
-		new JavaTestKit(system){{
-		new Within(duration("10 seconds")) {
-		@Override
-		protected void run() {
-
-			fakeJobManager = TestingUtils.createForwardingActor(system, getTestActor(), Option.<String>empty());
-			resourceManager = TestingUtils.createResourceManager(system, fakeJobManager.actor(), config);
-
-			// register with JM
-			expectMsgClass(RegisterResourceManager.class);
-			resourceManager.tell(
-				new RegisterResourceManagerSuccessful(fakeJobManager.actor(), Collections.<ResourceID>emptyList()),
-				fakeJobManager);
-
-			ResourceID resourceID = ResourceID.generate();
-
-			// Send task manager registration
-			resourceManager.tell(new NotifyResourceStarted(resourceID),
-				fakeJobManager);
-
-			expectMsgClass(Acknowledge.class);
-
-			// check for number registration of registered resources
-			resourceManager.tell(new TestingResourceManager.GetRegisteredResources(), fakeJobManager);
-			TestingResourceManager.GetRegisteredResourcesReply reply =
-				expectMsgClass(TestingResourceManager.GetRegisteredResourcesReply.class);
-
-			assertEquals(1, reply.resources.size());
-
-			// Send task manager registration again
-			resourceManager.tell(new NotifyResourceStarted(resourceID),
-				fakeJobManager);
-
-			expectMsgClass(Acknowledge.class);
-
-			// check for number registration of registered resources
-			resourceManager.tell(new TestingResourceManager.GetRegisteredResources(), fakeJobManager);
-			reply = expectMsgClass(TestingResourceManager.GetRegisteredResourcesReply.class);
-
-			assertEquals(1, reply.resources.size());
-
-			// Send invalid null resource id to throw an exception during resource registration
-			resourceManager.tell(new NotifyResourceStarted(null),
-				fakeJobManager);
-
-			expectMsgClass(Acknowledge.class);
-
-			// check for number registration of registered resources
-			resourceManager.tell(new TestingResourceManager.GetRegisteredResources(), fakeJobManager);
-			reply = expectMsgClass(TestingResourceManager.GetRegisteredResourcesReply.class);
-
-			assertEquals(1, reply.resources.size());
-		}};
-		}};
-	}
-
-	@Test
-	public void testResourceRemoval() {
-		new JavaTestKit(system){{
-		new Within(duration("10 seconds")) {
-		@Override
-		protected void run() {
-
-			fakeJobManager = TestingUtils.createForwardingActor(system, getTestActor(), Option.<String>empty());
-			resourceManager = TestingUtils.createResourceManager(system, fakeJobManager.actor(), config);
-
-			// register with JM
-			expectMsgClass(RegisterResourceManager.class);
-			resourceManager.tell(
-				new RegisterResourceManagerSuccessful(fakeJobManager.actor(), Collections.<ResourceID>emptyList()),
-				fakeJobManager);
-
-			ResourceID resourceID = ResourceID.generate();
-
-			// remove unknown resource
-			resourceManager.tell(new RemoveResource(resourceID), fakeJobManager);
-
-			// Send task manager registration
-			resourceManager.tell(new NotifyResourceStarted(resourceID),
-				fakeJobManager);
-
-			expectMsgClass(Acknowledge.class);
-
-			// check for number registration of registered resources
-			resourceManager.tell(new TestingResourceManager.GetRegisteredResources(), fakeJobManager);
-			TestingResourceManager.GetRegisteredResourcesReply reply =
-				expectMsgClass(TestingResourceManager.GetRegisteredResourcesReply.class);
-
-			assertEquals(1, reply.resources.size());
-			assertTrue(reply.resources.contains(resourceID));
-
-			// remove resource
-			resourceManager.tell(new RemoveResource(resourceID), fakeJobManager);
-
-			// check for number registration of registered resources
-			resourceManager.tell(new TestingResourceManager.GetRegisteredResources(), fakeJobManager);
-			reply =	expectMsgClass(TestingResourceManager.GetRegisteredResourcesReply.class);
-
-			assertEquals(0, reply.resources.size());
-
-		}};
-		}};
-	}
-
-	/**
-	 * Tests notification of JobManager about a failed resource.
-	 */
-	@Test
-	public void testResourceFailureNotification() {
-		new JavaTestKit(system){{
-		new Within(duration("10 seconds")) {
-		@Override
-		protected void run() {
-
-			fakeJobManager = TestingUtils.createForwardingActor(system, getTestActor(), Option.<String>empty());
-			resourceManager = TestingUtils.createResourceManager(system, fakeJobManager.actor(), config);
-
-			// register with JM
-			expectMsgClass(RegisterResourceManager.class);
-			resourceManager.tell(
-				new RegisterResourceManagerSuccessful(fakeJobManager.actor(), Collections.<ResourceID>emptyList()),
-				fakeJobManager);
-
-			ResourceID resourceID1 = ResourceID.generate();
-			ResourceID resourceID2 = ResourceID.generate();
-
-			// Send task manager registration
-			resourceManager.tell(new NotifyResourceStarted(resourceID1),
-				fakeJobManager);
-
-			expectMsgClass(Acknowledge.class);
-
-			// Send task manager registration
-			resourceManager.tell(new NotifyResourceStarted(resourceID2),
-				fakeJobManager);
-
-			expectMsgClass(Acknowledge.class);
-
-			// check for number registration of registered resources
-			resourceManager.tell(new TestingResourceManager.GetRegisteredResources(), fakeJobManager);
-			TestingResourceManager.GetRegisteredResourcesReply reply =
-				expectMsgClass(TestingResourceManager.GetRegisteredResourcesReply.class);
-
-			assertEquals(2, reply.resources.size());
-			assertTrue(reply.resources.contains(resourceID1));
-			assertTrue(reply.resources.contains(resourceID2));
-
-			// fail resources
-			resourceManager.tell(new TestingResourceManager.FailResource(resourceID1), fakeJobManager);
-			resourceManager.tell(new TestingResourceManager.FailResource(resourceID2), fakeJobManager);
-
-			ResourceRemoved answer = expectMsgClass(ResourceRemoved.class);
-			ResourceRemoved answer2 = expectMsgClass(ResourceRemoved.class);
-
-			assertEquals(resourceID1, answer.resourceId());
-			assertEquals(resourceID2, answer2.resourceId());
-
-		}};
-		}};
-	}
-}


[18/50] [abbrv] flink git commit: [FLINK-4456] Replace Akka specific types by interfaces in Task

Posted by tr...@apache.org.
[FLINK-4456] Replace Akka specific types by interfaces in Task

Introduce TaskExecutionStateListener for Task

Replace JobManagerGateway in Task by InputSplitProvider and CheckpointNotifier

Replace the TaskManager ActorGateway by TaskManagerConnection in Task

Rename taskmanager.CheckpointNotifier into CheckpointResponder; rename TaskExecutionStateListener.notifyTaskExecutionState into notifyTaskExecutionStateChanged

Remove InputSplitProvider.start; add ClassLoader parameter to InputSplitProvider.getNextInputSplit

Removes the unused class InputSplitIterator.

Update InputSplitProvider JavaDocs

This closes #2456.


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

Branch: refs/heads/flip-6
Commit: 0735b5b935b0c0757943e2d58047afcfb9949560
Parents: 02b852e
Author: Till Rohrmann <tr...@apache.org>
Authored: Thu Sep 1 14:41:44 2016 +0200
Committer: Till Rohrmann <tr...@apache.org>
Committed: Thu Sep 8 17:20:08 2016 +0200

----------------------------------------------------------------------
 .../ExecutionPlanAfterExecutionTest.java        |   5 +-
 .../jobgraph/tasks/InputSplitIterator.java      |  88 ----------------
 .../jobgraph/tasks/InputSplitProvider.java      |   5 +-
 .../flink/runtime/operators/DataSourceTask.java |   2 +-
 .../ActorGatewayCheckpointResponder.java        |  78 ++++++++++++++
 .../ActorGatewayTaskExecutionStateListener.java |  42 ++++++++
 .../ActorGatewayTaskManagerConnection.java      |  59 +++++++++++
 .../taskmanager/CheckpointResponder.java        |  63 +++++++++++
 .../runtime/taskmanager/RuntimeEnvironment.java |  21 ++--
 .../apache/flink/runtime/taskmanager/Task.java  | 105 ++++++++-----------
 .../taskmanager/TaskExecutionStateListener.java |  29 +++++
 .../taskmanager/TaskInputSplitProvider.java     |  55 +++++-----
 .../taskmanager/TaskManagerConnection.java      |  57 ++++++++++
 .../flink/runtime/taskmanager/TaskManager.scala |  17 ++-
 .../testutils/MockInputSplitProvider.java       |   3 +-
 .../runtime/taskmanager/TaskAsyncCallTest.java  |  13 +--
 .../taskmanager/TaskInputSplitProviderTest.java |  16 +--
 .../flink/runtime/taskmanager/TaskStopTest.java |   8 +-
 .../flink/runtime/taskmanager/TaskTest.java     |  60 +++++++----
 .../source/InputFormatSourceFunction.java       |   2 +-
 .../source/InputFormatSourceFunctionTest.java   |   5 +-
 .../tasks/InterruptSensitiveRestoreTest.java    |  13 ++-
 .../streaming/runtime/tasks/StreamTaskTest.java |  75 ++++---------
 23 files changed, 519 insertions(+), 302 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/0735b5b9/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 2bffba9..4ec0e47 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
@@ -25,12 +25,15 @@ import org.apache.flink.api.java.LocalEnvironment;
 import org.apache.flink.api.java.aggregation.Aggregations;
 import org.apache.flink.api.java.io.DiscardingOutputFormat;
 import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.util.TestLogger;
 import org.junit.Test;
 
+import java.io.Serializable;
+
 import static org.junit.Assert.fail;
 
 @SuppressWarnings("serial")
-public class ExecutionPlanAfterExecutionTest implements java.io.Serializable {
+public class ExecutionPlanAfterExecutionTest extends TestLogger implements Serializable {
 
 	@Test
 	public void testExecuteAfterGetExecutionPlan() {

http://git-wip-us.apache.org/repos/asf/flink/blob/0735b5b9/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/InputSplitIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/InputSplitIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/InputSplitIterator.java
deleted file mode 100644
index a3d700a..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/InputSplitIterator.java
+++ /dev/null
@@ -1,88 +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.tasks;
-
-import java.util.Iterator;
-
-import org.apache.flink.core.io.InputSplit;
-
-/**
- * The input split iterator allows a task to iterate over all input splits it is supposed to
- * consume. Internally, the input split iterator calls an {@link InputSplitProvider} on each <code>next</code> call in
- * order to facilitate lazy split assignment.
- * 
- * @param <T>
- */
-public class InputSplitIterator<T extends InputSplit> implements Iterator<T> {
-
-	/**
-	 * The {@link InputSplitProvider} that is called to provide new input splits.
-	 */
-	private final InputSplitProvider inputSplitProvider;
-
-	/**
-	 * Buffers the next input split to be returned by this iterator or <code>null</code> it no split is buffered.
-	 */
-	private T nextInputSplit = null;
-
-	/**
-	 * Constructs a new input split iterator.
-	 * 
-	 * @param inputSplitProvider
-	 *        the input split provider to be called for new input splits
-	 */
-	public InputSplitIterator(final InputSplitProvider inputSplitProvider) {
-		this.inputSplitProvider = inputSplitProvider;
-	}
-
-
-	@SuppressWarnings("unchecked")
-	@Override
-	public boolean hasNext() {
-
-		if (this.nextInputSplit == null) {
-			this.nextInputSplit = (T) inputSplitProvider.getNextInputSplit();
-		}
-
-		return this.nextInputSplit != null;
-	}
-
-
-	@SuppressWarnings("unchecked")
-	@Override
-	public T next() {
-		T retVal = null;
-
-		if (this.nextInputSplit == null) {
-			this.nextInputSplit = (T) inputSplitProvider.getNextInputSplit();
-		}
-
-		retVal = this.nextInputSplit;
-		this.nextInputSplit = null;
-
-		return retVal;
-	}
-
-
-	@Override
-	public void remove() {
-		throw new RuntimeException("The InputSplitIterator does not implement the remove method");
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/0735b5b9/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/InputSplitProvider.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/InputSplitProvider.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/InputSplitProvider.java
index 5e7a40f..e0cde17 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/InputSplitProvider.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/InputSplitProvider.java
@@ -30,9 +30,10 @@ public interface InputSplitProvider {
 
 	/**
 	 * Requests the next input split to be consumed by the calling task.
-	 * 
+	 *
+	 * @param userCodeClassLoader used to deserialize input splits
 	 * @return the next input split to be consumed by the calling task or <code>null</code> if the
 	 *         task shall not consume any further input splits.
 	 */
-	InputSplit getNextInputSplit();
+	InputSplit getNextInputSplit(ClassLoader userCodeClassLoader);
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/0735b5b9/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSourceTask.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSourceTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSourceTask.java
index 68e29b6..c062bf8 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSourceTask.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/DataSourceTask.java
@@ -333,7 +333,7 @@ public class DataSourceTask<OT> extends AbstractInvokable {
 					return true;
 				}
 				
-				InputSplit split = provider.getNextInputSplit();
+				InputSplit split = provider.getNextInputSplit(getUserCodeClassLoader());
 				
 				if (split != null) {
 					this.nextSplit = split;

http://git-wip-us.apache.org/repos/asf/flink/blob/0735b5b9/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ActorGatewayCheckpointResponder.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ActorGatewayCheckpointResponder.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ActorGatewayCheckpointResponder.java
new file mode 100644
index 0000000..56e5922
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ActorGatewayCheckpointResponder.java
@@ -0,0 +1,78 @@
+/*
+ * 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.taskmanager;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.instance.ActorGateway;
+import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint;
+import org.apache.flink.runtime.messages.checkpoint.DeclineCheckpoint;
+import org.apache.flink.runtime.state.ChainedStateHandle;
+import org.apache.flink.runtime.state.KeyGroupsStateHandle;
+import org.apache.flink.runtime.state.StreamStateHandle;
+import org.apache.flink.util.Preconditions;
+
+import java.util.List;
+
+/**
+ * Implementation using {@link ActorGateway} to forward the messages.
+ */
+public class ActorGatewayCheckpointResponder implements CheckpointResponder {
+
+	private final ActorGateway actorGateway;
+
+	public ActorGatewayCheckpointResponder(ActorGateway actorGateway) {
+		this.actorGateway = Preconditions.checkNotNull(actorGateway);
+	}
+
+	@Override
+	public void acknowledgeCheckpoint(
+		JobID jobID,
+		ExecutionAttemptID executionAttemptID,
+		long checkpointID,
+		ChainedStateHandle<StreamStateHandle> chainedStateHandle,
+		List<KeyGroupsStateHandle> keyGroupStateHandles) {
+
+		AcknowledgeCheckpoint message = new AcknowledgeCheckpoint(
+			jobID,
+			executionAttemptID,
+			checkpointID,
+			chainedStateHandle,
+			keyGroupStateHandles);
+
+		actorGateway.tell(message);
+	}
+
+	@Override
+	public void declineCheckpoint(
+		JobID jobID,
+		ExecutionAttemptID executionAttemptID,
+		long checkpointID,
+		long checkpointTimestamp) {
+
+		DeclineCheckpoint decline = new DeclineCheckpoint(
+			jobID,
+			executionAttemptID,
+			checkpointID,
+			checkpointTimestamp);
+
+		actorGateway.tell(decline);
+
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0735b5b9/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ActorGatewayTaskExecutionStateListener.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ActorGatewayTaskExecutionStateListener.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ActorGatewayTaskExecutionStateListener.java
new file mode 100644
index 0000000..d729dbb
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ActorGatewayTaskExecutionStateListener.java
@@ -0,0 +1,42 @@
+/*
+ * 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.taskmanager;
+
+import org.apache.flink.runtime.instance.ActorGateway;
+import org.apache.flink.runtime.messages.TaskMessages;
+import org.apache.flink.util.Preconditions;
+
+/**
+ * Implementation using {@link ActorGateway} to forward the messages.
+ */
+public class ActorGatewayTaskExecutionStateListener implements TaskExecutionStateListener {
+
+	private final ActorGateway actorGateway;
+
+	public ActorGatewayTaskExecutionStateListener(ActorGateway actorGateway) {
+		this.actorGateway = Preconditions.checkNotNull(actorGateway);
+	}
+
+	@Override
+	public void notifyTaskExecutionStateChanged(TaskExecutionState taskExecutionState) {
+		TaskMessages.UpdateTaskExecutionState actorMessage = new TaskMessages.UpdateTaskExecutionState(taskExecutionState);
+
+		actorGateway.tell(actorMessage);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0735b5b9/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ActorGatewayTaskManagerConnection.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ActorGatewayTaskManagerConnection.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ActorGatewayTaskManagerConnection.java
new file mode 100644
index 0000000..cddac55
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ActorGatewayTaskManagerConnection.java
@@ -0,0 +1,59 @@
+/*
+ * 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.taskmanager;
+
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.instance.ActorGateway;
+import org.apache.flink.runtime.messages.TaskManagerMessages;
+import org.apache.flink.runtime.messages.TaskMessages;
+import org.apache.flink.util.Preconditions;
+
+/**
+ * Implementation using {@link ActorGateway} to forward the messages.
+ */
+public class ActorGatewayTaskManagerConnection implements TaskManagerConnection {
+
+	private final ActorGateway actorGateway;
+
+	public ActorGatewayTaskManagerConnection(ActorGateway actorGateway) {
+		this.actorGateway = Preconditions.checkNotNull(actorGateway);
+	}
+
+	@Override
+	public void notifyFinalState(ExecutionAttemptID executionAttemptID) {
+		actorGateway.tell(new TaskMessages.TaskInFinalState(executionAttemptID));
+	}
+
+	@Override
+	public void notifyFatalError(String message, Throwable cause) {
+		actorGateway.tell(new TaskManagerMessages.FatalError(message, cause));
+	}
+
+	@Override
+	public void failTask(ExecutionAttemptID executionAttemptID, Throwable cause) {
+		actorGateway.tell(new TaskMessages.FailTask(executionAttemptID, cause));
+	}
+
+	@Override
+	public void updateTaskExecutionState(TaskExecutionState taskExecutionState) {
+		TaskMessages.UpdateTaskExecutionState actorMessage = new TaskMessages.UpdateTaskExecutionState(taskExecutionState);
+
+		actorGateway.tell(actorMessage);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0735b5b9/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/CheckpointResponder.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/CheckpointResponder.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/CheckpointResponder.java
new file mode 100644
index 0000000..9d5c4e1
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/CheckpointResponder.java
@@ -0,0 +1,63 @@
+/*
+ * 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.taskmanager;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.state.ChainedStateHandle;
+import org.apache.flink.runtime.state.KeyGroupsStateHandle;
+import org.apache.flink.runtime.state.StreamStateHandle;
+
+import java.util.List;
+
+/**
+ * Responder for checkpoint acknowledge and decline messages in the {@link Task}.
+ */
+public interface CheckpointResponder {
+
+	/**
+	 * Acknowledges the given checkpoint.
+	 *
+	 * @param jobID Job ID of the running job
+	 * @param executionAttemptID Execution attempt ID of the running task
+	 * @param checkpointID Checkpoint ID of the checkpoint
+	 * @param chainedStateHandle Chained state handle
+	 * @param keyGroupStateHandles State handles for key groups
+	 */
+	void acknowledgeCheckpoint(
+		JobID jobID,
+		ExecutionAttemptID executionAttemptID,
+		long checkpointID,
+		ChainedStateHandle<StreamStateHandle> chainedStateHandle,
+		List<KeyGroupsStateHandle> keyGroupStateHandles);
+
+	/**
+	 * Declines the given checkpoint.
+	 *
+	 * @param jobID Job ID of the running job
+	 * @param executionAttemptID Execution attempt ID of the running task
+	 * @param checkpointID Checkpoint ID of the checkpoint
+	 * @param checkpointTimestamp Timestamp of the checkpoint
+	 */
+	void declineCheckpoint(
+		JobID jobID,
+		ExecutionAttemptID executionAttemptID,
+		long checkpointID,
+		long checkpointTimestamp);
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0735b5b9/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/RuntimeEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/RuntimeEnvironment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/RuntimeEnvironment.java
index d54826a..3e4ba4d 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/RuntimeEnvironment.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/RuntimeEnvironment.java
@@ -27,14 +27,12 @@ import org.apache.flink.runtime.accumulators.AccumulatorRegistry;
 import org.apache.flink.runtime.broadcast.BroadcastVariableManager;
 import org.apache.flink.runtime.execution.Environment;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
-import org.apache.flink.runtime.instance.ActorGateway;
 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.JobVertexID;
 import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider;
 import org.apache.flink.runtime.memory.MemoryManager;
-import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint;
 import org.apache.flink.runtime.metrics.groups.TaskMetricGroup;
 import org.apache.flink.runtime.query.TaskKvStateRegistry;
 import org.apache.flink.runtime.state.ChainedStateHandle;
@@ -74,7 +72,7 @@ public class RuntimeEnvironment implements Environment {
 	private final ResultPartitionWriter[] writers;
 	private final InputGate[] inputGates;
 	
-	private final ActorGateway jobManager;
+	private final CheckpointResponder checkpointResponder;
 
 	private final AccumulatorRegistry accumulatorRegistry;
 
@@ -105,7 +103,7 @@ public class RuntimeEnvironment implements Environment {
 			Map<String, Future<Path>> distCacheEntries,
 			ResultPartitionWriter[] writers,
 			InputGate[] inputGates,
-			ActorGateway jobManager,
+			CheckpointResponder checkpointResponder,
 			TaskManagerRuntimeInfo taskManagerInfo,
 			TaskMetricGroup metrics,
 			Task containingTask) {
@@ -127,7 +125,7 @@ public class RuntimeEnvironment implements Environment {
 		this.distCacheEntries = checkNotNull(distCacheEntries);
 		this.writers = checkNotNull(writers);
 		this.inputGates = checkNotNull(inputGates);
-		this.jobManager = checkNotNull(jobManager);
+		this.checkpointResponder = checkNotNull(checkpointResponder);
 		this.taskManagerInfo = checkNotNull(taskManagerInfo);
 		this.containingTask = containingTask;
 		this.metrics = metrics;
@@ -251,14 +249,13 @@ public class RuntimeEnvironment implements Environment {
 			ChainedStateHandle<StreamStateHandle> chainedStateHandle,
 			List<KeyGroupsStateHandle> keyGroupStateHandles) {
 
-		AcknowledgeCheckpoint message = new AcknowledgeCheckpoint(
-				jobId,
-				executionId,
-				checkpointId,
-				chainedStateHandle,
-				keyGroupStateHandles);
 
-		jobManager.tell(message);
+		checkpointResponder.acknowledgeCheckpoint(
+			jobId,
+			executionId,
+			checkpointId,
+			chainedStateHandle,
+			keyGroupStateHandles);
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/0735b5b9/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 d09e03c..9994b7d 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,6 +26,7 @@ import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.runtime.io.network.netty.PartitionStateChecker;
+import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider;
 import org.apache.flink.runtime.io.network.partition.ResultPartitionConsumableNotifier;
 import org.apache.flink.runtime.metrics.groups.TaskMetricGroup;
 import org.apache.flink.runtime.accumulators.AccumulatorRegistry;
@@ -40,7 +41,6 @@ import org.apache.flink.runtime.execution.ExecutionState;
 import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.runtime.filecache.FileCache;
-import org.apache.flink.runtime.instance.ActorGateway;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.io.network.NetworkEnvironment;
 import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter;
@@ -54,11 +54,6 @@ import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
 import org.apache.flink.runtime.jobgraph.tasks.StatefulTask;
 import org.apache.flink.runtime.jobgraph.tasks.StoppableTask;
 import org.apache.flink.runtime.memory.MemoryManager;
-import org.apache.flink.runtime.messages.TaskManagerMessages.FatalError;
-import org.apache.flink.runtime.messages.TaskMessages.FailTask;
-import org.apache.flink.runtime.messages.TaskMessages.TaskInFinalState;
-import org.apache.flink.runtime.messages.TaskMessages.UpdateTaskExecutionState;
-import org.apache.flink.runtime.messages.checkpoint.DeclineCheckpoint;
 import org.apache.flink.runtime.query.TaskKvStateRegistry;
 import org.apache.flink.runtime.state.ChainedStateHandle;
 import org.apache.flink.runtime.state.KeyGroupsStateHandle;
@@ -68,8 +63,6 @@ import org.apache.flink.util.SerializedValue;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import scala.concurrent.duration.FiniteDuration;
-
 import java.io.IOException;
 import java.net.URL;
 import java.util.HashMap;
@@ -174,17 +167,17 @@ public class Task implements Runnable {
 
 	private final Map<IntermediateDataSetID, SingleInputGate> inputGatesById;
 
-	/** Gateway to the TaskManager that spawned this task */
-	private final ActorGateway taskManager;
+	/** Connection to the task manager */
+	private final TaskManagerConnection taskManagerConnection;
 
-	/** Gateway to the JobManager */
-	private final ActorGateway jobManager;
+	/** Input split provider for the task */
+	private final InputSplitProvider inputSplitProvider;
 
-	/** All actors that want to be notified about changes in the task's execution state */
-	private final List<ActorGateway> executionListenerActors;
+	/** Checkpoint notifier used to communicate with the CheckpointCoordinator */
+	private final CheckpointResponder checkpointResponder;
 
-	/** The timeout for all ask operations on actors */
-	private final FiniteDuration actorAskTimeout;
+	/** All listener that want to be notified about changes in the task's execution state */
+	private final List<TaskExecutionStateListener> taskExecutionStateListeners;
 
 	/** The library cache, from which the task can request its required JAR files */
 	private final LibraryCacheManager libraryCache;
@@ -244,20 +237,21 @@ public class Task implements Runnable {
 	 * <p><b>IMPORTANT:</b> This constructor may not start any work that would need to
 	 * be undone in the case of a failing task deployment.</p>
 	 */
-	public Task(TaskDeploymentDescriptor tdd,
-				MemoryManager memManager,
-				IOManager ioManager,
-				NetworkEnvironment networkEnvironment,
-				JobManagerCommunicationFactory jobManagerCommunicationFactory,
-				BroadcastVariableManager bcVarManager,
-				ActorGateway taskManagerActor,
-				ActorGateway jobManagerActor,
-				FiniteDuration actorAskTimeout,
-				LibraryCacheManager libraryCache,
-				FileCache fileCache,
-				TaskManagerRuntimeInfo taskManagerConfig,
-				TaskMetricGroup metricGroup)
-	{
+	public Task(
+		TaskDeploymentDescriptor tdd,
+		MemoryManager memManager,
+		IOManager ioManager,
+		NetworkEnvironment networkEnvironment,
+		JobManagerCommunicationFactory jobManagerCommunicationFactory,
+		BroadcastVariableManager bcVarManager,
+		TaskManagerConnection taskManagerConnection,
+		InputSplitProvider inputSplitProvider,
+		CheckpointResponder checkpointResponder,
+		LibraryCacheManager libraryCache,
+		FileCache fileCache,
+		TaskManagerRuntimeInfo taskManagerConfig,
+		TaskMetricGroup metricGroup) {
+
 		this.taskInfo = checkNotNull(tdd.getTaskInfo());
 		this.jobId = checkNotNull(tdd.getJobID());
 		this.vertexId = checkNotNull(tdd.getVertexID());
@@ -281,16 +275,16 @@ public class Task implements Runnable {
 		this.broadcastVariableManager = checkNotNull(bcVarManager);
 		this.accumulatorRegistry = new AccumulatorRegistry(jobId, executionId);
 
-		this.jobManager = checkNotNull(jobManagerActor);
-		this.taskManager = checkNotNull(taskManagerActor);
-		this.actorAskTimeout = checkNotNull(actorAskTimeout);
+		this.inputSplitProvider = checkNotNull(inputSplitProvider);
+		this.checkpointResponder = checkNotNull(checkpointResponder);
+		this.taskManagerConnection = checkNotNull(taskManagerConnection);
 
 		this.libraryCache = checkNotNull(libraryCache);
 		this.fileCache = checkNotNull(fileCache);
 		this.network = checkNotNull(networkEnvironment);
 		this.taskManagerConfig = checkNotNull(taskManagerConfig);
 
-		this.executionListenerActors = new CopyOnWriteArrayList<ActorGateway>();
+		this.taskExecutionStateListeners = new CopyOnWriteArrayList<>();
 		this.metrics = metricGroup;
 
 		// create the reader and writer structures
@@ -539,19 +533,16 @@ public class Task implements Runnable {
 			//  call the user code initialization methods
 			// ----------------------------------------------------------------
 
-			TaskInputSplitProvider splitProvider = new TaskInputSplitProvider(jobManager,
-					jobId, vertexId, executionId, userCodeClassLoader, actorAskTimeout);
-
 			TaskKvStateRegistry kvStateRegistry = network
 					.createKvStateTaskRegistry(jobId, getJobVertexId());
 
-			Environment env = new RuntimeEnvironment(jobId, vertexId, executionId,
-					executionConfig, taskInfo, jobConfiguration, taskConfiguration,
-					userCodeClassLoader, memoryManager, ioManager,
-					broadcastVariableManager, accumulatorRegistry,
-					kvStateRegistry,
-					splitProvider, distributedCacheEntries,
-					writers, inputGates, jobManager, taskManagerConfig, metrics, this);
+			Environment env = new RuntimeEnvironment(
+				jobId, vertexId, executionId, executionConfig, taskInfo,
+				jobConfiguration, taskConfiguration, userCodeClassLoader,
+				memoryManager, ioManager, broadcastVariableManager,
+				accumulatorRegistry, kvStateRegistry, inputSplitProvider,
+				distributedCacheEntries, writers, inputGates,
+				checkpointResponder, taskManagerConfig, metrics, this);
 
 			// let the task code create its readers and writers
 			invokable.setEnvironment(env);
@@ -588,11 +579,9 @@ public class Task implements Runnable {
 				throw new CancelTaskException();
 			}
 
-			// notify everyone that we switched to running. especially the TaskManager needs
-			// to know this!
+			// notify everyone that we switched to running
 			notifyObservers(ExecutionState.RUNNING, null);
-			taskManager.tell(new UpdateTaskExecutionState(
-					new TaskExecutionState(jobId, executionId, ExecutionState.RUNNING)));
+			taskManagerConnection.updateTaskExecutionState(new TaskExecutionState(jobId, executionId, ExecutionState.RUNNING));
 
 			// make sure the user code classloader is accessible thread-locally
 			executingThread.setContextClassLoader(userCodeClassLoader);
@@ -785,11 +774,11 @@ public class Task implements Runnable {
 	}
 
 	private void notifyFinalState() {
-		taskManager.tell(new TaskInFinalState(executionId));
+		taskManagerConnection.notifyFinalState(executionId);
 	}
 
 	private void notifyFatalError(String message, Throwable cause) {
-		taskManager.tell(new FatalError(message, cause));
+		taskManagerConnection.notifyFatalError(message, cause);
 	}
 
 	// ----------------------------------------------------------------------------------------------------------------
@@ -815,7 +804,7 @@ public class Task implements Runnable {
 						((StoppableTask)Task.this.invokable).stop();
 					} catch(RuntimeException e) {
 						LOG.error("Stopping task " + taskNameWithSubtask + " failed.", e);
-						taskManager.tell(new FailTask(executionId, e));
+						taskManagerConnection.failTask(executionId, e);
 					}
 				}
 			};
@@ -910,8 +899,8 @@ public class Task implements Runnable {
 	//  State Listeners
 	// ------------------------------------------------------------------------
 
-	public void registerExecutionListener(ActorGateway listener) {
-		executionListenerActors.add(listener);
+	public void registerExecutionListener(TaskExecutionStateListener listener) {
+		taskExecutionStateListeners.add(listener);
 	}
 
 	private void notifyObservers(ExecutionState newState, Throwable error) {
@@ -923,10 +912,9 @@ public class Task implements Runnable {
 		}
 
 		TaskExecutionState stateUpdate = new TaskExecutionState(jobId, executionId, newState, error);
-		UpdateTaskExecutionState actorMessage = new UpdateTaskExecutionState(stateUpdate);
 
-		for (ActorGateway listener : executionListenerActors) {
-			listener.tell(actorMessage);
+		for (TaskExecutionStateListener listener : taskExecutionStateListeners) {
+			listener.notifyTaskExecutionStateChanged(stateUpdate);
 		}
 	}
 
@@ -936,7 +924,7 @@ public class Task implements Runnable {
 
 	/**
 	 * Calls the invokable to trigger a checkpoint, if the invokable implements the interface
-	 * {@link org.apache.flink.runtime.jobgraph.tasks.StatefulTask}.
+	 * {@link StatefulTask}.
 	 * 
 	 * @param checkpointID The ID identifying the checkpoint.
 	 * @param checkpointTimestamp The timestamp associated with the checkpoint.
@@ -957,8 +945,7 @@ public class Task implements Runnable {
 						try {
 							boolean success = statefulTask.triggerCheckpoint(checkpointID, checkpointTimestamp);
 							if (!success) {
-								DeclineCheckpoint decline = new DeclineCheckpoint(jobId, getExecutionId(), checkpointID, checkpointTimestamp);
-								jobManager.tell(decline);
+								checkpointResponder.declineCheckpoint(jobId, getExecutionId(), checkpointID, checkpointTimestamp);
 							}
 						}
 						catch (Throwable t) {

http://git-wip-us.apache.org/repos/asf/flink/blob/0735b5b9/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskExecutionStateListener.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskExecutionStateListener.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskExecutionStateListener.java
new file mode 100644
index 0000000..9fa9c90
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskExecutionStateListener.java
@@ -0,0 +1,29 @@
+/*
+ * 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.taskmanager;
+
+public interface TaskExecutionStateListener {
+
+	/**
+	 * Called whenever the task's execution state changes
+	 *
+	 * @param taskExecutionState describing the task execution state change
+	 */
+	void notifyTaskExecutionStateChanged(TaskExecutionState taskExecutionState);
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0735b5b9/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 cb78c16..60beae0 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
@@ -27,55 +27,55 @@ import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider;
 import org.apache.flink.runtime.messages.JobManagerMessages;
 import org.apache.flink.util.InstantiationUtil;
 
+import org.apache.flink.util.Preconditions;
 import scala.concurrent.Await;
 import scala.concurrent.Future;
 import scala.concurrent.duration.FiniteDuration;
 
+/**
+ * Implementation using {@link ActorGateway} to forward the messages.
+ */
 public class TaskInputSplitProvider implements InputSplitProvider {
 
 	private final ActorGateway jobManager;
 	
-	private final JobID jobId;
+	private final JobID jobID;
 	
-	private final JobVertexID vertexId;
+	private final JobVertexID vertexID;
 
 	private final ExecutionAttemptID executionID;
 
-	private final ClassLoader usercodeClassLoader;
-	
 	private final FiniteDuration timeout;
-	
+
+
 	public TaskInputSplitProvider(
-			ActorGateway jobManager,
-			JobID jobId,
-			JobVertexID vertexId,
-			ExecutionAttemptID executionID,
-			ClassLoader userCodeClassLoader,
-			FiniteDuration timeout)
-	{
-		this.jobManager = jobManager;
-		this.jobId = jobId;
-		this.vertexId = vertexId;
-		this.executionID = executionID;
-		this.usercodeClassLoader = userCodeClassLoader;
-		this.timeout = timeout;
+		ActorGateway jobManager,
+		JobID jobID,
+		JobVertexID vertexID,
+		ExecutionAttemptID executionID,
+		FiniteDuration timeout) {
+
+		this.jobManager = Preconditions.checkNotNull(jobManager);
+		this.jobID = Preconditions.checkNotNull(jobID);
+		this.vertexID = Preconditions.checkNotNull(vertexID);
+		this.executionID = Preconditions.checkNotNull(executionID);
+		this.timeout = Preconditions.checkNotNull(timeout);
 	}
 
 	@Override
-	public InputSplit getNextInputSplit() {
+	public InputSplit getNextInputSplit(ClassLoader userCodeClassLoader) {
+		Preconditions.checkNotNull(userCodeClassLoader);
+
 		try {
 			final Future<Object> response = jobManager.ask(
-					new JobManagerMessages.RequestNextInputSplit(jobId, vertexId, executionID),
+					new JobManagerMessages.RequestNextInputSplit(jobID, vertexID, executionID),
 					timeout);
 
 			final Object result = Await.result(response, timeout);
 
-			if(!(result instanceof JobManagerMessages.NextInputSplit)){
-				throw new RuntimeException("RequestNextInputSplit requires a response of type " +
-						"NextInputSplit. Instead response is of type " + result.getClass() + ".");
-			} else {
+			if(result instanceof JobManagerMessages.NextInputSplit){
 				final JobManagerMessages.NextInputSplit nextInputSplit =
-						(JobManagerMessages.NextInputSplit) result;
+					(JobManagerMessages.NextInputSplit) result;
 
 				byte[] serializedData = nextInputSplit.splitData();
 
@@ -83,9 +83,12 @@ public class TaskInputSplitProvider implements InputSplitProvider {
 					return null;
 				} else {
 					Object deserialized = InstantiationUtil.deserializeObject(serializedData,
-							usercodeClassLoader);
+						userCodeClassLoader);
 					return (InputSplit) deserialized;
 				}
+			} else {
+				throw new Exception("RequestNextInputSplit requires a response of type " +
+					"NextInputSplit. Instead response is of type " + result.getClass() + '.');
 			}
 		} catch (Exception e) {
 			throw new RuntimeException("Requesting the next InputSplit failed.", e);

http://git-wip-us.apache.org/repos/asf/flink/blob/0735b5b9/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerConnection.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerConnection.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerConnection.java
new file mode 100644
index 0000000..dc1b40f
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManagerConnection.java
@@ -0,0 +1,57 @@
+/*
+ * 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.taskmanager;
+
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+
+/**
+ * Interface for the communication of the {@link Task} with the {@link TaskManager}.
+ */
+public interface TaskManagerConnection {
+
+	/**
+	 * Notifies the task manager that the given task is in a final state.
+	 *
+	 * @param executionAttemptID Execution attempt ID of the task
+	 */
+	void notifyFinalState(ExecutionAttemptID executionAttemptID);
+
+	/**
+	 * Notifies the task manager about a fatal error occurred in the task.
+	 *
+	 * @param message Message to report
+	 * @param cause Cause of the fatal error
+	 */
+	void notifyFatalError(String message, Throwable cause);
+
+	/**
+	 * Tells the task manager to fail the given task.
+	 *
+	 * @param executionAttemptID Execution attempt ID of the task to fail
+	 * @param cause Cause of the failure
+	 */
+	void failTask(ExecutionAttemptID executionAttemptID, Throwable cause);
+
+	/**
+	 * Notifies the task manager about the task execution state update.
+	 *
+	 * @param taskExecutionState Task execution state update
+	 */
+	void updateTaskExecutionState(TaskExecutionState taskExecutionState);
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0735b5b9/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala
index de85f30..8ebdd80 100644
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala
+++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala
@@ -1149,6 +1149,17 @@ class TaskManager(
       
       val taskMetricGroup = taskManagerMetricGroup.addTaskForJob(tdd)
 
+      val inputSplitProvider = new TaskInputSplitProvider(
+        jobManagerGateway,
+        tdd.getJobID,
+        tdd.getVertexID,
+        tdd.getExecutionId,
+        config.timeout)
+
+      val checkpointResponder = new ActorGatewayCheckpointResponder(jobManagerGateway);
+
+      val taskManagerConnection = new ActorGatewayTaskManagerConnection(selfGateway)
+
       val task = new Task(
         tdd,
         memoryManager,
@@ -1156,9 +1167,9 @@ class TaskManager(
         network,
         jmFactory,
         bcVarManager,
-        selfGateway,
-        jobManagerGateway,
-        config.timeout,
+        taskManagerConnection,
+        inputSplitProvider,
+        checkpointResponder,
         libCache,
         fileCache,
         runtimeInfo,

http://git-wip-us.apache.org/repos/asf/flink/blob/0735b5b9/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockInputSplitProvider.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockInputSplitProvider.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockInputSplitProvider.java
index 1d405f0..a17484f 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockInputSplitProvider.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockInputSplitProvider.java
@@ -84,7 +84,7 @@ public class MockInputSplitProvider implements InputSplitProvider {
 
 
 	@Override
-	public InputSplit getNextInputSplit() {
+	public InputSplit getNextInputSplit(ClassLoader userCodeClassLoader) {
 
 		if (this.nextSplit < this.inputSplits.length) {
 			return this.inputSplits[this.nextSplit++];
@@ -92,5 +92,4 @@ public class MockInputSplitProvider implements InputSplitProvider {
 
 		return null;
 	}
-
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/0735b5b9/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java
index 9501c7c..a5f4019 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java
@@ -22,6 +22,7 @@ import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.core.testutils.OneShotLatch;
+import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider;
 import org.apache.flink.runtime.metrics.groups.TaskMetricGroup;
 import org.apache.flink.runtime.blob.BlobKey;
 import org.apache.flink.runtime.broadcast.BroadcastVariableManager;
@@ -32,8 +33,6 @@ import org.apache.flink.runtime.execution.ExecutionState;
 import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.runtime.filecache.FileCache;
-import org.apache.flink.runtime.instance.ActorGateway;
-import org.apache.flink.runtime.instance.DummyActorGateway;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.io.network.NetworkEnvironment;
 import org.apache.flink.runtime.io.network.partition.ResultPartitionConsumableNotifier;
@@ -52,12 +51,9 @@ import org.apache.flink.util.SerializedValue;
 import org.junit.Before;
 import org.junit.Test;
 
-import scala.concurrent.duration.FiniteDuration;
-
 import java.net.URL;
 import java.util.Collections;
 import java.util.List;
-import java.util.concurrent.TimeUnit;
 
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.fail;
@@ -170,16 +166,15 @@ public class TaskAsyncCallTest {
 				Collections.<URL>emptyList(),
 				0);
 
-		ActorGateway taskManagerGateway = DummyActorGateway.INSTANCE;
 		return new Task(tdd,
 			mock(MemoryManager.class),
 			mock(IOManager.class),
 			networkEnvironment,
 			jobManagerCommunicationFactory,
 			mock(BroadcastVariableManager.class),
-			taskManagerGateway,
-			DummyActorGateway.INSTANCE,
-			new FiniteDuration(60, TimeUnit.SECONDS),
+			mock(TaskManagerConnection.class),
+			mock(InputSplitProvider.class),
+			mock(CheckpointResponder.class),
 			libCache,
 			mock(FileCache.class),
 			new TaskManagerRuntimeInfo("localhost", new Configuration(), System.getProperty("java.io.tmpdir")),

http://git-wip-us.apache.org/repos/asf/flink/blob/0735b5b9/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskInputSplitProviderTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskInputSplitProviderTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskInputSplitProviderTest.java
index 4ccce1d..642300d 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskInputSplitProviderTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskInputSplitProviderTest.java
@@ -47,22 +47,22 @@ public class TaskInputSplitProviderTest {
 
 
 		final TaskInputSplitProvider provider = new TaskInputSplitProvider(
-				gateway,
-				jobID,
-				vertexID,
-				executionID,
-				getClass().getClassLoader(),
-				timeout
-		);
+			gateway,
+			jobID,
+			vertexID,
+			executionID,
+			timeout);
 
 		// The jobManager will return a
-		InputSplit nextInputSplit = provider.getNextInputSplit();
+		InputSplit nextInputSplit = provider.getNextInputSplit(getClass().getClassLoader());
 
 		assertTrue(nextInputSplit == null);
 	}
 
 	public static class NullInputSplitGateway extends BaseTestingActorGateway {
 
+		private static final long serialVersionUID = -7733997150554492926L;
+
 		public NullInputSplitGateway() {
 			super(TestingUtils.defaultExecutionContext());
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/0735b5b9/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskStopTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskStopTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskStopTest.java
index cfa7fb6..d041465 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskStopTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskStopTest.java
@@ -20,6 +20,7 @@ package org.apache.flink.runtime.taskmanager;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.api.common.TaskInfo;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider;
 import org.apache.flink.runtime.metrics.groups.TaskMetricGroup;
 import org.apache.flink.runtime.broadcast.BroadcastVariableManager;
 import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor;
@@ -27,7 +28,6 @@ import org.apache.flink.runtime.execution.ExecutionState;
 import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.runtime.filecache.FileCache;
-import org.apache.flink.runtime.instance.ActorGateway;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.io.network.NetworkEnvironment;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
@@ -73,9 +73,9 @@ public class TaskStopTest {
 			mock(NetworkEnvironment.class),
 			mock(JobManagerCommunicationFactory.class),
 			mock(BroadcastVariableManager.class),
-			mock(ActorGateway.class),
-			mock(ActorGateway.class),
-			mock(FiniteDuration.class),
+			mock(TaskManagerConnection.class),
+			mock(InputSplitProvider.class),
+			mock(CheckpointResponder.class),
 			mock(LibraryCacheManager.class),
 			mock(FileCache.class),
 			mock(TaskManagerRuntimeInfo.class),

http://git-wip-us.apache.org/repos/asf/flink/blob/0735b5b9/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 9e8f8f8..e5fdf32 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
@@ -42,6 +42,7 @@ import org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate;
 import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider;
 import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.messages.TaskMessages;
 import org.apache.flink.runtime.metrics.groups.TaskMetricGroup;
@@ -93,6 +94,9 @@ public class TaskTest {
 	private ActorGateway jobManagerGateway;
 	private ActorGateway listenerGateway;
 
+	private ActorGatewayTaskExecutionStateListener listener;
+	private ActorGatewayTaskManagerConnection taskManagerConnection;
+
 	private BlockingQueue<Object> taskManagerMessages;
 	private BlockingQueue<Object> jobManagerMessages;
 	private BlockingQueue<Object> listenerMessages;
@@ -105,6 +109,9 @@ public class TaskTest {
 		taskManagerGateway = new ForwardingActorGateway(taskManagerMessages);
 		jobManagerGateway = new ForwardingActorGateway(jobManagerMessages);
 		listenerGateway = new ForwardingActorGateway(listenerMessages);
+
+		listener = new ActorGatewayTaskExecutionStateListener(listenerGateway);
+		taskManagerConnection = new ActorGatewayTaskManagerConnection(taskManagerGateway);
 		
 		awaitLatch = new OneShotLatch();
 		triggerLatch = new OneShotLatch();
@@ -135,7 +142,7 @@ public class TaskTest {
 			assertFalse(task.isCanceledOrFailed());
 			assertNull(task.getFailureCause());
 			
-			task.registerExecutionListener(listenerGateway);
+			task.registerExecutionListener(listener);
 			
 			// go into the run method. we should switch to DEPLOYING, RUNNING, then
 			// FINISHED, and all should be good
@@ -210,7 +217,7 @@ public class TaskTest {
 			assertFalse(task.isCanceledOrFailed());
 			assertNull(task.getFailureCause());
 
-			task.registerExecutionListener(listenerGateway);
+			task.registerExecutionListener(listener);
 
 			// should fail
 			task.run();
@@ -253,7 +260,7 @@ public class TaskTest {
 			
 			Task task = createTask(TestInvokableCorrect.class, libCache, network, jobManagerCommunicationFactory);
 
-			task.registerExecutionListener(listenerGateway);
+			task.registerExecutionListener(listener);
 
 			task.run();
 
@@ -274,7 +281,7 @@ public class TaskTest {
 	public void testInvokableInstantiationFailed() {
 		try {
 			Task task = createTask(InvokableNonInstantiable.class);
-			task.registerExecutionListener(listenerGateway);
+			task.registerExecutionListener(listener);
 
 			task.run();
 
@@ -295,7 +302,7 @@ public class TaskTest {
 	public void testExecutionFailsInInvoke() {
 		try {
 			Task task = createTask(InvokableWithExceptionInInvoke.class);
-			task.registerExecutionListener(listenerGateway);
+			task.registerExecutionListener(listener);
 			
 			task.run();
 
@@ -319,7 +326,7 @@ public class TaskTest {
 	public void testCancelDuringInvoke() {
 		try {
 			Task task = createTask(InvokableBlockingInInvoke.class);
-			task.registerExecutionListener(listenerGateway);
+			task.registerExecutionListener(listener);
 
 			// run the task asynchronous
 			task.startTaskThread();
@@ -353,7 +360,7 @@ public class TaskTest {
 	public void testFailExternallyDuringInvoke() {
 		try {
 			Task task = createTask(InvokableBlockingInInvoke.class);
-			task.registerExecutionListener(listenerGateway);
+			task.registerExecutionListener(listener);
 
 			// run the task asynchronous
 			task.startTaskThread();
@@ -386,7 +393,7 @@ public class TaskTest {
 	public void testCanceledAfterExecutionFailedInInvoke() {
 		try {
 			Task task = createTask(InvokableWithExceptionInInvoke.class);
-			task.registerExecutionListener(listenerGateway);
+			task.registerExecutionListener(listener);
 
 			task.run();
 
@@ -413,7 +420,7 @@ public class TaskTest {
 	public void testExecutionFailesAfterCanceling() {
 		try {
 			Task task = createTask(InvokableWithExceptionOnTrigger.class);
-			task.registerExecutionListener(listenerGateway);
+			task.registerExecutionListener(listener);
 
 			// run the task asynchronous
 			task.startTaskThread();
@@ -450,7 +457,7 @@ public class TaskTest {
 	public void testExecutionFailsAfterTaskMarkedFailed() {
 		try {
 			Task task = createTask(InvokableWithExceptionOnTrigger.class);
-			task.registerExecutionListener(listenerGateway);
+			task.registerExecutionListener(listener);
 
 			// run the task asynchronous
 			task.startTaskThread();
@@ -618,21 +625,30 @@ public class TaskTest {
 		JobManagerCommunicationFactory jobManagerCommunicationFactory) {
 		
 		TaskDeploymentDescriptor tdd = createTaskDeploymentDescriptor(invokable);
+
+		InputSplitProvider inputSplitProvider = new TaskInputSplitProvider(
+			jobManagerGateway,
+			tdd.getJobID(),
+			tdd.getVertexID(),
+			tdd.getExecutionId(),
+			new FiniteDuration(60, TimeUnit.SECONDS));
+
+		CheckpointResponder checkpointResponder = new ActorGatewayCheckpointResponder(jobManagerGateway);
 		
 		return new Task(
-				tdd,
-				mock(MemoryManager.class),
-				mock(IOManager.class),
-				networkEnvironment,
+			tdd,
+			mock(MemoryManager.class),
+			mock(IOManager.class),
+			networkEnvironment,
 			jobManagerCommunicationFactory,
-				mock(BroadcastVariableManager.class),
-				taskManagerGateway,
-				jobManagerGateway,
-				new FiniteDuration(60, TimeUnit.SECONDS),
-				libCache,
-				mock(FileCache.class),
-				new TaskManagerRuntimeInfo("localhost", new Configuration(), System.getProperty("java.io.tmpdir")),
-				mock(TaskMetricGroup.class));
+			mock(BroadcastVariableManager.class),
+			taskManagerConnection,
+			inputSplitProvider,
+			checkpointResponder,
+			libCache,
+			mock(FileCache.class),
+			new TaskManagerRuntimeInfo("localhost", new Configuration(), System.getProperty("java.io.tmpdir")),
+			mock(TaskMetricGroup.class));
 	}
 
 	private TaskDeploymentDescriptor createTaskDeploymentDescriptor(Class<? extends AbstractInvokable> invokable) {

http://git-wip-us.apache.org/repos/asf/flink/blob/0735b5b9/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/InputFormatSourceFunction.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/InputFormatSourceFunction.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/InputFormatSourceFunction.java
index e3e5c54..343affe 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/InputFormatSourceFunction.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/InputFormatSourceFunction.java
@@ -146,7 +146,7 @@ public class InputFormatSourceFunction<OUT> extends RichParallelSourceFunction<O
 					return true;
 				}
 
-				InputSplit split = provider.getNextInputSplit();
+				InputSplit split = provider.getNextInputSplit(getRuntimeContext().getUserCodeClassLoader());
 
 				if (split != null) {
 					this.nextSplit = split;

http://git-wip-us.apache.org/repos/asf/flink/blob/0735b5b9/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/source/InputFormatSourceFunctionTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/source/InputFormatSourceFunctionTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/source/InputFormatSourceFunctionTest.java
index a41c7db..d1131b4 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/source/InputFormatSourceFunctionTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/source/InputFormatSourceFunctionTest.java
@@ -84,6 +84,7 @@ public class InputFormatSourceFunctionTest {
 
 	private static class LifeCycleTestInputFormat extends RichInputFormat<Integer,InputSplit> {
 
+		private static final long serialVersionUID = 7408902249499583273L;
 		private boolean isConfigured = false;
 		private boolean isInputFormatOpen = false;
 		private boolean isSplitOpen = false;
@@ -128,6 +129,8 @@ public class InputFormatSourceFunctionTest {
 			for (int i = 0; i < minNumSplits; i++) {
 				final int idx = i;
 				splits[idx] = new InputSplit() {
+					private static final long serialVersionUID = -1480792932361908285L;
+
 					@Override
 					public int getSplitNumber() {
 						return idx;
@@ -270,7 +273,7 @@ public class InputFormatSourceFunctionTest {
 
 			return new InputSplitProvider() {
 				@Override
-				public InputSplit getNextInputSplit() {
+				public InputSplit getNextInputSplit(ClassLoader userCodeClassLoader) {
 					if (nextSplit < inputSplits.length) {
 						return inputSplits[nextSplit++];
 					}

http://git-wip-us.apache.org/repos/asf/flink/blob/0735b5b9/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/InterruptSensitiveRestoreTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/InterruptSensitiveRestoreTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/InterruptSensitiveRestoreTest.java
index e1c9407..47f1bd5 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/InterruptSensitiveRestoreTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/InterruptSensitiveRestoreTest.java
@@ -32,10 +32,10 @@ import org.apache.flink.runtime.execution.ExecutionState;
 import org.apache.flink.runtime.execution.librarycache.FallbackLibraryCacheManager;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.runtime.filecache.FileCache;
-import org.apache.flink.runtime.instance.ActorGateway;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.io.network.NetworkEnvironment;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider;
 import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.operators.testutils.UnregisteredTaskMetricsGroup;
 import org.apache.flink.runtime.query.TaskKvStateRegistry;
@@ -44,7 +44,9 @@ import org.apache.flink.runtime.state.ChainedStateHandle;
 import org.apache.flink.runtime.taskmanager.JobManagerCommunicationFactory;
 import org.apache.flink.runtime.state.KeyGroupsStateHandle;
 import org.apache.flink.runtime.state.StreamStateHandle;
+import org.apache.flink.runtime.taskmanager.CheckpointResponder;
 import org.apache.flink.runtime.taskmanager.Task;
+import org.apache.flink.runtime.taskmanager.TaskManagerConnection;
 import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo;
 import org.apache.flink.runtime.util.EnvironmentInformation;
 import org.apache.flink.streaming.api.TimeCharacteristic;
@@ -56,15 +58,12 @@ import org.apache.flink.util.SerializedValue;
 
 import org.junit.Test;
 
-import scala.concurrent.duration.FiniteDuration;
-
 import java.io.EOFException;
 import java.io.IOException;
 import java.io.Serializable;
 import java.net.URL;
 import java.util.Collections;
 import java.util.List;
-import java.util.concurrent.TimeUnit;
 
 import static org.junit.Assert.*;
 import static org.mockito.Mockito.*;
@@ -153,9 +152,9 @@ public class InterruptSensitiveRestoreTest {
 			networkEnvironment,
 			mock(JobManagerCommunicationFactory.class),
 			mock(BroadcastVariableManager.class),
-			mock(ActorGateway.class),
-			mock(ActorGateway.class),
-			new FiniteDuration(10, TimeUnit.SECONDS),
+				mock(TaskManagerConnection.class),
+				mock(InputSplitProvider.class),
+				mock(CheckpointResponder.class),
 			new FallbackLibraryCacheManager(),
 			new FileCache(new Configuration()),
 			new TaskManagerRuntimeInfo(

http://git-wip-us.apache.org/repos/asf/flink/blob/0735b5b9/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java
index 0a9d2fa..0c79c4e 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java
@@ -41,12 +41,16 @@ import org.apache.flink.runtime.io.network.partition.ResultPartitionConsumableNo
 import org.apache.flink.runtime.io.network.partition.ResultPartitionManager;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider;
 import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.metrics.groups.TaskMetricGroup;
-import org.apache.flink.runtime.messages.TaskMessages;
 import org.apache.flink.runtime.query.TaskKvStateRegistry;
+import org.apache.flink.runtime.taskmanager.CheckpointResponder;
 import org.apache.flink.runtime.taskmanager.JobManagerCommunicationFactory;
 import org.apache.flink.runtime.taskmanager.Task;
+import org.apache.flink.runtime.taskmanager.TaskExecutionState;
+import org.apache.flink.runtime.taskmanager.TaskExecutionStateListener;
+import org.apache.flink.runtime.taskmanager.TaskManagerConnection;
 import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo;
 import org.apache.flink.streaming.api.TimeCharacteristic;
 import org.apache.flink.streaming.api.functions.source.SourceFunction;
@@ -98,12 +102,12 @@ public class StreamTaskTest {
 
 		Task task = createTask(SourceStreamTask.class, cfg);
 
-		ExecutionStateListener executionStateListener = new ExecutionStateListener();
+		TestingExecutionStateListener testingExecutionStateListener = new TestingExecutionStateListener();
 
-		task.registerExecutionListener(executionStateListener);
+		task.registerExecutionListener(testingExecutionStateListener);
 		task.startTaskThread();
 
-		Future<ExecutionState> running = executionStateListener.notifyWhenExecutionState(ExecutionState.RUNNING);
+		Future<ExecutionState> running = testingExecutionStateListener.notifyWhenExecutionState(ExecutionState.RUNNING);
 
 		// wait until the task thread reached state RUNNING
 		ExecutionState executionState = Await.result(running, deadline.timeLeft());
@@ -118,7 +122,7 @@ public class StreamTaskTest {
 		// hit the task before the operator is deserialized
 		task.cancelExecution();
 
-		Future<ExecutionState> canceling = executionStateListener.notifyWhenExecutionState(ExecutionState.CANCELING);
+		Future<ExecutionState> canceling = testingExecutionStateListener.notifyWhenExecutionState(ExecutionState.CANCELING);
 
 		executionState = Await.result(canceling, deadline.timeLeft());
 
@@ -137,9 +141,7 @@ public class StreamTaskTest {
 	//  Test Utilities
 	// ------------------------------------------------------------------------
 
-	private static class ExecutionStateListener implements ActorGateway {
-
-		private static final long serialVersionUID = 8926442805035692182L;
+	private static class TestingExecutionStateListener implements TaskExecutionStateListener {
 
 		ExecutionState executionState = null;
 
@@ -167,56 +169,17 @@ public class StreamTaskTest {
 		}
 
 		@Override
-		public Future<Object> ask(Object message, FiniteDuration timeout) {
-			return null;
-		}
-
-		@Override
-		public void tell(Object message) {
-			this.tell(message, null);
-		}
-
-		@Override
-		public void tell(Object message, ActorGateway sender) {
-			if (message instanceof TaskMessages.UpdateTaskExecutionState) {
-				TaskMessages.UpdateTaskExecutionState updateTaskExecutionState = (TaskMessages.UpdateTaskExecutionState) message;
-
-				synchronized (priorityQueue) {
-					this.executionState = updateTaskExecutionState.taskExecutionState().getExecutionState();
+		public void notifyTaskExecutionStateChanged(TaskExecutionState taskExecutionState) {
+			synchronized (priorityQueue) {
+				this.executionState = taskExecutionState.getExecutionState();
 
-					while (!priorityQueue.isEmpty() && priorityQueue.peek().f0.ordinal() <= this.executionState.ordinal()) {
-						Promise<ExecutionState> promise = priorityQueue.poll().f1;
+				while (!priorityQueue.isEmpty() && priorityQueue.peek().f0.ordinal() <= executionState.ordinal()) {
+					Promise<ExecutionState> promise = priorityQueue.poll().f1;
 
-						promise.success(this.executionState);
-					}
+					promise.success(executionState);
 				}
 			}
 		}
-
-		@Override
-		public void forward(Object message, ActorGateway sender) {
-
-		}
-
-		@Override
-		public Future<Object> retry(Object message, int numberRetries, FiniteDuration timeout, ExecutionContext executionContext) {
-			return null;
-		}
-
-		@Override
-		public String path() {
-			return null;
-		}
-
-		@Override
-		public ActorRef actor() {
-			return null;
-		}
-
-		@Override
-		public UUID leaderSessionID() {
-			return null;
-		}
 	}
 
 	private Task createTask(Class<? extends AbstractInvokable> invokable, StreamConfig taskConfig) throws Exception {
@@ -254,9 +217,9 @@ public class StreamTaskTest {
 			network,
 			jobManagerCommunicationFactory,
 			mock(BroadcastVariableManager.class),
-			new DummyGateway(),
-			new DummyGateway(),
-			new FiniteDuration(60, TimeUnit.SECONDS),
+			mock(TaskManagerConnection.class),
+			mock(InputSplitProvider.class),
+			mock(CheckpointResponder.class),
 			libCache,
 			mock(FileCache.class),
 			new TaskManagerRuntimeInfo("localhost", new Configuration(), System.getProperty("java.io.tmpdir")),


[41/50] [abbrv] flink git commit: [hotfix] [clustermgnt] Set pending registration properly in TaskExecutorToResourceManagerConnection

Posted by tr...@apache.org.
[hotfix] [clustermgnt] Set pending registration properly in TaskExecutorToResourceManagerConnection


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

Branch: refs/heads/flip-6
Commit: 20fefaae286ca3bd61b6d6bde91ca3f5b715c99b
Parents: 282566d
Author: Till Rohrmann <tr...@apache.org>
Authored: Mon Aug 29 17:40:57 2016 +0200
Committer: Till Rohrmann <tr...@apache.org>
Committed: Thu Sep 8 17:27:00 2016 +0200

----------------------------------------------------------------------
 .../TaskExecutorToResourceManagerConnection.java        | 12 ++++++------
 1 file changed, 6 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/20fefaae/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorToResourceManagerConnection.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorToResourceManagerConnection.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorToResourceManagerConnection.java
index f398b7d..7ccc879 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorToResourceManagerConnection.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorToResourceManagerConnection.java
@@ -55,7 +55,7 @@ public class TaskExecutorToResourceManagerConnection {
 
 	private final String resourceManagerAddress;
 
-	private ResourceManagerRegistration pendingRegistration;
+	private TaskExecutorToResourceManagerConnection.ResourceManagerRegistration pendingRegistration;
 
 	private ResourceManagerGateway registeredResourceManager;
 
@@ -86,13 +86,13 @@ public class TaskExecutorToResourceManagerConnection {
 		checkState(!closed, "The connection is already closed");
 		checkState(!isRegistered() && pendingRegistration == null, "The connection is already started");
 
-		ResourceManagerRegistration registration = new ResourceManagerRegistration(
+		pendingRegistration = new TaskExecutorToResourceManagerConnection.ResourceManagerRegistration(
 				log, taskExecutor.getRpcService(),
 				resourceManagerAddress, resourceManagerLeaderId,
 				taskExecutor.getAddress(), taskExecutor.getResourceID());
-		registration.startRegistration();
+		pendingRegistration.startRegistration();
 
-		Future<Tuple2<ResourceManagerGateway, TaskExecutorRegistrationSuccess>> future = registration.getFuture();
+		Future<Tuple2<ResourceManagerGateway, TaskExecutorRegistrationSuccess>> future = pendingRegistration.getFuture();
 		
 		future.onSuccess(new OnSuccess<Tuple2<ResourceManagerGateway, TaskExecutorRegistrationSuccess>>() {
 			@Override
@@ -167,14 +167,14 @@ public class TaskExecutorToResourceManagerConnection {
 	//  Utilities
 	// ------------------------------------------------------------------------
 
-	static class ResourceManagerRegistration
+	private static class ResourceManagerRegistration
 			extends RetryingRegistration<ResourceManagerGateway, TaskExecutorRegistrationSuccess> {
 
 		private final String taskExecutorAddress;
 		
 		private final ResourceID resourceID;
 
-		public ResourceManagerRegistration(
+		ResourceManagerRegistration(
 				Logger log,
 				RpcService rpcService,
 				String targetAddress,


[47/50] [abbrv] flink git commit: [FLINK-4529] [flip-6] Move TaskExecutor, JobMaster and ResourceManager out of the rpc package

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/72468d14/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequest.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequest.java
new file mode 100644
index 0000000..896421b
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotRequest.java
@@ -0,0 +1,74 @@
+/*
+ * 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.resourcemanager;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+
+import java.io.Serializable;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * This describes the requirement of the slot, mainly used by JobManager requesting slot from ResourceManager.
+ */
+public class SlotRequest implements Serializable {
+
+	private static final long serialVersionUID = -6586877187990445986L;
+
+	/** The JobID of the slot requested for */
+	private final JobID jobId;
+
+	/** The unique identification of this request */
+	private final AllocationID allocationId;
+
+	/** The resource profile of the required slot */
+	private final ResourceProfile resourceProfile;
+
+	public SlotRequest(JobID jobId, AllocationID allocationId, ResourceProfile resourceProfile) {
+		this.jobId = checkNotNull(jobId);
+		this.allocationId = checkNotNull(allocationId);
+		this.resourceProfile = checkNotNull(resourceProfile);
+	}
+
+	/**
+	 * Get the JobID of the slot requested for.
+	 * @return The job id
+	 */
+	public JobID getJobId() {
+		return jobId;
+	}
+
+	/**
+	 * Get the unique identification of this request
+	 * @return the allocation id
+	 */
+	public AllocationID getAllocationId() {
+		return allocationId;
+	}
+
+	/**
+	 * Get the resource profile of the desired slot
+	 * @return The resource profile
+	 */
+	public ResourceProfile getResourceProfile() {
+		return resourceProfile;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/72468d14/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/jobmaster/JobMaster.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/jobmaster/JobMaster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/jobmaster/JobMaster.java
deleted file mode 100644
index a046cb8..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/jobmaster/JobMaster.java
+++ /dev/null
@@ -1,244 +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.rpc.jobmaster;
-
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
-import org.apache.flink.runtime.jobgraph.JobGraph;
-import org.apache.flink.runtime.leaderelection.LeaderContender;
-import org.apache.flink.runtime.leaderelection.LeaderElectionService;
-import org.apache.flink.runtime.messages.Acknowledge;
-import org.apache.flink.runtime.rpc.RpcMethod;
-import org.apache.flink.runtime.rpc.resourcemanager.ResourceManagerGateway;
-import org.apache.flink.runtime.rpc.RpcEndpoint;
-import org.apache.flink.runtime.rpc.RpcService;
-import org.apache.flink.runtime.taskmanager.TaskExecutionState;
-import org.apache.flink.util.Preconditions;
-
-import java.util.UUID;
-
-/**
- * JobMaster implementation. The job master is responsible for the execution of a single
- * {@link org.apache.flink.runtime.jobgraph.JobGraph}.
- * <p>
- * It offers the following methods as part of its rpc interface to interact with the JobMaster
- * remotely:
- * <ul>
- *     <li>{@link #updateTaskExecutionState(TaskExecutionState)} updates the task execution state for
- * given task</li>
- * </ul>
- */
-public class JobMaster extends RpcEndpoint<JobMasterGateway> {
-
-	/** Gateway to connected resource manager, null iff not connected */
-	private ResourceManagerGateway resourceManager = null;
-
-	/** Logical representation of the job */
-	private final JobGraph jobGraph;
-	private final JobID jobID;
-
-	/** Configuration of the job */
-	private final Configuration configuration;
-
-	/** Service to contend for and retrieve the leadership of JM and RM */
-	private final HighAvailabilityServices highAvailabilityServices;
-
-	/** Leader Management */
-	private LeaderElectionService leaderElectionService = null;
-	private UUID leaderSessionID;
-
-	/**
-	 * The JM's Constructor
-	 *
-	 * @param jobGraph The representation of the job's execution plan
-	 * @param configuration The job's configuration
-	 * @param rpcService The RPC service at which the JM serves
-	 * @param highAvailabilityService The cluster's HA service from the JM can elect and retrieve leaders.
-	 */
-	public JobMaster(
-		JobGraph jobGraph,
-		Configuration configuration,
-		RpcService rpcService,
-		HighAvailabilityServices highAvailabilityService) {
-
-		super(rpcService);
-
-		this.jobGraph = Preconditions.checkNotNull(jobGraph);
-		this.jobID = Preconditions.checkNotNull(jobGraph.getJobID());
-
-		this.configuration = Preconditions.checkNotNull(configuration);
-
-		this.highAvailabilityServices = Preconditions.checkNotNull(highAvailabilityService);
-	}
-
-	public ResourceManagerGateway getResourceManager() {
-		return resourceManager;
-	}
-
-	//----------------------------------------------------------------------------------------------
-	// Initialization methods
-	//----------------------------------------------------------------------------------------------
-	public void start() {
-		super.start();
-
-		// register at the election once the JM starts
-		registerAtElectionService();
-	}
-
-
-	//----------------------------------------------------------------------------------------------
-	// JobMaster Leadership methods
-	//----------------------------------------------------------------------------------------------
-
-	/**
-	 * Retrieves the election service and contend for the leadership.
-	 */
-	private void registerAtElectionService() {
-		try {
-			leaderElectionService = highAvailabilityServices.getJobMasterLeaderElectionService(jobID);
-			leaderElectionService.start(new JobMasterLeaderContender());
-		} catch (Exception e) {
-			throw new RuntimeException("Fail to register at the election of JobMaster", e);
-		}
-	}
-
-	/**
-	 * Start the execution when the leadership is granted.
-	 *
-	 * @param newLeaderSessionID The identifier of the new leadership session
-	 */
-	public void grantJobMasterLeadership(final UUID newLeaderSessionID) {
-		runAsync(new Runnable() {
-			@Override
-			public void run() {
-				log.info("JobManager {} grants leadership with session id {}.", getAddress(), newLeaderSessionID);
-
-				// The operation may be blocking, but since JM is idle before it grants the leadership, it's okay that
-				// JM waits here for the operation's completeness.
-				leaderSessionID = newLeaderSessionID;
-				leaderElectionService.confirmLeaderSessionID(newLeaderSessionID);
-
-				// TODO:: execute the job when the leadership is granted.
-			}
-		});
-	}
-
-	/**
-	 * Stop the execution when the leadership is revoked.
-	 */
-	public void revokeJobMasterLeadership() {
-		runAsync(new Runnable() {
-			@Override
-			public void run() {
-				log.info("JobManager {} was revoked leadership.", getAddress());
-
-				// TODO:: cancel the job's execution and notify all listeners
-				cancelAndClearEverything(new Exception("JobManager is no longer the leader."));
-
-				leaderSessionID = null;
-			}
-		});
-	}
-
-	/**
-	 * Handles error occurring in the leader election service
-	 *
-	 * @param exception Exception thrown in the leader election service
-	 */
-	public void onJobMasterElectionError(final Exception exception) {
-		runAsync(new Runnable() {
-			@Override
-			public void run() {
-				log.error("Received an error from the LeaderElectionService.", exception);
-
-				// TODO:: cancel the job's execution and shutdown the JM
-				cancelAndClearEverything(exception);
-
-				leaderSessionID = null;
-			}
-		});
-
-	}
-
-	//----------------------------------------------------------------------------------------------
-	// RPC methods
-	//----------------------------------------------------------------------------------------------
-
-	/**
-	 * Updates the task execution state for a given task.
-	 *
-	 * @param taskExecutionState New task execution state for a given task
-	 * @return Acknowledge the task execution state update
-	 */
-	@RpcMethod
-	public Acknowledge updateTaskExecutionState(TaskExecutionState taskExecutionState) {
-		System.out.println("TaskExecutionState: " + taskExecutionState);
-		return Acknowledge.get();
-	}
-
-	/**
-	 * Triggers the registration of the job master at the resource manager.
-	 *
-	 * @param address Address of the resource manager
-	 */
-	@RpcMethod
-	public void registerAtResourceManager(final String address) {
-		//TODO:: register at the RM
-	}
-
-	//----------------------------------------------------------------------------------------------
-	// Helper methods
-	//----------------------------------------------------------------------------------------------
-
-	/**
-	 * Cancel the current job and notify all listeners the job's cancellation.
-	 *
-	 * @param cause Cause for the cancelling.
-	 */
-	private void cancelAndClearEverything(Throwable cause) {
-		// currently, nothing to do here
-	}
-
-	// ------------------------------------------------------------------------
-	//  Utility classes
-	// ------------------------------------------------------------------------
-	private class JobMasterLeaderContender implements LeaderContender {
-
-		@Override
-		public void grantLeadership(UUID leaderSessionID) {
-			JobMaster.this.grantJobMasterLeadership(leaderSessionID);
-		}
-
-		@Override
-		public void revokeLeadership() {
-			JobMaster.this.revokeJobMasterLeadership();
-		}
-
-		@Override
-		public String getAddress() {
-			return JobMaster.this.getAddress();
-		}
-
-		@Override
-		public void handleError(Exception exception) {
-			onJobMasterElectionError(exception);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/72468d14/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/jobmaster/JobMasterGateway.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/jobmaster/JobMasterGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/jobmaster/JobMasterGateway.java
deleted file mode 100644
index 17a4c3a..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/jobmaster/JobMasterGateway.java
+++ /dev/null
@@ -1,45 +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.rpc.jobmaster;
-
-import org.apache.flink.runtime.messages.Acknowledge;
-import org.apache.flink.runtime.rpc.RpcGateway;
-import org.apache.flink.runtime.taskmanager.TaskExecutionState;
-import scala.concurrent.Future;
-
-/**
- * {@link JobMaster} rpc gateway interface
- */
-public interface JobMasterGateway extends RpcGateway {
-
-	/**
-	 * Updates the task execution state for a given task.
-	 *
-	 * @param taskExecutionState New task execution state for a given task
-	 * @return Future acknowledge of the task execution state update
-	 */
-	Future<Acknowledge> updateTaskExecutionState(TaskExecutionState taskExecutionState);
-
-	/**
-	 * Triggers the registration of the job master at the resource manager.
-	 *
-	 * @param address Address of the resource manager
-	 */
-	void registerAtResourceManager(final String address);
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/72468d14/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/registration/RegistrationResponse.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/registration/RegistrationResponse.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/registration/RegistrationResponse.java
deleted file mode 100644
index 2de560a..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/registration/RegistrationResponse.java
+++ /dev/null
@@ -1,84 +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.rpc.registration;
-
-import java.io.Serializable;
-
-/**
- * Base class for responses given to registration attempts from {@link RetryingRegistration}.
- */
-public abstract class RegistrationResponse implements Serializable {
-
-	private static final long serialVersionUID = 1L;
-
-	// ----------------------------------------------------------------------------
-	
-	/**
-	 * Base class for a successful registration. Concrete registration implementations
-	 * will typically extend this class to attach more information.
-	 */
-	public static class Success extends RegistrationResponse {
-		private static final long serialVersionUID = 1L;
-		
-		@Override
-		public String toString() {
-			return "Registration Successful";
-		}
-	}
-
-	// ----------------------------------------------------------------------------
-
-	/**
-	 * A rejected (declined) registration.
-	 */
-	public static final class Decline extends RegistrationResponse {
-		private static final long serialVersionUID = 1L;
-
-		/** the rejection reason */
-		private final String reason;
-
-		/**
-		 * Creates a new rejection message.
-		 * 
-		 * @param reason The reason for the rejection.
-		 */
-		public Decline(String reason) {
-			this.reason = reason != null ? reason : "(unknown)";
-		}
-
-		/**
-		 * Gets the reason for the rejection.
-		 */
-		public String getReason() {
-			return reason;
-		}
-
-		@Override
-		public String toString() {
-			return "Registration Declined (" + reason + ')';
-		}
-	}
-}
-
-
-
-
-
-
-

http://git-wip-us.apache.org/repos/asf/flink/blob/72468d14/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/registration/RetryingRegistration.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/registration/RetryingRegistration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/registration/RetryingRegistration.java
deleted file mode 100644
index dcb5011..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/registration/RetryingRegistration.java
+++ /dev/null
@@ -1,296 +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.rpc.registration;
-
-import akka.dispatch.OnFailure;
-import akka.dispatch.OnSuccess;
-
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.runtime.rpc.RpcGateway;
-import org.apache.flink.runtime.rpc.RpcService;
-
-import org.slf4j.Logger;
-
-import scala.concurrent.Future;
-import scala.concurrent.Promise;
-import scala.concurrent.impl.Promise.DefaultPromise;
-
-import java.util.UUID;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-
-import static org.apache.flink.util.Preconditions.checkArgument;
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-
-/**
- * This utility class implements the basis of registering one component at another component,
- * for example registering the TaskExecutor at the ResourceManager.
- * This {@code RetryingRegistration} implements both the initial address resolution
- * and the retries-with-backoff strategy.
- * 
- * <p>The registration gives access to a future that is completed upon successful registration.
- * The registration can be canceled, for example when the target where it tries to register
- * at looses leader status.
- * 
- * @param <Gateway> The type of the gateway to connect to.
- * @param <Success> The type of the successful registration responses.
- */
-public abstract class RetryingRegistration<Gateway extends RpcGateway, Success extends RegistrationResponse.Success> {
-
-	// ------------------------------------------------------------------------
-	//  default configuration values
-	// ------------------------------------------------------------------------
-
-	/** default value for the initial registration timeout (milliseconds) */
-	private static final long INITIAL_REGISTRATION_TIMEOUT_MILLIS = 100;
-
-	/** default value for the maximum registration timeout, after exponential back-off (milliseconds) */
-	private static final long MAX_REGISTRATION_TIMEOUT_MILLIS = 30000;
-
-	/** The pause (milliseconds) made after an registration attempt caused an exception (other than timeout) */
-	private static final long ERROR_REGISTRATION_DELAY_MILLIS = 10000;
-
-	/** The pause (milliseconds) made after the registration attempt was refused */
-	private static final long REFUSED_REGISTRATION_DELAY_MILLIS = 30000;
-
-	// ------------------------------------------------------------------------
-	// Fields
-	// ------------------------------------------------------------------------
-
-	private final Logger log;
-
-	private final RpcService rpcService;
-
-	private final String targetName;
-
-	private final Class<Gateway> targetType;
-
-	private final String targetAddress;
-
-	private final UUID leaderId;
-
-	private final Promise<Tuple2<Gateway, Success>> completionPromise;
-
-	private final long initialRegistrationTimeout;
-
-	private final long maxRegistrationTimeout;
-
-	private final long delayOnError;
-
-	private final long delayOnRefusedRegistration;
-
-	private volatile boolean canceled;
-
-	// ------------------------------------------------------------------------
-
-	public RetryingRegistration(
-			Logger log,
-			RpcService rpcService,
-			String targetName,
-			Class<Gateway> targetType,
-			String targetAddress,
-			UUID leaderId) {
-		this(log, rpcService, targetName, targetType, targetAddress, leaderId,
-				INITIAL_REGISTRATION_TIMEOUT_MILLIS, MAX_REGISTRATION_TIMEOUT_MILLIS,
-				ERROR_REGISTRATION_DELAY_MILLIS, REFUSED_REGISTRATION_DELAY_MILLIS);
-	}
-
-	public RetryingRegistration(
-			Logger log,
-			RpcService rpcService,
-			String targetName, 
-			Class<Gateway> targetType,
-			String targetAddress,
-			UUID leaderId,
-			long initialRegistrationTimeout,
-			long maxRegistrationTimeout,
-			long delayOnError,
-			long delayOnRefusedRegistration) {
-
-		checkArgument(initialRegistrationTimeout > 0, "initial registration timeout must be greater than zero");
-		checkArgument(maxRegistrationTimeout > 0, "maximum registration timeout must be greater than zero");
-		checkArgument(delayOnError >= 0, "delay on error must be non-negative");
-		checkArgument(delayOnRefusedRegistration >= 0, "delay on refused registration must be non-negative");
-
-		this.log = checkNotNull(log);
-		this.rpcService = checkNotNull(rpcService);
-		this.targetName = checkNotNull(targetName);
-		this.targetType = checkNotNull(targetType);
-		this.targetAddress = checkNotNull(targetAddress);
-		this.leaderId = checkNotNull(leaderId);
-		this.initialRegistrationTimeout = initialRegistrationTimeout;
-		this.maxRegistrationTimeout = maxRegistrationTimeout;
-		this.delayOnError = delayOnError;
-		this.delayOnRefusedRegistration = delayOnRefusedRegistration;
-
-		this.completionPromise = new DefaultPromise<>();
-	}
-
-	// ------------------------------------------------------------------------
-	//  completion and cancellation
-	// ------------------------------------------------------------------------
-
-	public Future<Tuple2<Gateway, Success>> getFuture() {
-		return completionPromise.future();
-	}
-
-	/**
-	 * Cancels the registration procedure.
-	 */
-	public void cancel() {
-		canceled = true;
-	}
-
-	/**
-	 * Checks if the registration was canceled.
-	 * @return True if the registration was canceled, false otherwise.
-	 */
-	public boolean isCanceled() {
-		return canceled;
-	}
-
-	// ------------------------------------------------------------------------
-	//  registration
-	// ------------------------------------------------------------------------
-
-	protected abstract Future<RegistrationResponse> invokeRegistration(
-			Gateway gateway, UUID leaderId, long timeoutMillis) throws Exception;
-
-	/**
-	 * This method resolves the target address to a callable gateway and starts the
-	 * registration after that.
-	 */
-	@SuppressWarnings("unchecked")
-	public void startRegistration() {
-		try {
-			// trigger resolution of the resource manager address to a callable gateway
-			Future<Gateway> resourceManagerFuture = rpcService.connect(targetAddress, targetType);
-	
-			// upon success, start the registration attempts
-			resourceManagerFuture.onSuccess(new OnSuccess<Gateway>() {
-				@Override
-				public void onSuccess(Gateway result) {
-					log.info("Resolved {} address, beginning registration", targetName);
-					register(result, 1, initialRegistrationTimeout);
-				}
-			}, rpcService.getExecutionContext());
-	
-			// upon failure, retry, unless this is cancelled
-			resourceManagerFuture.onFailure(new OnFailure() {
-				@Override
-				public void onFailure(Throwable failure) {
-					if (!isCanceled()) {
-						log.warn("Could not resolve {} address {}, retrying...", targetName, targetAddress);
-						startRegistration();
-					}
-				}
-			}, rpcService.getExecutionContext());
-		}
-		catch (Throwable t) {
-			cancel();
-			completionPromise.tryFailure(t);
-		}
-	}
-
-	/**
-	 * This method performs a registration attempt and triggers either a success notification or a retry,
-	 * depending on the result.
-	 */
-	@SuppressWarnings("unchecked")
-	private void register(final Gateway gateway, final int attempt, final long timeoutMillis) {
-		// eager check for canceling to avoid some unnecessary work
-		if (canceled) {
-			return;
-		}
-
-		try {
-			log.info("Registration at {} attempt {} (timeout={}ms)", targetName, attempt, timeoutMillis);
-			Future<RegistrationResponse> registrationFuture = invokeRegistration(gateway, leaderId, timeoutMillis);
-	
-			// if the registration was successful, let the TaskExecutor know
-			registrationFuture.onSuccess(new OnSuccess<RegistrationResponse>() {
-				
-				@Override
-				public void onSuccess(RegistrationResponse result) throws Throwable {
-					if (!isCanceled()) {
-						if (result instanceof RegistrationResponse.Success) {
-							// registration successful!
-							Success success = (Success) result;
-							completionPromise.success(new Tuple2<>(gateway, success));
-						}
-						else {
-							// registration refused or unknown
-							if (result instanceof RegistrationResponse.Decline) {
-								RegistrationResponse.Decline decline = (RegistrationResponse.Decline) result;
-								log.info("Registration at {} was declined: {}", targetName, decline.getReason());
-							} else {
-								log.error("Received unknown response to registration attempt: " + result);
-							}
-
-							log.info("Pausing and re-attempting registration in {} ms", delayOnRefusedRegistration);
-							registerLater(gateway, 1, initialRegistrationTimeout, delayOnRefusedRegistration);
-						}
-					}
-				}
-			}, rpcService.getExecutionContext());
-	
-			// upon failure, retry
-			registrationFuture.onFailure(new OnFailure() {
-				@Override
-				public void onFailure(Throwable failure) {
-					if (!isCanceled()) {
-						if (failure instanceof TimeoutException) {
-							// we simply have not received a response in time. maybe the timeout was
-							// very low (initial fast registration attempts), maybe the target endpoint is
-							// currently down.
-							if (log.isDebugEnabled()) {
-								log.debug("Registration at {} ({}) attempt {} timed out after {} ms",
-										targetName, targetAddress, attempt, timeoutMillis);
-							}
-	
-							long newTimeoutMillis = Math.min(2 * timeoutMillis, maxRegistrationTimeout);
-							register(gateway, attempt + 1, newTimeoutMillis);
-						}
-						else {
-							// a serious failure occurred. we still should not give up, but keep trying
-							log.error("Registration at " + targetName + " failed due to an error", failure);
-							log.info("Pausing and re-attempting registration in {} ms", delayOnError);
-	
-							registerLater(gateway, 1, initialRegistrationTimeout, delayOnError);
-						}
-					}
-				}
-			}, rpcService.getExecutionContext());
-		}
-		catch (Throwable t) {
-			cancel();
-			completionPromise.tryFailure(t);
-		}
-	}
-
-	private void registerLater(final Gateway gateway, final int attempt, final long timeoutMillis, long delay) {
-		rpcService.scheduleRunnable(new Runnable() {
-			@Override
-			public void run() {
-				register(gateway, attempt, timeoutMillis);
-			}
-		}, delay, TimeUnit.MILLISECONDS);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/72468d14/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/JobMasterRegistration.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/JobMasterRegistration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/JobMasterRegistration.java
deleted file mode 100644
index 7a2deae..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/JobMasterRegistration.java
+++ /dev/null
@@ -1,35 +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.rpc.resourcemanager;
-
-import java.io.Serializable;
-
-public class JobMasterRegistration implements Serializable {
-	private static final long serialVersionUID = 8411214999193765202L;
-
-	private final String address;
-
-	public JobMasterRegistration(String address) {
-		this.address = address;
-	}
-
-	public String getAddress() {
-		return address;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/72468d14/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/RegistrationResponse.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/RegistrationResponse.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/RegistrationResponse.java
deleted file mode 100644
index 8ac9e49..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/RegistrationResponse.java
+++ /dev/null
@@ -1,43 +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.rpc.resourcemanager;
-
-import org.apache.flink.runtime.instance.InstanceID;
-
-import java.io.Serializable;
-
-public class RegistrationResponse implements Serializable {
-	private static final long serialVersionUID = -2379003255993119993L;
-
-	private final boolean isSuccess;
-	private final InstanceID instanceID;
-
-	public RegistrationResponse(boolean isSuccess, InstanceID instanceID) {
-		this.isSuccess = isSuccess;
-		this.instanceID = instanceID;
-	}
-
-	public boolean isSuccess() {
-		return isSuccess;
-	}
-
-	public InstanceID getInstanceID() {
-		return instanceID;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/72468d14/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManager.java
deleted file mode 100644
index f7147c9..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManager.java
+++ /dev/null
@@ -1,214 +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.rpc.resourcemanager;
-
-import akka.dispatch.Mapper;
-
-import org.apache.flink.annotation.VisibleForTesting;
-import org.apache.flink.runtime.clusterframework.types.ResourceID;
-import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
-import org.apache.flink.runtime.instance.InstanceID;
-import org.apache.flink.runtime.leaderelection.LeaderContender;
-import org.apache.flink.runtime.leaderelection.LeaderElectionService;
-import org.apache.flink.runtime.rpc.RpcMethod;
-import org.apache.flink.runtime.rpc.RpcEndpoint;
-import org.apache.flink.runtime.rpc.RpcService;
-import org.apache.flink.runtime.rpc.jobmaster.JobMaster;
-import org.apache.flink.runtime.rpc.jobmaster.JobMasterGateway;
-import org.apache.flink.runtime.rpc.taskexecutor.TaskExecutorRegistrationSuccess;
-
-import scala.concurrent.Future;
-
-import java.util.HashMap;
-import java.util.Map;
-import java.util.UUID;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/**
- * ResourceManager implementation. The resource manager is responsible for resource de-/allocation
- * and bookkeeping.
- *
- * It offers the following methods as part of its rpc interface to interact with the him remotely:
- * <ul>
- *     <li>{@link #registerJobMaster(JobMasterRegistration)} registers a {@link JobMaster} at the resource manager</li>
- *     <li>{@link #requestSlot(SlotRequest)} requests a slot from the resource manager</li>
- * </ul>
- */
-public class ResourceManager extends RpcEndpoint<ResourceManagerGateway> {
-	private final Map<JobMasterGateway, InstanceID> jobMasterGateways;
-	private final HighAvailabilityServices highAvailabilityServices;
-	private LeaderElectionService leaderElectionService = null;
-	private UUID leaderSessionID = null;
-
-	public ResourceManager(RpcService rpcService, HighAvailabilityServices highAvailabilityServices) {
-		super(rpcService);
-		this.highAvailabilityServices = checkNotNull(highAvailabilityServices);
-		this.jobMasterGateways = new HashMap<>();
-	}
-
-	@Override
-	public void start() {
-		// start a leader
-		try {
-			super.start();
-			leaderElectionService = highAvailabilityServices.getResourceManagerLeaderElectionService();
-			leaderElectionService.start(new ResourceManagerLeaderContender());
-		} catch (Throwable e) {
-			log.error("A fatal error happened when starting the ResourceManager", e);
-			throw new RuntimeException("A fatal error happened when starting the ResourceManager", e);
-		}
-	}
-
-	@Override
-	public void shutDown() {
-		try {
-			leaderElectionService.stop();
-			super.shutDown();
-		} catch(Throwable e) {
-			log.error("A fatal error happened when shutdown the ResourceManager", e);
-			throw new RuntimeException("A fatal error happened when shutdown the ResourceManager", e);
-		}
-	}
-
-	/**
-	 * Gets the leader session id of current resourceManager.
-	 *
-	 * @return return the leaderSessionId of current resourceManager, this returns null until the current resourceManager is granted leadership.
-	 */
-	@VisibleForTesting
-	UUID getLeaderSessionID() {
-		return leaderSessionID;
-	}
-
-	/**
-	 * Register a {@link JobMaster} at the resource manager.
-	 *
-	 * @param jobMasterRegistration Job master registration information
-	 * @return Future registration response
-	 */
-	@RpcMethod
-	public Future<RegistrationResponse> registerJobMaster(JobMasterRegistration jobMasterRegistration) {
-		Future<JobMasterGateway> jobMasterFuture = getRpcService().connect(jobMasterRegistration.getAddress(), JobMasterGateway.class);
-
-		return jobMasterFuture.map(new Mapper<JobMasterGateway, RegistrationResponse>() {
-			@Override
-			public RegistrationResponse apply(final JobMasterGateway jobMasterGateway) {
-				InstanceID instanceID;
-
-				if (jobMasterGateways.containsKey(jobMasterGateway)) {
-					instanceID = jobMasterGateways.get(jobMasterGateway);
-				} else {
-					instanceID = new InstanceID();
-					jobMasterGateways.put(jobMasterGateway, instanceID);
-				}
-
-				return new RegistrationResponse(true, instanceID);
-			}
-		}, getMainThreadExecutionContext());
-	}
-
-	/**
-	 * Requests a slot from the resource manager.
-	 *
-	 * @param slotRequest Slot request
-	 * @return Slot assignment
-	 */
-	@RpcMethod
-	public SlotAssignment requestSlot(SlotRequest slotRequest) {
-		System.out.println("SlotRequest: " + slotRequest);
-		return new SlotAssignment();
-	}
-
-
-	/**
-	 *
-	 * @param resourceManagerLeaderId  The fencing token for the ResourceManager leader 
-	 * @param taskExecutorAddress      The address of the TaskExecutor that registers
-	 * @param resourceID               The resource ID of the TaskExecutor that registers
-	 *
-	 * @return The response by the ResourceManager.
-	 */
-	@RpcMethod
-	public org.apache.flink.runtime.rpc.registration.RegistrationResponse registerTaskExecutor(
-			UUID resourceManagerLeaderId,
-			String taskExecutorAddress,
-			ResourceID resourceID) {
-
-		return new TaskExecutorRegistrationSuccess(new InstanceID(), 5000);
-	}
-
-	private class ResourceManagerLeaderContender implements LeaderContender {
-
-		/**
-		 * Callback method when current resourceManager is granted leadership
-		 *
-		 * @param leaderSessionID unique leadershipID
-		 */
-		@Override
-		public void grantLeadership(final UUID leaderSessionID) {
-			runAsync(new Runnable() {
-				@Override
-				public void run() {
-					log.info("ResourceManager {} was granted leadership with leader session ID {}", getAddress(), leaderSessionID);
-					ResourceManager.this.leaderSessionID = leaderSessionID;
-					// confirming the leader session ID might be blocking,
-					leaderElectionService.confirmLeaderSessionID(leaderSessionID);
-				}
-			});
-		}
-
-		/**
-		 * Callback method when current resourceManager lose leadership.
-		 */
-		@Override
-		public void revokeLeadership() {
-			runAsync(new Runnable() {
-				@Override
-				public void run() {
-					log.info("ResourceManager {} was revoked leadership.", getAddress());
-					jobMasterGateways.clear();
-					leaderSessionID = null;
-				}
-			});
-		}
-
-		@Override
-		public String getAddress() {
-			return ResourceManager.this.getAddress();
-		}
-
-		/**
-		 * Handles error occurring in the leader election service
-		 *
-		 * @param exception Exception being thrown in the leader election service
-		 */
-		@Override
-		public void handleError(final Exception exception) {
-			runAsync(new Runnable() {
-				@Override
-				public void run() {
-					log.error("ResourceManager received an error from the LeaderElectionService.", exception);
-					// terminate ResourceManager in case of an error
-					shutDown();
-				}
-			});
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/72468d14/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManagerGateway.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManagerGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManagerGateway.java
deleted file mode 100644
index afddb01..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManagerGateway.java
+++ /dev/null
@@ -1,77 +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.rpc.resourcemanager;
-
-import org.apache.flink.runtime.clusterframework.types.ResourceID;
-import org.apache.flink.runtime.rpc.RpcGateway;
-import org.apache.flink.runtime.rpc.RpcTimeout;
-import org.apache.flink.runtime.rpc.jobmaster.JobMaster;
-
-import scala.concurrent.Future;
-import scala.concurrent.duration.FiniteDuration;
-
-import java.util.UUID;
-
-/**
- * The {@link ResourceManager}'s RPC gateway interface.
- */
-public interface ResourceManagerGateway extends RpcGateway {
-
-	/**
-	 * Register a {@link JobMaster} at the resource manager.
-	 *
-	 * @param jobMasterRegistration Job master registration information
-	 * @param timeout Timeout for the future to complete
-	 * @return Future registration response
-	 */
-	Future<RegistrationResponse> registerJobMaster(
-		JobMasterRegistration jobMasterRegistration,
-		@RpcTimeout FiniteDuration timeout);
-
-	/**
-	 * Register a {@link JobMaster} at the resource manager.
-	 *
-	 * @param jobMasterRegistration Job master registration information
-	 * @return Future registration response
-	 */
-	Future<RegistrationResponse> registerJobMaster(JobMasterRegistration jobMasterRegistration);
-
-	/**
-	 * Requests a slot from the resource manager.
-	 *
-	 * @param slotRequest Slot request
-	 * @return Future slot assignment
-	 */
-	Future<SlotAssignment> requestSlot(SlotRequest slotRequest);
-
-	/**
-	 * 
-	 * @param resourceManagerLeaderId  The fencing token for the ResourceManager leader 
-	 * @param taskExecutorAddress      The address of the TaskExecutor that registers
-	 * @param resourceID               The resource ID of the TaskExecutor that registers
-	 * @param timeout                  The timeout for the response.
-	 * 
-	 * @return The future to the response by the ResourceManager.
-	 */
-	Future<org.apache.flink.runtime.rpc.registration.RegistrationResponse> registerTaskExecutor(
-			UUID resourceManagerLeaderId,
-			String taskExecutorAddress,
-			ResourceID resourceID,
-			@RpcTimeout FiniteDuration timeout);
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/72468d14/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/SlotAssignment.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/SlotAssignment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/SlotAssignment.java
deleted file mode 100644
index 86cd8b7..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/SlotAssignment.java
+++ /dev/null
@@ -1,25 +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.rpc.resourcemanager;
-
-import java.io.Serializable;
-
-public class SlotAssignment implements Serializable{
-	private static final long serialVersionUID = -6990813455942742322L;
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/72468d14/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/SlotRequest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/SlotRequest.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/SlotRequest.java
deleted file mode 100644
index 74c7c39..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/SlotRequest.java
+++ /dev/null
@@ -1,74 +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.rpc.resourcemanager;
-
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.runtime.clusterframework.types.AllocationID;
-import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
-
-import java.io.Serializable;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/**
- * This describes the requirement of the slot, mainly used by JobManager requesting slot from ResourceManager.
- */
-public class SlotRequest implements Serializable {
-
-	private static final long serialVersionUID = -6586877187990445986L;
-
-	/** The JobID of the slot requested for */
-	private final JobID jobId;
-
-	/** The unique identification of this request */
-	private final AllocationID allocationId;
-
-	/** The resource profile of the required slot */
-	private final ResourceProfile resourceProfile;
-
-	public SlotRequest(JobID jobId, AllocationID allocationId, ResourceProfile resourceProfile) {
-		this.jobId = checkNotNull(jobId);
-		this.allocationId = checkNotNull(allocationId);
-		this.resourceProfile = checkNotNull(resourceProfile);
-	}
-
-	/**
-	 * Get the JobID of the slot requested for.
-	 * @return The job id
-	 */
-	public JobID getJobId() {
-		return jobId;
-	}
-
-	/**
-	 * Get the unique identification of this request
-	 * @return the allocation id
-	 */
-	public AllocationID getAllocationId() {
-		return allocationId;
-	}
-
-	/**
-	 * Get the resource profile of the desired slot
-	 * @return The resource profile
-	 */
-	public ResourceProfile getResourceProfile() {
-		return resourceProfile;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/72468d14/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/SlotReport.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/SlotReport.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/SlotReport.java
deleted file mode 100644
index c372ecb..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/SlotReport.java
+++ /dev/null
@@ -1,56 +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.rpc.taskexecutor;
-
-import org.apache.flink.runtime.clusterframework.types.ResourceID;
-
-import java.io.Serializable;
-import java.util.List;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/**
- * A report about the current status of all slots of the TaskExecutor, describing
- * which slots are available and allocated, and what jobs (JobManagers) the allocated slots
- * have been allocated to.
- */
-public class SlotReport implements Serializable {
-
-	private static final long serialVersionUID = -3150175198722481689L;
-
-	/** The slots status of the TaskManager */
-	private final List<SlotStatus> slotsStatus;
-
-	/** The resource id which identifies the TaskManager */
-	private final ResourceID resourceID;
-
-	public SlotReport(final List<SlotStatus> slotsStatus, final ResourceID resourceID) {
-		this.slotsStatus = checkNotNull(slotsStatus);
-		this.resourceID = checkNotNull(resourceID);
-	}
-
-	public List<SlotStatus> getSlotsStatus() {
-		return slotsStatus;
-	}
-
-	public ResourceID getResourceID() {
-		return resourceID;
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/72468d14/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/SlotStatus.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/SlotStatus.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/SlotStatus.java
deleted file mode 100644
index e8e2084..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/SlotStatus.java
+++ /dev/null
@@ -1,129 +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.rpc.taskexecutor;
-
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.runtime.clusterframework.types.AllocationID;
-import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
-import org.apache.flink.runtime.clusterframework.types.SlotID;
-
-import java.io.Serializable;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/**
- * This describes the slot current status which located in TaskManager.
- */
-public class SlotStatus implements Serializable {
-
-	private static final long serialVersionUID = 5099191707339664493L;
-
-	/** slotID to identify a slot */
-	private final SlotID slotID;
-
-	/** the resource profile of the slot */
-	private final ResourceProfile profiler;
-
-	/** if the slot is allocated, allocationId identify its allocation; else, allocationId is null */
-	private final AllocationID allocationID;
-
-	/** if the slot is allocated, jobId identify which job this slot is allocated to; else, jobId is null */
-	private final JobID jobID;
-
-	public SlotStatus(SlotID slotID, ResourceProfile profiler) {
-		this(slotID, profiler, null, null);
-	}
-
-	public SlotStatus(SlotID slotID, ResourceProfile profiler, AllocationID allocationID, JobID jobID) {
-		this.slotID = checkNotNull(slotID, "slotID cannot be null");
-		this.profiler = checkNotNull(profiler, "profile cannot be null");
-		this.allocationID = allocationID;
-		this.jobID = jobID;
-	}
-
-	/**
-	 * Get the unique identification of this slot
-	 *
-	 * @return The slot id
-	 */
-	public SlotID getSlotID() {
-		return slotID;
-	}
-
-	/**
-	 * Get the resource profile of this slot
-	 *
-	 * @return The resource profile
-	 */
-	public ResourceProfile getProfiler() {
-		return profiler;
-	}
-
-	/**
-	 * Get the allocation id of this slot
-	 *
-	 * @return The allocation id if this slot is allocated, otherwise null
-	 */
-	public AllocationID getAllocationID() {
-		return allocationID;
-	}
-
-	/**
-	 * Get the job id of the slot allocated for
-	 *
-	 * @return The job id if this slot is allocated, otherwise null
-	 */
-	public JobID getJobID() {
-		return jobID;
-	}
-
-	@Override
-	public boolean equals(Object o) {
-		if (this == o) {
-			return true;
-		}
-		if (o == null || getClass() != o.getClass()) {
-			return false;
-		}
-
-		SlotStatus that = (SlotStatus) o;
-
-		if (!slotID.equals(that.slotID)) {
-			return false;
-		}
-		if (!profiler.equals(that.profiler)) {
-			return false;
-		}
-		if (allocationID != null ? !allocationID.equals(that.allocationID) : that.allocationID != null) {
-			return false;
-		}
-		return jobID != null ? jobID.equals(that.jobID) : that.jobID == null;
-
-	}
-
-	@Override
-	public int hashCode() {
-		int result = slotID.hashCode();
-		result = 31 * result + profiler.hashCode();
-		result = 31 * result + (allocationID != null ? allocationID.hashCode() : 0);
-		result = 31 * result + (jobID != null ? jobID.hashCode() : 0);
-		return result;
-	}
-
-}


[27/50] [abbrv] flink git commit: [FLINK-4373] [cluster management] Introduce SlotID, AllocationID, ResourceProfile

Posted by tr...@apache.org.
[FLINK-4373] [cluster management] Introduce SlotID, AllocationID, ResourceProfile

[FLINK-4373] [cluster management] address comments

This closes #2370.


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

Branch: refs/heads/flip-6
Commit: e966f8217b8f009410f7ae2a741e65475017c1c3
Parents: dfbbe55
Author: Kurt Young <yk...@gmail.com>
Authored: Fri Aug 12 11:05:48 2016 +0800
Committer: Till Rohrmann <tr...@apache.org>
Committed: Thu Sep 8 17:26:57 2016 +0200

----------------------------------------------------------------------
 .../clusterframework/types/AllocationID.java    | 32 ++++++++
 .../clusterframework/types/ResourceProfile.java | 68 ++++++++++++++++
 .../runtime/clusterframework/types/SlotID.java  | 83 ++++++++++++++++++++
 .../types/ResourceProfileTest.java              | 49 ++++++++++++
 4 files changed, 232 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/e966f821/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/AllocationID.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/AllocationID.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/AllocationID.java
new file mode 100644
index 0000000..f7ae6ee
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/AllocationID.java
@@ -0,0 +1,32 @@
+/*
+ * 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.clusterframework.types;
+
+import org.apache.flink.util.AbstractID;
+
+/**
+ * Unique identifier for the attempt to allocate a slot, normally created by JobManager when requesting a slot,
+ * constant across re-tries. This can also be used to identify responses by the ResourceManager and to identify
+ * deployment calls towards the TaskManager that was allocated from.
+ */
+public class AllocationID extends AbstractID {
+
+	private static final long serialVersionUID = 1L;
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/e966f821/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceProfile.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceProfile.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceProfile.java
new file mode 100644
index 0000000..cbe709f
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceProfile.java
@@ -0,0 +1,68 @@
+/*
+ * 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.clusterframework.types;
+
+import java.io.Serializable;
+
+/**
+ * Describe the resource profile of the slot, either when requiring or offering it. The profile can be
+ * checked whether it can match another profile's requirement, and furthermore we may calculate a matching
+ * score to decide which profile we should choose when we have lots of candidate slots.
+ */
+public class ResourceProfile implements Serializable {
+
+	private static final long serialVersionUID = -784900073893060124L;
+
+	/** How many cpu cores are needed, use double so we can specify cpu like 0.1 */
+	private final double cpuCores;
+
+	/** How many memory in mb are needed */
+	private final long memoryInMB;
+
+	public ResourceProfile(double cpuCores, long memoryInMB) {
+		this.cpuCores = cpuCores;
+		this.memoryInMB = memoryInMB;
+	}
+
+	/**
+	 * Get the cpu cores needed
+	 * @return The cpu cores, 1.0 means a full cpu thread
+	 */
+	public double getCpuCores() {
+		return cpuCores;
+	}
+
+	/**
+	 * Get the memory needed in MB
+	 * @return The memory in MB
+	 */
+	public long getMemoryInMB() {
+		return memoryInMB;
+	}
+
+	/**
+	 * Check whether required resource profile can be matched
+	 *
+	 * @param required the required resource profile
+	 * @return true if the requirement is matched, otherwise false
+	 */
+	public boolean isMatching(ResourceProfile required) {
+		return Double.compare(cpuCores, required.getCpuCores()) >= 0 && memoryInMB >= required.getMemoryInMB();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/e966f821/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/SlotID.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/SlotID.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/SlotID.java
new file mode 100644
index 0000000..d1b072d
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/SlotID.java
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.clusterframework.types;
+
+import java.io.Serializable;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Unique identifier for a slot which located in TaskManager.
+ */
+public class SlotID implements ResourceIDRetrievable, Serializable {
+
+	private static final long serialVersionUID = -6399206032549807771L;
+
+	/** The resource id which this slot located */
+	private final ResourceID resourceId;
+
+	/** The numeric id for single slot */
+	private final int slotId;
+
+	public SlotID(ResourceID resourceId, int slotId) {
+		this.resourceId = checkNotNull(resourceId, "ResourceID must not be null");
+		this.slotId = slotId;
+	}
+
+	// ------------------------------------------------------------------------
+
+	@Override
+	public ResourceID getResourceID() {
+		return resourceId;
+	}
+
+	// ------------------------------------------------------------------------
+
+	@Override
+	public boolean equals(Object o) {
+		if (this == o) {
+			return true;
+		}
+		if (o == null || getClass() != o.getClass()) {
+			return false;
+		}
+
+		SlotID slotID = (SlotID) o;
+
+		if (slotId != slotID.slotId) {
+			return false;
+		}
+		return resourceId.equals(slotID.resourceId);
+	}
+
+	@Override
+	public int hashCode() {
+		int result = resourceId.hashCode();
+		result = 31 * result + slotId;
+		return result;
+	}
+
+	@Override
+	public String toString() {
+		return "SlotID{" +
+			"resourceId=" + resourceId +
+			", slotId=" + slotId +
+			'}';
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/e966f821/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/types/ResourceProfileTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/types/ResourceProfileTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/types/ResourceProfileTest.java
new file mode 100644
index 0000000..bc5ddaa
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/types/ResourceProfileTest.java
@@ -0,0 +1,49 @@
+/*
+ * 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.clusterframework.types;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+public class ResourceProfileTest {
+
+	@Test
+	public void testMatchRequirement() throws Exception {
+		ResourceProfile rp1 = new ResourceProfile(1.0, 100);
+		ResourceProfile rp2 = new ResourceProfile(1.0, 200);
+		ResourceProfile rp3 = new ResourceProfile(2.0, 100);
+		ResourceProfile rp4 = new ResourceProfile(2.0, 200);
+
+		assertFalse(rp1.isMatching(rp2));
+		assertTrue(rp2.isMatching(rp1));
+
+		assertFalse(rp1.isMatching(rp3));
+		assertTrue(rp3.isMatching(rp1));
+
+		assertFalse(rp2.isMatching(rp3));
+		assertFalse(rp3.isMatching(rp2));
+
+		assertTrue(rp4.isMatching(rp1));
+		assertTrue(rp4.isMatching(rp2));
+		assertTrue(rp4.isMatching(rp3));
+		assertTrue(rp4.isMatching(rp4));
+	}
+}


[45/50] [abbrv] flink git commit: [FLINK-4529] [flip-6] Move TaskExecutor, JobMaster and ResourceManager out of the rpc package

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/72468d14/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
new file mode 100644
index 0000000..4871b96
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
@@ -0,0 +1,827 @@
+/*
+ * 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.taskexecutor;
+
+import akka.actor.ActorSystem;
+import akka.dispatch.ExecutionContexts$;
+import akka.util.Timeout;
+import com.typesafe.config.Config;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.IllegalConfigurationException;
+import org.apache.flink.core.memory.HeapMemorySegment;
+import org.apache.flink.core.memory.HybridMemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
+import org.apache.flink.core.memory.MemoryType;
+import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.instance.InstanceConnectionInfo;
+import org.apache.flink.runtime.io.disk.iomanager.IOManager;
+import org.apache.flink.runtime.io.disk.iomanager.IOManager.IOMode;
+import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
+import org.apache.flink.runtime.io.network.NetworkEnvironment;
+import org.apache.flink.runtime.io.network.netty.NettyConfig;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.runtime.memory.MemoryManager;
+import org.apache.flink.runtime.rpc.RpcEndpoint;
+import org.apache.flink.runtime.rpc.RpcMethod;
+import org.apache.flink.runtime.rpc.RpcService;
+import org.apache.flink.runtime.rpc.akka.AkkaRpcService;
+import org.apache.flink.runtime.taskmanager.MemoryLogger;
+import org.apache.flink.runtime.util.EnvironmentInformation;
+import org.apache.flink.runtime.util.LeaderRetrievalUtils;
+import org.apache.flink.runtime.taskmanager.NetworkEnvironmentConfiguration;
+import org.apache.flink.util.MathUtils;
+import org.apache.flink.util.NetUtils;
+
+import scala.Tuple2;
+import scala.Option;
+import scala.Some;
+import scala.concurrent.ExecutionContext;
+import scala.concurrent.duration.Duration;
+import scala.concurrent.duration.FiniteDuration;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.util.UUID;
+import java.util.concurrent.ForkJoinPool;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * TaskExecutor implementation. The task executor is responsible for the execution of multiple
+ * {@link org.apache.flink.runtime.taskmanager.Task}.
+ */
+public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
+
+	private static final Logger LOG = LoggerFactory.getLogger(TaskExecutor.class);
+
+	/** The unique resource ID of this TaskExecutor */
+	private final ResourceID resourceID;
+
+	/** The access to the leader election and metadata storage services */
+	private final HighAvailabilityServices haServices;
+
+	/** The task manager configuration */
+	private final TaskExecutorConfiguration taskExecutorConfig;
+
+	/** The I/O manager component in the task manager */
+	private final IOManager ioManager;
+
+	/** The memory manager component in the task manager */
+	private final MemoryManager memoryManager;
+
+	/** The network component in the task manager */
+	private final NetworkEnvironment networkEnvironment;
+
+	/** The number of slots in the task manager, should be 1 for YARN */
+	private final int numberOfSlots;
+
+	// --------- resource manager --------
+
+	private TaskExecutorToResourceManagerConnection resourceManagerConnection;
+
+	// ------------------------------------------------------------------------
+
+	public TaskExecutor(
+			TaskExecutorConfiguration taskExecutorConfig,
+			ResourceID resourceID,
+			MemoryManager memoryManager,
+			IOManager ioManager,
+			NetworkEnvironment networkEnvironment,
+			int numberOfSlots,
+			RpcService rpcService,
+			HighAvailabilityServices haServices) {
+
+		super(rpcService);
+
+		this.taskExecutorConfig = checkNotNull(taskExecutorConfig);
+		this.resourceID = checkNotNull(resourceID);
+		this.memoryManager = checkNotNull(memoryManager);
+		this.ioManager = checkNotNull(ioManager);
+		this.networkEnvironment = checkNotNull(networkEnvironment);
+		this.numberOfSlots = checkNotNull(numberOfSlots);
+		this.haServices = checkNotNull(haServices);
+	}
+
+	// ------------------------------------------------------------------------
+	//  Life cycle
+	// ------------------------------------------------------------------------
+
+	@Override
+	public void start() {
+		super.start();
+
+		// start by connecting to the ResourceManager
+		try {
+			haServices.getResourceManagerLeaderRetriever().start(new ResourceManagerLeaderListener());
+		} catch (Exception e) {
+			onFatalErrorAsync(e);
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  RPC methods - ResourceManager related
+	// ------------------------------------------------------------------------
+
+	@RpcMethod
+	public void notifyOfNewResourceManagerLeader(String newLeaderAddress, UUID newLeaderId) {
+		if (resourceManagerConnection != null) {
+			if (newLeaderAddress != null) {
+				// the resource manager switched to a new leader
+				log.info("ResourceManager leader changed from {} to {}. Registering at new leader.",
+					resourceManagerConnection.getResourceManagerAddress(), newLeaderAddress);
+			}
+			else {
+				// address null means that the current leader is lost without a new leader being there, yet
+				log.info("Current ResourceManager {} lost leader status. Waiting for new ResourceManager leader.",
+					resourceManagerConnection.getResourceManagerAddress());
+			}
+
+			// drop the current connection or connection attempt
+			if (resourceManagerConnection != null) {
+				resourceManagerConnection.close();
+				resourceManagerConnection = null;
+			}
+		}
+
+		// establish a connection to the new leader
+		if (newLeaderAddress != null) {
+			log.info("Attempting to register at ResourceManager {}", newLeaderAddress);
+			resourceManagerConnection =
+				new TaskExecutorToResourceManagerConnection(log, this, newLeaderAddress, newLeaderId);
+			resourceManagerConnection.start();
+		}
+	}
+
+	/**
+	 * Starts and runs the TaskManager.
+	 * <p/>
+	 * This method first tries to select the network interface to use for the TaskManager
+	 * communication. The network interface is used both for the actor communication
+	 * (coordination) as well as for the data exchange between task managers. Unless
+	 * the hostname/interface is explicitly configured in the configuration, this
+	 * method will try out various interfaces and methods to connect to the JobManager
+	 * and select the one where the connection attempt is successful.
+	 * <p/>
+	 * After selecting the network interface, this method brings up an actor system
+	 * for the TaskManager and its actors, starts the TaskManager's services
+	 * (library cache, shuffle network stack, ...), and starts the TaskManager itself.
+	 *
+	 * @param configuration    The configuration for the TaskManager.
+	 * @param resourceID       The id of the resource which the task manager will run on.
+	 */
+	public static void selectNetworkInterfaceAndRunTaskManager(
+		Configuration configuration,
+		ResourceID resourceID) throws Exception {
+
+		final InetSocketAddress taskManagerAddress = selectNetworkInterfaceAndPort(configuration);
+
+		runTaskManager(taskManagerAddress.getHostName(), resourceID, taskManagerAddress.getPort(), configuration);
+	}
+
+	private static InetSocketAddress selectNetworkInterfaceAndPort(Configuration configuration)
+		throws Exception {
+		String taskManagerHostname = configuration.getString(ConfigConstants.TASK_MANAGER_HOSTNAME_KEY, null);
+		if (taskManagerHostname != null) {
+			LOG.info("Using configured hostname/address for TaskManager: " + taskManagerHostname);
+		} else {
+			LeaderRetrievalService leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
+			FiniteDuration lookupTimeout = AkkaUtils.getLookupTimeout(configuration);
+
+			InetAddress taskManagerAddress = LeaderRetrievalUtils.findConnectingAddress(leaderRetrievalService, lookupTimeout);
+			taskManagerHostname = taskManagerAddress.getHostName();
+			LOG.info("TaskManager will use hostname/address '{}' ({}) for communication.",
+				taskManagerHostname, taskManagerAddress.getHostAddress());
+		}
+
+		// if no task manager port has been configured, use 0 (system will pick any free port)
+		final int actorSystemPort = configuration.getInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, 0);
+		if (actorSystemPort < 0 || actorSystemPort > 65535) {
+			throw new IllegalConfigurationException("Invalid value for '" +
+				ConfigConstants.TASK_MANAGER_IPC_PORT_KEY +
+				"' (port for the TaskManager actor system) : " + actorSystemPort +
+				" - Leave config parameter empty or use 0 to let the system choose a port automatically.");
+		}
+
+		return new InetSocketAddress(taskManagerHostname, actorSystemPort);
+	}
+
+	/**
+	 * Starts and runs the TaskManager. Brings up an actor system for the TaskManager and its
+	 * actors, starts the TaskManager's services (library cache, shuffle network stack, ...),
+	 * and starts the TaskManager itself.
+	 * <p/>
+	 * This method will also spawn a process reaper for the TaskManager (kill the process if
+	 * the actor fails) and optionally start the JVM memory logging thread.
+	 *
+	 * @param taskManagerHostname The hostname/address of the interface where the actor system
+	 *                            will communicate.
+	 * @param resourceID          The id of the resource which the task manager will run on.
+	 * @param actorSystemPort   The port at which the actor system will communicate.
+	 * @param configuration       The configuration for the TaskManager.
+	 */
+	private static void runTaskManager(
+		String taskManagerHostname,
+		ResourceID resourceID,
+		int actorSystemPort,
+		final Configuration configuration) throws Exception {
+
+		LOG.info("Starting TaskManager");
+
+		// Bring up the TaskManager actor system first, bind it to the given address.
+
+		LOG.info("Starting TaskManager actor system at " +
+			NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort));
+
+		final ActorSystem taskManagerSystem;
+		try {
+			Tuple2<String, Object> address = new Tuple2<String, Object>(taskManagerHostname, actorSystemPort);
+			Config akkaConfig = AkkaUtils.getAkkaConfig(configuration, new Some<>(address));
+			LOG.debug("Using akka configuration\n " + akkaConfig);
+			taskManagerSystem = AkkaUtils.createActorSystem(akkaConfig);
+		} catch (Throwable t) {
+			if (t instanceof org.jboss.netty.channel.ChannelException) {
+				Throwable cause = t.getCause();
+				if (cause != null && t.getCause() instanceof java.net.BindException) {
+					String address = NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort);
+					throw new IOException("Unable to bind TaskManager actor system to address " +
+						address + " - " + cause.getMessage(), t);
+				}
+			}
+			throw new Exception("Could not create TaskManager actor system", t);
+		}
+
+		// start akka rpc service based on actor system
+		final Timeout timeout = new Timeout(AkkaUtils.getTimeout(configuration).toMillis(), TimeUnit.MILLISECONDS);
+		final AkkaRpcService akkaRpcService = new AkkaRpcService(taskManagerSystem, timeout);
+
+		// start high availability service to implement getResourceManagerLeaderRetriever method only
+		final HighAvailabilityServices haServices = new HighAvailabilityServices() {
+			@Override
+			public LeaderRetrievalService getResourceManagerLeaderRetriever() throws Exception {
+				return LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
+			}
+
+			@Override
+			public LeaderElectionService getResourceManagerLeaderElectionService() throws Exception {
+				return null;
+			}
+
+			@Override
+			public LeaderElectionService getJobMasterLeaderElectionService(JobID jobID) throws Exception {
+				return null;
+			}
+		};
+
+		// start all the TaskManager services (network stack,  library cache, ...)
+		// and the TaskManager actor
+		try {
+			LOG.info("Starting TaskManager actor");
+			TaskExecutor taskExecutor = startTaskManagerComponentsAndActor(
+				configuration,
+				resourceID,
+				akkaRpcService,
+				taskManagerHostname,
+				haServices,
+				false);
+
+			taskExecutor.start();
+
+			// if desired, start the logging daemon that periodically logs the memory usage information
+			if (LOG.isInfoEnabled() && configuration.getBoolean(
+				ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD,
+				ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD)) {
+				LOG.info("Starting periodic memory usage logger");
+
+				long interval = configuration.getLong(
+					ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS,
+					ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS);
+
+				MemoryLogger logger = new MemoryLogger(LOG, interval, taskManagerSystem);
+				logger.start();
+			}
+
+			// block until everything is done
+			taskManagerSystem.awaitTermination();
+		} catch (Throwable t) {
+			LOG.error("Error while starting up taskManager", t);
+			try {
+				taskManagerSystem.shutdown();
+			} catch (Throwable tt) {
+				LOG.warn("Could not cleanly shut down actor system", tt);
+			}
+			throw t;
+		}
+	}
+
+	// --------------------------------------------------------------------------
+	//  Starting and running the TaskManager
+	// --------------------------------------------------------------------------
+
+	/**
+	 * @param configuration                 The configuration for the TaskManager.
+	 * @param resourceID                    The id of the resource which the task manager will run on.
+	 * @param rpcService                  The rpc service which is used to start and connect to the TaskManager RpcEndpoint .
+	 * @param taskManagerHostname       The hostname/address that describes the TaskManager's data location.
+	 * @param haServices        Optionally, a high availability service can be provided. If none is given,
+	 *                                      then a HighAvailabilityServices is constructed from the configuration.
+	 * @param localTaskManagerCommunication     If true, the TaskManager will not initiate the TCP network stack.
+	 * @return An ActorRef to the TaskManager actor.
+	 * @throws org.apache.flink.configuration.IllegalConfigurationException     Thrown, if the given config contains illegal values.
+	 * @throws java.io.IOException      Thrown, if any of the I/O components (such as buffer pools,
+	 *                                       I/O manager, ...) cannot be properly started.
+	 * @throws java.lang.Exception      Thrown is some other error occurs while parsing the configuration
+	 *                                      or starting the TaskManager components.
+	 */
+	public static TaskExecutor startTaskManagerComponentsAndActor(
+		Configuration configuration,
+		ResourceID resourceID,
+		RpcService rpcService,
+		String taskManagerHostname,
+		HighAvailabilityServices haServices,
+		boolean localTaskManagerCommunication) throws Exception {
+
+		final TaskExecutorConfiguration taskExecutorConfig = parseTaskManagerConfiguration(
+			configuration, taskManagerHostname, localTaskManagerCommunication);
+
+		MemoryType memType = taskExecutorConfig.getNetworkConfig().memoryType();
+
+		// pre-start checks
+		checkTempDirs(taskExecutorConfig.getTmpDirPaths());
+
+		ExecutionContext executionContext = ExecutionContexts$.MODULE$.fromExecutor(new ForkJoinPool());
+
+		// we start the network first, to make sure it can allocate its buffers first
+		final NetworkEnvironment network = new NetworkEnvironment(
+			executionContext,
+			taskExecutorConfig.getTimeout(),
+			taskExecutorConfig.getNetworkConfig(),
+			taskExecutorConfig.getConnectionInfo());
+
+		// computing the amount of memory to use depends on how much memory is available
+		// it strictly needs to happen AFTER the network stack has been initialized
+
+		// check if a value has been configured
+		long configuredMemory = configuration.getLong(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, -1L);
+		checkConfigParameter(configuredMemory == -1 || configuredMemory > 0, configuredMemory,
+			ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY,
+			"MemoryManager needs at least one MB of memory. " +
+				"If you leave this config parameter empty, the system automatically " +
+				"pick a fraction of the available memory.");
+
+		final long memorySize;
+		boolean preAllocateMemory = configuration.getBoolean(
+			ConfigConstants.TASK_MANAGER_MEMORY_PRE_ALLOCATE_KEY,
+			ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_PRE_ALLOCATE);
+		if (configuredMemory > 0) {
+			if (preAllocateMemory) {
+				LOG.info("Using {} MB for managed memory." , configuredMemory);
+			} else {
+				LOG.info("Limiting managed memory to {} MB, memory will be allocated lazily." , configuredMemory);
+			}
+			memorySize = configuredMemory << 20; // megabytes to bytes
+		} else {
+			float fraction = configuration.getFloat(
+				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
+				ConfigConstants.DEFAULT_MEMORY_MANAGER_MEMORY_FRACTION);
+			checkConfigParameter(fraction > 0.0f && fraction < 1.0f, fraction,
+				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
+				"MemoryManager fraction of the free memory must be between 0.0 and 1.0");
+
+			if (memType == MemoryType.HEAP) {
+				long relativeMemSize = (long) (EnvironmentInformation.getSizeOfFreeHeapMemoryWithDefrag() * fraction);
+				if (preAllocateMemory) {
+					LOG.info("Using {} of the currently free heap space for managed heap memory ({} MB)." ,
+						fraction , relativeMemSize >> 20);
+				} else {
+					LOG.info("Limiting managed memory to {} of the currently free heap space ({} MB), " +
+						"memory will be allocated lazily." , fraction , relativeMemSize >> 20);
+				}
+				memorySize = relativeMemSize;
+			} else if (memType == MemoryType.OFF_HEAP) {
+				// The maximum heap memory has been adjusted according to the fraction
+				long maxMemory = EnvironmentInformation.getMaxJvmHeapMemory();
+				long directMemorySize = (long) (maxMemory / (1.0 - fraction) * fraction);
+				if (preAllocateMemory) {
+					LOG.info("Using {} of the maximum memory size for managed off-heap memory ({} MB)." ,
+						fraction, directMemorySize >> 20);
+				} else {
+					LOG.info("Limiting managed memory to {} of the maximum memory size ({} MB)," +
+						" memory will be allocated lazily.", fraction, directMemorySize >> 20);
+				}
+				memorySize = directMemorySize;
+			} else {
+				throw new RuntimeException("No supported memory type detected.");
+			}
+		}
+
+		// now start the memory manager
+		final MemoryManager memoryManager;
+		try {
+			memoryManager = new MemoryManager(
+				memorySize,
+				taskExecutorConfig.getNumberOfSlots(),
+				taskExecutorConfig.getNetworkConfig().networkBufferSize(),
+				memType,
+				preAllocateMemory);
+		} catch (OutOfMemoryError e) {
+			if (memType == MemoryType.HEAP) {
+				throw new Exception("OutOfMemory error (" + e.getMessage() +
+					") while allocating the TaskManager heap memory (" + memorySize + " bytes).", e);
+			} else if (memType == MemoryType.OFF_HEAP) {
+				throw new Exception("OutOfMemory error (" + e.getMessage() +
+					") while allocating the TaskManager off-heap memory (" + memorySize +
+					" bytes).Try increasing the maximum direct memory (-XX:MaxDirectMemorySize)", e);
+			} else {
+				throw e;
+			}
+		}
+
+		// start the I/O manager, it will create some temp directories.
+		final IOManager ioManager = new IOManagerAsync(taskExecutorConfig.getTmpDirPaths());
+
+		final TaskExecutor taskExecutor = new TaskExecutor(
+			taskExecutorConfig,
+			resourceID,
+			memoryManager,
+			ioManager,
+			network,
+			taskExecutorConfig.getNumberOfSlots(),
+			rpcService,
+			haServices);
+
+		return taskExecutor;
+	}
+
+	// --------------------------------------------------------------------------
+	//  Parsing and checking the TaskManager Configuration
+	// --------------------------------------------------------------------------
+
+	/**
+	 * Utility method to extract TaskManager config parameters from the configuration and to
+	 * sanity check them.
+	 *
+	 * @param configuration                 The configuration.
+	 * @param taskManagerHostname           The host name under which the TaskManager communicates.
+	 * @param localTaskManagerCommunication             True, to skip initializing the network stack.
+	 *                                      Use only in cases where only one task manager runs.
+	 * @return TaskExecutorConfiguration that wrappers InstanceConnectionInfo, NetworkEnvironmentConfiguration, etc.
+	 */
+	private static TaskExecutorConfiguration parseTaskManagerConfiguration(
+		Configuration configuration,
+		String taskManagerHostname,
+		boolean localTaskManagerCommunication) throws Exception {
+
+		// ------- read values from the config and check them ---------
+		//                      (a lot of them)
+
+		// ----> hosts / ports for communication and data exchange
+
+		int dataport = configuration.getInteger(ConfigConstants.TASK_MANAGER_DATA_PORT_KEY,
+			ConfigConstants.DEFAULT_TASK_MANAGER_DATA_PORT);
+		if (dataport == 0) {
+			dataport = NetUtils.getAvailablePort();
+		}
+		checkConfigParameter(dataport > 0, dataport, ConfigConstants.TASK_MANAGER_DATA_PORT_KEY,
+			"Leave config parameter empty or use 0 to let the system choose a port automatically.");
+
+		InetAddress taskManagerAddress = InetAddress.getByName(taskManagerHostname);
+		final InstanceConnectionInfo connectionInfo = new InstanceConnectionInfo(taskManagerAddress, dataport);
+
+		// ----> memory / network stack (shuffles/broadcasts), task slots, temp directories
+
+		// we need this because many configs have been written with a "-1" entry
+		int slots = configuration.getInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 1);
+		if (slots == -1) {
+			slots = 1;
+		}
+		checkConfigParameter(slots >= 1, slots, ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS,
+			"Number of task slots must be at least one.");
+
+		final int numNetworkBuffers = configuration.getInteger(
+			ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY,
+			ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_NUM_BUFFERS);
+		checkConfigParameter(numNetworkBuffers > 0, numNetworkBuffers,
+			ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY, "");
+
+		final int pageSize = configuration.getInteger(
+			ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
+			ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_SEGMENT_SIZE);
+		// check page size of for minimum size
+		checkConfigParameter(pageSize >= MemoryManager.MIN_PAGE_SIZE, pageSize,
+			ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
+			"Minimum memory segment size is " + MemoryManager.MIN_PAGE_SIZE);
+		// check page size for power of two
+		checkConfigParameter(MathUtils.isPowerOf2(pageSize), pageSize,
+			ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
+			"Memory segment size must be a power of 2.");
+
+		// check whether we use heap or off-heap memory
+		final MemoryType memType;
+		if (configuration.getBoolean(ConfigConstants.TASK_MANAGER_MEMORY_OFF_HEAP_KEY, false)) {
+			memType = MemoryType.OFF_HEAP;
+		} else {
+			memType = MemoryType.HEAP;
+		}
+
+		// initialize the memory segment factory accordingly
+		if (memType == MemoryType.HEAP) {
+			if (!MemorySegmentFactory.initializeIfNotInitialized(HeapMemorySegment.FACTORY)) {
+				throw new Exception("Memory type is set to heap memory, but memory segment " +
+					"factory has been initialized for off-heap memory segments");
+			}
+		} else {
+			if (!MemorySegmentFactory.initializeIfNotInitialized(HybridMemorySegment.FACTORY)) {
+				throw new Exception("Memory type is set to off-heap memory, but memory segment " +
+					"factory has been initialized for heap memory segments");
+			}
+		}
+
+		final String[] tmpDirs = configuration.getString(
+			ConfigConstants.TASK_MANAGER_TMP_DIR_KEY,
+			ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH).split(",|" + File.pathSeparator);
+
+		final NettyConfig nettyConfig;
+		if (!localTaskManagerCommunication) {
+			nettyConfig = new NettyConfig(connectionInfo.address(), connectionInfo.dataPort(), pageSize, slots, configuration);
+		} else {
+			nettyConfig = null;
+		}
+
+		// Default spill I/O mode for intermediate results
+		final String syncOrAsync = configuration.getString(
+			ConfigConstants.TASK_MANAGER_NETWORK_DEFAULT_IO_MODE,
+			ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_DEFAULT_IO_MODE);
+
+		final IOMode ioMode;
+		if (syncOrAsync.equals("async")) {
+			ioMode = IOManager.IOMode.ASYNC;
+		} else {
+			ioMode = IOManager.IOMode.SYNC;
+		}
+
+		final int queryServerPort =  configuration.getInteger(
+			ConfigConstants.QUERYABLE_STATE_SERVER_PORT,
+			ConfigConstants.DEFAULT_QUERYABLE_STATE_SERVER_PORT);
+
+		final int queryServerNetworkThreads =  configuration.getInteger(
+			ConfigConstants.QUERYABLE_STATE_SERVER_NETWORK_THREADS,
+			ConfigConstants.DEFAULT_QUERYABLE_STATE_SERVER_NETWORK_THREADS);
+
+		final int queryServerQueryThreads =  configuration.getInteger(
+			ConfigConstants.QUERYABLE_STATE_SERVER_QUERY_THREADS,
+			ConfigConstants.DEFAULT_QUERYABLE_STATE_SERVER_QUERY_THREADS);
+
+		final NetworkEnvironmentConfiguration networkConfig = new NetworkEnvironmentConfiguration(
+			numNetworkBuffers,
+			pageSize,
+			memType,
+			ioMode,
+			queryServerPort,
+			queryServerNetworkThreads,
+			queryServerQueryThreads,
+			localTaskManagerCommunication ? Option.<NettyConfig>empty() : new Some<>(nettyConfig),
+			new Tuple2<>(500, 3000));
+
+		// ----> timeouts, library caching, profiling
+
+		final FiniteDuration timeout;
+		try {
+			timeout = AkkaUtils.getTimeout(configuration);
+		} catch (Exception e) {
+			throw new IllegalArgumentException(
+				"Invalid format for '" + ConfigConstants.AKKA_ASK_TIMEOUT +
+					"'.Use formats like '50 s' or '1 min' to specify the timeout.");
+		}
+		LOG.info("Messages between TaskManager and JobManager have a max timeout of " + timeout);
+
+		final long cleanupInterval = configuration.getLong(
+			ConfigConstants.LIBRARY_CACHE_MANAGER_CLEANUP_INTERVAL,
+			ConfigConstants.DEFAULT_LIBRARY_CACHE_MANAGER_CLEANUP_INTERVAL) * 1000;
+
+		final FiniteDuration finiteRegistrationDuration;
+		try {
+			Duration maxRegistrationDuration = Duration.create(configuration.getString(
+				ConfigConstants.TASK_MANAGER_MAX_REGISTRATION_DURATION,
+				ConfigConstants.DEFAULT_TASK_MANAGER_MAX_REGISTRATION_DURATION));
+			if (maxRegistrationDuration.isFinite()) {
+				finiteRegistrationDuration = new FiniteDuration(maxRegistrationDuration.toSeconds(), TimeUnit.SECONDS);
+			} else {
+				finiteRegistrationDuration = null;
+			}
+		} catch (NumberFormatException e) {
+			throw new IllegalArgumentException("Invalid format for parameter " +
+				ConfigConstants.TASK_MANAGER_MAX_REGISTRATION_DURATION, e);
+		}
+
+		final FiniteDuration initialRegistrationPause;
+		try {
+			Duration pause = Duration.create(configuration.getString(
+				ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE,
+				ConfigConstants.DEFAULT_TASK_MANAGER_INITIAL_REGISTRATION_PAUSE));
+			if (pause.isFinite()) {
+				initialRegistrationPause = new FiniteDuration(pause.toSeconds(), TimeUnit.SECONDS);
+			} else {
+				throw new IllegalArgumentException("The initial registration pause must be finite: " + pause);
+			}
+		} catch (NumberFormatException e) {
+			throw new IllegalArgumentException("Invalid format for parameter " +
+				ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE, e);
+		}
+
+		final FiniteDuration maxRegistrationPause;
+		try {
+			Duration pause = Duration.create(configuration.getString(
+				ConfigConstants.TASK_MANAGER_MAX_REGISTARTION_PAUSE,
+				ConfigConstants.DEFAULT_TASK_MANAGER_MAX_REGISTRATION_PAUSE));
+			if (pause.isFinite()) {
+				maxRegistrationPause = new FiniteDuration(pause.toSeconds(), TimeUnit.SECONDS);
+			} else {
+				throw new IllegalArgumentException("The maximum registration pause must be finite: " + pause);
+			}
+		} catch (NumberFormatException e) {
+			throw new IllegalArgumentException("Invalid format for parameter " +
+				ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE, e);
+		}
+
+		final FiniteDuration refusedRegistrationPause;
+		try {
+			Duration pause = Duration.create(configuration.getString(
+				ConfigConstants.TASK_MANAGER_REFUSED_REGISTRATION_PAUSE,
+				ConfigConstants.DEFAULT_TASK_MANAGER_REFUSED_REGISTRATION_PAUSE));
+			if (pause.isFinite()) {
+				refusedRegistrationPause = new FiniteDuration(pause.toSeconds(), TimeUnit.SECONDS);
+			} else {
+				throw new IllegalArgumentException("The refused registration pause must be finite: " + pause);
+			}
+		} catch (NumberFormatException e) {
+			throw new IllegalArgumentException("Invalid format for parameter " +
+				ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE, e);
+		}
+
+		return new TaskExecutorConfiguration(
+			tmpDirs,
+			cleanupInterval,
+			connectionInfo,
+			networkConfig,
+			timeout,
+			finiteRegistrationDuration,
+			slots,
+			configuration,
+			initialRegistrationPause,
+			maxRegistrationPause,
+			refusedRegistrationPause);
+	}
+
+	/**
+	 * Validates a condition for a config parameter and displays a standard exception, if the
+	 * the condition does not hold.
+	 *
+	 * @param condition    The condition that must hold. If the condition is false, an exception is thrown.
+	 * @param parameter    The parameter value. Will be shown in the exception message.
+	 * @param name         The name of the config parameter. Will be shown in the exception message.
+	 * @param errorMessage The optional custom error message to append to the exception message.
+	 */
+	private static void checkConfigParameter(
+		boolean condition,
+		Object parameter,
+		String name,
+		String errorMessage) {
+		if (!condition) {
+			throw new IllegalConfigurationException("Invalid configuration value for " + name + " : " + parameter + " - " + errorMessage);
+		}
+	}
+
+	/**
+	 * Validates that all the directories denoted by the strings do actually exist, are proper
+	 * directories (not files), and are writable.
+	 *
+	 * @param tmpDirs The array of directory paths to check.
+	 * @throws Exception Thrown if any of the directories does not exist or is not writable
+	 *                   or is a file, rather than a directory.
+	 */
+	private static void checkTempDirs(String[] tmpDirs) throws IOException {
+		for (String dir : tmpDirs) {
+			if (dir != null && !dir.equals("")) {
+				File file = new File(dir);
+				if (!file.exists()) {
+					throw new IOException("Temporary file directory " + file.getAbsolutePath() + " does not exist.");
+				}
+				if (!file.isDirectory()) {
+					throw new IOException("Temporary file directory " + file.getAbsolutePath() + " is not a directory.");
+				}
+				if (!file.canWrite()) {
+					throw new IOException("Temporary file directory " + file.getAbsolutePath() + " is not writable.");
+				}
+
+				if (LOG.isInfoEnabled()) {
+					long totalSpaceGb = file.getTotalSpace() >> 30;
+					long usableSpaceGb = file.getUsableSpace() >> 30;
+					double usablePercentage = (double)usableSpaceGb / totalSpaceGb * 100;
+					String path = file.getAbsolutePath();
+					LOG.info(String.format("Temporary file directory '%s': total %d GB, " + "usable %d GB (%.2f%% usable)",
+						path, totalSpaceGb, usableSpaceGb, usablePercentage));
+				}
+			} else {
+				throw new IllegalArgumentException("Temporary file directory #$id is null.");
+			}
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  Properties
+	// ------------------------------------------------------------------------
+
+	public ResourceID getResourceID() {
+		return resourceID;
+	}
+
+	// ------------------------------------------------------------------------
+	//  Error Handling
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Notifies the TaskExecutor that a fatal error has occurred and it cannot proceed.
+	 * This method should be used when asynchronous threads want to notify the
+	 * TaskExecutor of a fatal error.
+	 *
+	 * @param t The exception describing the fatal error
+	 */
+	void onFatalErrorAsync(final Throwable t) {
+		runAsync(new Runnable() {
+			@Override
+			public void run() {
+				onFatalError(t);
+			}
+		});
+	}
+
+	/**
+	 * Notifies the TaskExecutor that a fatal error has occurred and it cannot proceed.
+	 * This method must only be called from within the TaskExecutor's main thread.
+	 *
+	 * @param t The exception describing the fatal error
+	 */
+	void onFatalError(Throwable t) {
+		// to be determined, probably delegate to a fatal error handler that 
+		// would either log (mini cluster) ot kill the process (yarn, mesos, ...)
+		log.error("FATAL ERROR", t);
+	}
+
+	// ------------------------------------------------------------------------
+	//  Access to fields for testing
+	// ------------------------------------------------------------------------
+
+	@VisibleForTesting
+	TaskExecutorToResourceManagerConnection getResourceManagerConnection() {
+		return resourceManagerConnection;
+	}
+
+	// ------------------------------------------------------------------------
+	//  Utility classes
+	// ------------------------------------------------------------------------
+
+	/**
+	 * The listener for leader changes of the resource manager
+	 */
+	private class ResourceManagerLeaderListener implements LeaderRetrievalListener {
+
+		@Override
+		public void notifyLeaderAddress(String leaderAddress, UUID leaderSessionID) {
+			getSelf().notifyOfNewResourceManagerLeader(leaderAddress, leaderSessionID);
+		}
+
+		@Override
+		public void handleError(Exception exception) {
+			onFatalErrorAsync(exception);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/72468d14/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorConfiguration.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorConfiguration.java
new file mode 100644
index 0000000..3707a47
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorConfiguration.java
@@ -0,0 +1,151 @@
+/*
+ * 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.taskexecutor;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.instance.InstanceConnectionInfo;
+import org.apache.flink.runtime.taskmanager.NetworkEnvironmentConfiguration;
+
+import scala.concurrent.duration.FiniteDuration;
+
+import java.io.Serializable;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * {@link TaskExecutor} Configuration
+ */
+public class TaskExecutorConfiguration implements Serializable {
+
+	private static final long serialVersionUID = 1L;
+
+	private final String[] tmpDirPaths;
+
+	private final long cleanupInterval;
+
+	private final int numberOfSlots;
+
+	private final Configuration configuration;
+
+	private final FiniteDuration timeout;
+	private final FiniteDuration maxRegistrationDuration;
+	private final FiniteDuration initialRegistrationPause;
+	private final FiniteDuration maxRegistrationPause;
+	private final FiniteDuration refusedRegistrationPause;
+
+	private final NetworkEnvironmentConfiguration networkConfig;
+
+	private final InstanceConnectionInfo connectionInfo;
+
+	public TaskExecutorConfiguration(
+			String[] tmpDirPaths,
+			long cleanupInterval,
+			InstanceConnectionInfo connectionInfo,
+			NetworkEnvironmentConfiguration networkConfig,
+			FiniteDuration timeout,
+			FiniteDuration maxRegistrationDuration,
+			int numberOfSlots,
+			Configuration configuration) {
+
+		this (tmpDirPaths,
+			cleanupInterval,
+			connectionInfo,
+			networkConfig,
+			timeout,
+			maxRegistrationDuration,
+			numberOfSlots,
+			configuration,
+			new FiniteDuration(500, TimeUnit.MILLISECONDS),
+			new FiniteDuration(30, TimeUnit.SECONDS),
+			new FiniteDuration(10, TimeUnit.SECONDS));
+	}
+
+	public TaskExecutorConfiguration(
+			String[] tmpDirPaths,
+			long cleanupInterval,
+			InstanceConnectionInfo connectionInfo,
+			NetworkEnvironmentConfiguration networkConfig,
+			FiniteDuration timeout,
+			FiniteDuration maxRegistrationDuration,
+			int numberOfSlots,
+			Configuration configuration,
+			FiniteDuration initialRegistrationPause,
+			FiniteDuration maxRegistrationPause,
+			FiniteDuration refusedRegistrationPause) {
+
+		this.tmpDirPaths = checkNotNull(tmpDirPaths);
+		this.cleanupInterval = checkNotNull(cleanupInterval);
+		this.connectionInfo = checkNotNull(connectionInfo);
+		this.networkConfig = checkNotNull(networkConfig);
+		this.timeout = checkNotNull(timeout);
+		this.maxRegistrationDuration = maxRegistrationDuration;
+		this.numberOfSlots = checkNotNull(numberOfSlots);
+		this.configuration = checkNotNull(configuration);
+		this.initialRegistrationPause = checkNotNull(initialRegistrationPause);
+		this.maxRegistrationPause = checkNotNull(maxRegistrationPause);
+		this.refusedRegistrationPause = checkNotNull(refusedRegistrationPause);
+	}
+
+	// --------------------------------------------------------------------------------------------
+	//  Properties
+	// --------------------------------------------------------------------------------------------
+
+	public String[] getTmpDirPaths() {
+		return tmpDirPaths;
+	}
+
+	public long getCleanupInterval() {
+		return cleanupInterval;
+	}
+
+	public InstanceConnectionInfo getConnectionInfo() { return connectionInfo; }
+
+	public NetworkEnvironmentConfiguration getNetworkConfig() { return networkConfig; }
+
+	public FiniteDuration getTimeout() {
+		return timeout;
+	}
+
+	public FiniteDuration getMaxRegistrationDuration() {
+		return maxRegistrationDuration;
+	}
+
+	public int getNumberOfSlots() {
+		return numberOfSlots;
+	}
+
+	public Configuration getConfiguration() {
+		return configuration;
+	}
+
+	public FiniteDuration getInitialRegistrationPause() {
+		return initialRegistrationPause;
+	}
+
+	public FiniteDuration getMaxRegistrationPause() {
+		return maxRegistrationPause;
+	}
+
+	public FiniteDuration getRefusedRegistrationPause() {
+		return refusedRegistrationPause;
+	}
+
+}
+

http://git-wip-us.apache.org/repos/asf/flink/blob/72468d14/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorGateway.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorGateway.java
new file mode 100644
index 0000000..6c99706
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorGateway.java
@@ -0,0 +1,35 @@
+/*
+ * 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.taskexecutor;
+
+import org.apache.flink.runtime.rpc.RpcGateway;
+
+import java.util.UUID;
+
+/**
+ * {@link TaskExecutor} RPC gateway interface
+ */
+public interface TaskExecutorGateway extends RpcGateway {
+
+	// ------------------------------------------------------------------------
+	//  ResourceManager handlers
+	// ------------------------------------------------------------------------
+
+	void notifyOfNewResourceManagerLeader(String address, UUID resourceManagerLeaderId);
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/72468d14/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorRegistrationSuccess.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorRegistrationSuccess.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorRegistrationSuccess.java
new file mode 100644
index 0000000..b357f52
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorRegistrationSuccess.java
@@ -0,0 +1,75 @@
+/*
+ * 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.taskexecutor;
+
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.registration.RegistrationResponse;
+
+import java.io.Serializable;
+
+/**
+ * Base class for responses from the ResourceManager to a registration attempt by a
+ * TaskExecutor.
+ */
+public final class TaskExecutorRegistrationSuccess extends RegistrationResponse.Success implements Serializable {
+
+	private static final long serialVersionUID = 1L;
+
+	private final InstanceID registrationId;
+
+	private final long heartbeatInterval;
+
+	/**
+	 * Create a new {@code TaskExecutorRegistrationSuccess} message.
+	 * 
+	 * @param registrationId     The ID that the ResourceManager assigned the registration.
+	 * @param heartbeatInterval  The interval in which the ResourceManager will heartbeat the TaskExecutor.
+	 */
+	public TaskExecutorRegistrationSuccess(InstanceID registrationId, long heartbeatInterval) {
+		this.registrationId = registrationId;
+		this.heartbeatInterval = heartbeatInterval;
+	}
+
+	/**
+	 * Gets the ID that the ResourceManager assigned the registration.
+	 */
+	public InstanceID getRegistrationId() {
+		return registrationId;
+	}
+
+	/**
+	 * Gets the interval in which the ResourceManager will heartbeat the TaskExecutor.
+	 */
+	public long getHeartbeatInterval() {
+		return heartbeatInterval;
+	}
+
+	@Override
+	public String toString() {
+		return "TaskExecutorRegistrationSuccess (" + registrationId + " / " + heartbeatInterval + ')';
+	}
+
+}
+
+
+
+
+
+
+

http://git-wip-us.apache.org/repos/asf/flink/blob/72468d14/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorToResourceManagerConnection.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorToResourceManagerConnection.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorToResourceManagerConnection.java
new file mode 100644
index 0000000..25332a0
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorToResourceManagerConnection.java
@@ -0,0 +1,198 @@
+/*
+ * 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.taskexecutor;
+
+import akka.dispatch.OnFailure;
+import akka.dispatch.OnSuccess;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.rpc.RpcService;
+import org.apache.flink.runtime.registration.RegistrationResponse;
+import org.apache.flink.runtime.registration.RetryingRegistration;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway;
+
+import org.slf4j.Logger;
+
+import scala.concurrent.Future;
+import scala.concurrent.duration.FiniteDuration;
+
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * The connection between a TaskExecutor and the ResourceManager.
+ */
+public class TaskExecutorToResourceManagerConnection {
+
+	/** the logger for all log messages of this class */
+	private final Logger log;
+
+	/** the TaskExecutor whose connection to the ResourceManager this represents */
+	private final TaskExecutor taskExecutor;
+
+	private final UUID resourceManagerLeaderId;
+
+	private final String resourceManagerAddress;
+
+	private TaskExecutorToResourceManagerConnection.ResourceManagerRegistration pendingRegistration;
+
+	private ResourceManagerGateway registeredResourceManager;
+
+	private InstanceID registrationId;
+
+	/** flag indicating that the connection is closed */
+	private volatile boolean closed;
+
+
+	public TaskExecutorToResourceManagerConnection(
+			Logger log,
+			TaskExecutor taskExecutor,
+			String resourceManagerAddress,
+			UUID resourceManagerLeaderId) {
+
+		this.log = checkNotNull(log);
+		this.taskExecutor = checkNotNull(taskExecutor);
+		this.resourceManagerAddress = checkNotNull(resourceManagerAddress);
+		this.resourceManagerLeaderId = checkNotNull(resourceManagerLeaderId);
+	}
+
+	// ------------------------------------------------------------------------
+	//  Life cycle
+	// ------------------------------------------------------------------------
+
+	@SuppressWarnings("unchecked")
+	public void start() {
+		checkState(!closed, "The connection is already closed");
+		checkState(!isRegistered() && pendingRegistration == null, "The connection is already started");
+
+		pendingRegistration = new TaskExecutorToResourceManagerConnection.ResourceManagerRegistration(
+				log, taskExecutor.getRpcService(),
+				resourceManagerAddress, resourceManagerLeaderId,
+				taskExecutor.getAddress(), taskExecutor.getResourceID());
+		pendingRegistration.startRegistration();
+
+		Future<Tuple2<ResourceManagerGateway, TaskExecutorRegistrationSuccess>> future = pendingRegistration.getFuture();
+		
+		future.onSuccess(new OnSuccess<Tuple2<ResourceManagerGateway, TaskExecutorRegistrationSuccess>>() {
+			@Override
+			public void onSuccess(Tuple2<ResourceManagerGateway, TaskExecutorRegistrationSuccess> result) {
+				registeredResourceManager = result.f0;
+				registrationId = result.f1.getRegistrationId();
+			}
+		}, taskExecutor.getMainThreadExecutionContext());
+		
+		// this future should only ever fail if there is a bug, not if the registration is declined
+		future.onFailure(new OnFailure() {
+			@Override
+			public void onFailure(Throwable failure) {
+				taskExecutor.onFatalError(failure);
+			}
+		}, taskExecutor.getMainThreadExecutionContext());
+	}
+
+	public void close() {
+		closed = true;
+
+		// make sure we do not keep re-trying forever
+		if (pendingRegistration != null) {
+			pendingRegistration.cancel();
+		}
+	}
+
+	public boolean isClosed() {
+		return closed;
+	}
+
+	// ------------------------------------------------------------------------
+	//  Properties
+	// ------------------------------------------------------------------------
+
+	public UUID getResourceManagerLeaderId() {
+		return resourceManagerLeaderId;
+	}
+
+	public String getResourceManagerAddress() {
+		return resourceManagerAddress;
+	}
+
+	/**
+	 * Gets the ResourceManagerGateway. This returns null until the registration is completed.
+	 */
+	public ResourceManagerGateway getResourceManager() {
+		return registeredResourceManager;
+	}
+
+	/**
+	 * Gets the ID under which the TaskExecutor is registered at the ResourceManager.
+	 * This returns null until the registration is completed.
+	 */
+	public InstanceID getRegistrationId() {
+		return registrationId;
+	}
+
+	public boolean isRegistered() {
+		return registeredResourceManager != null;
+	}
+
+	// ------------------------------------------------------------------------
+
+	@Override
+	public String toString() {
+		return String.format("Connection to ResourceManager %s (leaderId=%s)",
+				resourceManagerAddress, resourceManagerLeaderId); 
+	}
+
+	// ------------------------------------------------------------------------
+	//  Utilities
+	// ------------------------------------------------------------------------
+
+	private static class ResourceManagerRegistration
+			extends RetryingRegistration<ResourceManagerGateway, TaskExecutorRegistrationSuccess> {
+
+		private final String taskExecutorAddress;
+		
+		private final ResourceID resourceID;
+
+		ResourceManagerRegistration(
+				Logger log,
+				RpcService rpcService,
+				String targetAddress,
+				UUID leaderId,
+				String taskExecutorAddress,
+				ResourceID resourceID) {
+
+			super(log, rpcService, "ResourceManager", ResourceManagerGateway.class, targetAddress, leaderId);
+			this.taskExecutorAddress = checkNotNull(taskExecutorAddress);
+			this.resourceID = checkNotNull(resourceID);
+		}
+
+		@Override
+		protected Future<RegistrationResponse> invokeRegistration(
+				ResourceManagerGateway resourceManager, UUID leaderId, long timeoutMillis) throws Exception {
+
+			FiniteDuration timeout = new FiniteDuration(timeoutMillis, TimeUnit.MILLISECONDS);
+			return resourceManager.registerTaskExecutor(leaderId, taskExecutorAddress, resourceID, timeout);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/72468d14/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ClusterShutdownITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ClusterShutdownITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ClusterShutdownITCase.java
new file mode 100644
index 0000000..744308c
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ClusterShutdownITCase.java
@@ -0,0 +1,156 @@
+/*
+ * 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.clusterframework;
+
+import akka.actor.ActorSystem;
+import akka.testkit.JavaTestKit;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.clusterframework.ApplicationStatus;
+import org.apache.flink.runtime.clusterframework.messages.StopCluster;
+import org.apache.flink.runtime.clusterframework.messages.StopClusterSuccessful;
+import org.apache.flink.runtime.instance.ActorGateway;
+import org.apache.flink.runtime.messages.Messages;
+import org.apache.flink.runtime.testingUtils.TestingMessages;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.runtime.testutils.TestingResourceManager;
+import org.apache.flink.util.TestLogger;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import scala.Option;
+
+
+/**
+ * Runs tests to ensure that a cluster is shutdown properly.
+ */
+public class ClusterShutdownITCase extends TestLogger {
+
+	private static ActorSystem system;
+
+	private static Configuration config = new Configuration();
+
+	@BeforeClass
+	public static void setup() {
+		system = AkkaUtils.createActorSystem(AkkaUtils.getDefaultAkkaConfig());
+	}
+
+	@AfterClass
+	public static void teardown() {
+		JavaTestKit.shutdownActorSystem(system);
+	}
+
+	/**
+	 * Tests a faked cluster shutdown procedure without the ResourceManager.
+	 */
+	@Test
+	public void testClusterShutdownWithoutResourceManager() {
+
+		new JavaTestKit(system){{
+		new Within(duration("30 seconds")) {
+		@Override
+		protected void run() {
+
+			ActorGateway me =
+				TestingUtils.createForwardingActor(system, getTestActor(), Option.<String>empty());
+
+			// start job manager which doesn't shutdown the actor system
+			ActorGateway jobManager =
+				TestingUtils.createJobManager(system, config, "jobmanager1");
+
+			// Tell the JobManager to inform us of shutdown actions
+			jobManager.tell(TestingMessages.getNotifyOfComponentShutdown(), me);
+
+			// Register a TaskManager
+			ActorGateway taskManager =
+				TestingUtils.createTaskManager(system, jobManager, config, true, true);
+
+			// Tell the TaskManager to inform us of TaskManager shutdowns
+			taskManager.tell(TestingMessages.getNotifyOfComponentShutdown(), me);
+
+
+			// No resource manager connected
+			jobManager.tell(new StopCluster(ApplicationStatus.SUCCEEDED, "Shutting down."), me);
+
+			expectMsgAllOf(
+				new TestingMessages.ComponentShutdown(taskManager.actor()),
+				new TestingMessages.ComponentShutdown(jobManager.actor()),
+				StopClusterSuccessful.getInstance()
+			);
+
+		}};
+		}};
+	}
+
+	/**
+	 * Tests a faked cluster shutdown procedure with the ResourceManager.
+	 */
+	@Test
+	public void testClusterShutdownWithResourceManager() {
+
+		new JavaTestKit(system){{
+		new Within(duration("30 seconds")) {
+		@Override
+		protected void run() {
+
+			ActorGateway me =
+				TestingUtils.createForwardingActor(system, getTestActor(), Option.<String>empty());
+
+			// start job manager which doesn't shutdown the actor system
+			ActorGateway jobManager =
+				TestingUtils.createJobManager(system, config, "jobmanager2");
+
+			// Tell the JobManager to inform us of shutdown actions
+			jobManager.tell(TestingMessages.getNotifyOfComponentShutdown(), me);
+
+			// Register a TaskManager
+			ActorGateway taskManager =
+				TestingUtils.createTaskManager(system, jobManager, config, true, true);
+
+			// Tell the TaskManager to inform us of TaskManager shutdowns
+			taskManager.tell(TestingMessages.getNotifyOfComponentShutdown(), me);
+
+			// Start resource manager and let it register
+			ActorGateway resourceManager =
+				TestingUtils.createResourceManager(system, jobManager.actor(), config);
+
+			// Tell the ResourceManager to inform us of ResourceManager shutdowns
+			resourceManager.tell(TestingMessages.getNotifyOfComponentShutdown(), me);
+
+			// notify about a resource manager registration at the job manager
+			resourceManager.tell(new TestingResourceManager.NotifyWhenResourceManagerConnected(), me);
+
+			// Wait for resource manager
+			expectMsgEquals(Messages.getAcknowledge());
+
+
+			// Shutdown cluster with resource manager connected
+			jobManager.tell(new StopCluster(ApplicationStatus.SUCCEEDED, "Shutting down."), me);
+
+			expectMsgAllOf(
+				new TestingMessages.ComponentShutdown(taskManager.actor()),
+				new TestingMessages.ComponentShutdown(jobManager.actor()),
+				new TestingMessages.ComponentShutdown(resourceManager.actor()),
+				StopClusterSuccessful.getInstance()
+			);
+
+		}};
+		}};
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/72468d14/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ResourceManagerITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ResourceManagerITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ResourceManagerITCase.java
new file mode 100644
index 0000000..1565dc3
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ResourceManagerITCase.java
@@ -0,0 +1,162 @@
+/*
+ * 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.clusterframework;
+
+import akka.actor.ActorSystem;
+import akka.testkit.JavaTestKit;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.instance.ActorGateway;
+import org.apache.flink.runtime.instance.HardwareDescription;
+import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
+import org.apache.flink.runtime.messages.Messages;
+import org.apache.flink.runtime.messages.RegistrationMessages;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.runtime.testutils.TestingResourceManager;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import scala.Option;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+/**
+ * It cases which test the interaction of the resource manager with job manager and task managers.
+ * Runs all tests in one Actor system.
+ */
+public class ResourceManagerITCase extends TestLogger {
+
+	private static ActorSystem system;
+
+	private static Configuration config = new Configuration();
+
+	@BeforeClass
+	public static void setup() {
+		system = AkkaUtils.createActorSystem(AkkaUtils.getDefaultAkkaConfig());
+	}
+
+	@AfterClass
+	public static void teardown() {
+		JavaTestKit.shutdownActorSystem(system);
+	}
+
+	/**
+	 * Tests whether the resource manager connects and reconciles existing task managers.
+	 */
+	@Test
+	public void testResourceManagerReconciliation() {
+
+		new JavaTestKit(system){{
+		new Within(duration("10 seconds")) {
+		@Override
+		protected void run() {
+
+			ActorGateway jobManager =
+				TestingUtils.createJobManager(system, config, "ReconciliationTest");
+			ActorGateway me =
+				TestingUtils.createForwardingActor(system, getTestActor(), Option.<String>empty());
+
+			// !! no resource manager started !!
+
+			ResourceID resourceID = ResourceID.generate();
+
+			TaskManagerLocation location = mock(TaskManagerLocation.class);
+			when(location.getResourceID()).thenReturn(resourceID);
+
+			HardwareDescription resourceProfile = HardwareDescription.extractFromSystem(1_000_000);
+
+			jobManager.tell(
+				new RegistrationMessages.RegisterTaskManager(resourceID, location, resourceProfile, 1),
+				me);
+
+			expectMsgClass(RegistrationMessages.AcknowledgeRegistration.class);
+
+			// now start the resource manager
+			ActorGateway resourceManager =
+				TestingUtils.createResourceManager(system, jobManager.actor(), config);
+
+			// register at testing job manager to receive a message once a resource manager registers
+			resourceManager.tell(new TestingResourceManager.NotifyWhenResourceManagerConnected(), me);
+
+			// Wait for resource manager
+			expectMsgEquals(Messages.getAcknowledge());
+
+			// check if we registered the task manager resource
+			resourceManager.tell(new TestingResourceManager.GetRegisteredResources(), me);
+
+			TestingResourceManager.GetRegisteredResourcesReply reply =
+				expectMsgClass(TestingResourceManager.GetRegisteredResourcesReply.class);
+
+			assertEquals(1, reply.resources.size());
+			assertTrue(reply.resources.contains(resourceID));
+
+		}};
+		}};
+	}
+
+	/**
+	 * Tests whether the resource manager gets informed upon TaskManager registration.
+	 */
+	@Test
+	public void testResourceManagerTaskManagerRegistration() {
+
+		new JavaTestKit(system){{
+		new Within(duration("30 seconds")) {
+		@Override
+		protected void run() {
+
+			ActorGateway jobManager =
+				TestingUtils.createJobManager(system, config, "RegTest");
+			ActorGateway me =
+				TestingUtils.createForwardingActor(system, getTestActor(), Option.<String>empty());
+
+			// start the resource manager
+			ActorGateway resourceManager =
+				TestingUtils.createResourceManager(system, jobManager.actor(), config);
+
+			// notify about a resource manager registration at the job manager
+			resourceManager.tell(new TestingResourceManager.NotifyWhenResourceManagerConnected(), me);
+
+			// Wait for resource manager
+			expectMsgEquals(Messages.getAcknowledge());
+
+			// start task manager and wait for registration
+			ActorGateway taskManager =
+				TestingUtils.createTaskManager(system, jobManager.actor(), config, true, true);
+
+			// check if we registered the task manager resource
+			resourceManager.tell(new TestingResourceManager.GetRegisteredResources(), me);
+
+			TestingResourceManager.GetRegisteredResourcesReply reply =
+				expectMsgClass(TestingResourceManager.GetRegisteredResourcesReply.class);
+
+			assertEquals(1, reply.resources.size());
+
+		}};
+		}};
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/72468d14/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ResourceManagerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ResourceManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ResourceManagerTest.java
new file mode 100644
index 0000000..ca8a07a
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ResourceManagerTest.java
@@ -0,0 +1,338 @@
+/*
+ * 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.clusterframework;
+
+import akka.actor.ActorSystem;
+import akka.testkit.JavaTestKit;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.clusterframework.messages.NotifyResourceStarted;
+import org.apache.flink.runtime.clusterframework.messages.RegisterResourceManager;
+import org.apache.flink.runtime.clusterframework.messages.RegisterResourceManagerSuccessful;
+import org.apache.flink.runtime.clusterframework.messages.RemoveResource;
+import org.apache.flink.runtime.clusterframework.messages.ResourceRemoved;
+import org.apache.flink.runtime.clusterframework.messages.TriggerRegistrationAtJobManager;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.instance.ActorGateway;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.messages.JobManagerMessages;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.runtime.testutils.TestingResourceManager;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import scala.Option;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import static org.junit.Assert.*;
+
+/**
+ * General tests for the resource manager component.
+ */
+public class ResourceManagerTest {
+
+	private static ActorSystem system;
+
+	private static ActorGateway fakeJobManager;
+	private static ActorGateway resourceManager;
+
+	private static Configuration config = new Configuration();
+
+	@BeforeClass
+	public static void setup() {
+		system = AkkaUtils.createLocalActorSystem(config);
+	}
+
+	@AfterClass
+	public static void teardown() {
+		JavaTestKit.shutdownActorSystem(system);
+	}
+
+	/**
+	 * Tests the registration and reconciliation of the ResourceManager with the JobManager
+	 */
+	@Test
+	public void testJobManagerRegistrationAndReconciliation() {
+		new JavaTestKit(system){{
+		new Within(duration("10 seconds")) {
+		@Override
+		protected void run() {
+			fakeJobManager = TestingUtils.createForwardingActor(system, getTestActor(), Option.<String>empty());
+			resourceManager = TestingUtils.createResourceManager(system, fakeJobManager.actor(), config);
+
+			expectMsgClass(RegisterResourceManager.class);
+
+			List<ResourceID> resourceList = new ArrayList<>();
+			resourceList.add(ResourceID.generate());
+			resourceList.add(ResourceID.generate());
+			resourceList.add(ResourceID.generate());
+
+			resourceManager.tell(
+				new RegisterResourceManagerSuccessful(fakeJobManager.actor(), resourceList),
+				fakeJobManager);
+
+			resourceManager.tell(new TestingResourceManager.GetRegisteredResources(), fakeJobManager);
+			TestingResourceManager.GetRegisteredResourcesReply reply =
+				expectMsgClass(TestingResourceManager.GetRegisteredResourcesReply.class);
+
+			for (ResourceID id : resourceList) {
+				if (!reply.resources.contains(id)) {
+					fail("Expected to find all resources that were provided during registration.");
+				}
+			}
+		}};
+		}};
+	}
+
+	/**
+	 * Tests delayed or erroneous registration of the ResourceManager with the JobManager
+	 */
+	@Test
+	public void testDelayedJobManagerRegistration() {
+		new JavaTestKit(system){{
+		new Within(duration("10 seconds")) {
+		@Override
+		protected void run() {
+
+			// set a short timeout for lookups
+			Configuration shortTimeoutConfig = config.clone();
+			shortTimeoutConfig.setString(ConfigConstants.AKKA_LOOKUP_TIMEOUT, "1 s");
+
+			fakeJobManager = TestingUtils.createForwardingActor(system, getTestActor(), Option.<String>empty());
+			resourceManager = TestingUtils.createResourceManager(system, fakeJobManager.actor(), shortTimeoutConfig);
+
+			// wait for registration message
+			RegisterResourceManager msg = expectMsgClass(RegisterResourceManager.class);
+			// give wrong response
+			getLastSender().tell(new JobManagerMessages.LeaderSessionMessage(null, new Object()),
+				fakeJobManager.actor());
+
+			// expect another retry and let it time out
+			expectMsgClass(RegisterResourceManager.class);
+
+			// wait for next try after timeout
+			expectMsgClass(RegisterResourceManager.class);
+
+		}};
+		}};
+	}
+
+	@Test
+	public void testTriggerReconnect() {
+		new JavaTestKit(system){{
+		new Within(duration("10 seconds")) {
+		@Override
+		protected void run() {
+
+			// set a long timeout for lookups such that the test fails in case of timeouts
+			Configuration shortTimeoutConfig = config.clone();
+			shortTimeoutConfig.setString(ConfigConstants.AKKA_LOOKUP_TIMEOUT, "99999 s");
+
+			fakeJobManager = TestingUtils.createForwardingActor(system, getTestActor(), Option.<String>empty());
+			resourceManager = TestingUtils.createResourceManager(system, fakeJobManager.actor(), shortTimeoutConfig);
+
+			// wait for registration message
+			RegisterResourceManager msg = expectMsgClass(RegisterResourceManager.class);
+			// all went well
+			resourceManager.tell(
+				new RegisterResourceManagerSuccessful(fakeJobManager.actor(), Collections.<ResourceID>emptyList()),
+				fakeJobManager);
+
+			// force a reconnect
+			resourceManager.tell(
+				new TriggerRegistrationAtJobManager(fakeJobManager.actor()),
+				fakeJobManager);
+
+			// new registration attempt should come in
+			expectMsgClass(RegisterResourceManager.class);
+
+		}};
+		}};
+	}
+
+	/**
+	 * Tests the registration and accounting of resources at the ResourceManager.
+	 */
+	@Test
+	public void testTaskManagerRegistration() {
+		new JavaTestKit(system){{
+		new Within(duration("10 seconds")) {
+		@Override
+		protected void run() {
+
+			fakeJobManager = TestingUtils.createForwardingActor(system, getTestActor(), Option.<String>empty());
+			resourceManager = TestingUtils.createResourceManager(system, fakeJobManager.actor(), config);
+
+			// register with JM
+			expectMsgClass(RegisterResourceManager.class);
+			resourceManager.tell(
+				new RegisterResourceManagerSuccessful(fakeJobManager.actor(), Collections.<ResourceID>emptyList()),
+				fakeJobManager);
+
+			ResourceID resourceID = ResourceID.generate();
+
+			// Send task manager registration
+			resourceManager.tell(new NotifyResourceStarted(resourceID),
+				fakeJobManager);
+
+			expectMsgClass(Acknowledge.class);
+
+			// check for number registration of registered resources
+			resourceManager.tell(new TestingResourceManager.GetRegisteredResources(), fakeJobManager);
+			TestingResourceManager.GetRegisteredResourcesReply reply =
+				expectMsgClass(TestingResourceManager.GetRegisteredResourcesReply.class);
+
+			assertEquals(1, reply.resources.size());
+
+			// Send task manager registration again
+			resourceManager.tell(new NotifyResourceStarted(resourceID),
+				fakeJobManager);
+
+			expectMsgClass(Acknowledge.class);
+
+			// check for number registration of registered resources
+			resourceManager.tell(new TestingResourceManager.GetRegisteredResources(), fakeJobManager);
+			reply = expectMsgClass(TestingResourceManager.GetRegisteredResourcesReply.class);
+
+			assertEquals(1, reply.resources.size());
+
+			// Send invalid null resource id to throw an exception during resource registration
+			resourceManager.tell(new NotifyResourceStarted(null),
+				fakeJobManager);
+
+			expectMsgClass(Acknowledge.class);
+
+			// check for number registration of registered resources
+			resourceManager.tell(new TestingResourceManager.GetRegisteredResources(), fakeJobManager);
+			reply = expectMsgClass(TestingResourceManager.GetRegisteredResourcesReply.class);
+
+			assertEquals(1, reply.resources.size());
+		}};
+		}};
+	}
+
+	@Test
+	public void testResourceRemoval() {
+		new JavaTestKit(system){{
+		new Within(duration("10 seconds")) {
+		@Override
+		protected void run() {
+
+			fakeJobManager = TestingUtils.createForwardingActor(system, getTestActor(), Option.<String>empty());
+			resourceManager = TestingUtils.createResourceManager(system, fakeJobManager.actor(), config);
+
+			// register with JM
+			expectMsgClass(RegisterResourceManager.class);
+			resourceManager.tell(
+				new RegisterResourceManagerSuccessful(fakeJobManager.actor(), Collections.<ResourceID>emptyList()),
+				fakeJobManager);
+
+			ResourceID resourceID = ResourceID.generate();
+
+			// remove unknown resource
+			resourceManager.tell(new RemoveResource(resourceID), fakeJobManager);
+
+			// Send task manager registration
+			resourceManager.tell(new NotifyResourceStarted(resourceID),
+				fakeJobManager);
+
+			expectMsgClass(Acknowledge.class);
+
+			// check for number registration of registered resources
+			resourceManager.tell(new TestingResourceManager.GetRegisteredResources(), fakeJobManager);
+			TestingResourceManager.GetRegisteredResourcesReply reply =
+				expectMsgClass(TestingResourceManager.GetRegisteredResourcesReply.class);
+
+			assertEquals(1, reply.resources.size());
+			assertTrue(reply.resources.contains(resourceID));
+
+			// remove resource
+			resourceManager.tell(new RemoveResource(resourceID), fakeJobManager);
+
+			// check for number registration of registered resources
+			resourceManager.tell(new TestingResourceManager.GetRegisteredResources(), fakeJobManager);
+			reply =	expectMsgClass(TestingResourceManager.GetRegisteredResourcesReply.class);
+
+			assertEquals(0, reply.resources.size());
+
+		}};
+		}};
+	}
+
+	/**
+	 * Tests notification of JobManager about a failed resource.
+	 */
+	@Test
+	public void testResourceFailureNotification() {
+		new JavaTestKit(system){{
+		new Within(duration("10 seconds")) {
+		@Override
+		protected void run() {
+
+			fakeJobManager = TestingUtils.createForwardingActor(system, getTestActor(), Option.<String>empty());
+			resourceManager = TestingUtils.createResourceManager(system, fakeJobManager.actor(), config);
+
+			// register with JM
+			expectMsgClass(RegisterResourceManager.class);
+			resourceManager.tell(
+				new RegisterResourceManagerSuccessful(fakeJobManager.actor(), Collections.<ResourceID>emptyList()),
+				fakeJobManager);
+
+			ResourceID resourceID1 = ResourceID.generate();
+			ResourceID resourceID2 = ResourceID.generate();
+
+			// Send task manager registration
+			resourceManager.tell(new NotifyResourceStarted(resourceID1),
+				fakeJobManager);
+
+			expectMsgClass(Acknowledge.class);
+
+			// Send task manager registration
+			resourceManager.tell(new NotifyResourceStarted(resourceID2),
+				fakeJobManager);
+
+			expectMsgClass(Acknowledge.class);
+
+			// check for number registration of registered resources
+			resourceManager.tell(new TestingResourceManager.GetRegisteredResources(), fakeJobManager);
+			TestingResourceManager.GetRegisteredResourcesReply reply =
+				expectMsgClass(TestingResourceManager.GetRegisteredResourcesReply.class);
+
+			assertEquals(2, reply.resources.size());
+			assertTrue(reply.resources.contains(resourceID1));
+			assertTrue(reply.resources.contains(resourceID2));
+
+			// fail resources
+			resourceManager.tell(new TestingResourceManager.FailResource(resourceID1), fakeJobManager);
+			resourceManager.tell(new TestingResourceManager.FailResource(resourceID2), fakeJobManager);
+
+			ResourceRemoved answer = expectMsgClass(ResourceRemoved.class);
+			ResourceRemoved answer2 = expectMsgClass(ResourceRemoved.class);
+
+			assertEquals(resourceID1, answer.resourceId());
+			assertEquals(resourceID2, answer2.resourceId());
+
+		}};
+		}};
+	}
+}


[48/50] [abbrv] flink git commit: [FLINK-4529] [flip-6] Move TaskExecutor, JobMaster and ResourceManager out of the rpc package

Posted by tr...@apache.org.
[FLINK-4529] [flip-6] Move TaskExecutor, JobMaster and ResourceManager out of the rpc package

The TaskExecutor, the JobMaster and the ResourceManager were still contained in the rpc
package. With this commit, they will be moved out of this package. Now they are contained
in dedicated packages on the o.a.f.runtime level.

This closes #2438.


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

Branch: refs/heads/flip-6
Commit: 72468d14b390b57d6fb9562fd60272ba564c38cf
Parents: 26082bd
Author: Till Rohrmann <tr...@apache.org>
Authored: Mon Aug 29 16:35:29 2016 +0200
Committer: Till Rohrmann <tr...@apache.org>
Committed: Thu Sep 8 17:27:01 2016 +0200

----------------------------------------------------------------------
 .../runtime/clusterframework/SlotManager.java   | 525 ------------
 .../flink/runtime/jobmaster/JobMaster.java      | 244 ++++++
 .../runtime/jobmaster/JobMasterGateway.java     |  45 +
 .../registration/RegistrationResponse.java      |  84 ++
 .../registration/RetryingRegistration.java      | 296 +++++++
 .../resourcemanager/JobMasterRegistration.java  |  35 +
 .../resourcemanager/RegistrationResponse.java   |  43 +
 .../resourcemanager/ResourceManager.java        | 214 +++++
 .../resourcemanager/ResourceManagerGateway.java |  77 ++
 .../runtime/resourcemanager/SlotAssignment.java |  25 +
 .../runtime/resourcemanager/SlotManager.java    | 523 ++++++++++++
 .../runtime/resourcemanager/SlotRequest.java    |  74 ++
 .../flink/runtime/rpc/jobmaster/JobMaster.java  | 244 ------
 .../runtime/rpc/jobmaster/JobMasterGateway.java |  45 -
 .../rpc/registration/RegistrationResponse.java  |  84 --
 .../rpc/registration/RetryingRegistration.java  | 296 -------
 .../resourcemanager/JobMasterRegistration.java  |  35 -
 .../resourcemanager/RegistrationResponse.java   |  43 -
 .../rpc/resourcemanager/ResourceManager.java    | 214 -----
 .../resourcemanager/ResourceManagerGateway.java |  77 --
 .../rpc/resourcemanager/SlotAssignment.java     |  25 -
 .../rpc/resourcemanager/SlotRequest.java        |  74 --
 .../runtime/rpc/taskexecutor/SlotReport.java    |  56 --
 .../runtime/rpc/taskexecutor/SlotStatus.java    | 129 ---
 .../runtime/rpc/taskexecutor/TaskExecutor.java  | 827 -------------------
 .../taskexecutor/TaskExecutorConfiguration.java | 151 ----
 .../rpc/taskexecutor/TaskExecutorGateway.java   |  35 -
 .../TaskExecutorRegistrationSuccess.java        |  75 --
 ...TaskExecutorToResourceManagerConnection.java | 198 -----
 .../flink/runtime/taskexecutor/SlotReport.java  |  56 ++
 .../flink/runtime/taskexecutor/SlotStatus.java  | 129 +++
 .../runtime/taskexecutor/TaskExecutor.java      | 827 +++++++++++++++++++
 .../taskexecutor/TaskExecutorConfiguration.java | 151 ++++
 .../taskexecutor/TaskExecutorGateway.java       |  35 +
 .../TaskExecutorRegistrationSuccess.java        |  75 ++
 ...TaskExecutorToResourceManagerConnection.java | 198 +++++
 .../clusterframework/ClusterShutdownITCase.java | 156 ++++
 .../clusterframework/ResourceManagerITCase.java | 162 ++++
 .../clusterframework/ResourceManagerTest.java   | 338 ++++++++
 .../clusterframework/SlotManagerTest.java       | 540 ------------
 .../registration/RetryingRegistrationTest.java  | 336 ++++++++
 .../registration/TestRegistrationGateway.java   |  85 ++
 .../resourcemanager/ClusterShutdownITCase.java  | 156 ----
 .../resourcemanager/ResourceManagerHATest.java  |  76 ++
 .../resourcemanager/ResourceManagerITCase.java  | 162 ----
 .../resourcemanager/ResourceManagerTest.java    | 338 --------
 .../resourcemanager/SlotManagerTest.java        | 538 ++++++++++++
 .../runtime/rpc/akka/AkkaRpcServiceTest.java    |  14 -
 .../registration/RetryingRegistrationTest.java  | 336 --------
 .../registration/TestRegistrationGateway.java   |  85 --
 .../resourcemanager/ResourceManagerHATest.java  |  76 --
 .../rpc/taskexecutor/TaskExecutorTest.java      | 117 ---
 .../runtime/taskexecutor/TaskExecutorTest.java  | 117 +++
 53 files changed, 4939 insertions(+), 4957 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/72468d14/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/SlotManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/SlotManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/SlotManager.java
deleted file mode 100644
index cc140a1..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/SlotManager.java
+++ /dev/null
@@ -1,525 +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.clusterframework;
-
-import org.apache.flink.annotation.VisibleForTesting;
-import org.apache.flink.runtime.clusterframework.types.AllocationID;
-import org.apache.flink.runtime.clusterframework.types.ResourceID;
-import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
-import org.apache.flink.runtime.clusterframework.types.ResourceSlot;
-import org.apache.flink.runtime.clusterframework.types.SlotID;
-import org.apache.flink.runtime.rpc.resourcemanager.ResourceManagerGateway;
-import org.apache.flink.runtime.rpc.resourcemanager.SlotRequest;
-import org.apache.flink.runtime.rpc.taskexecutor.SlotReport;
-import org.apache.flink.runtime.rpc.taskexecutor.SlotStatus;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.HashMap;
-import java.util.LinkedHashMap;
-import java.util.Map;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/**
- * SlotManager is responsible for receiving slot requests and do slot allocations. It allows to request
- * slots from registered TaskManagers and issues container allocation requests in case of there are not
- * enough available slots. Besides, it should sync its slot allocation with TaskManager's heartbeat.
- * <p>
- * The main operation principle of SlotManager is:
- * <ul>
- * <li>1. All slot allocation status should be synced with TaskManager, which is the ground truth.</li>
- * <li>2. All slots that have registered must be tracked, either by free pool or allocated pool.</li>
- * <li>3. All slot requests will be handled by best efforts, there is no guarantee that one request will be
- * fulfilled in time or correctly allocated. Conflicts or timeout or some special error will happen, it should
- * be handled outside SlotManager. SlotManager will make each decision based on the information it currently
- * holds.</li>
- * </ul>
- * <b>IMPORTANT:</b> This class is <b>Not Thread-safe</b>.
- */
-public abstract class SlotManager {
-
-	private static final Logger LOG = LoggerFactory.getLogger(SlotManager.class);
-
-	/** Gateway to communicate with ResourceManager */
-	private final ResourceManagerGateway resourceManagerGateway;
-
-	/** All registered slots, including free and allocated slots */
-	private final Map<ResourceID, Map<SlotID, ResourceSlot>> registeredSlots;
-
-	/** All pending slot requests, waiting available slots to fulfil */
-	private final Map<AllocationID, SlotRequest> pendingSlotRequests;
-
-	/** All free slots that can be used to be allocated */
-	private final Map<SlotID, ResourceSlot> freeSlots;
-
-	/** All allocations, we can lookup allocations either by SlotID or AllocationID */
-	private final AllocationMap allocationMap;
-
-	public SlotManager(ResourceManagerGateway resourceManagerGateway) {
-		this.resourceManagerGateway = checkNotNull(resourceManagerGateway);
-		this.registeredSlots = new HashMap<>(16);
-		this.pendingSlotRequests = new LinkedHashMap<>(16);
-		this.freeSlots = new HashMap<>(16);
-		this.allocationMap = new AllocationMap();
-	}
-
-	// ------------------------------------------------------------------------
-	//  slot managements
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Request a slot with requirements, we may either fulfill the request or pending it. Trigger container
-	 * allocation if we don't have enough resource. If we have free slot which can match the request, record
-	 * this allocation and forward the request to TaskManager through ResourceManager (we want this done by
-	 * RPC's main thread to avoid race condition).
-	 *
-	 * @param request The detailed request of the slot
-	 */
-	public void requestSlot(final SlotRequest request) {
-		if (isRequestDuplicated(request)) {
-			LOG.warn("Duplicated slot request, AllocationID:{}", request.getAllocationId());
-			return;
-		}
-
-		// try to fulfil the request with current free slots
-		ResourceSlot slot = chooseSlotToUse(request, freeSlots);
-		if (slot != null) {
-			LOG.info("Assigning SlotID({}) to AllocationID({}), JobID:{}", slot.getSlotId(),
-				request.getAllocationId(), request.getJobId());
-
-			// record this allocation in bookkeeping
-			allocationMap.addAllocation(slot.getSlotId(), request.getAllocationId());
-
-			// remove selected slot from free pool
-			freeSlots.remove(slot.getSlotId());
-
-			// TODO: send slot request to TaskManager
-		} else {
-			LOG.info("Cannot fulfil slot request, try to allocate a new container for it, " +
-				"AllocationID:{}, JobID:{}", request.getAllocationId(), request.getJobId());
-			allocateContainer(request.getResourceProfile());
-			pendingSlotRequests.put(request.getAllocationId(), request);
-		}
-	}
-
-	/**
-	 * Sync slot status with TaskManager's SlotReport.
-	 */
-	public void updateSlotStatus(final SlotReport slotReport) {
-		for (SlotStatus slotStatus : slotReport.getSlotsStatus()) {
-			updateSlotStatus(slotStatus);
-		}
-	}
-
-	/**
-	 * The slot request to TaskManager may be either failed by rpc communication (timeout, network error, etc.)
-	 * or really rejected by TaskManager. We shall retry this request by:
-	 * <ul>
-	 * <li>1. verify and clear all the previous allocate information for this request
-	 * <li>2. try to request slot again
-	 * </ul>
-	 * <p>
-	 * This may cause some duplicate allocation, e.g. the slot request to TaskManager is successful but the response
-	 * is lost somehow, so we may request a slot in another TaskManager, this causes two slots assigned to one request,
-	 * but it can be taken care of by rejecting registration at JobManager.
-	 *
-	 * @param originalRequest The original slot request
-	 * @param slotId          The target SlotID
-	 */
-	public void handleSlotRequestFailedAtTaskManager(final SlotRequest originalRequest, final SlotID slotId) {
-		final AllocationID originalAllocationId = originalRequest.getAllocationId();
-		LOG.info("Slot request failed at TaskManager, SlotID:{}, AllocationID:{}, JobID:{}",
-			slotId, originalAllocationId, originalRequest.getJobId());
-
-		// verify the allocation info before we do anything
-		if (freeSlots.containsKey(slotId)) {
-			// this slot is currently empty, no need to de-allocate it from our allocations
-			LOG.info("Original slot is somehow empty, retrying this request");
-
-			// before retry, we should double check whether this request was allocated by some other ways
-			if (!allocationMap.isAllocated(originalAllocationId)) {
-				requestSlot(originalRequest);
-			} else {
-				LOG.info("The failed request has somehow been allocated, SlotID:{}",
-					allocationMap.getSlotID(originalAllocationId));
-			}
-		} else if (allocationMap.isAllocated(slotId)) {
-			final AllocationID currentAllocationId = allocationMap.getAllocationID(slotId);
-
-			// check whether we have an agreement on whom this slot belongs to
-			if (originalAllocationId.equals(currentAllocationId)) {
-				LOG.info("De-allocate this request and retry");
-				allocationMap.removeAllocation(currentAllocationId);
-
-				// put this slot back to free pool
-				ResourceSlot slot = checkNotNull(getRegisteredSlot(slotId));
-				freeSlots.put(slotId, slot);
-
-				// retry the request
-				requestSlot(originalRequest);
-			} else {
-				// the slot is taken by someone else, no need to de-allocate it from our allocations
-				LOG.info("Original slot is taken by someone else, current AllocationID:{}", currentAllocationId);
-
-				// before retry, we should double check whether this request was allocated by some other ways
-				if (!allocationMap.isAllocated(originalAllocationId)) {
-					requestSlot(originalRequest);
-				} else {
-					LOG.info("The failed request is somehow been allocated, SlotID:{}",
-						allocationMap.getSlotID(originalAllocationId));
-				}
-			}
-		} else {
-			LOG.error("BUG! {} is neither in free pool nor in allocated pool", slotId);
-		}
-	}
-
-	/**
-	 * Callback for TaskManager failures. In case that a TaskManager fails, we have to clean up all its slots.
-	 *
-	 * @param resourceId The ResourceID of the TaskManager
-	 */
-	public void notifyTaskManagerFailure(final ResourceID resourceId) {
-		LOG.info("Resource:{} been notified failure", resourceId);
-		final Map<SlotID, ResourceSlot> slotIdsToRemove = registeredSlots.remove(resourceId);
-		if (slotIdsToRemove != null) {
-			for (SlotID slotId : slotIdsToRemove.keySet()) {
-				LOG.info("Removing Slot:{} upon resource failure", slotId);
-				if (freeSlots.containsKey(slotId)) {
-					freeSlots.remove(slotId);
-				} else if (allocationMap.isAllocated(slotId)) {
-					allocationMap.removeAllocation(slotId);
-				} else {
-					LOG.error("BUG! {} is neither in free pool nor in allocated pool", slotId);
-				}
-			}
-		}
-	}
-
-	// ------------------------------------------------------------------------
-	//  internal behaviors
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Update slot status based on TaskManager's report. There are mainly two situations when we receive the report:
-	 * <ul>
-	 * <li>1. The slot is newly registered.</li>
-	 * <li>2. The slot has registered, it contains its current status.</li>
-	 * </ul>
-	 * <p>
-	 * Regarding 1: It's fairly simple, we just record this slot's status, and trigger schedule if slot is empty.
-	 * <p>
-	 * Regarding 2: It will cause some weird situation since we may have some time-gap on how the slot's status really
-	 * is. We may have some updates on the slot's allocation, but it doesn't reflected by TaskManager's heartbeat yet,
-	 * and we may make some wrong decision if we cannot guarantee we have the exact status about all the slots. So
-	 * the principle here is: We always trust TaskManager's heartbeat, we will correct our information based on that
-	 * and take next action based on the diff between our information and heartbeat status.
-	 *
-	 * @param reportedStatus Reported slot status
-	 */
-	void updateSlotStatus(final SlotStatus reportedStatus) {
-		final SlotID slotId = reportedStatus.getSlotID();
-		final ResourceSlot slot = new ResourceSlot(slotId, reportedStatus.getProfiler());
-
-		if (registerNewSlot(slot)) {
-			// we have a newly registered slot
-			LOG.info("New slot appeared, SlotID:{}, AllocationID:{}", slotId, reportedStatus.getAllocationID());
-
-			if (reportedStatus.getAllocationID() != null) {
-				// slot in use, record this in bookkeeping
-				allocationMap.addAllocation(slotId, reportedStatus.getAllocationID());
-			} else {
-				handleFreeSlot(new ResourceSlot(slotId, reportedStatus.getProfiler()));
-			}
-		} else {
-			// slot exists, update current information
-			if (reportedStatus.getAllocationID() != null) {
-				// slot is reported in use
-				final AllocationID reportedAllocationId = reportedStatus.getAllocationID();
-
-				// check whether we also thought this slot is in use
-				if (allocationMap.isAllocated(slotId)) {
-					// we also think that slot is in use, check whether the AllocationID matches
-					final AllocationID currentAllocationId = allocationMap.getAllocationID(slotId);
-
-					if (!reportedAllocationId.equals(currentAllocationId)) {
-						LOG.info("Slot allocation info mismatch! SlotID:{}, current:{}, reported:{}",
-							slotId, currentAllocationId, reportedAllocationId);
-
-						// seems we have a disagreement about the slot assignments, need to correct it
-						allocationMap.removeAllocation(slotId);
-						allocationMap.addAllocation(slotId, reportedAllocationId);
-					}
-				} else {
-					LOG.info("Slot allocation info mismatch! SlotID:{}, current:null, reported:{}",
-						slotId, reportedAllocationId);
-
-					// we thought the slot is free, should correct this information
-					allocationMap.addAllocation(slotId, reportedStatus.getAllocationID());
-
-					// remove this slot from free slots pool
-					freeSlots.remove(slotId);
-				}
-			} else {
-				// slot is reported empty
-
-				// check whether we also thought this slot is empty
-				if (allocationMap.isAllocated(slotId)) {
-					LOG.info("Slot allocation info mismatch! SlotID:{}, current:{}, reported:null",
-						slotId, allocationMap.getAllocationID(slotId));
-
-					// we thought the slot is in use, correct it
-					allocationMap.removeAllocation(slotId);
-
-					// we have a free slot!
-					handleFreeSlot(new ResourceSlot(slotId, reportedStatus.getProfiler()));
-				}
-			}
-		}
-	}
-
-	/**
-	 * When we have a free slot, try to fulfill the pending request first. If any request can be fulfilled,
-	 * record this allocation in bookkeeping and send slot request to TaskManager, else we just add this slot
-	 * to the free pool.
-	 *
-	 * @param freeSlot The free slot
-	 */
-	private void handleFreeSlot(final ResourceSlot freeSlot) {
-		SlotRequest chosenRequest = chooseRequestToFulfill(freeSlot, pendingSlotRequests);
-
-		if (chosenRequest != null) {
-			pendingSlotRequests.remove(chosenRequest.getAllocationId());
-
-			LOG.info("Assigning SlotID({}) to AllocationID({}), JobID:{}", freeSlot.getSlotId(),
-				chosenRequest.getAllocationId(), chosenRequest.getJobId());
-			allocationMap.addAllocation(freeSlot.getSlotId(), chosenRequest.getAllocationId());
-
-			// TODO: send slot request to TaskManager
-		} else {
-			freeSlots.put(freeSlot.getSlotId(), freeSlot);
-		}
-	}
-
-	/**
-	 * Check whether the request is duplicated. We use AllocationID to identify slot request, for each
-	 * formerly received slot request, it is either in pending list or already been allocated.
-	 *
-	 * @param request The slot request
-	 * @return <tt>true</tt> if the request is duplicated
-	 */
-	private boolean isRequestDuplicated(final SlotRequest request) {
-		final AllocationID allocationId = request.getAllocationId();
-		return pendingSlotRequests.containsKey(allocationId)
-			|| allocationMap.isAllocated(allocationId);
-	}
-
-	/**
-	 * Try to register slot, and tell if this slot is newly registered.
-	 *
-	 * @param slot The ResourceSlot which will be checked and registered
-	 * @return <tt>true</tt> if we meet a new slot
-	 */
-	private boolean registerNewSlot(final ResourceSlot slot) {
-		final SlotID slotId = slot.getSlotId();
-		final ResourceID resourceId = slotId.getResourceID();
-		if (!registeredSlots.containsKey(resourceId)) {
-			registeredSlots.put(resourceId, new HashMap<SlotID, ResourceSlot>());
-		}
-		return registeredSlots.get(resourceId).put(slotId, slot) == null;
-	}
-
-	private ResourceSlot getRegisteredSlot(final SlotID slotId) {
-		final ResourceID resourceId = slotId.getResourceID();
-		if (!registeredSlots.containsKey(resourceId)) {
-			return null;
-		}
-		return registeredSlots.get(resourceId).get(slotId);
-	}
-
-	// ------------------------------------------------------------------------
-	//  Framework specific behavior
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Choose a slot to use among all free slots, the behavior is framework specified.
-	 *
-	 * @param request   The slot request
-	 * @param freeSlots All slots which can be used
-	 * @return The slot we choose to use, <tt>null</tt> if we did not find a match
-	 */
-	protected abstract ResourceSlot chooseSlotToUse(final SlotRequest request,
-		final Map<SlotID, ResourceSlot> freeSlots);
-
-	/**
-	 * Choose a pending request to fulfill when we have a free slot, the behavior is framework specified.
-	 *
-	 * @param offeredSlot     The free slot
-	 * @param pendingRequests All the pending slot requests
-	 * @return The chosen SlotRequest, <tt>null</tt> if we did not find a match
-	 */
-	protected abstract SlotRequest chooseRequestToFulfill(final ResourceSlot offeredSlot,
-		final Map<AllocationID, SlotRequest> pendingRequests);
-
-	/**
-	 * The framework specific code for allocating a container for specified resource profile.
-	 *
-	 * @param resourceProfile The resource profile
-	 */
-	protected abstract void allocateContainer(final ResourceProfile resourceProfile);
-
-
-	// ------------------------------------------------------------------------
-	//  Helper classes
-	// ------------------------------------------------------------------------
-
-	/**
-	 * We maintain all the allocations with SlotID and AllocationID. We are able to get or remove the allocation info
-	 * either by SlotID or AllocationID.
-	 */
-	private static class AllocationMap {
-
-		/** All allocated slots (by SlotID) */
-		private final Map<SlotID, AllocationID> allocatedSlots;
-
-		/** All allocated slots (by AllocationID), it'a a inverse view of allocatedSlots */
-		private final Map<AllocationID, SlotID> allocatedSlotsByAllocationId;
-
-		AllocationMap() {
-			this.allocatedSlots = new HashMap<>(16);
-			this.allocatedSlotsByAllocationId = new HashMap<>(16);
-		}
-
-		/**
-		 * Add a allocation
-		 *
-		 * @param slotId       The slot id
-		 * @param allocationId The allocation id
-		 */
-		void addAllocation(final SlotID slotId, final AllocationID allocationId) {
-			allocatedSlots.put(slotId, allocationId);
-			allocatedSlotsByAllocationId.put(allocationId, slotId);
-		}
-
-		/**
-		 * De-allocation with slot id
-		 *
-		 * @param slotId The slot id
-		 */
-		void removeAllocation(final SlotID slotId) {
-			if (allocatedSlots.containsKey(slotId)) {
-				final AllocationID allocationId = allocatedSlots.get(slotId);
-				allocatedSlots.remove(slotId);
-				allocatedSlotsByAllocationId.remove(allocationId);
-			}
-		}
-
-		/**
-		 * De-allocation with allocation id
-		 *
-		 * @param allocationId The allocation id
-		 */
-		void removeAllocation(final AllocationID allocationId) {
-			if (allocatedSlotsByAllocationId.containsKey(allocationId)) {
-				SlotID slotId = allocatedSlotsByAllocationId.get(allocationId);
-				allocatedSlotsByAllocationId.remove(allocationId);
-				allocatedSlots.remove(slotId);
-			}
-		}
-
-		/**
-		 * Check whether allocation exists by slot id
-		 *
-		 * @param slotId The slot id
-		 * @return true if the allocation exists
-		 */
-		boolean isAllocated(final SlotID slotId) {
-			return allocatedSlots.containsKey(slotId);
-		}
-
-		/**
-		 * Check whether allocation exists by allocation id
-		 *
-		 * @param allocationId The allocation id
-		 * @return true if the allocation exists
-		 */
-		boolean isAllocated(final AllocationID allocationId) {
-			return allocatedSlotsByAllocationId.containsKey(allocationId);
-		}
-
-		AllocationID getAllocationID(final SlotID slotId) {
-			return allocatedSlots.get(slotId);
-		}
-
-		SlotID getSlotID(final AllocationID allocationId) {
-			return allocatedSlotsByAllocationId.get(allocationId);
-		}
-
-		public int size() {
-			return allocatedSlots.size();
-		}
-	}
-
-	// ------------------------------------------------------------------------
-	//  Testing utilities
-	// ------------------------------------------------------------------------
-
-	@VisibleForTesting
-	boolean isAllocated(final SlotID slotId) {
-		return allocationMap.isAllocated(slotId);
-	}
-
-	@VisibleForTesting
-	boolean isAllocated(final AllocationID allocationId) {
-		return allocationMap.isAllocated(allocationId);
-	}
-
-	/**
-	 * Add free slots directly to the free pool, this will not trigger pending requests allocation
-	 *
-	 * @param slot The resource slot
-	 */
-	@VisibleForTesting
-	void addFreeSlot(final ResourceSlot slot) {
-		final ResourceID resourceId = slot.getResourceID();
-		final SlotID slotId = slot.getSlotId();
-
-		if (!registeredSlots.containsKey(resourceId)) {
-			registeredSlots.put(resourceId, new HashMap<SlotID, ResourceSlot>());
-		}
-		registeredSlots.get(resourceId).put(slot.getSlotId(), slot);
-		freeSlots.put(slotId, slot);
-	}
-
-	@VisibleForTesting
-	int getAllocatedSlotCount() {
-		return allocationMap.size();
-	}
-
-	@VisibleForTesting
-	int getFreeSlotCount() {
-		return freeSlots.size();
-	}
-
-	@VisibleForTesting
-	int getPendingRequestCount() {
-		return pendingSlotRequests.size();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/72468d14/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java
new file mode 100644
index 0000000..0a6a7ef
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMaster.java
@@ -0,0 +1,244 @@
+/*
+ * 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.jobmaster;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.leaderelection.LeaderContender;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.rpc.RpcMethod;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway;
+import org.apache.flink.runtime.rpc.RpcEndpoint;
+import org.apache.flink.runtime.rpc.RpcService;
+import org.apache.flink.runtime.taskmanager.TaskExecutionState;
+import org.apache.flink.util.Preconditions;
+
+import java.util.UUID;
+
+/**
+ * JobMaster implementation. The job master is responsible for the execution of a single
+ * {@link org.apache.flink.runtime.jobgraph.JobGraph}.
+ * <p>
+ * It offers the following methods as part of its rpc interface to interact with the JobMaster
+ * remotely:
+ * <ul>
+ *     <li>{@link #updateTaskExecutionState(TaskExecutionState)} updates the task execution state for
+ * given task</li>
+ * </ul>
+ */
+public class JobMaster extends RpcEndpoint<JobMasterGateway> {
+
+	/** Gateway to connected resource manager, null iff not connected */
+	private ResourceManagerGateway resourceManager = null;
+
+	/** Logical representation of the job */
+	private final JobGraph jobGraph;
+	private final JobID jobID;
+
+	/** Configuration of the job */
+	private final Configuration configuration;
+
+	/** Service to contend for and retrieve the leadership of JM and RM */
+	private final HighAvailabilityServices highAvailabilityServices;
+
+	/** Leader Management */
+	private LeaderElectionService leaderElectionService = null;
+	private UUID leaderSessionID;
+
+	/**
+	 * The JM's Constructor
+	 *
+	 * @param jobGraph The representation of the job's execution plan
+	 * @param configuration The job's configuration
+	 * @param rpcService The RPC service at which the JM serves
+	 * @param highAvailabilityService The cluster's HA service from the JM can elect and retrieve leaders.
+	 */
+	public JobMaster(
+		JobGraph jobGraph,
+		Configuration configuration,
+		RpcService rpcService,
+		HighAvailabilityServices highAvailabilityService) {
+
+		super(rpcService);
+
+		this.jobGraph = Preconditions.checkNotNull(jobGraph);
+		this.jobID = Preconditions.checkNotNull(jobGraph.getJobID());
+
+		this.configuration = Preconditions.checkNotNull(configuration);
+
+		this.highAvailabilityServices = Preconditions.checkNotNull(highAvailabilityService);
+	}
+
+	public ResourceManagerGateway getResourceManager() {
+		return resourceManager;
+	}
+
+	//----------------------------------------------------------------------------------------------
+	// Initialization methods
+	//----------------------------------------------------------------------------------------------
+	public void start() {
+		super.start();
+
+		// register at the election once the JM starts
+		registerAtElectionService();
+	}
+
+
+	//----------------------------------------------------------------------------------------------
+	// JobMaster Leadership methods
+	//----------------------------------------------------------------------------------------------
+
+	/**
+	 * Retrieves the election service and contend for the leadership.
+	 */
+	private void registerAtElectionService() {
+		try {
+			leaderElectionService = highAvailabilityServices.getJobMasterLeaderElectionService(jobID);
+			leaderElectionService.start(new JobMasterLeaderContender());
+		} catch (Exception e) {
+			throw new RuntimeException("Fail to register at the election of JobMaster", e);
+		}
+	}
+
+	/**
+	 * Start the execution when the leadership is granted.
+	 *
+	 * @param newLeaderSessionID The identifier of the new leadership session
+	 */
+	public void grantJobMasterLeadership(final UUID newLeaderSessionID) {
+		runAsync(new Runnable() {
+			@Override
+			public void run() {
+				log.info("JobManager {} grants leadership with session id {}.", getAddress(), newLeaderSessionID);
+
+				// The operation may be blocking, but since JM is idle before it grants the leadership, it's okay that
+				// JM waits here for the operation's completeness.
+				leaderSessionID = newLeaderSessionID;
+				leaderElectionService.confirmLeaderSessionID(newLeaderSessionID);
+
+				// TODO:: execute the job when the leadership is granted.
+			}
+		});
+	}
+
+	/**
+	 * Stop the execution when the leadership is revoked.
+	 */
+	public void revokeJobMasterLeadership() {
+		runAsync(new Runnable() {
+			@Override
+			public void run() {
+				log.info("JobManager {} was revoked leadership.", getAddress());
+
+				// TODO:: cancel the job's execution and notify all listeners
+				cancelAndClearEverything(new Exception("JobManager is no longer the leader."));
+
+				leaderSessionID = null;
+			}
+		});
+	}
+
+	/**
+	 * Handles error occurring in the leader election service
+	 *
+	 * @param exception Exception thrown in the leader election service
+	 */
+	public void onJobMasterElectionError(final Exception exception) {
+		runAsync(new Runnable() {
+			@Override
+			public void run() {
+				log.error("Received an error from the LeaderElectionService.", exception);
+
+				// TODO:: cancel the job's execution and shutdown the JM
+				cancelAndClearEverything(exception);
+
+				leaderSessionID = null;
+			}
+		});
+
+	}
+
+	//----------------------------------------------------------------------------------------------
+	// RPC methods
+	//----------------------------------------------------------------------------------------------
+
+	/**
+	 * Updates the task execution state for a given task.
+	 *
+	 * @param taskExecutionState New task execution state for a given task
+	 * @return Acknowledge the task execution state update
+	 */
+	@RpcMethod
+	public Acknowledge updateTaskExecutionState(TaskExecutionState taskExecutionState) {
+		System.out.println("TaskExecutionState: " + taskExecutionState);
+		return Acknowledge.get();
+	}
+
+	/**
+	 * Triggers the registration of the job master at the resource manager.
+	 *
+	 * @param address Address of the resource manager
+	 */
+	@RpcMethod
+	public void registerAtResourceManager(final String address) {
+		//TODO:: register at the RM
+	}
+
+	//----------------------------------------------------------------------------------------------
+	// Helper methods
+	//----------------------------------------------------------------------------------------------
+
+	/**
+	 * Cancel the current job and notify all listeners the job's cancellation.
+	 *
+	 * @param cause Cause for the cancelling.
+	 */
+	private void cancelAndClearEverything(Throwable cause) {
+		// currently, nothing to do here
+	}
+
+	// ------------------------------------------------------------------------
+	//  Utility classes
+	// ------------------------------------------------------------------------
+	private class JobMasterLeaderContender implements LeaderContender {
+
+		@Override
+		public void grantLeadership(UUID leaderSessionID) {
+			JobMaster.this.grantJobMasterLeadership(leaderSessionID);
+		}
+
+		@Override
+		public void revokeLeadership() {
+			JobMaster.this.revokeJobMasterLeadership();
+		}
+
+		@Override
+		public String getAddress() {
+			return JobMaster.this.getAddress();
+		}
+
+		@Override
+		public void handleError(Exception exception) {
+			onJobMasterElectionError(exception);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/72468d14/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java
new file mode 100644
index 0000000..a53e383
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmaster/JobMasterGateway.java
@@ -0,0 +1,45 @@
+/*
+ * 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.jobmaster;
+
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.rpc.RpcGateway;
+import org.apache.flink.runtime.taskmanager.TaskExecutionState;
+import scala.concurrent.Future;
+
+/**
+ * {@link JobMaster} rpc gateway interface
+ */
+public interface JobMasterGateway extends RpcGateway {
+
+	/**
+	 * Updates the task execution state for a given task.
+	 *
+	 * @param taskExecutionState New task execution state for a given task
+	 * @return Future acknowledge of the task execution state update
+	 */
+	Future<Acknowledge> updateTaskExecutionState(TaskExecutionState taskExecutionState);
+
+	/**
+	 * Triggers the registration of the job master at the resource manager.
+	 *
+	 * @param address Address of the resource manager
+	 */
+	void registerAtResourceManager(final String address);
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/72468d14/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RegistrationResponse.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RegistrationResponse.java b/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RegistrationResponse.java
new file mode 100644
index 0000000..fefcc78
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RegistrationResponse.java
@@ -0,0 +1,84 @@
+/*
+ * 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.registration;
+
+import java.io.Serializable;
+
+/**
+ * Base class for responses given to registration attempts from {@link RetryingRegistration}.
+ */
+public abstract class RegistrationResponse implements Serializable {
+
+	private static final long serialVersionUID = 1L;
+
+	// ----------------------------------------------------------------------------
+	
+	/**
+	 * Base class for a successful registration. Concrete registration implementations
+	 * will typically extend this class to attach more information.
+	 */
+	public static class Success extends RegistrationResponse {
+		private static final long serialVersionUID = 1L;
+		
+		@Override
+		public String toString() {
+			return "Registration Successful";
+		}
+	}
+
+	// ----------------------------------------------------------------------------
+
+	/**
+	 * A rejected (declined) registration.
+	 */
+	public static final class Decline extends RegistrationResponse {
+		private static final long serialVersionUID = 1L;
+
+		/** the rejection reason */
+		private final String reason;
+
+		/**
+		 * Creates a new rejection message.
+		 * 
+		 * @param reason The reason for the rejection.
+		 */
+		public Decline(String reason) {
+			this.reason = reason != null ? reason : "(unknown)";
+		}
+
+		/**
+		 * Gets the reason for the rejection.
+		 */
+		public String getReason() {
+			return reason;
+		}
+
+		@Override
+		public String toString() {
+			return "Registration Declined (" + reason + ')';
+		}
+	}
+}
+
+
+
+
+
+
+

http://git-wip-us.apache.org/repos/asf/flink/blob/72468d14/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RetryingRegistration.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RetryingRegistration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RetryingRegistration.java
new file mode 100644
index 0000000..88fe9b5
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RetryingRegistration.java
@@ -0,0 +1,296 @@
+/*
+ * 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.registration;
+
+import akka.dispatch.OnFailure;
+import akka.dispatch.OnSuccess;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.runtime.rpc.RpcGateway;
+import org.apache.flink.runtime.rpc.RpcService;
+
+import org.slf4j.Logger;
+
+import scala.concurrent.Future;
+import scala.concurrent.Promise;
+import scala.concurrent.impl.Promise.DefaultPromise;
+
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+
+/**
+ * This utility class implements the basis of registering one component at another component,
+ * for example registering the TaskExecutor at the ResourceManager.
+ * This {@code RetryingRegistration} implements both the initial address resolution
+ * and the retries-with-backoff strategy.
+ * 
+ * <p>The registration gives access to a future that is completed upon successful registration.
+ * The registration can be canceled, for example when the target where it tries to register
+ * at looses leader status.
+ * 
+ * @param <Gateway> The type of the gateway to connect to.
+ * @param <Success> The type of the successful registration responses.
+ */
+public abstract class RetryingRegistration<Gateway extends RpcGateway, Success extends RegistrationResponse.Success> {
+
+	// ------------------------------------------------------------------------
+	//  default configuration values
+	// ------------------------------------------------------------------------
+
+	/** default value for the initial registration timeout (milliseconds) */
+	private static final long INITIAL_REGISTRATION_TIMEOUT_MILLIS = 100;
+
+	/** default value for the maximum registration timeout, after exponential back-off (milliseconds) */
+	private static final long MAX_REGISTRATION_TIMEOUT_MILLIS = 30000;
+
+	/** The pause (milliseconds) made after an registration attempt caused an exception (other than timeout) */
+	private static final long ERROR_REGISTRATION_DELAY_MILLIS = 10000;
+
+	/** The pause (milliseconds) made after the registration attempt was refused */
+	private static final long REFUSED_REGISTRATION_DELAY_MILLIS = 30000;
+
+	// ------------------------------------------------------------------------
+	// Fields
+	// ------------------------------------------------------------------------
+
+	private final Logger log;
+
+	private final RpcService rpcService;
+
+	private final String targetName;
+
+	private final Class<Gateway> targetType;
+
+	private final String targetAddress;
+
+	private final UUID leaderId;
+
+	private final Promise<Tuple2<Gateway, Success>> completionPromise;
+
+	private final long initialRegistrationTimeout;
+
+	private final long maxRegistrationTimeout;
+
+	private final long delayOnError;
+
+	private final long delayOnRefusedRegistration;
+
+	private volatile boolean canceled;
+
+	// ------------------------------------------------------------------------
+
+	public RetryingRegistration(
+			Logger log,
+			RpcService rpcService,
+			String targetName,
+			Class<Gateway> targetType,
+			String targetAddress,
+			UUID leaderId) {
+		this(log, rpcService, targetName, targetType, targetAddress, leaderId,
+				INITIAL_REGISTRATION_TIMEOUT_MILLIS, MAX_REGISTRATION_TIMEOUT_MILLIS,
+				ERROR_REGISTRATION_DELAY_MILLIS, REFUSED_REGISTRATION_DELAY_MILLIS);
+	}
+
+	public RetryingRegistration(
+			Logger log,
+			RpcService rpcService,
+			String targetName, 
+			Class<Gateway> targetType,
+			String targetAddress,
+			UUID leaderId,
+			long initialRegistrationTimeout,
+			long maxRegistrationTimeout,
+			long delayOnError,
+			long delayOnRefusedRegistration) {
+
+		checkArgument(initialRegistrationTimeout > 0, "initial registration timeout must be greater than zero");
+		checkArgument(maxRegistrationTimeout > 0, "maximum registration timeout must be greater than zero");
+		checkArgument(delayOnError >= 0, "delay on error must be non-negative");
+		checkArgument(delayOnRefusedRegistration >= 0, "delay on refused registration must be non-negative");
+
+		this.log = checkNotNull(log);
+		this.rpcService = checkNotNull(rpcService);
+		this.targetName = checkNotNull(targetName);
+		this.targetType = checkNotNull(targetType);
+		this.targetAddress = checkNotNull(targetAddress);
+		this.leaderId = checkNotNull(leaderId);
+		this.initialRegistrationTimeout = initialRegistrationTimeout;
+		this.maxRegistrationTimeout = maxRegistrationTimeout;
+		this.delayOnError = delayOnError;
+		this.delayOnRefusedRegistration = delayOnRefusedRegistration;
+
+		this.completionPromise = new DefaultPromise<>();
+	}
+
+	// ------------------------------------------------------------------------
+	//  completion and cancellation
+	// ------------------------------------------------------------------------
+
+	public Future<Tuple2<Gateway, Success>> getFuture() {
+		return completionPromise.future();
+	}
+
+	/**
+	 * Cancels the registration procedure.
+	 */
+	public void cancel() {
+		canceled = true;
+	}
+
+	/**
+	 * Checks if the registration was canceled.
+	 * @return True if the registration was canceled, false otherwise.
+	 */
+	public boolean isCanceled() {
+		return canceled;
+	}
+
+	// ------------------------------------------------------------------------
+	//  registration
+	// ------------------------------------------------------------------------
+
+	protected abstract Future<RegistrationResponse> invokeRegistration(
+			Gateway gateway, UUID leaderId, long timeoutMillis) throws Exception;
+
+	/**
+	 * This method resolves the target address to a callable gateway and starts the
+	 * registration after that.
+	 */
+	@SuppressWarnings("unchecked")
+	public void startRegistration() {
+		try {
+			// trigger resolution of the resource manager address to a callable gateway
+			Future<Gateway> resourceManagerFuture = rpcService.connect(targetAddress, targetType);
+	
+			// upon success, start the registration attempts
+			resourceManagerFuture.onSuccess(new OnSuccess<Gateway>() {
+				@Override
+				public void onSuccess(Gateway result) {
+					log.info("Resolved {} address, beginning registration", targetName);
+					register(result, 1, initialRegistrationTimeout);
+				}
+			}, rpcService.getExecutionContext());
+	
+			// upon failure, retry, unless this is cancelled
+			resourceManagerFuture.onFailure(new OnFailure() {
+				@Override
+				public void onFailure(Throwable failure) {
+					if (!isCanceled()) {
+						log.warn("Could not resolve {} address {}, retrying...", targetName, targetAddress);
+						startRegistration();
+					}
+				}
+			}, rpcService.getExecutionContext());
+		}
+		catch (Throwable t) {
+			cancel();
+			completionPromise.tryFailure(t);
+		}
+	}
+
+	/**
+	 * This method performs a registration attempt and triggers either a success notification or a retry,
+	 * depending on the result.
+	 */
+	@SuppressWarnings("unchecked")
+	private void register(final Gateway gateway, final int attempt, final long timeoutMillis) {
+		// eager check for canceling to avoid some unnecessary work
+		if (canceled) {
+			return;
+		}
+
+		try {
+			log.info("Registration at {} attempt {} (timeout={}ms)", targetName, attempt, timeoutMillis);
+			Future<RegistrationResponse> registrationFuture = invokeRegistration(gateway, leaderId, timeoutMillis);
+	
+			// if the registration was successful, let the TaskExecutor know
+			registrationFuture.onSuccess(new OnSuccess<RegistrationResponse>() {
+				
+				@Override
+				public void onSuccess(RegistrationResponse result) throws Throwable {
+					if (!isCanceled()) {
+						if (result instanceof RegistrationResponse.Success) {
+							// registration successful!
+							Success success = (Success) result;
+							completionPromise.success(new Tuple2<>(gateway, success));
+						}
+						else {
+							// registration refused or unknown
+							if (result instanceof RegistrationResponse.Decline) {
+								RegistrationResponse.Decline decline = (RegistrationResponse.Decline) result;
+								log.info("Registration at {} was declined: {}", targetName, decline.getReason());
+							} else {
+								log.error("Received unknown response to registration attempt: " + result);
+							}
+
+							log.info("Pausing and re-attempting registration in {} ms", delayOnRefusedRegistration);
+							registerLater(gateway, 1, initialRegistrationTimeout, delayOnRefusedRegistration);
+						}
+					}
+				}
+			}, rpcService.getExecutionContext());
+	
+			// upon failure, retry
+			registrationFuture.onFailure(new OnFailure() {
+				@Override
+				public void onFailure(Throwable failure) {
+					if (!isCanceled()) {
+						if (failure instanceof TimeoutException) {
+							// we simply have not received a response in time. maybe the timeout was
+							// very low (initial fast registration attempts), maybe the target endpoint is
+							// currently down.
+							if (log.isDebugEnabled()) {
+								log.debug("Registration at {} ({}) attempt {} timed out after {} ms",
+										targetName, targetAddress, attempt, timeoutMillis);
+							}
+	
+							long newTimeoutMillis = Math.min(2 * timeoutMillis, maxRegistrationTimeout);
+							register(gateway, attempt + 1, newTimeoutMillis);
+						}
+						else {
+							// a serious failure occurred. we still should not give up, but keep trying
+							log.error("Registration at " + targetName + " failed due to an error", failure);
+							log.info("Pausing and re-attempting registration in {} ms", delayOnError);
+	
+							registerLater(gateway, 1, initialRegistrationTimeout, delayOnError);
+						}
+					}
+				}
+			}, rpcService.getExecutionContext());
+		}
+		catch (Throwable t) {
+			cancel();
+			completionPromise.tryFailure(t);
+		}
+	}
+
+	private void registerLater(final Gateway gateway, final int attempt, final long timeoutMillis, long delay) {
+		rpcService.scheduleRunnable(new Runnable() {
+			@Override
+			public void run() {
+				register(gateway, attempt, timeoutMillis);
+			}
+		}, delay, TimeUnit.MILLISECONDS);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/72468d14/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobMasterRegistration.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobMasterRegistration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobMasterRegistration.java
new file mode 100644
index 0000000..309dcc1
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/JobMasterRegistration.java
@@ -0,0 +1,35 @@
+/*
+ * 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.resourcemanager;
+
+import java.io.Serializable;
+
+public class JobMasterRegistration implements Serializable {
+	private static final long serialVersionUID = 8411214999193765202L;
+
+	private final String address;
+
+	public JobMasterRegistration(String address) {
+		this.address = address;
+	}
+
+	public String getAddress() {
+		return address;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/72468d14/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/RegistrationResponse.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/RegistrationResponse.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/RegistrationResponse.java
new file mode 100644
index 0000000..fb6c401
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/RegistrationResponse.java
@@ -0,0 +1,43 @@
+/*
+ * 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.resourcemanager;
+
+import org.apache.flink.runtime.instance.InstanceID;
+
+import java.io.Serializable;
+
+public class RegistrationResponse implements Serializable {
+	private static final long serialVersionUID = -2379003255993119993L;
+
+	private final boolean isSuccess;
+	private final InstanceID instanceID;
+
+	public RegistrationResponse(boolean isSuccess, InstanceID instanceID) {
+		this.isSuccess = isSuccess;
+		this.instanceID = instanceID;
+	}
+
+	public boolean isSuccess() {
+		return isSuccess;
+	}
+
+	public InstanceID getInstanceID() {
+		return instanceID;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/72468d14/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
new file mode 100644
index 0000000..44c022b
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManager.java
@@ -0,0 +1,214 @@
+/*
+ * 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.resourcemanager;
+
+import akka.dispatch.Mapper;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.leaderelection.LeaderContender;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.rpc.RpcMethod;
+import org.apache.flink.runtime.rpc.RpcEndpoint;
+import org.apache.flink.runtime.rpc.RpcService;
+import org.apache.flink.runtime.jobmaster.JobMaster;
+import org.apache.flink.runtime.jobmaster.JobMasterGateway;
+import org.apache.flink.runtime.taskexecutor.TaskExecutorRegistrationSuccess;
+
+import scala.concurrent.Future;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.UUID;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * ResourceManager implementation. The resource manager is responsible for resource de-/allocation
+ * and bookkeeping.
+ *
+ * It offers the following methods as part of its rpc interface to interact with the him remotely:
+ * <ul>
+ *     <li>{@link #registerJobMaster(JobMasterRegistration)} registers a {@link JobMaster} at the resource manager</li>
+ *     <li>{@link #requestSlot(SlotRequest)} requests a slot from the resource manager</li>
+ * </ul>
+ */
+public class ResourceManager extends RpcEndpoint<ResourceManagerGateway> {
+	private final Map<JobMasterGateway, InstanceID> jobMasterGateways;
+	private final HighAvailabilityServices highAvailabilityServices;
+	private LeaderElectionService leaderElectionService = null;
+	private UUID leaderSessionID = null;
+
+	public ResourceManager(RpcService rpcService, HighAvailabilityServices highAvailabilityServices) {
+		super(rpcService);
+		this.highAvailabilityServices = checkNotNull(highAvailabilityServices);
+		this.jobMasterGateways = new HashMap<>();
+	}
+
+	@Override
+	public void start() {
+		// start a leader
+		try {
+			super.start();
+			leaderElectionService = highAvailabilityServices.getResourceManagerLeaderElectionService();
+			leaderElectionService.start(new ResourceManagerLeaderContender());
+		} catch (Throwable e) {
+			log.error("A fatal error happened when starting the ResourceManager", e);
+			throw new RuntimeException("A fatal error happened when starting the ResourceManager", e);
+		}
+	}
+
+	@Override
+	public void shutDown() {
+		try {
+			leaderElectionService.stop();
+			super.shutDown();
+		} catch(Throwable e) {
+			log.error("A fatal error happened when shutdown the ResourceManager", e);
+			throw new RuntimeException("A fatal error happened when shutdown the ResourceManager", e);
+		}
+	}
+
+	/**
+	 * Gets the leader session id of current resourceManager.
+	 *
+	 * @return return the leaderSessionId of current resourceManager, this returns null until the current resourceManager is granted leadership.
+	 */
+	@VisibleForTesting
+	UUID getLeaderSessionID() {
+		return leaderSessionID;
+	}
+
+	/**
+	 * Register a {@link JobMaster} at the resource manager.
+	 *
+	 * @param jobMasterRegistration Job master registration information
+	 * @return Future registration response
+	 */
+	@RpcMethod
+	public Future<RegistrationResponse> registerJobMaster(JobMasterRegistration jobMasterRegistration) {
+		Future<JobMasterGateway> jobMasterFuture = getRpcService().connect(jobMasterRegistration.getAddress(), JobMasterGateway.class);
+
+		return jobMasterFuture.map(new Mapper<JobMasterGateway, RegistrationResponse>() {
+			@Override
+			public RegistrationResponse apply(final JobMasterGateway jobMasterGateway) {
+				InstanceID instanceID;
+
+				if (jobMasterGateways.containsKey(jobMasterGateway)) {
+					instanceID = jobMasterGateways.get(jobMasterGateway);
+				} else {
+					instanceID = new InstanceID();
+					jobMasterGateways.put(jobMasterGateway, instanceID);
+				}
+
+				return new RegistrationResponse(true, instanceID);
+			}
+		}, getMainThreadExecutionContext());
+	}
+
+	/**
+	 * Requests a slot from the resource manager.
+	 *
+	 * @param slotRequest Slot request
+	 * @return Slot assignment
+	 */
+	@RpcMethod
+	public SlotAssignment requestSlot(SlotRequest slotRequest) {
+		System.out.println("SlotRequest: " + slotRequest);
+		return new SlotAssignment();
+	}
+
+
+	/**
+	 *
+	 * @param resourceManagerLeaderId  The fencing token for the ResourceManager leader 
+	 * @param taskExecutorAddress      The address of the TaskExecutor that registers
+	 * @param resourceID               The resource ID of the TaskExecutor that registers
+	 *
+	 * @return The response by the ResourceManager.
+	 */
+	@RpcMethod
+	public org.apache.flink.runtime.registration.RegistrationResponse registerTaskExecutor(
+			UUID resourceManagerLeaderId,
+			String taskExecutorAddress,
+			ResourceID resourceID) {
+
+		return new TaskExecutorRegistrationSuccess(new InstanceID(), 5000);
+	}
+
+	private class ResourceManagerLeaderContender implements LeaderContender {
+
+		/**
+		 * Callback method when current resourceManager is granted leadership
+		 *
+		 * @param leaderSessionID unique leadershipID
+		 */
+		@Override
+		public void grantLeadership(final UUID leaderSessionID) {
+			runAsync(new Runnable() {
+				@Override
+				public void run() {
+					log.info("ResourceManager {} was granted leadership with leader session ID {}", getAddress(), leaderSessionID);
+					ResourceManager.this.leaderSessionID = leaderSessionID;
+					// confirming the leader session ID might be blocking,
+					leaderElectionService.confirmLeaderSessionID(leaderSessionID);
+				}
+			});
+		}
+
+		/**
+		 * Callback method when current resourceManager lose leadership.
+		 */
+		@Override
+		public void revokeLeadership() {
+			runAsync(new Runnable() {
+				@Override
+				public void run() {
+					log.info("ResourceManager {} was revoked leadership.", getAddress());
+					jobMasterGateways.clear();
+					leaderSessionID = null;
+				}
+			});
+		}
+
+		@Override
+		public String getAddress() {
+			return ResourceManager.this.getAddress();
+		}
+
+		/**
+		 * Handles error occurring in the leader election service
+		 *
+		 * @param exception Exception being thrown in the leader election service
+		 */
+		@Override
+		public void handleError(final Exception exception) {
+			runAsync(new Runnable() {
+				@Override
+				public void run() {
+					log.error("ResourceManager received an error from the LeaderElectionService.", exception);
+					// terminate ResourceManager in case of an error
+					shutDown();
+				}
+			});
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/72468d14/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java
new file mode 100644
index 0000000..b5782b0
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/ResourceManagerGateway.java
@@ -0,0 +1,77 @@
+/*
+ * 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.resourcemanager;
+
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.rpc.RpcGateway;
+import org.apache.flink.runtime.rpc.RpcTimeout;
+import org.apache.flink.runtime.jobmaster.JobMaster;
+
+import scala.concurrent.Future;
+import scala.concurrent.duration.FiniteDuration;
+
+import java.util.UUID;
+
+/**
+ * The {@link ResourceManager}'s RPC gateway interface.
+ */
+public interface ResourceManagerGateway extends RpcGateway {
+
+	/**
+	 * Register a {@link JobMaster} at the resource manager.
+	 *
+	 * @param jobMasterRegistration Job master registration information
+	 * @param timeout Timeout for the future to complete
+	 * @return Future registration response
+	 */
+	Future<RegistrationResponse> registerJobMaster(
+		JobMasterRegistration jobMasterRegistration,
+		@RpcTimeout FiniteDuration timeout);
+
+	/**
+	 * Register a {@link JobMaster} at the resource manager.
+	 *
+	 * @param jobMasterRegistration Job master registration information
+	 * @return Future registration response
+	 */
+	Future<RegistrationResponse> registerJobMaster(JobMasterRegistration jobMasterRegistration);
+
+	/**
+	 * Requests a slot from the resource manager.
+	 *
+	 * @param slotRequest Slot request
+	 * @return Future slot assignment
+	 */
+	Future<SlotAssignment> requestSlot(SlotRequest slotRequest);
+
+	/**
+	 * 
+	 * @param resourceManagerLeaderId  The fencing token for the ResourceManager leader 
+	 * @param taskExecutorAddress      The address of the TaskExecutor that registers
+	 * @param resourceID               The resource ID of the TaskExecutor that registers
+	 * @param timeout                  The timeout for the response.
+	 * 
+	 * @return The future to the response by the ResourceManager.
+	 */
+	Future<org.apache.flink.runtime.registration.RegistrationResponse> registerTaskExecutor(
+			UUID resourceManagerLeaderId,
+			String taskExecutorAddress,
+			ResourceID resourceID,
+			@RpcTimeout FiniteDuration timeout);
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/72468d14/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotAssignment.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotAssignment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotAssignment.java
new file mode 100644
index 0000000..695204d
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotAssignment.java
@@ -0,0 +1,25 @@
+/*
+ * 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.resourcemanager;
+
+import java.io.Serializable;
+
+public class SlotAssignment implements Serializable{
+	private static final long serialVersionUID = -6990813455942742322L;
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/72468d14/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotManager.java
new file mode 100644
index 0000000..5c06648
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/SlotManager.java
@@ -0,0 +1,523 @@
+/*
+ * 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.resourcemanager;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.ResourceSlot;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.taskexecutor.SlotReport;
+import org.apache.flink.runtime.taskexecutor.SlotStatus;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * SlotManager is responsible for receiving slot requests and do slot allocations. It allows to request
+ * slots from registered TaskManagers and issues container allocation requests in case of there are not
+ * enough available slots. Besides, it should sync its slot allocation with TaskManager's heartbeat.
+ * <p>
+ * The main operation principle of SlotManager is:
+ * <ul>
+ * <li>1. All slot allocation status should be synced with TaskManager, which is the ground truth.</li>
+ * <li>2. All slots that have registered must be tracked, either by free pool or allocated pool.</li>
+ * <li>3. All slot requests will be handled by best efforts, there is no guarantee that one request will be
+ * fulfilled in time or correctly allocated. Conflicts or timeout or some special error will happen, it should
+ * be handled outside SlotManager. SlotManager will make each decision based on the information it currently
+ * holds.</li>
+ * </ul>
+ * <b>IMPORTANT:</b> This class is <b>Not Thread-safe</b>.
+ */
+public abstract class SlotManager {
+
+	private static final Logger LOG = LoggerFactory.getLogger(SlotManager.class);
+
+	/** Gateway to communicate with ResourceManager */
+	private final ResourceManagerGateway resourceManagerGateway;
+
+	/** All registered slots, including free and allocated slots */
+	private final Map<ResourceID, Map<SlotID, ResourceSlot>> registeredSlots;
+
+	/** All pending slot requests, waiting available slots to fulfil */
+	private final Map<AllocationID, SlotRequest> pendingSlotRequests;
+
+	/** All free slots that can be used to be allocated */
+	private final Map<SlotID, ResourceSlot> freeSlots;
+
+	/** All allocations, we can lookup allocations either by SlotID or AllocationID */
+	private final AllocationMap allocationMap;
+
+	public SlotManager(ResourceManagerGateway resourceManagerGateway) {
+		this.resourceManagerGateway = checkNotNull(resourceManagerGateway);
+		this.registeredSlots = new HashMap<>(16);
+		this.pendingSlotRequests = new LinkedHashMap<>(16);
+		this.freeSlots = new HashMap<>(16);
+		this.allocationMap = new AllocationMap();
+	}
+
+	// ------------------------------------------------------------------------
+	//  slot managements
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Request a slot with requirements, we may either fulfill the request or pending it. Trigger container
+	 * allocation if we don't have enough resource. If we have free slot which can match the request, record
+	 * this allocation and forward the request to TaskManager through ResourceManager (we want this done by
+	 * RPC's main thread to avoid race condition).
+	 *
+	 * @param request The detailed request of the slot
+	 */
+	public void requestSlot(final SlotRequest request) {
+		if (isRequestDuplicated(request)) {
+			LOG.warn("Duplicated slot request, AllocationID:{}", request.getAllocationId());
+			return;
+		}
+
+		// try to fulfil the request with current free slots
+		ResourceSlot slot = chooseSlotToUse(request, freeSlots);
+		if (slot != null) {
+			LOG.info("Assigning SlotID({}) to AllocationID({}), JobID:{}", slot.getSlotId(),
+				request.getAllocationId(), request.getJobId());
+
+			// record this allocation in bookkeeping
+			allocationMap.addAllocation(slot.getSlotId(), request.getAllocationId());
+
+			// remove selected slot from free pool
+			freeSlots.remove(slot.getSlotId());
+
+			// TODO: send slot request to TaskManager
+		} else {
+			LOG.info("Cannot fulfil slot request, try to allocate a new container for it, " +
+				"AllocationID:{}, JobID:{}", request.getAllocationId(), request.getJobId());
+			allocateContainer(request.getResourceProfile());
+			pendingSlotRequests.put(request.getAllocationId(), request);
+		}
+	}
+
+	/**
+	 * Sync slot status with TaskManager's SlotReport.
+	 */
+	public void updateSlotStatus(final SlotReport slotReport) {
+		for (SlotStatus slotStatus : slotReport.getSlotsStatus()) {
+			updateSlotStatus(slotStatus);
+		}
+	}
+
+	/**
+	 * The slot request to TaskManager may be either failed by rpc communication (timeout, network error, etc.)
+	 * or really rejected by TaskManager. We shall retry this request by:
+	 * <ul>
+	 * <li>1. verify and clear all the previous allocate information for this request
+	 * <li>2. try to request slot again
+	 * </ul>
+	 * <p>
+	 * This may cause some duplicate allocation, e.g. the slot request to TaskManager is successful but the response
+	 * is lost somehow, so we may request a slot in another TaskManager, this causes two slots assigned to one request,
+	 * but it can be taken care of by rejecting registration at JobManager.
+	 *
+	 * @param originalRequest The original slot request
+	 * @param slotId          The target SlotID
+	 */
+	public void handleSlotRequestFailedAtTaskManager(final SlotRequest originalRequest, final SlotID slotId) {
+		final AllocationID originalAllocationId = originalRequest.getAllocationId();
+		LOG.info("Slot request failed at TaskManager, SlotID:{}, AllocationID:{}, JobID:{}",
+			slotId, originalAllocationId, originalRequest.getJobId());
+
+		// verify the allocation info before we do anything
+		if (freeSlots.containsKey(slotId)) {
+			// this slot is currently empty, no need to de-allocate it from our allocations
+			LOG.info("Original slot is somehow empty, retrying this request");
+
+			// before retry, we should double check whether this request was allocated by some other ways
+			if (!allocationMap.isAllocated(originalAllocationId)) {
+				requestSlot(originalRequest);
+			} else {
+				LOG.info("The failed request has somehow been allocated, SlotID:{}",
+					allocationMap.getSlotID(originalAllocationId));
+			}
+		} else if (allocationMap.isAllocated(slotId)) {
+			final AllocationID currentAllocationId = allocationMap.getAllocationID(slotId);
+
+			// check whether we have an agreement on whom this slot belongs to
+			if (originalAllocationId.equals(currentAllocationId)) {
+				LOG.info("De-allocate this request and retry");
+				allocationMap.removeAllocation(currentAllocationId);
+
+				// put this slot back to free pool
+				ResourceSlot slot = checkNotNull(getRegisteredSlot(slotId));
+				freeSlots.put(slotId, slot);
+
+				// retry the request
+				requestSlot(originalRequest);
+			} else {
+				// the slot is taken by someone else, no need to de-allocate it from our allocations
+				LOG.info("Original slot is taken by someone else, current AllocationID:{}", currentAllocationId);
+
+				// before retry, we should double check whether this request was allocated by some other ways
+				if (!allocationMap.isAllocated(originalAllocationId)) {
+					requestSlot(originalRequest);
+				} else {
+					LOG.info("The failed request is somehow been allocated, SlotID:{}",
+						allocationMap.getSlotID(originalAllocationId));
+				}
+			}
+		} else {
+			LOG.error("BUG! {} is neither in free pool nor in allocated pool", slotId);
+		}
+	}
+
+	/**
+	 * Callback for TaskManager failures. In case that a TaskManager fails, we have to clean up all its slots.
+	 *
+	 * @param resourceId The ResourceID of the TaskManager
+	 */
+	public void notifyTaskManagerFailure(final ResourceID resourceId) {
+		LOG.info("Resource:{} been notified failure", resourceId);
+		final Map<SlotID, ResourceSlot> slotIdsToRemove = registeredSlots.remove(resourceId);
+		if (slotIdsToRemove != null) {
+			for (SlotID slotId : slotIdsToRemove.keySet()) {
+				LOG.info("Removing Slot:{} upon resource failure", slotId);
+				if (freeSlots.containsKey(slotId)) {
+					freeSlots.remove(slotId);
+				} else if (allocationMap.isAllocated(slotId)) {
+					allocationMap.removeAllocation(slotId);
+				} else {
+					LOG.error("BUG! {} is neither in free pool nor in allocated pool", slotId);
+				}
+			}
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  internal behaviors
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Update slot status based on TaskManager's report. There are mainly two situations when we receive the report:
+	 * <ul>
+	 * <li>1. The slot is newly registered.</li>
+	 * <li>2. The slot has registered, it contains its current status.</li>
+	 * </ul>
+	 * <p>
+	 * Regarding 1: It's fairly simple, we just record this slot's status, and trigger schedule if slot is empty.
+	 * <p>
+	 * Regarding 2: It will cause some weird situation since we may have some time-gap on how the slot's status really
+	 * is. We may have some updates on the slot's allocation, but it doesn't reflected by TaskManager's heartbeat yet,
+	 * and we may make some wrong decision if we cannot guarantee we have the exact status about all the slots. So
+	 * the principle here is: We always trust TaskManager's heartbeat, we will correct our information based on that
+	 * and take next action based on the diff between our information and heartbeat status.
+	 *
+	 * @param reportedStatus Reported slot status
+	 */
+	void updateSlotStatus(final SlotStatus reportedStatus) {
+		final SlotID slotId = reportedStatus.getSlotID();
+		final ResourceSlot slot = new ResourceSlot(slotId, reportedStatus.getProfiler());
+
+		if (registerNewSlot(slot)) {
+			// we have a newly registered slot
+			LOG.info("New slot appeared, SlotID:{}, AllocationID:{}", slotId, reportedStatus.getAllocationID());
+
+			if (reportedStatus.getAllocationID() != null) {
+				// slot in use, record this in bookkeeping
+				allocationMap.addAllocation(slotId, reportedStatus.getAllocationID());
+			} else {
+				handleFreeSlot(new ResourceSlot(slotId, reportedStatus.getProfiler()));
+			}
+		} else {
+			// slot exists, update current information
+			if (reportedStatus.getAllocationID() != null) {
+				// slot is reported in use
+				final AllocationID reportedAllocationId = reportedStatus.getAllocationID();
+
+				// check whether we also thought this slot is in use
+				if (allocationMap.isAllocated(slotId)) {
+					// we also think that slot is in use, check whether the AllocationID matches
+					final AllocationID currentAllocationId = allocationMap.getAllocationID(slotId);
+
+					if (!reportedAllocationId.equals(currentAllocationId)) {
+						LOG.info("Slot allocation info mismatch! SlotID:{}, current:{}, reported:{}",
+							slotId, currentAllocationId, reportedAllocationId);
+
+						// seems we have a disagreement about the slot assignments, need to correct it
+						allocationMap.removeAllocation(slotId);
+						allocationMap.addAllocation(slotId, reportedAllocationId);
+					}
+				} else {
+					LOG.info("Slot allocation info mismatch! SlotID:{}, current:null, reported:{}",
+						slotId, reportedAllocationId);
+
+					// we thought the slot is free, should correct this information
+					allocationMap.addAllocation(slotId, reportedStatus.getAllocationID());
+
+					// remove this slot from free slots pool
+					freeSlots.remove(slotId);
+				}
+			} else {
+				// slot is reported empty
+
+				// check whether we also thought this slot is empty
+				if (allocationMap.isAllocated(slotId)) {
+					LOG.info("Slot allocation info mismatch! SlotID:{}, current:{}, reported:null",
+						slotId, allocationMap.getAllocationID(slotId));
+
+					// we thought the slot is in use, correct it
+					allocationMap.removeAllocation(slotId);
+
+					// we have a free slot!
+					handleFreeSlot(new ResourceSlot(slotId, reportedStatus.getProfiler()));
+				}
+			}
+		}
+	}
+
+	/**
+	 * When we have a free slot, try to fulfill the pending request first. If any request can be fulfilled,
+	 * record this allocation in bookkeeping and send slot request to TaskManager, else we just add this slot
+	 * to the free pool.
+	 *
+	 * @param freeSlot The free slot
+	 */
+	private void handleFreeSlot(final ResourceSlot freeSlot) {
+		SlotRequest chosenRequest = chooseRequestToFulfill(freeSlot, pendingSlotRequests);
+
+		if (chosenRequest != null) {
+			pendingSlotRequests.remove(chosenRequest.getAllocationId());
+
+			LOG.info("Assigning SlotID({}) to AllocationID({}), JobID:{}", freeSlot.getSlotId(),
+				chosenRequest.getAllocationId(), chosenRequest.getJobId());
+			allocationMap.addAllocation(freeSlot.getSlotId(), chosenRequest.getAllocationId());
+
+			// TODO: send slot request to TaskManager
+		} else {
+			freeSlots.put(freeSlot.getSlotId(), freeSlot);
+		}
+	}
+
+	/**
+	 * Check whether the request is duplicated. We use AllocationID to identify slot request, for each
+	 * formerly received slot request, it is either in pending list or already been allocated.
+	 *
+	 * @param request The slot request
+	 * @return <tt>true</tt> if the request is duplicated
+	 */
+	private boolean isRequestDuplicated(final SlotRequest request) {
+		final AllocationID allocationId = request.getAllocationId();
+		return pendingSlotRequests.containsKey(allocationId)
+			|| allocationMap.isAllocated(allocationId);
+	}
+
+	/**
+	 * Try to register slot, and tell if this slot is newly registered.
+	 *
+	 * @param slot The ResourceSlot which will be checked and registered
+	 * @return <tt>true</tt> if we meet a new slot
+	 */
+	private boolean registerNewSlot(final ResourceSlot slot) {
+		final SlotID slotId = slot.getSlotId();
+		final ResourceID resourceId = slotId.getResourceID();
+		if (!registeredSlots.containsKey(resourceId)) {
+			registeredSlots.put(resourceId, new HashMap<SlotID, ResourceSlot>());
+		}
+		return registeredSlots.get(resourceId).put(slotId, slot) == null;
+	}
+
+	private ResourceSlot getRegisteredSlot(final SlotID slotId) {
+		final ResourceID resourceId = slotId.getResourceID();
+		if (!registeredSlots.containsKey(resourceId)) {
+			return null;
+		}
+		return registeredSlots.get(resourceId).get(slotId);
+	}
+
+	// ------------------------------------------------------------------------
+	//  Framework specific behavior
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Choose a slot to use among all free slots, the behavior is framework specified.
+	 *
+	 * @param request   The slot request
+	 * @param freeSlots All slots which can be used
+	 * @return The slot we choose to use, <tt>null</tt> if we did not find a match
+	 */
+	protected abstract ResourceSlot chooseSlotToUse(final SlotRequest request,
+		final Map<SlotID, ResourceSlot> freeSlots);
+
+	/**
+	 * Choose a pending request to fulfill when we have a free slot, the behavior is framework specified.
+	 *
+	 * @param offeredSlot     The free slot
+	 * @param pendingRequests All the pending slot requests
+	 * @return The chosen SlotRequest, <tt>null</tt> if we did not find a match
+	 */
+	protected abstract SlotRequest chooseRequestToFulfill(final ResourceSlot offeredSlot,
+		final Map<AllocationID, SlotRequest> pendingRequests);
+
+	/**
+	 * The framework specific code for allocating a container for specified resource profile.
+	 *
+	 * @param resourceProfile The resource profile
+	 */
+	protected abstract void allocateContainer(final ResourceProfile resourceProfile);
+
+
+	// ------------------------------------------------------------------------
+	//  Helper classes
+	// ------------------------------------------------------------------------
+
+	/**
+	 * We maintain all the allocations with SlotID and AllocationID. We are able to get or remove the allocation info
+	 * either by SlotID or AllocationID.
+	 */
+	private static class AllocationMap {
+
+		/** All allocated slots (by SlotID) */
+		private final Map<SlotID, AllocationID> allocatedSlots;
+
+		/** All allocated slots (by AllocationID), it'a a inverse view of allocatedSlots */
+		private final Map<AllocationID, SlotID> allocatedSlotsByAllocationId;
+
+		AllocationMap() {
+			this.allocatedSlots = new HashMap<>(16);
+			this.allocatedSlotsByAllocationId = new HashMap<>(16);
+		}
+
+		/**
+		 * Add a allocation
+		 *
+		 * @param slotId       The slot id
+		 * @param allocationId The allocation id
+		 */
+		void addAllocation(final SlotID slotId, final AllocationID allocationId) {
+			allocatedSlots.put(slotId, allocationId);
+			allocatedSlotsByAllocationId.put(allocationId, slotId);
+		}
+
+		/**
+		 * De-allocation with slot id
+		 *
+		 * @param slotId The slot id
+		 */
+		void removeAllocation(final SlotID slotId) {
+			if (allocatedSlots.containsKey(slotId)) {
+				final AllocationID allocationId = allocatedSlots.get(slotId);
+				allocatedSlots.remove(slotId);
+				allocatedSlotsByAllocationId.remove(allocationId);
+			}
+		}
+
+		/**
+		 * De-allocation with allocation id
+		 *
+		 * @param allocationId The allocation id
+		 */
+		void removeAllocation(final AllocationID allocationId) {
+			if (allocatedSlotsByAllocationId.containsKey(allocationId)) {
+				SlotID slotId = allocatedSlotsByAllocationId.get(allocationId);
+				allocatedSlotsByAllocationId.remove(allocationId);
+				allocatedSlots.remove(slotId);
+			}
+		}
+
+		/**
+		 * Check whether allocation exists by slot id
+		 *
+		 * @param slotId The slot id
+		 * @return true if the allocation exists
+		 */
+		boolean isAllocated(final SlotID slotId) {
+			return allocatedSlots.containsKey(slotId);
+		}
+
+		/**
+		 * Check whether allocation exists by allocation id
+		 *
+		 * @param allocationId The allocation id
+		 * @return true if the allocation exists
+		 */
+		boolean isAllocated(final AllocationID allocationId) {
+			return allocatedSlotsByAllocationId.containsKey(allocationId);
+		}
+
+		AllocationID getAllocationID(final SlotID slotId) {
+			return allocatedSlots.get(slotId);
+		}
+
+		SlotID getSlotID(final AllocationID allocationId) {
+			return allocatedSlotsByAllocationId.get(allocationId);
+		}
+
+		public int size() {
+			return allocatedSlots.size();
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  Testing utilities
+	// ------------------------------------------------------------------------
+
+	@VisibleForTesting
+	boolean isAllocated(final SlotID slotId) {
+		return allocationMap.isAllocated(slotId);
+	}
+
+	@VisibleForTesting
+	boolean isAllocated(final AllocationID allocationId) {
+		return allocationMap.isAllocated(allocationId);
+	}
+
+	/**
+	 * Add free slots directly to the free pool, this will not trigger pending requests allocation
+	 *
+	 * @param slot The resource slot
+	 */
+	@VisibleForTesting
+	void addFreeSlot(final ResourceSlot slot) {
+		final ResourceID resourceId = slot.getResourceID();
+		final SlotID slotId = slot.getSlotId();
+
+		if (!registeredSlots.containsKey(resourceId)) {
+			registeredSlots.put(resourceId, new HashMap<SlotID, ResourceSlot>());
+		}
+		registeredSlots.get(resourceId).put(slot.getSlotId(), slot);
+		freeSlots.put(slotId, slot);
+	}
+
+	@VisibleForTesting
+	int getAllocatedSlotCount() {
+		return allocationMap.size();
+	}
+
+	@VisibleForTesting
+	int getFreeSlotCount() {
+		return freeSlots.size();
+	}
+
+	@VisibleForTesting
+	int getPendingRequestCount() {
+		return pendingSlotRequests.size();
+	}
+}


[09/50] [abbrv] flink git commit: [FLINK-4073] remove unstable testing code in TaskManagerFailure test

Posted by tr...@apache.org.
[FLINK-4073] remove unstable testing code in TaskManagerFailure test

Yarn reports null or (1, maxVcores) depending on its internal logic. The
test only worked in the past because it summed up the used vcores of the
RM and the TM containers. We have checks in place to ensure the vcores
config value is passed on to the Flink ResourceManager.


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

Branch: refs/heads/flip-6
Commit: 87aea72f181d0945092538384037acb0da364b9e
Parents: 2f87f61
Author: Maximilian Michels <mx...@apache.org>
Authored: Tue Sep 6 12:22:04 2016 +0200
Committer: Maximilian Michels <mx...@apache.org>
Committed: Tue Sep 6 12:55:38 2016 +0200

----------------------------------------------------------------------
 .../YARNSessionCapacitySchedulerITCase.java     | 21 --------------------
 1 file changed, 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/87aea72f/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java
----------------------------------------------------------------------
diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java
index 886ec32..ddea4dd 100644
--- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java
+++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java
@@ -32,9 +32,6 @@ import org.apache.hadoop.yarn.api.protocolrecords.StopContainersRequest;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.NodeReport;
-import org.apache.hadoop.yarn.api.records.NodeState;
-import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.client.api.YarnClient;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
@@ -246,24 +243,6 @@ public class YARNSessionCapacitySchedulerITCase extends YarnTestBase {
 		Assert.assertNotNull("Unable to find container with TaskManager", taskManagerContainer);
 		Assert.assertNotNull("Illegal state", nodeManager);
 
-		try {
-			List<NodeReport> nodeReports = yc.getNodeReports(NodeState.RUNNING);
-
-			// we asked for one node with 2 vcores so we expect 2 vcores
-			// note that the JobManager may also run on the NodeManager
-			boolean foundVCoresSetting = false;
-			for (NodeReport rep: nodeReports) {
-				Resource resource = rep.getUsed();
-				if (resource != null && resource.getVirtualCores() == 2) {
-					foundVCoresSetting = true;
-					break;
-				}
-			}
-			Assert.assertTrue(foundVCoresSetting);
-		} catch (Exception e) {
-			Assert.fail("Test failed: " + e.getMessage());
-		}
-
 		yc.stop();
 
 		List<ContainerId> toStop = new LinkedList<ContainerId>();


[32/50] [abbrv] flink git commit: [FLINK-4355] [cluster management] Implement TaskManager side of registration at ResourceManager.

Posted by tr...@apache.org.
[FLINK-4355] [cluster management] Implement TaskManager side of registration at ResourceManager.

This closes #2353


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

Branch: refs/heads/flip-6
Commit: f94ae5321199b2c803690278668c1094e18537e5
Parents: 2c54a61
Author: Stephan Ewen <se...@apache.org>
Authored: Wed Aug 10 20:42:45 2016 +0200
Committer: Till Rohrmann <tr...@apache.org>
Committed: Thu Sep 8 17:26:58 2016 +0200

----------------------------------------------------------------------
 .../HighAvailabilityServices.java               |  39 +++
 .../runtime/highavailability/NonHaServices.java |  59 ++++
 .../StandaloneLeaderRetrievalService.java       |  72 +++--
 .../apache/flink/runtime/rpc/RpcEndpoint.java   |   1 -
 .../apache/flink/runtime/rpc/RpcService.java    |  27 ++
 .../flink/runtime/rpc/akka/AkkaRpcService.java  |  18 ++
 .../runtime/rpc/akka/messages/RunAsync.java     |   1 +
 .../rpc/registration/RegistrationResponse.java  |  84 ++++++
 .../rpc/registration/RetryingRegistration.java  | 292 +++++++++++++++++++
 .../rpc/resourcemanager/ResourceManager.java    |  23 ++
 .../resourcemanager/ResourceManagerGateway.java |  21 +-
 .../runtime/rpc/taskexecutor/SlotReport.java    |  38 +++
 .../runtime/rpc/taskexecutor/TaskExecutor.java  | 169 ++++++++---
 .../rpc/taskexecutor/TaskExecutorGateway.java   |  29 +-
 .../TaskExecutorRegistrationSuccess.java        |  75 +++++
 ...TaskExecutorToResourceManagerConnection.java | 194 ++++++++++++
 .../runtime/rpc/akka/AkkaRpcServiceTest.java    |  51 +++-
 .../rpc/taskexecutor/TaskExecutorTest.java      |  87 +-----
 18 files changed, 1105 insertions(+), 175 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/f94ae532/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServices.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServices.java
new file mode 100644
index 0000000..094d36f
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServices.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.runtime.highavailability;
+
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+
+/**
+ * This class gives access to all services needed for
+ *
+ * <ul>
+ *     <li>ResourceManager leader election and leader retrieval</li>
+ *     <li>JobManager leader election and leader retrieval</li>
+ *     <li>Persistence for checkpoint metadata</li>
+ *     <li>Registering the latest completed checkpoint(s)</li>
+ * </ul>
+ */
+public interface HighAvailabilityServices {
+
+	/**
+	 * Gets the leader retriever for the cluster's resource manager.
+	 */
+	LeaderRetrievalService getResourceManagerLeaderRetriever() throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f94ae532/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/NonHaServices.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/NonHaServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/NonHaServices.java
new file mode 100644
index 0000000..b8c2ed8
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/NonHaServices.java
@@ -0,0 +1,59 @@
+/*
+ * 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.highavailability;
+
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.runtime.leaderretrieval.StandaloneLeaderRetrievalService;
+
+import java.util.UUID;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * An implementation of the {@link HighAvailabilityServices} for the non-high-availability case.
+ * This implementation can be used for testing, and for cluster setups that do not
+ * tolerate failures of the master processes (JobManager, ResourceManager).
+ * 
+ * <p>This implementation has no dependencies on any external services. It returns fix
+ * pre-configured leaders, and stores checkpoints and metadata simply on the heap and therefore
+ * in volatile memory.
+ */
+public class NonHaServices implements HighAvailabilityServices {
+
+	/** The fix address of the ResourceManager */
+	private final String resourceManagerAddress;
+
+	/**
+	 * Creates a new services class for the fix pre-defined leaders.
+	 * 
+	 * @param resourceManagerAddress    The fix address of the ResourceManager
+	 */
+	public NonHaServices(String resourceManagerAddress) {
+		this.resourceManagerAddress = checkNotNull(resourceManagerAddress);
+	}
+
+	// ------------------------------------------------------------------------
+	//  Services
+	// ------------------------------------------------------------------------
+
+	@Override
+	public LeaderRetrievalService getResourceManagerLeaderRetriever() throws Exception {
+		return new StandaloneLeaderRetrievalService(resourceManagerAddress, new UUID(0, 0));
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f94ae532/flink-runtime/src/main/java/org/apache/flink/runtime/leaderretrieval/StandaloneLeaderRetrievalService.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/leaderretrieval/StandaloneLeaderRetrievalService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/leaderretrieval/StandaloneLeaderRetrievalService.java
index 26a34aa..16b163c 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/leaderretrieval/StandaloneLeaderRetrievalService.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/leaderretrieval/StandaloneLeaderRetrievalService.java
@@ -18,44 +18,74 @@
 
 package org.apache.flink.runtime.leaderretrieval;
 
-import org.apache.flink.util.Preconditions;
+import java.util.UUID;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
 
 /**
- * Standalone implementation of the {@link LeaderRetrievalService}. The standalone implementation
- * assumes that there is only a single {@link org.apache.flink.runtime.jobmanager.JobManager} whose
- * address is given to the service when creating it. This address is directly given to the
- * {@link LeaderRetrievalListener} when the service is started.
+ * Standalone implementation of the {@link LeaderRetrievalService}. This implementation
+ * assumes that there is only a single contender for leadership
+ * (e.g., a single JobManager or ResourceManager process) and that this process is
+ * reachable under a constant address.
+ * 
+ * <p>As soon as this service is started, it immediately notifies the leader listener
+ * of the leader contender with the pre-configured address.
  */
 public class StandaloneLeaderRetrievalService implements LeaderRetrievalService {
 
-	/** Address of the only JobManager */
-	private final String jobManagerAddress;
+	private final Object startStopLock = new Object();
+	
+	/** The fix address of the leader */
+	private final String leaderAddress;
+
+	/** The fix leader ID (leader lock fencing token) */
+	private final UUID leaderId;
 
-	/** Listener which wants to be notified about the new leader */
-	private LeaderRetrievalListener leaderListener;
+	/** Flag whether this service is started */
+	private boolean started;
 
 	/**
-	 * Creates a StandaloneLeaderRetrievalService with the given JobManager address.
+	 * Creates a StandaloneLeaderRetrievalService with the given leader address.
+	 * The leaderId will be null.
 	 *
-	 * @param jobManagerAddress The JobManager's address which is returned to the
-	 * 							{@link LeaderRetrievalListener}
+	 * @param leaderAddress The leader's pre-configured address
 	 */
-	public StandaloneLeaderRetrievalService(String jobManagerAddress) {
-		this.jobManagerAddress = jobManagerAddress;
+	public StandaloneLeaderRetrievalService(String leaderAddress) {
+		this.leaderAddress = checkNotNull(leaderAddress);
+		this.leaderId = null;
 	}
 
+	/**
+	 * Creates a StandaloneLeaderRetrievalService with the given leader address.
+	 *
+	 * @param leaderAddress The leader's pre-configured address
+	 * @param leaderId      The constant leaderId.
+	 */
+	public StandaloneLeaderRetrievalService(String leaderAddress, UUID leaderId) {
+		this.leaderAddress = checkNotNull(leaderAddress);
+		this.leaderId = checkNotNull(leaderId);
+	}
+
+	// ------------------------------------------------------------------------
+
 	@Override
 	public void start(LeaderRetrievalListener listener) {
-		Preconditions.checkNotNull(listener, "Listener must not be null.");
-		Preconditions.checkState(leaderListener == null, "StandaloneLeaderRetrievalService can " +
-				"only be started once.");
+		checkNotNull(listener, "Listener must not be null.");
 
-		leaderListener = listener;
+		synchronized (startStopLock) {
+			checkState(!started, "StandaloneLeaderRetrievalService can only be started once.");
+			started = true;
 
-		// directly notify the listener, because we already know the leading JobManager's address
-		leaderListener.notifyLeaderAddress(jobManagerAddress, null);
+			// directly notify the listener, because we already know the leading JobManager's address
+			listener.notifyLeaderAddress(leaderAddress, leaderId);
+		}
 	}
 
 	@Override
-	public void stop() {}
+	public void stop() {
+		synchronized (startStopLock) {
+			started = false;
+		}
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/f94ae532/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java
index 67ac182..a28bc14 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java
@@ -237,7 +237,6 @@ public abstract class RpcEndpoint<C extends RpcGateway> {
 	 * }</pre>
 	 */
 	public void validateRunsInMainThread() {
-		// because the initialization is lazy, it can be that certain methods are
 		assert currentMainThread.get() == Thread.currentThread();
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/f94ae532/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java
index f93be83..fabdb05 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java
@@ -18,8 +18,11 @@
 
 package org.apache.flink.runtime.rpc;
 
+import scala.concurrent.ExecutionContext;
 import scala.concurrent.Future;
 
+import java.util.concurrent.TimeUnit;
+
 /**
  * Interface for rpc services. An rpc service is used to start and connect to a {@link RpcEndpoint}.
  * Connecting to a rpc server will return a {@link RpcGateway} which can be used to call remote
@@ -71,4 +74,28 @@ public interface RpcService {
 	 * @return Fully qualified address
 	 */
 	<C extends RpcGateway> String getAddress(C selfGateway);
+
+	/**
+	 * Gets the execution context, provided by this RPC service. This execution
+	 * context can be used for example for the {@code onComplete(...)} or {@code onSuccess(...)}
+	 * methods of Futures.
+	 * 
+	 * <p><b>IMPORTANT:</b> This execution context does not isolate the method invocations against
+	 * any concurrent invocations and is therefore not suitable to run completion methods of futures
+	 * that modify state of an {@link RpcEndpoint}. For such operations, one needs to use the
+	 * {@link RpcEndpoint#getMainThreadExecutionContext() MainThreadExecutionContext} of that
+	 * {@code RpcEndpoint}.
+	 * 
+	 * @return The execution context provided by the RPC service
+	 */
+	ExecutionContext getExecutionContext();
+
+	/**
+	 * Execute the runnable in the execution context of this RPC Service, as returned by
+	 * {@link #getExecutionContext()}, after a scheduled delay.
+	 *
+	 * @param runnable Runnable to be executed
+	 * @param delay    The delay after which the runnable will be executed
+	 */
+	void scheduleRunnable(Runnable runnable, long delay, TimeUnit unit);
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/f94ae532/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java
index 7b33524..b647bbd 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java
@@ -38,14 +38,18 @@ import org.apache.flink.runtime.rpc.StartStoppable;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import scala.concurrent.ExecutionContext;
 import scala.concurrent.Future;
+import scala.concurrent.duration.FiniteDuration;
 
 import javax.annotation.concurrent.ThreadSafe;
 import java.lang.reflect.InvocationHandler;
 import java.lang.reflect.Proxy;
 import java.util.HashSet;
 import java.util.Set;
+import java.util.concurrent.TimeUnit;
 
+import static org.apache.flink.util.Preconditions.checkArgument;
 import static org.apache.flink.util.Preconditions.checkNotNull;
 import static org.apache.flink.util.Preconditions.checkState;
 
@@ -199,4 +203,18 @@ public class AkkaRpcService implements RpcService {
 			throw new IllegalArgumentException("Cannot get address for non " + className + '.');
 		}
 	}
+
+	@Override
+	public ExecutionContext getExecutionContext() {
+		return actorSystem.dispatcher();
+	}
+
+	@Override
+	public void scheduleRunnable(Runnable runnable, long delay, TimeUnit unit) {
+		checkNotNull(runnable, "runnable");
+		checkNotNull(unit, "unit");
+		checkArgument(delay >= 0, "delay must be zero or larger");
+
+		actorSystem.scheduler().scheduleOnce(new FiniteDuration(delay, unit), runnable, getExecutionContext());
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/f94ae532/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RunAsync.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RunAsync.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RunAsync.java
index c18906c..ce4f9d6 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RunAsync.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RunAsync.java
@@ -36,6 +36,7 @@ public final class RunAsync implements Serializable {
 	private final long delay;
 
 	/**
+	 * Creates a new {@code RunAsync} message.
 	 * 
 	 * @param runnable  The Runnable to run.
 	 * @param delay     The delay in milliseconds. Zero indicates immediate execution.

http://git-wip-us.apache.org/repos/asf/flink/blob/f94ae532/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/registration/RegistrationResponse.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/registration/RegistrationResponse.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/registration/RegistrationResponse.java
new file mode 100644
index 0000000..2de560a
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/registration/RegistrationResponse.java
@@ -0,0 +1,84 @@
+/*
+ * 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.rpc.registration;
+
+import java.io.Serializable;
+
+/**
+ * Base class for responses given to registration attempts from {@link RetryingRegistration}.
+ */
+public abstract class RegistrationResponse implements Serializable {
+
+	private static final long serialVersionUID = 1L;
+
+	// ----------------------------------------------------------------------------
+	
+	/**
+	 * Base class for a successful registration. Concrete registration implementations
+	 * will typically extend this class to attach more information.
+	 */
+	public static class Success extends RegistrationResponse {
+		private static final long serialVersionUID = 1L;
+		
+		@Override
+		public String toString() {
+			return "Registration Successful";
+		}
+	}
+
+	// ----------------------------------------------------------------------------
+
+	/**
+	 * A rejected (declined) registration.
+	 */
+	public static final class Decline extends RegistrationResponse {
+		private static final long serialVersionUID = 1L;
+
+		/** the rejection reason */
+		private final String reason;
+
+		/**
+		 * Creates a new rejection message.
+		 * 
+		 * @param reason The reason for the rejection.
+		 */
+		public Decline(String reason) {
+			this.reason = reason != null ? reason : "(unknown)";
+		}
+
+		/**
+		 * Gets the reason for the rejection.
+		 */
+		public String getReason() {
+			return reason;
+		}
+
+		@Override
+		public String toString() {
+			return "Registration Declined (" + reason + ')';
+		}
+	}
+}
+
+
+
+
+
+
+

http://git-wip-us.apache.org/repos/asf/flink/blob/f94ae532/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/registration/RetryingRegistration.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/registration/RetryingRegistration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/registration/RetryingRegistration.java
new file mode 100644
index 0000000..4c93684
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/registration/RetryingRegistration.java
@@ -0,0 +1,292 @@
+/*
+ * 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.rpc.registration;
+
+import akka.dispatch.OnFailure;
+import akka.dispatch.OnSuccess;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.runtime.rpc.RpcGateway;
+import org.apache.flink.runtime.rpc.RpcService;
+
+import org.slf4j.Logger;
+
+import scala.concurrent.Future;
+import scala.concurrent.Promise;
+import scala.concurrent.impl.Promise.DefaultPromise;
+
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+
+/**
+ * This utility class implements the basis of registering one component at another component,
+ * for example registering the TaskExecutor at the ResourceManager.
+ * This {@code RetryingRegistration} implements both the initial address resolution
+ * and the retries-with-backoff strategy.
+ * 
+ * <p>The registration gives access to a future that is completed upon successful registration.
+ * The registration can be canceled, for example when the target where it tries to register
+ * at looses leader status.
+ * 
+ * @param <Gateway> The type of the gateway to connect to.
+ * @param <Success> The type of the successful registration responses.
+ */
+public abstract class RetryingRegistration<Gateway extends RpcGateway, Success extends RegistrationResponse.Success> {
+
+	// ------------------------------------------------------------------------
+	//  default configuration values
+	// ------------------------------------------------------------------------
+
+	private static final long INITIAL_REGISTRATION_TIMEOUT_MILLIS = 100;
+
+	private static final long MAX_REGISTRATION_TIMEOUT_MILLIS = 30000;
+
+	private static final long ERROR_REGISTRATION_DELAY_MILLIS = 10000;
+
+	private static final long REFUSED_REGISTRATION_DELAY_MILLIS = 30000;
+
+	// ------------------------------------------------------------------------
+	// Fields
+	// ------------------------------------------------------------------------
+
+	private final Logger log;
+
+	private final RpcService rpcService;
+
+	private final String targetName;
+
+	private final Class<Gateway> targetType;
+
+	private final String targetAddress;
+
+	private final UUID leaderId;
+
+	private final Promise<Tuple2<Gateway, Success>> completionPromise;
+
+	private final long initialRegistrationTimeout;
+
+	private final long maxRegistrationTimeout;
+
+	private final long delayOnError;
+
+	private final long delayOnRefusedRegistration;
+
+	private volatile boolean canceled;
+
+	// ------------------------------------------------------------------------
+
+	public RetryingRegistration(
+			Logger log,
+			RpcService rpcService,
+			String targetName,
+			Class<Gateway> targetType,
+			String targetAddress,
+			UUID leaderId) {
+		this(log, rpcService, targetName, targetType, targetAddress, leaderId,
+				INITIAL_REGISTRATION_TIMEOUT_MILLIS, MAX_REGISTRATION_TIMEOUT_MILLIS,
+				ERROR_REGISTRATION_DELAY_MILLIS, REFUSED_REGISTRATION_DELAY_MILLIS);
+	}
+
+	public RetryingRegistration(
+			Logger log,
+			RpcService rpcService,
+			String targetName, 
+			Class<Gateway> targetType,
+			String targetAddress,
+			UUID leaderId,
+			long initialRegistrationTimeout,
+			long maxRegistrationTimeout,
+			long delayOnError,
+			long delayOnRefusedRegistration) {
+
+		checkArgument(initialRegistrationTimeout > 0, "initial registration timeout must be greater than zero");
+		checkArgument(maxRegistrationTimeout > 0, "maximum registration timeout must be greater than zero");
+		checkArgument(delayOnError >= 0, "delay on error must be non-negative");
+		checkArgument(delayOnRefusedRegistration >= 0, "delay on refused registration must be non-negative");
+
+		this.log = checkNotNull(log);
+		this.rpcService = checkNotNull(rpcService);
+		this.targetName = checkNotNull(targetName);
+		this.targetType = checkNotNull(targetType);
+		this.targetAddress = checkNotNull(targetAddress);
+		this.leaderId = checkNotNull(leaderId);
+		this.initialRegistrationTimeout = initialRegistrationTimeout;
+		this.maxRegistrationTimeout = maxRegistrationTimeout;
+		this.delayOnError = delayOnError;
+		this.delayOnRefusedRegistration = delayOnRefusedRegistration;
+
+		this.completionPromise = new DefaultPromise<>();
+	}
+
+	// ------------------------------------------------------------------------
+	//  completion and cancellation
+	// ------------------------------------------------------------------------
+
+	public Future<Tuple2<Gateway, Success>> getFuture() {
+		return completionPromise.future();
+	}
+
+	/**
+	 * Cancels the registration procedure.
+	 */
+	public void cancel() {
+		canceled = true;
+	}
+
+	/**
+	 * Checks if the registration was canceled.
+	 * @return True if the registration was canceled, false otherwise.
+	 */
+	public boolean isCanceled() {
+		return canceled;
+	}
+
+	// ------------------------------------------------------------------------
+	//  registration
+	// ------------------------------------------------------------------------
+
+	protected abstract Future<RegistrationResponse> invokeRegistration(
+			Gateway gateway, UUID leaderId, long timeoutMillis) throws Exception;
+
+	/**
+	 * This method resolves the target address to a callable gateway and starts the
+	 * registration after that.
+	 */
+	@SuppressWarnings("unchecked")
+	public void startRegistration() {
+		try {
+			// trigger resolution of the resource manager address to a callable gateway
+			Future<Gateway> resourceManagerFuture = rpcService.connect(targetAddress, targetType);
+	
+			// upon success, start the registration attempts
+			resourceManagerFuture.onSuccess(new OnSuccess<Gateway>() {
+				@Override
+				public void onSuccess(Gateway result) {
+					log.info("Resolved {} address, beginning registration", targetName);
+					register(result, 1, initialRegistrationTimeout);
+				}
+			}, rpcService.getExecutionContext());
+	
+			// upon failure, retry, unless this is cancelled
+			resourceManagerFuture.onFailure(new OnFailure() {
+				@Override
+				public void onFailure(Throwable failure) {
+					if (!isCanceled()) {
+						log.warn("Could not resolve {} address {}, retrying...", targetName, targetAddress);
+						startRegistration();
+					}
+				}
+			}, rpcService.getExecutionContext());
+		}
+		catch (Throwable t) {
+			cancel();
+			completionPromise.tryFailure(t);
+		}
+	}
+
+	/**
+	 * This method performs a registration attempt and triggers either a success notification or a retry,
+	 * depending on the result.
+	 */
+	@SuppressWarnings("unchecked")
+	private void register(final Gateway gateway, final int attempt, final long timeoutMillis) {
+		// eager check for canceling to avoid some unnecessary work
+		if (canceled) {
+			return;
+		}
+
+		try {
+			log.info("Registration at {} attempt {} (timeout={}ms)", targetName, attempt, timeoutMillis);
+			Future<RegistrationResponse> registrationFuture = invokeRegistration(gateway, leaderId, timeoutMillis);
+	
+			// if the registration was successful, let the TaskExecutor know
+			registrationFuture.onSuccess(new OnSuccess<RegistrationResponse>() {
+				
+				@Override
+				public void onSuccess(RegistrationResponse result) throws Throwable {
+					if (!isCanceled()) {
+						if (result instanceof RegistrationResponse.Success) {
+							// registration successful!
+							Success success = (Success) result;
+							completionPromise.success(new Tuple2<>(gateway, success));
+						}
+						else {
+							// registration refused or unknown
+							if (result instanceof RegistrationResponse.Decline) {
+								RegistrationResponse.Decline decline = (RegistrationResponse.Decline) result;
+								log.info("Registration at {} was declined: {}", targetName, decline.getReason());
+							} else {
+								log.error("Received unknown response to registration attempt: " + result);
+							}
+
+							log.info("Pausing and re-attempting registration in {} ms", delayOnRefusedRegistration);
+							registerLater(gateway, 1, initialRegistrationTimeout, delayOnRefusedRegistration);
+						}
+					}
+				}
+			}, rpcService.getExecutionContext());
+	
+			// upon failure, retry
+			registrationFuture.onFailure(new OnFailure() {
+				@Override
+				public void onFailure(Throwable failure) {
+					if (!isCanceled()) {
+						if (failure instanceof TimeoutException) {
+							// we simply have not received a response in time. maybe the timeout was
+							// very low (initial fast registration attempts), maybe the target endpoint is
+							// currently down.
+							if (log.isDebugEnabled()) {
+								log.debug("Registration at {} ({}) attempt {} timed out after {} ms",
+										targetName, targetAddress, attempt, timeoutMillis);
+							}
+	
+							long newTimeoutMillis = Math.min(2 * timeoutMillis, maxRegistrationTimeout);
+							register(gateway, attempt + 1, newTimeoutMillis);
+						}
+						else {
+							// a serious failure occurred. we still should not give up, but keep trying
+							log.error("Registration at " + targetName + " failed due to an error", failure);
+							log.info("Pausing and re-attempting registration in {} ms", delayOnError);
+	
+							registerLater(gateway, 1, initialRegistrationTimeout, delayOnError);
+						}
+					}
+				}
+			}, rpcService.getExecutionContext());
+		}
+		catch (Throwable t) {
+			cancel();
+			completionPromise.tryFailure(t);
+		}
+	}
+
+	private void registerLater(final Gateway gateway, final int attempt, final long timeoutMillis, long delay) {
+		rpcService.scheduleRunnable(new Runnable() {
+			@Override
+			public void run() {
+				register(gateway, attempt, timeoutMillis);
+			}
+		}, delay, TimeUnit.MILLISECONDS);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f94ae532/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManager.java
index 729ef0c..6f34465 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManager.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManager.java
@@ -19,19 +19,24 @@
 package org.apache.flink.runtime.rpc.resourcemanager;
 
 import akka.dispatch.Mapper;
+
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
 import org.apache.flink.runtime.instance.InstanceID;
 import org.apache.flink.runtime.rpc.RpcMethod;
 import org.apache.flink.runtime.rpc.RpcEndpoint;
 import org.apache.flink.runtime.rpc.RpcService;
 import org.apache.flink.runtime.rpc.jobmaster.JobMaster;
 import org.apache.flink.runtime.rpc.jobmaster.JobMasterGateway;
+import org.apache.flink.runtime.rpc.taskexecutor.TaskExecutorRegistrationSuccess;
 import org.apache.flink.util.Preconditions;
+
 import scala.concurrent.ExecutionContext;
 import scala.concurrent.ExecutionContext$;
 import scala.concurrent.Future;
 
 import java.util.HashMap;
 import java.util.Map;
+import java.util.UUID;
 import java.util.concurrent.ExecutorService;
 
 /**
@@ -93,4 +98,22 @@ public class ResourceManager extends RpcEndpoint<ResourceManagerGateway> {
 		System.out.println("SlotRequest: " + slotRequest);
 		return new SlotAssignment();
 	}
+
+
+	/**
+	 *
+	 * @param resourceManagerLeaderId  The fencing token for the ResourceManager leader 
+	 * @param taskExecutorAddress      The address of the TaskExecutor that registers
+	 * @param resourceID               The resource ID of the TaskExecutor that registers
+	 *
+	 * @return The response by the ResourceManager.
+	 */
+	@RpcMethod
+	public org.apache.flink.runtime.rpc.registration.RegistrationResponse registerTaskExecutor(
+			UUID resourceManagerLeaderId,
+			String taskExecutorAddress,
+			ResourceID resourceID) {
+
+		return new TaskExecutorRegistrationSuccess(new InstanceID(), 5000);
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/f94ae532/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManagerGateway.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManagerGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManagerGateway.java
index 464a261..afddb01 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManagerGateway.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManagerGateway.java
@@ -18,14 +18,18 @@
 
 package org.apache.flink.runtime.rpc.resourcemanager;
 
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
 import org.apache.flink.runtime.rpc.RpcGateway;
 import org.apache.flink.runtime.rpc.RpcTimeout;
 import org.apache.flink.runtime.rpc.jobmaster.JobMaster;
+
 import scala.concurrent.Future;
 import scala.concurrent.duration.FiniteDuration;
 
+import java.util.UUID;
+
 /**
- * {@link ResourceManager} rpc gateway interface.
+ * The {@link ResourceManager}'s RPC gateway interface.
  */
 public interface ResourceManagerGateway extends RpcGateway {
 
@@ -55,4 +59,19 @@ public interface ResourceManagerGateway extends RpcGateway {
 	 * @return Future slot assignment
 	 */
 	Future<SlotAssignment> requestSlot(SlotRequest slotRequest);
+
+	/**
+	 * 
+	 * @param resourceManagerLeaderId  The fencing token for the ResourceManager leader 
+	 * @param taskExecutorAddress      The address of the TaskExecutor that registers
+	 * @param resourceID               The resource ID of the TaskExecutor that registers
+	 * @param timeout                  The timeout for the response.
+	 * 
+	 * @return The future to the response by the ResourceManager.
+	 */
+	Future<org.apache.flink.runtime.rpc.registration.RegistrationResponse> registerTaskExecutor(
+			UUID resourceManagerLeaderId,
+			String taskExecutorAddress,
+			ResourceID resourceID,
+			@RpcTimeout FiniteDuration timeout);
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/f94ae532/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/SlotReport.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/SlotReport.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/SlotReport.java
new file mode 100644
index 0000000..e42fa4a
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/SlotReport.java
@@ -0,0 +1,38 @@
+/*
+ * 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.rpc.taskexecutor;
+
+import java.io.Serializable;
+
+/**
+ * A report about the current status of all slots of the TaskExecutor, describing
+ * which slots are available and allocated, and what jobs (JobManagers) the allocated slots
+ * have been allocated to.
+ */
+public class SlotReport implements Serializable{
+
+	private static final long serialVersionUID = 1L;
+
+	// ------------------------------------------------------------------------
+	
+	@Override
+	public String toString() {
+		return "SlotReport";
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f94ae532/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java
index 3a7dd9f..1a637bb 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java
@@ -18,67 +18,152 @@
 
 package org.apache.flink.runtime.rpc.taskexecutor;
 
-import akka.dispatch.ExecutionContexts$;
-import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor;
-import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
-import org.apache.flink.runtime.messages.Acknowledge;
-import org.apache.flink.runtime.rpc.RpcMethod;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
 import org.apache.flink.runtime.rpc.RpcEndpoint;
+import org.apache.flink.runtime.rpc.RpcMethod;
 import org.apache.flink.runtime.rpc.RpcService;
-import org.apache.flink.util.Preconditions;
-import scala.concurrent.ExecutionContext;
 
-import java.util.HashSet;
-import java.util.Set;
-import java.util.concurrent.ExecutorService;
+import java.util.UUID;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**
  * TaskExecutor implementation. The task executor is responsible for the execution of multiple
  * {@link org.apache.flink.runtime.taskmanager.Task}.
- *
- * It offers the following methods as part of its rpc interface to interact with him remotely:
- * <ul>
- *     <li>{@link #executeTask(TaskDeploymentDescriptor)} executes a given task on the TaskExecutor</li>
- *     <li>{@link #cancelTask(ExecutionAttemptID)} cancels a given task identified by the {@link ExecutionAttemptID}</li>
- * </ul>
  */
 public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
-	private final ExecutionContext executionContext;
-	private final Set<ExecutionAttemptID> tasks = new HashSet<>();
 
-	public TaskExecutor(RpcService rpcService, ExecutorService executorService) {
+	/** The unique resource ID of this TaskExecutor */
+	private final ResourceID resourceID;
+
+	/** The access to the leader election and metadata storage services */
+	private final HighAvailabilityServices haServices;
+
+	// --------- resource manager --------
+
+	private TaskExecutorToResourceManagerConnection resourceManagerConnection;
+
+	// ------------------------------------------------------------------------
+
+	public TaskExecutor(
+			RpcService rpcService,
+			HighAvailabilityServices haServices,
+			ResourceID resourceID) {
+
 		super(rpcService);
-		this.executionContext = ExecutionContexts$.MODULE$.fromExecutor(
-			Preconditions.checkNotNull(executorService));
+
+		this.haServices = checkNotNull(haServices);
+		this.resourceID = checkNotNull(resourceID);
+	}
+
+	// ------------------------------------------------------------------------
+	//  Properties
+	// ------------------------------------------------------------------------
+
+	public ResourceID getResourceID() {
+		return resourceID;
+	}
+
+	// ------------------------------------------------------------------------
+	//  Life cycle
+	// ------------------------------------------------------------------------
+
+	@Override
+	public void start() {
+		// start by connecting to the ResourceManager
+		try {
+			haServices.getResourceManagerLeaderRetriever().start(new ResourceManagerLeaderListener());
+		} catch (Exception e) {
+			onFatalErrorAsync(e);
+		}
+	}
+
+
+	// ------------------------------------------------------------------------
+	//  RPC methods - ResourceManager related
+	// ------------------------------------------------------------------------
+
+	@RpcMethod
+	public void notifyOfNewResourceManagerLeader(String newLeaderAddress, UUID newLeaderId) {
+		if (resourceManagerConnection != null) {
+			if (newLeaderAddress != null) {
+				// the resource manager switched to a new leader
+				log.info("ResourceManager leader changed from {} to {}. Registering at new leader.",
+						resourceManagerConnection.getResourceManagerAddress(), newLeaderAddress);
+			}
+			else {
+				// address null means that the current leader is lost without a new leader being there, yet
+				log.info("Current ResourceManager {} lost leader status. Waiting for new ResourceManager leader.",
+						resourceManagerConnection.getResourceManagerAddress());
+			}
+
+			// drop the current connection or connection attempt
+			if (resourceManagerConnection != null) {
+				resourceManagerConnection.close();
+				resourceManagerConnection = null;
+			}
+		}
+
+		// establish a connection to the new leader
+		if (newLeaderAddress != null) {
+			log.info("Attempting to register at ResourceManager {}", newLeaderAddress);
+			resourceManagerConnection = 
+					new TaskExecutorToResourceManagerConnection(log, this, newLeaderAddress, newLeaderId);
+			resourceManagerConnection.start();
+		}
 	}
 
+	// ------------------------------------------------------------------------
+	//  Error handling
+	// ------------------------------------------------------------------------
+
 	/**
-	 * Execute the given task on the task executor. The task is described by the provided
-	 * {@link TaskDeploymentDescriptor}.
-	 *
-	 * @param taskDeploymentDescriptor Descriptor for the task to be executed
-	 * @return Acknowledge the start of the task execution
+	 * Notifies the TaskExecutor that a fatal error has occurred and it cannot proceed.
+	 * This method should be used when asynchronous threads want to notify the
+	 * TaskExecutor of a fatal error.
+	 * 
+	 * @param t The exception describing the fatal error
 	 */
-	@RpcMethod
-	public Acknowledge executeTask(TaskDeploymentDescriptor taskDeploymentDescriptor) {
-		tasks.add(taskDeploymentDescriptor.getExecutionId());
-		return Acknowledge.get();
+	void onFatalErrorAsync(final Throwable t) {
+		runAsync(new Runnable() {
+			@Override
+			public void run() {
+				onFatalError(t);
+			}
+		});
 	}
 
 	/**
-	 * Cancel a task identified by it {@link ExecutionAttemptID}. If the task cannot be found, then
-	 * the method throws an {@link Exception}.
-	 *
-	 * @param executionAttemptId Execution attempt ID identifying the task to be canceled.
-	 * @return Acknowledge the task canceling
-	 * @throws Exception if the task with the given execution attempt id could not be found
+	 * Notifies the TaskExecutor that a fatal error has occurred and it cannot proceed.
+	 * This method must only be called from within the TaskExecutor's main thread.
+	 * 
+	 * @param t The exception describing the fatal error
 	 */
-	@RpcMethod
-	public Acknowledge cancelTask(ExecutionAttemptID executionAttemptId) throws Exception {
-		if (tasks.contains(executionAttemptId)) {
-			return Acknowledge.get();
-		} else {
-			throw new Exception("Could not find task.");
+	void onFatalError(Throwable t) {
+		// to be determined, probably delegate to a fatal error handler that 
+		// would either log (mini cluster) ot kill the process (yarn, mesos, ...)
+		log.error("FATAL ERROR", t);
+	}
+
+	// ------------------------------------------------------------------------
+	//  Utility classes
+	// ------------------------------------------------------------------------
+
+	/**
+	 * The listener for leader changes of the resource manager
+	 */
+	private class ResourceManagerLeaderListener implements LeaderRetrievalListener {
+
+		@Override
+		public void notifyLeaderAddress(String leaderAddress, UUID leaderSessionID) {
+			getSelf().notifyOfNewResourceManagerLeader(leaderAddress, leaderSessionID);
+		}
+
+		@Override
+		public void handleError(Exception exception) {
+			onFatalErrorAsync(exception);
 		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/f94ae532/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorGateway.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorGateway.java
index 450423e..b0b21b0 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorGateway.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorGateway.java
@@ -18,31 +18,18 @@
 
 package org.apache.flink.runtime.rpc.taskexecutor;
 
-import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor;
-import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
-import org.apache.flink.runtime.messages.Acknowledge;
 import org.apache.flink.runtime.rpc.RpcGateway;
-import scala.concurrent.Future;
+
+import java.util.UUID;
 
 /**
- * {@link TaskExecutor} rpc gateway interface
+ * {@link TaskExecutor} RPC gateway interface
  */
 public interface TaskExecutorGateway extends RpcGateway {
-	/**
-	 * Execute the given task on the task executor. The task is described by the provided
-	 * {@link TaskDeploymentDescriptor}.
-	 *
-	 * @param taskDeploymentDescriptor Descriptor for the task to be executed
-	 * @return Future acknowledge of the start of the task execution
-	 */
-	Future<Acknowledge> executeTask(TaskDeploymentDescriptor taskDeploymentDescriptor);
 
-	/**
-	 * Cancel a task identified by it {@link ExecutionAttemptID}. If the task cannot be found, then
-	 * the method throws an {@link Exception}.
-	 *
-	 * @param executionAttemptId Execution attempt ID identifying the task to be canceled.
-	 * @return Future acknowledge of the task canceling
-	 */
-	Future<Acknowledge> cancelTask(ExecutionAttemptID executionAttemptId);
+	// ------------------------------------------------------------------------
+	//  ResourceManager handlers
+	// ------------------------------------------------------------------------
+
+	void notifyOfNewResourceManagerLeader(String address, UUID resourceManagerLeaderId);
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/f94ae532/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorRegistrationSuccess.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorRegistrationSuccess.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorRegistrationSuccess.java
new file mode 100644
index 0000000..641102d
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorRegistrationSuccess.java
@@ -0,0 +1,75 @@
+/*
+ * 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.rpc.taskexecutor;
+
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.rpc.registration.RegistrationResponse;
+
+import java.io.Serializable;
+
+/**
+ * Base class for responses from the ResourceManager to a registration attempt by a
+ * TaskExecutor.
+ */
+public final class TaskExecutorRegistrationSuccess extends RegistrationResponse.Success implements Serializable {
+
+	private static final long serialVersionUID = 1L;
+
+	private final InstanceID registrationId;
+
+	private final long heartbeatInterval;
+
+	/**
+	 * Create a new {@code TaskExecutorRegistrationSuccess} message.
+	 * 
+	 * @param registrationId     The ID that the ResourceManager assigned the registration.
+	 * @param heartbeatInterval  The interval in which the ResourceManager will heartbeat the TaskExecutor.
+	 */
+	public TaskExecutorRegistrationSuccess(InstanceID registrationId, long heartbeatInterval) {
+		this.registrationId = registrationId;
+		this.heartbeatInterval = heartbeatInterval;
+	}
+
+	/**
+	 * Gets the ID that the ResourceManager assigned the registration.
+	 */
+	public InstanceID getRegistrationId() {
+		return registrationId;
+	}
+
+	/**
+	 * Gets the interval in which the ResourceManager will heartbeat the TaskExecutor.
+	 */
+	public long getHeartbeatInterval() {
+		return heartbeatInterval;
+	}
+
+	@Override
+	public String toString() {
+		return "TaskExecutorRegistrationSuccess (" + registrationId + " / " + heartbeatInterval + ')';
+	}
+
+}
+
+
+
+
+
+
+

http://git-wip-us.apache.org/repos/asf/flink/blob/f94ae532/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorToResourceManagerConnection.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorToResourceManagerConnection.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorToResourceManagerConnection.java
new file mode 100644
index 0000000..ef75862
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorToResourceManagerConnection.java
@@ -0,0 +1,194 @@
+/*
+ * 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.rpc.taskexecutor;
+
+import akka.dispatch.OnFailure;
+import akka.dispatch.OnSuccess;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.rpc.RpcService;
+import org.apache.flink.runtime.rpc.registration.RegistrationResponse;
+import org.apache.flink.runtime.rpc.registration.RetryingRegistration;
+import org.apache.flink.runtime.rpc.resourcemanager.ResourceManagerGateway;
+
+import org.slf4j.Logger;
+
+import scala.concurrent.Future;
+import scala.concurrent.duration.FiniteDuration;
+
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+public class TaskExecutorToResourceManagerConnection {
+
+	/** the logger for all log messages of this class */
+	private final Logger log;
+
+	/** the TaskExecutor whose connection to the ResourceManager this represents */
+	private final TaskExecutor taskExecutor;
+
+	private final UUID resourceManagerLeaderId;
+
+	private final String resourceManagerAddress;
+
+	private ResourceManagerRegistration pendingRegistration;
+
+	private ResourceManagerGateway registeredResourceManager;
+
+	private InstanceID registrationId;
+
+	/** flag indicating that the connection is closed */
+	private volatile boolean closed;
+
+
+	public TaskExecutorToResourceManagerConnection(
+			Logger log,
+			TaskExecutor taskExecutor,
+			String resourceManagerAddress,
+			UUID resourceManagerLeaderId) {
+
+		this.log = checkNotNull(log);
+		this.taskExecutor = checkNotNull(taskExecutor);
+		this.resourceManagerAddress = checkNotNull(resourceManagerAddress);
+		this.resourceManagerLeaderId = checkNotNull(resourceManagerLeaderId);
+	}
+
+	// ------------------------------------------------------------------------
+	//  Life cycle
+	// ------------------------------------------------------------------------
+
+	@SuppressWarnings("unchecked")
+	public void start() {
+		checkState(!closed, "The connection is already closed");
+		checkState(!isRegistered() && pendingRegistration == null, "The connection is already started");
+
+		ResourceManagerRegistration registration = new ResourceManagerRegistration(
+				log, taskExecutor.getRpcService(),
+				resourceManagerAddress, resourceManagerLeaderId,
+				taskExecutor.getAddress(), taskExecutor.getResourceID());
+
+		Future<Tuple2<ResourceManagerGateway, TaskExecutorRegistrationSuccess>> future = registration.getFuture();
+		
+		future.onSuccess(new OnSuccess<Tuple2<ResourceManagerGateway, TaskExecutorRegistrationSuccess>>() {
+			@Override
+			public void onSuccess(Tuple2<ResourceManagerGateway, TaskExecutorRegistrationSuccess> result) {
+				registeredResourceManager = result.f0;
+				registrationId = result.f1.getRegistrationId();
+			}
+		}, taskExecutor.getMainThreadExecutionContext());
+		
+		// this future should only ever fail if there is a bug, not if the registration is declined
+		future.onFailure(new OnFailure() {
+			@Override
+			public void onFailure(Throwable failure) {
+				taskExecutor.onFatalError(failure);
+			}
+		}, taskExecutor.getMainThreadExecutionContext());
+	}
+
+	public void close() {
+		closed = true;
+
+		// make sure we do not keep re-trying forever
+		if (pendingRegistration != null) {
+			pendingRegistration.cancel();
+		}
+	}
+
+	public boolean isClosed() {
+		return closed;
+	}
+
+	// ------------------------------------------------------------------------
+	//  Properties
+	// ------------------------------------------------------------------------
+
+	public UUID getResourceManagerLeaderId() {
+		return resourceManagerLeaderId;
+	}
+
+	public String getResourceManagerAddress() {
+		return resourceManagerAddress;
+	}
+
+	/**
+	 * Gets the ResourceManagerGateway. This returns null until the registration is completed.
+	 */
+	public ResourceManagerGateway getResourceManager() {
+		return registeredResourceManager;
+	}
+
+	/**
+	 * Gets the ID under which the TaskExecutor is registered at the ResourceManager.
+	 * This returns null until the registration is completed.
+	 */
+	public InstanceID getRegistrationId() {
+		return registrationId;
+	}
+
+	public boolean isRegistered() {
+		return registeredResourceManager != null;
+	}
+
+	// ------------------------------------------------------------------------
+
+	@Override
+	public String toString() {
+		return String.format("Connection to ResourceManager %s (leaderId=%s)",
+				resourceManagerAddress, resourceManagerLeaderId); 
+	}
+
+	// ------------------------------------------------------------------------
+	//  Utilities
+	// ------------------------------------------------------------------------
+
+	static class ResourceManagerRegistration
+			extends RetryingRegistration<ResourceManagerGateway, TaskExecutorRegistrationSuccess> {
+
+		private final String taskExecutorAddress;
+		
+		private final ResourceID resourceID;
+
+		public ResourceManagerRegistration(
+				Logger log,
+				RpcService rpcService,
+				String targetAddress,
+				UUID leaderId,
+				String taskExecutorAddress,
+				ResourceID resourceID) {
+
+			super(log, rpcService, "ResourceManager", ResourceManagerGateway.class, targetAddress, leaderId);
+			this.taskExecutorAddress = checkNotNull(taskExecutorAddress);
+			this.resourceID = checkNotNull(resourceID);
+		}
+
+		@Override
+		protected Future<RegistrationResponse> invokeRegistration(
+				ResourceManagerGateway resourceManager, UUID leaderId, long timeoutMillis) throws Exception {
+
+			FiniteDuration timeout = new FiniteDuration(timeoutMillis, TimeUnit.MILLISECONDS);
+			return resourceManager.registerTaskExecutor(leaderId, taskExecutorAddress, resourceID, timeout);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/f94ae532/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java
index fd55904..7b4ab89 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java
@@ -20,15 +20,17 @@ package org.apache.flink.runtime.rpc.akka;
 
 import akka.actor.ActorSystem;
 import akka.util.Timeout;
+
+import org.apache.flink.core.testutils.OneShotLatch;
 import org.apache.flink.runtime.akka.AkkaUtils;
-import org.apache.flink.runtime.rpc.RpcEndpoint;
-import org.apache.flink.runtime.rpc.RpcGateway;
-import org.apache.flink.runtime.rpc.RpcService;
 import org.apache.flink.runtime.rpc.jobmaster.JobMaster;
 import org.apache.flink.runtime.rpc.resourcemanager.ResourceManagerGateway;
 import org.apache.flink.runtime.rpc.resourcemanager.ResourceManager;
 import org.apache.flink.util.TestLogger;
+
+import org.junit.AfterClass;
 import org.junit.Test;
+
 import scala.concurrent.duration.Deadline;
 import scala.concurrent.duration.FiniteDuration;
 
@@ -41,6 +43,49 @@ import static org.junit.Assert.assertTrue;
 
 public class AkkaRpcServiceTest extends TestLogger {
 
+	// ------------------------------------------------------------------------
+	//  shared test members
+	// ------------------------------------------------------------------------
+
+	private static ActorSystem actorSystem = AkkaUtils.createDefaultActorSystem();
+
+	private static AkkaRpcService akkaRpcService =
+			new AkkaRpcService(actorSystem, new Timeout(10000, TimeUnit.MILLISECONDS));
+
+	@AfterClass
+	public static void shutdown() {
+		akkaRpcService.stopService();
+		actorSystem.shutdown();
+	}
+
+	// ------------------------------------------------------------------------
+	//  tests
+	// ------------------------------------------------------------------------
+
+	@Test
+	public void testScheduleRunnable() throws Exception {
+		final OneShotLatch latch = new OneShotLatch();
+		final long delay = 100;
+		final long start = System.nanoTime();
+
+		akkaRpcService.scheduleRunnable(new Runnable() {
+			@Override
+			public void run() {
+				latch.trigger();
+			}
+		}, delay, TimeUnit.MILLISECONDS);
+
+		latch.await();
+		final long stop = System.nanoTime();
+
+		assertTrue("call was not properly delayed", ((stop - start) / 1000000) >= delay);
+	}
+
+	// ------------------------------------------------------------------------
+	//  specific component tests - should be moved to the test classes
+	//  for those components
+	// ------------------------------------------------------------------------
+
 	/**
 	 * Tests that the {@link JobMaster} can connect to the {@link ResourceManager} using the
 	 * {@link AkkaRpcService}.

http://git-wip-us.apache.org/repos/asf/flink/blob/f94ae532/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorTest.java
index c96f4f6..9f9bab3 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorTest.java
@@ -18,93 +18,8 @@
 
 package org.apache.flink.runtime.rpc.taskexecutor;
 
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.blob.BlobKey;
-import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor;
-import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor;
-import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor;
-import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
-import org.apache.flink.runtime.jobgraph.JobVertexID;
-import org.apache.flink.runtime.messages.Acknowledge;
-import org.apache.flink.runtime.rpc.MainThreadExecutor;
-import org.apache.flink.runtime.rpc.RpcEndpoint;
-import org.apache.flink.runtime.rpc.RpcGateway;
-import org.apache.flink.runtime.rpc.RpcService;
-import org.apache.flink.runtime.rpc.StartStoppable;
-import org.apache.flink.runtime.util.DirectExecutorService;
-import org.apache.flink.util.SerializedValue;
 import org.apache.flink.util.TestLogger;
-import org.junit.Test;
-import org.mockito.Matchers;
-import org.mockito.cglib.proxy.InvocationHandler;
-import org.mockito.cglib.proxy.Proxy;
-import scala.concurrent.Future;
-
-import java.net.URL;
-import java.util.Collections;
-
-import static org.junit.Assert.fail;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
 
 public class TaskExecutorTest extends TestLogger {
-
-	/**
-	 * Tests that we can deploy and cancel a task on the TaskExecutor without exceptions
-	 */
-	@Test
-	public void testTaskExecution() throws Exception {
-		RpcService testingRpcService = mock(RpcService.class);
-		InvocationHandler invocationHandler = mock(InvocationHandler.class);
-		Object selfGateway = Proxy.newProxyInstance(ClassLoader.getSystemClassLoader(), new Class<?>[] {TaskExecutorGateway.class, MainThreadExecutor.class, StartStoppable.class}, invocationHandler);
-		when(testingRpcService.startServer(Matchers.any(RpcEndpoint.class))).thenReturn((RpcGateway)selfGateway);
-
-		DirectExecutorService directExecutorService = new DirectExecutorService();
-		TaskExecutor taskExecutor = new TaskExecutor(testingRpcService, directExecutorService);
-		taskExecutor.start();
-
-		TaskDeploymentDescriptor tdd = new TaskDeploymentDescriptor(
-			new JobID(),
-			"Test job",
-			new JobVertexID(),
-			new ExecutionAttemptID(),
-			new SerializedValue<ExecutionConfig>(null),
-			"Test task",
-			0,
-			1,
-			0,
-			new Configuration(),
-			new Configuration(),
-			"Invokable",
-			Collections.<ResultPartitionDeploymentDescriptor>emptyList(),
-			Collections.<InputGateDeploymentDescriptor>emptyList(),
-			Collections.<BlobKey>emptyList(),
-			Collections.<URL>emptyList(),
-			0
-		);
-
-		Acknowledge ack = taskExecutor.executeTask(tdd);
-
-		ack = taskExecutor.cancelTask(tdd.getExecutionId());
-	}
-
-	/**
-	 * Tests that cancelling a non-existing task will return an exception
-	 */
-	@Test(expected=Exception.class)
-	public void testWrongTaskCancellation() throws Exception {
-		RpcService testingRpcService = mock(RpcService.class);
-		InvocationHandler invocationHandler = mock(InvocationHandler.class);
-		Object selfGateway = Proxy.newProxyInstance(ClassLoader.getSystemClassLoader(), new Class<?>[] {TaskExecutorGateway.class, MainThreadExecutor.class, StartStoppable.class}, invocationHandler);
-		when(testingRpcService.startServer(Matchers.any(RpcEndpoint.class))).thenReturn((RpcGateway)selfGateway);
-		DirectExecutorService directExecutorService = null;
-		TaskExecutor taskExecutor = new TaskExecutor(testingRpcService, directExecutorService);
-		taskExecutor.start();
-
-		taskExecutor.cancelTask(new ExecutionAttemptID());
-
-		fail("The cancellation should have thrown an exception.");
-	}
+	
 }


[28/50] [abbrv] flink git commit: [FLINK-4392] [rpc] Make RPC Service thread-safe

Posted by tr...@apache.org.
[FLINK-4392] [rpc] Make RPC Service thread-safe


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

Branch: refs/heads/flip-6
Commit: 518fb681c4c585596e0bf15e1a26d1801f72fc7b
Parents: 2bf6727
Author: Stephan Ewen <se...@apache.org>
Authored: Sat Aug 13 19:11:47 2016 +0200
Committer: Till Rohrmann <tr...@apache.org>
Committed: Thu Sep 8 17:26:57 2016 +0200

----------------------------------------------------------------------
 .../flink/runtime/rpc/akka/AkkaGateway.java     |  3 +-
 .../flink/runtime/rpc/akka/AkkaRpcService.java  | 92 +++++++++++++++-----
 2 files changed, 70 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/518fb681/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaGateway.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaGateway.java
index a826e7d..ec3091c 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaGateway.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaGateway.java
@@ -19,11 +19,12 @@
 package org.apache.flink.runtime.rpc.akka;
 
 import akka.actor.ActorRef;
+import org.apache.flink.runtime.rpc.RpcGateway;
 
 /**
  * Interface for Akka based rpc gateways
  */
-interface AkkaGateway {
+interface AkkaGateway extends RpcGateway {
 
 	ActorRef getRpcServer();
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/518fb681/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java
index 17983d0..448216c 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java
@@ -28,47 +28,61 @@ import akka.actor.Props;
 import akka.dispatch.Mapper;
 import akka.pattern.AskableActorSelection;
 import akka.util.Timeout;
+
 import org.apache.flink.runtime.akka.AkkaUtils;
 import org.apache.flink.runtime.rpc.MainThreadExecutor;
 import org.apache.flink.runtime.rpc.RpcGateway;
 import org.apache.flink.runtime.rpc.RpcEndpoint;
 import org.apache.flink.runtime.rpc.RpcService;
-import org.apache.flink.util.Preconditions;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import scala.concurrent.Future;
 
+import javax.annotation.concurrent.ThreadSafe;
 import java.lang.reflect.InvocationHandler;
 import java.lang.reflect.Proxy;
-import java.util.Collection;
 import java.util.HashSet;
+import java.util.Set;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
 
 /**
- * Akka based {@link RpcService} implementation. The rpc service starts an Akka actor to receive
- * rpcs from a {@link RpcGateway}.
+ * Akka based {@link RpcService} implementation. The RPC service starts an Akka actor to receive
+ * RPC invocations from a {@link RpcGateway}.
  */
+@ThreadSafe
 public class AkkaRpcService implements RpcService {
+
 	private static final Logger LOG = LoggerFactory.getLogger(AkkaRpcService.class);
 
+	private final Object lock = new Object();
+
 	private final ActorSystem actorSystem;
 	private final Timeout timeout;
-	private final Collection<ActorRef> actors = new HashSet<>(4);
+	private final Set<ActorRef> actors = new HashSet<>(4);
+
+	private volatile boolean stopped;
 
 	public AkkaRpcService(final ActorSystem actorSystem, final Timeout timeout) {
-		this.actorSystem = Preconditions.checkNotNull(actorSystem, "actor system");
-		this.timeout = Preconditions.checkNotNull(timeout, "timeout");
+		this.actorSystem = checkNotNull(actorSystem, "actor system");
+		this.timeout = checkNotNull(timeout, "timeout");
 	}
 
+	// this method does not mutate state and is thus thread-safe
 	@Override
 	public <C extends RpcGateway> Future<C> connect(final String address, final Class<C> clazz) {
-		LOG.info("Try to connect to remote rpc server with address {}. Returning a {} gateway.", address, clazz.getName());
+		checkState(!stopped, "RpcService is stopped");
 
-		final ActorSelection actorSel = actorSystem.actorSelection(address);
+		LOG.debug("Try to connect to remote RPC endpoint with address {}. Returning a {} gateway.",
+				address, clazz.getName());
 
+		final ActorSelection actorSel = actorSystem.actorSelection(address);
 		final AskableActorSelection asker = new AskableActorSelection(actorSel);
 
 		final Future<Object> identify = asker.ask(new Identify(42), timeout);
-
 		return identify.map(new Mapper<Object, C>(){
 			@Override
 			public C apply(Object obj) {
@@ -89,20 +103,29 @@ public class AkkaRpcService implements RpcService {
 
 	@Override
 	public <C extends RpcGateway, S extends RpcEndpoint<C>> C startServer(S rpcEndpoint) {
-		Preconditions.checkNotNull(rpcEndpoint, "rpc endpoint");
-
-		LOG.info("Start Akka rpc actor to handle rpcs for {}.", rpcEndpoint.getClass().getName());
+		checkNotNull(rpcEndpoint, "rpc endpoint");
 
 		Props akkaRpcActorProps = Props.create(AkkaRpcActor.class, rpcEndpoint);
+		ActorRef actorRef;
+
+		synchronized (lock) {
+			checkState(!stopped, "RpcService is stopped");
+			actorRef = actorSystem.actorOf(akkaRpcActorProps);
+			actors.add(actorRef);
+		}
 
-		ActorRef actorRef = actorSystem.actorOf(akkaRpcActorProps);
-		actors.add(actorRef);
+		LOG.info("Starting RPC endpoint for {} at {} .", rpcEndpoint.getClass().getName(), actorRef.path());
 
 		InvocationHandler akkaInvocationHandler = new AkkaInvocationHandler(actorRef, timeout);
 
+		// Rather than using the System ClassLoader directly, we derive the ClassLoader
+		// from this class . That works better in cases where Flink runs embedded and all Flink
+		// code is loaded dynamically (for example from an OSGI bundle) through a custom ClassLoader
+		ClassLoader classLoader = getClass().getClassLoader();
+
 		@SuppressWarnings("unchecked")
 		C self = (C) Proxy.newProxyInstance(
-			ClassLoader.getSystemClassLoader(),
+			classLoader,
 			new Class<?>[]{rpcEndpoint.getSelfGatewayType(), MainThreadExecutor.class, AkkaGateway.class},
 			akkaInvocationHandler);
 
@@ -110,35 +133,56 @@ public class AkkaRpcService implements RpcService {
 	}
 
 	@Override
-	public <C extends RpcGateway> void stopServer(C selfGateway) {
+	public void stopServer(RpcGateway selfGateway) {
 		if (selfGateway instanceof AkkaGateway) {
 			AkkaGateway akkaClient = (AkkaGateway) selfGateway;
 
-			if (actors.contains(akkaClient.getRpcServer())) {
-				ActorRef selfActorRef = akkaClient.getRpcServer();
-
-				LOG.info("Stop Akka rpc actor {}.", selfActorRef.path());
+			boolean fromThisService;
+			synchronized (lock) {
+				if (stopped) {
+					return;
+				} else {
+					fromThisService = actors.remove(akkaClient.getRpcServer());
+				}
+			}
 
+			if (fromThisService) {
+				ActorRef selfActorRef = akkaClient.getRpcServer();
+				LOG.info("Stopping RPC endpoint {}.", selfActorRef.path());
 				selfActorRef.tell(PoisonPill.getInstance(), ActorRef.noSender());
+			} else {
+				LOG.debug("RPC endpoint {} already stopped or from different RPC service");
 			}
 		}
 	}
 
 	@Override
 	public void stopService() {
-		LOG.info("Stop Akka rpc service.");
-		actorSystem.shutdown();
+		LOG.info("Stopping Akka RPC service.");
+
+		synchronized (lock) {
+			if (stopped) {
+				return;
+			}
+
+			stopped = true;
+			actorSystem.shutdown();
+			actors.clear();
+		}
+
 		actorSystem.awaitTermination();
 	}
 
 	@Override
 	public <C extends RpcGateway> String getAddress(C selfGateway) {
+		checkState(!stopped, "RpcService is stopped");
+
 		if (selfGateway instanceof AkkaGateway) {
 			ActorRef actorRef = ((AkkaGateway) selfGateway).getRpcServer();
 			return AkkaUtils.getAkkaURL(actorSystem, actorRef);
 		} else {
 			String className = AkkaGateway.class.getName();
-			throw new RuntimeException("Cannot get address for non " + className + '.');
+			throw new IllegalArgumentException("Cannot get address for non " + className + '.');
 		}
 	}
 }


[39/50] [abbrv] flink git commit: [FLINK-4347][cluster management] Implement SlotManager core

Posted by tr...@apache.org.
[FLINK-4347][cluster management] Implement SlotManager core

This closes #2388


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

Branch: refs/heads/flip-6
Commit: 108f43f7e9dec63815df1272b6d819715d76b78d
Parents: 73de842
Author: Kurt Young <yk...@gmail.com>
Authored: Thu Aug 18 15:48:30 2016 +0800
Committer: Till Rohrmann <tr...@apache.org>
Committed: Thu Sep 8 17:27:00 2016 +0200

----------------------------------------------------------------------
 .../runtime/clusterframework/SlotManager.java   | 525 ++++++++++++++++++
 .../clusterframework/types/ResourceID.java      |   4 +-
 .../clusterframework/types/ResourceProfile.java |   5 +
 .../clusterframework/types/ResourceSlot.java    |  66 +++
 .../runtime/clusterframework/types/SlotID.java  |  14 +-
 .../rpc/resourcemanager/SlotRequest.java        |  51 +-
 .../runtime/rpc/taskexecutor/SlotReport.java    |  56 ++
 .../runtime/rpc/taskexecutor/SlotStatus.java    | 129 +++++
 .../clusterframework/SlotManagerTest.java       | 540 +++++++++++++++++++
 9 files changed, 1382 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/108f43f7/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/SlotManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/SlotManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/SlotManager.java
new file mode 100644
index 0000000..cc140a1
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/SlotManager.java
@@ -0,0 +1,525 @@
+/*
+ * 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.clusterframework;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.ResourceSlot;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.rpc.resourcemanager.ResourceManagerGateway;
+import org.apache.flink.runtime.rpc.resourcemanager.SlotRequest;
+import org.apache.flink.runtime.rpc.taskexecutor.SlotReport;
+import org.apache.flink.runtime.rpc.taskexecutor.SlotStatus;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * SlotManager is responsible for receiving slot requests and do slot allocations. It allows to request
+ * slots from registered TaskManagers and issues container allocation requests in case of there are not
+ * enough available slots. Besides, it should sync its slot allocation with TaskManager's heartbeat.
+ * <p>
+ * The main operation principle of SlotManager is:
+ * <ul>
+ * <li>1. All slot allocation status should be synced with TaskManager, which is the ground truth.</li>
+ * <li>2. All slots that have registered must be tracked, either by free pool or allocated pool.</li>
+ * <li>3. All slot requests will be handled by best efforts, there is no guarantee that one request will be
+ * fulfilled in time or correctly allocated. Conflicts or timeout or some special error will happen, it should
+ * be handled outside SlotManager. SlotManager will make each decision based on the information it currently
+ * holds.</li>
+ * </ul>
+ * <b>IMPORTANT:</b> This class is <b>Not Thread-safe</b>.
+ */
+public abstract class SlotManager {
+
+	private static final Logger LOG = LoggerFactory.getLogger(SlotManager.class);
+
+	/** Gateway to communicate with ResourceManager */
+	private final ResourceManagerGateway resourceManagerGateway;
+
+	/** All registered slots, including free and allocated slots */
+	private final Map<ResourceID, Map<SlotID, ResourceSlot>> registeredSlots;
+
+	/** All pending slot requests, waiting available slots to fulfil */
+	private final Map<AllocationID, SlotRequest> pendingSlotRequests;
+
+	/** All free slots that can be used to be allocated */
+	private final Map<SlotID, ResourceSlot> freeSlots;
+
+	/** All allocations, we can lookup allocations either by SlotID or AllocationID */
+	private final AllocationMap allocationMap;
+
+	public SlotManager(ResourceManagerGateway resourceManagerGateway) {
+		this.resourceManagerGateway = checkNotNull(resourceManagerGateway);
+		this.registeredSlots = new HashMap<>(16);
+		this.pendingSlotRequests = new LinkedHashMap<>(16);
+		this.freeSlots = new HashMap<>(16);
+		this.allocationMap = new AllocationMap();
+	}
+
+	// ------------------------------------------------------------------------
+	//  slot managements
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Request a slot with requirements, we may either fulfill the request or pending it. Trigger container
+	 * allocation if we don't have enough resource. If we have free slot which can match the request, record
+	 * this allocation and forward the request to TaskManager through ResourceManager (we want this done by
+	 * RPC's main thread to avoid race condition).
+	 *
+	 * @param request The detailed request of the slot
+	 */
+	public void requestSlot(final SlotRequest request) {
+		if (isRequestDuplicated(request)) {
+			LOG.warn("Duplicated slot request, AllocationID:{}", request.getAllocationId());
+			return;
+		}
+
+		// try to fulfil the request with current free slots
+		ResourceSlot slot = chooseSlotToUse(request, freeSlots);
+		if (slot != null) {
+			LOG.info("Assigning SlotID({}) to AllocationID({}), JobID:{}", slot.getSlotId(),
+				request.getAllocationId(), request.getJobId());
+
+			// record this allocation in bookkeeping
+			allocationMap.addAllocation(slot.getSlotId(), request.getAllocationId());
+
+			// remove selected slot from free pool
+			freeSlots.remove(slot.getSlotId());
+
+			// TODO: send slot request to TaskManager
+		} else {
+			LOG.info("Cannot fulfil slot request, try to allocate a new container for it, " +
+				"AllocationID:{}, JobID:{}", request.getAllocationId(), request.getJobId());
+			allocateContainer(request.getResourceProfile());
+			pendingSlotRequests.put(request.getAllocationId(), request);
+		}
+	}
+
+	/**
+	 * Sync slot status with TaskManager's SlotReport.
+	 */
+	public void updateSlotStatus(final SlotReport slotReport) {
+		for (SlotStatus slotStatus : slotReport.getSlotsStatus()) {
+			updateSlotStatus(slotStatus);
+		}
+	}
+
+	/**
+	 * The slot request to TaskManager may be either failed by rpc communication (timeout, network error, etc.)
+	 * or really rejected by TaskManager. We shall retry this request by:
+	 * <ul>
+	 * <li>1. verify and clear all the previous allocate information for this request
+	 * <li>2. try to request slot again
+	 * </ul>
+	 * <p>
+	 * This may cause some duplicate allocation, e.g. the slot request to TaskManager is successful but the response
+	 * is lost somehow, so we may request a slot in another TaskManager, this causes two slots assigned to one request,
+	 * but it can be taken care of by rejecting registration at JobManager.
+	 *
+	 * @param originalRequest The original slot request
+	 * @param slotId          The target SlotID
+	 */
+	public void handleSlotRequestFailedAtTaskManager(final SlotRequest originalRequest, final SlotID slotId) {
+		final AllocationID originalAllocationId = originalRequest.getAllocationId();
+		LOG.info("Slot request failed at TaskManager, SlotID:{}, AllocationID:{}, JobID:{}",
+			slotId, originalAllocationId, originalRequest.getJobId());
+
+		// verify the allocation info before we do anything
+		if (freeSlots.containsKey(slotId)) {
+			// this slot is currently empty, no need to de-allocate it from our allocations
+			LOG.info("Original slot is somehow empty, retrying this request");
+
+			// before retry, we should double check whether this request was allocated by some other ways
+			if (!allocationMap.isAllocated(originalAllocationId)) {
+				requestSlot(originalRequest);
+			} else {
+				LOG.info("The failed request has somehow been allocated, SlotID:{}",
+					allocationMap.getSlotID(originalAllocationId));
+			}
+		} else if (allocationMap.isAllocated(slotId)) {
+			final AllocationID currentAllocationId = allocationMap.getAllocationID(slotId);
+
+			// check whether we have an agreement on whom this slot belongs to
+			if (originalAllocationId.equals(currentAllocationId)) {
+				LOG.info("De-allocate this request and retry");
+				allocationMap.removeAllocation(currentAllocationId);
+
+				// put this slot back to free pool
+				ResourceSlot slot = checkNotNull(getRegisteredSlot(slotId));
+				freeSlots.put(slotId, slot);
+
+				// retry the request
+				requestSlot(originalRequest);
+			} else {
+				// the slot is taken by someone else, no need to de-allocate it from our allocations
+				LOG.info("Original slot is taken by someone else, current AllocationID:{}", currentAllocationId);
+
+				// before retry, we should double check whether this request was allocated by some other ways
+				if (!allocationMap.isAllocated(originalAllocationId)) {
+					requestSlot(originalRequest);
+				} else {
+					LOG.info("The failed request is somehow been allocated, SlotID:{}",
+						allocationMap.getSlotID(originalAllocationId));
+				}
+			}
+		} else {
+			LOG.error("BUG! {} is neither in free pool nor in allocated pool", slotId);
+		}
+	}
+
+	/**
+	 * Callback for TaskManager failures. In case that a TaskManager fails, we have to clean up all its slots.
+	 *
+	 * @param resourceId The ResourceID of the TaskManager
+	 */
+	public void notifyTaskManagerFailure(final ResourceID resourceId) {
+		LOG.info("Resource:{} been notified failure", resourceId);
+		final Map<SlotID, ResourceSlot> slotIdsToRemove = registeredSlots.remove(resourceId);
+		if (slotIdsToRemove != null) {
+			for (SlotID slotId : slotIdsToRemove.keySet()) {
+				LOG.info("Removing Slot:{} upon resource failure", slotId);
+				if (freeSlots.containsKey(slotId)) {
+					freeSlots.remove(slotId);
+				} else if (allocationMap.isAllocated(slotId)) {
+					allocationMap.removeAllocation(slotId);
+				} else {
+					LOG.error("BUG! {} is neither in free pool nor in allocated pool", slotId);
+				}
+			}
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  internal behaviors
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Update slot status based on TaskManager's report. There are mainly two situations when we receive the report:
+	 * <ul>
+	 * <li>1. The slot is newly registered.</li>
+	 * <li>2. The slot has registered, it contains its current status.</li>
+	 * </ul>
+	 * <p>
+	 * Regarding 1: It's fairly simple, we just record this slot's status, and trigger schedule if slot is empty.
+	 * <p>
+	 * Regarding 2: It will cause some weird situation since we may have some time-gap on how the slot's status really
+	 * is. We may have some updates on the slot's allocation, but it doesn't reflected by TaskManager's heartbeat yet,
+	 * and we may make some wrong decision if we cannot guarantee we have the exact status about all the slots. So
+	 * the principle here is: We always trust TaskManager's heartbeat, we will correct our information based on that
+	 * and take next action based on the diff between our information and heartbeat status.
+	 *
+	 * @param reportedStatus Reported slot status
+	 */
+	void updateSlotStatus(final SlotStatus reportedStatus) {
+		final SlotID slotId = reportedStatus.getSlotID();
+		final ResourceSlot slot = new ResourceSlot(slotId, reportedStatus.getProfiler());
+
+		if (registerNewSlot(slot)) {
+			// we have a newly registered slot
+			LOG.info("New slot appeared, SlotID:{}, AllocationID:{}", slotId, reportedStatus.getAllocationID());
+
+			if (reportedStatus.getAllocationID() != null) {
+				// slot in use, record this in bookkeeping
+				allocationMap.addAllocation(slotId, reportedStatus.getAllocationID());
+			} else {
+				handleFreeSlot(new ResourceSlot(slotId, reportedStatus.getProfiler()));
+			}
+		} else {
+			// slot exists, update current information
+			if (reportedStatus.getAllocationID() != null) {
+				// slot is reported in use
+				final AllocationID reportedAllocationId = reportedStatus.getAllocationID();
+
+				// check whether we also thought this slot is in use
+				if (allocationMap.isAllocated(slotId)) {
+					// we also think that slot is in use, check whether the AllocationID matches
+					final AllocationID currentAllocationId = allocationMap.getAllocationID(slotId);
+
+					if (!reportedAllocationId.equals(currentAllocationId)) {
+						LOG.info("Slot allocation info mismatch! SlotID:{}, current:{}, reported:{}",
+							slotId, currentAllocationId, reportedAllocationId);
+
+						// seems we have a disagreement about the slot assignments, need to correct it
+						allocationMap.removeAllocation(slotId);
+						allocationMap.addAllocation(slotId, reportedAllocationId);
+					}
+				} else {
+					LOG.info("Slot allocation info mismatch! SlotID:{}, current:null, reported:{}",
+						slotId, reportedAllocationId);
+
+					// we thought the slot is free, should correct this information
+					allocationMap.addAllocation(slotId, reportedStatus.getAllocationID());
+
+					// remove this slot from free slots pool
+					freeSlots.remove(slotId);
+				}
+			} else {
+				// slot is reported empty
+
+				// check whether we also thought this slot is empty
+				if (allocationMap.isAllocated(slotId)) {
+					LOG.info("Slot allocation info mismatch! SlotID:{}, current:{}, reported:null",
+						slotId, allocationMap.getAllocationID(slotId));
+
+					// we thought the slot is in use, correct it
+					allocationMap.removeAllocation(slotId);
+
+					// we have a free slot!
+					handleFreeSlot(new ResourceSlot(slotId, reportedStatus.getProfiler()));
+				}
+			}
+		}
+	}
+
+	/**
+	 * When we have a free slot, try to fulfill the pending request first. If any request can be fulfilled,
+	 * record this allocation in bookkeeping and send slot request to TaskManager, else we just add this slot
+	 * to the free pool.
+	 *
+	 * @param freeSlot The free slot
+	 */
+	private void handleFreeSlot(final ResourceSlot freeSlot) {
+		SlotRequest chosenRequest = chooseRequestToFulfill(freeSlot, pendingSlotRequests);
+
+		if (chosenRequest != null) {
+			pendingSlotRequests.remove(chosenRequest.getAllocationId());
+
+			LOG.info("Assigning SlotID({}) to AllocationID({}), JobID:{}", freeSlot.getSlotId(),
+				chosenRequest.getAllocationId(), chosenRequest.getJobId());
+			allocationMap.addAllocation(freeSlot.getSlotId(), chosenRequest.getAllocationId());
+
+			// TODO: send slot request to TaskManager
+		} else {
+			freeSlots.put(freeSlot.getSlotId(), freeSlot);
+		}
+	}
+
+	/**
+	 * Check whether the request is duplicated. We use AllocationID to identify slot request, for each
+	 * formerly received slot request, it is either in pending list or already been allocated.
+	 *
+	 * @param request The slot request
+	 * @return <tt>true</tt> if the request is duplicated
+	 */
+	private boolean isRequestDuplicated(final SlotRequest request) {
+		final AllocationID allocationId = request.getAllocationId();
+		return pendingSlotRequests.containsKey(allocationId)
+			|| allocationMap.isAllocated(allocationId);
+	}
+
+	/**
+	 * Try to register slot, and tell if this slot is newly registered.
+	 *
+	 * @param slot The ResourceSlot which will be checked and registered
+	 * @return <tt>true</tt> if we meet a new slot
+	 */
+	private boolean registerNewSlot(final ResourceSlot slot) {
+		final SlotID slotId = slot.getSlotId();
+		final ResourceID resourceId = slotId.getResourceID();
+		if (!registeredSlots.containsKey(resourceId)) {
+			registeredSlots.put(resourceId, new HashMap<SlotID, ResourceSlot>());
+		}
+		return registeredSlots.get(resourceId).put(slotId, slot) == null;
+	}
+
+	private ResourceSlot getRegisteredSlot(final SlotID slotId) {
+		final ResourceID resourceId = slotId.getResourceID();
+		if (!registeredSlots.containsKey(resourceId)) {
+			return null;
+		}
+		return registeredSlots.get(resourceId).get(slotId);
+	}
+
+	// ------------------------------------------------------------------------
+	//  Framework specific behavior
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Choose a slot to use among all free slots, the behavior is framework specified.
+	 *
+	 * @param request   The slot request
+	 * @param freeSlots All slots which can be used
+	 * @return The slot we choose to use, <tt>null</tt> if we did not find a match
+	 */
+	protected abstract ResourceSlot chooseSlotToUse(final SlotRequest request,
+		final Map<SlotID, ResourceSlot> freeSlots);
+
+	/**
+	 * Choose a pending request to fulfill when we have a free slot, the behavior is framework specified.
+	 *
+	 * @param offeredSlot     The free slot
+	 * @param pendingRequests All the pending slot requests
+	 * @return The chosen SlotRequest, <tt>null</tt> if we did not find a match
+	 */
+	protected abstract SlotRequest chooseRequestToFulfill(final ResourceSlot offeredSlot,
+		final Map<AllocationID, SlotRequest> pendingRequests);
+
+	/**
+	 * The framework specific code for allocating a container for specified resource profile.
+	 *
+	 * @param resourceProfile The resource profile
+	 */
+	protected abstract void allocateContainer(final ResourceProfile resourceProfile);
+
+
+	// ------------------------------------------------------------------------
+	//  Helper classes
+	// ------------------------------------------------------------------------
+
+	/**
+	 * We maintain all the allocations with SlotID and AllocationID. We are able to get or remove the allocation info
+	 * either by SlotID or AllocationID.
+	 */
+	private static class AllocationMap {
+
+		/** All allocated slots (by SlotID) */
+		private final Map<SlotID, AllocationID> allocatedSlots;
+
+		/** All allocated slots (by AllocationID), it'a a inverse view of allocatedSlots */
+		private final Map<AllocationID, SlotID> allocatedSlotsByAllocationId;
+
+		AllocationMap() {
+			this.allocatedSlots = new HashMap<>(16);
+			this.allocatedSlotsByAllocationId = new HashMap<>(16);
+		}
+
+		/**
+		 * Add a allocation
+		 *
+		 * @param slotId       The slot id
+		 * @param allocationId The allocation id
+		 */
+		void addAllocation(final SlotID slotId, final AllocationID allocationId) {
+			allocatedSlots.put(slotId, allocationId);
+			allocatedSlotsByAllocationId.put(allocationId, slotId);
+		}
+
+		/**
+		 * De-allocation with slot id
+		 *
+		 * @param slotId The slot id
+		 */
+		void removeAllocation(final SlotID slotId) {
+			if (allocatedSlots.containsKey(slotId)) {
+				final AllocationID allocationId = allocatedSlots.get(slotId);
+				allocatedSlots.remove(slotId);
+				allocatedSlotsByAllocationId.remove(allocationId);
+			}
+		}
+
+		/**
+		 * De-allocation with allocation id
+		 *
+		 * @param allocationId The allocation id
+		 */
+		void removeAllocation(final AllocationID allocationId) {
+			if (allocatedSlotsByAllocationId.containsKey(allocationId)) {
+				SlotID slotId = allocatedSlotsByAllocationId.get(allocationId);
+				allocatedSlotsByAllocationId.remove(allocationId);
+				allocatedSlots.remove(slotId);
+			}
+		}
+
+		/**
+		 * Check whether allocation exists by slot id
+		 *
+		 * @param slotId The slot id
+		 * @return true if the allocation exists
+		 */
+		boolean isAllocated(final SlotID slotId) {
+			return allocatedSlots.containsKey(slotId);
+		}
+
+		/**
+		 * Check whether allocation exists by allocation id
+		 *
+		 * @param allocationId The allocation id
+		 * @return true if the allocation exists
+		 */
+		boolean isAllocated(final AllocationID allocationId) {
+			return allocatedSlotsByAllocationId.containsKey(allocationId);
+		}
+
+		AllocationID getAllocationID(final SlotID slotId) {
+			return allocatedSlots.get(slotId);
+		}
+
+		SlotID getSlotID(final AllocationID allocationId) {
+			return allocatedSlotsByAllocationId.get(allocationId);
+		}
+
+		public int size() {
+			return allocatedSlots.size();
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  Testing utilities
+	// ------------------------------------------------------------------------
+
+	@VisibleForTesting
+	boolean isAllocated(final SlotID slotId) {
+		return allocationMap.isAllocated(slotId);
+	}
+
+	@VisibleForTesting
+	boolean isAllocated(final AllocationID allocationId) {
+		return allocationMap.isAllocated(allocationId);
+	}
+
+	/**
+	 * Add free slots directly to the free pool, this will not trigger pending requests allocation
+	 *
+	 * @param slot The resource slot
+	 */
+	@VisibleForTesting
+	void addFreeSlot(final ResourceSlot slot) {
+		final ResourceID resourceId = slot.getResourceID();
+		final SlotID slotId = slot.getSlotId();
+
+		if (!registeredSlots.containsKey(resourceId)) {
+			registeredSlots.put(resourceId, new HashMap<SlotID, ResourceSlot>());
+		}
+		registeredSlots.get(resourceId).put(slot.getSlotId(), slot);
+		freeSlots.put(slotId, slot);
+	}
+
+	@VisibleForTesting
+	int getAllocatedSlotCount() {
+		return allocationMap.size();
+	}
+
+	@VisibleForTesting
+	int getFreeSlotCount() {
+		return freeSlots.size();
+	}
+
+	@VisibleForTesting
+	int getPendingRequestCount() {
+		return pendingSlotRequests.size();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/108f43f7/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceID.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceID.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceID.java
index 8cf9ccb..6b8a037 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceID.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceID.java
@@ -63,9 +63,7 @@ public final class ResourceID implements ResourceIDRetrievable, Serializable {
 
 	@Override
 	public String toString() {
-		return "ResourceID{" +
-			"resourceId='" + resourceId + '\'' +
-			'}';
+		return resourceId;
 	}
 
 	/**

http://git-wip-us.apache.org/repos/asf/flink/blob/108f43f7/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceProfile.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceProfile.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceProfile.java
index cbe709f..ff1c4bf 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceProfile.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceProfile.java
@@ -40,6 +40,11 @@ public class ResourceProfile implements Serializable {
 		this.memoryInMB = memoryInMB;
 	}
 
+	public ResourceProfile(ResourceProfile other) {
+		this.cpuCores = other.cpuCores;
+		this.memoryInMB = other.memoryInMB;
+	}
+
 	/**
 	 * Get the cpu cores needed
 	 * @return The cpu cores, 1.0 means a full cpu thread

http://git-wip-us.apache.org/repos/asf/flink/blob/108f43f7/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceSlot.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceSlot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceSlot.java
new file mode 100644
index 0000000..8a6db5f
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/ResourceSlot.java
@@ -0,0 +1,66 @@
+/*
+ * 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.clusterframework.types;
+
+import java.io.Serializable;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A ResourceSlot represents a slot located in TaskManager from ResourceManager's view. It has a unique
+ * identification and resource profile which we can compare to the resource request.
+ */
+public class ResourceSlot implements ResourceIDRetrievable, Serializable {
+
+	private static final long serialVersionUID = -5853720153136840674L;
+
+	/** The unique identification of this slot */
+	private final SlotID slotId;
+
+	/** The resource profile of this slot */
+	private final ResourceProfile resourceProfile;
+
+	public ResourceSlot(SlotID slotId, ResourceProfile resourceProfile) {
+		this.slotId = checkNotNull(slotId);
+		this.resourceProfile = checkNotNull(resourceProfile);
+	}
+
+	@Override
+	public ResourceID getResourceID() {
+		return slotId.getResourceID();
+	}
+
+	public SlotID getSlotId() {
+		return slotId;
+	}
+
+	public ResourceProfile getResourceProfile() {
+		return resourceProfile;
+	}
+
+	/**
+	 * Check whether required resource profile can be matched by this slot.
+	 *
+	 * @param required The required resource profile
+	 * @return true if requirement can be matched
+	 */
+	public boolean isMatchingRequirement(ResourceProfile required) {
+		return resourceProfile.isMatching(required);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/108f43f7/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/SlotID.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/SlotID.java b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/SlotID.java
index d1b072d..e831a5d 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/SlotID.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/types/SlotID.java
@@ -75,9 +75,15 @@ public class SlotID implements ResourceIDRetrievable, Serializable {
 
 	@Override
 	public String toString() {
-		return "SlotID{" +
-			"resourceId=" + resourceId +
-			", slotId=" + slotId +
-			'}';
+		return resourceId + "_" + slotId;
+	}
+
+	/**
+	 * Generate a random slot id.
+	 *
+	 * @return A random slot id.
+	 */
+	public static SlotID generate() {
+		return new SlotID(ResourceID.generate(), 0);
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/108f43f7/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/SlotRequest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/SlotRequest.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/SlotRequest.java
index d8fe268..74c7c39 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/SlotRequest.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/SlotRequest.java
@@ -18,8 +18,57 @@
 
 package org.apache.flink.runtime.rpc.resourcemanager;
 
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+
 import java.io.Serializable;
 
-public class SlotRequest implements Serializable{
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * This describes the requirement of the slot, mainly used by JobManager requesting slot from ResourceManager.
+ */
+public class SlotRequest implements Serializable {
+
 	private static final long serialVersionUID = -6586877187990445986L;
+
+	/** The JobID of the slot requested for */
+	private final JobID jobId;
+
+	/** The unique identification of this request */
+	private final AllocationID allocationId;
+
+	/** The resource profile of the required slot */
+	private final ResourceProfile resourceProfile;
+
+	public SlotRequest(JobID jobId, AllocationID allocationId, ResourceProfile resourceProfile) {
+		this.jobId = checkNotNull(jobId);
+		this.allocationId = checkNotNull(allocationId);
+		this.resourceProfile = checkNotNull(resourceProfile);
+	}
+
+	/**
+	 * Get the JobID of the slot requested for.
+	 * @return The job id
+	 */
+	public JobID getJobId() {
+		return jobId;
+	}
+
+	/**
+	 * Get the unique identification of this request
+	 * @return the allocation id
+	 */
+	public AllocationID getAllocationId() {
+		return allocationId;
+	}
+
+	/**
+	 * Get the resource profile of the desired slot
+	 * @return The resource profile
+	 */
+	public ResourceProfile getResourceProfile() {
+		return resourceProfile;
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/108f43f7/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/SlotReport.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/SlotReport.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/SlotReport.java
new file mode 100644
index 0000000..c372ecb
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/SlotReport.java
@@ -0,0 +1,56 @@
+/*
+ * 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.rpc.taskexecutor;
+
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+
+import java.io.Serializable;
+import java.util.List;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * A report about the current status of all slots of the TaskExecutor, describing
+ * which slots are available and allocated, and what jobs (JobManagers) the allocated slots
+ * have been allocated to.
+ */
+public class SlotReport implements Serializable {
+
+	private static final long serialVersionUID = -3150175198722481689L;
+
+	/** The slots status of the TaskManager */
+	private final List<SlotStatus> slotsStatus;
+
+	/** The resource id which identifies the TaskManager */
+	private final ResourceID resourceID;
+
+	public SlotReport(final List<SlotStatus> slotsStatus, final ResourceID resourceID) {
+		this.slotsStatus = checkNotNull(slotsStatus);
+		this.resourceID = checkNotNull(resourceID);
+	}
+
+	public List<SlotStatus> getSlotsStatus() {
+		return slotsStatus;
+	}
+
+	public ResourceID getResourceID() {
+		return resourceID;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/108f43f7/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/SlotStatus.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/SlotStatus.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/SlotStatus.java
new file mode 100644
index 0000000..e8e2084
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/SlotStatus.java
@@ -0,0 +1,129 @@
+/*
+ * 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.rpc.taskexecutor;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+
+import java.io.Serializable;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * This describes the slot current status which located in TaskManager.
+ */
+public class SlotStatus implements Serializable {
+
+	private static final long serialVersionUID = 5099191707339664493L;
+
+	/** slotID to identify a slot */
+	private final SlotID slotID;
+
+	/** the resource profile of the slot */
+	private final ResourceProfile profiler;
+
+	/** if the slot is allocated, allocationId identify its allocation; else, allocationId is null */
+	private final AllocationID allocationID;
+
+	/** if the slot is allocated, jobId identify which job this slot is allocated to; else, jobId is null */
+	private final JobID jobID;
+
+	public SlotStatus(SlotID slotID, ResourceProfile profiler) {
+		this(slotID, profiler, null, null);
+	}
+
+	public SlotStatus(SlotID slotID, ResourceProfile profiler, AllocationID allocationID, JobID jobID) {
+		this.slotID = checkNotNull(slotID, "slotID cannot be null");
+		this.profiler = checkNotNull(profiler, "profile cannot be null");
+		this.allocationID = allocationID;
+		this.jobID = jobID;
+	}
+
+	/**
+	 * Get the unique identification of this slot
+	 *
+	 * @return The slot id
+	 */
+	public SlotID getSlotID() {
+		return slotID;
+	}
+
+	/**
+	 * Get the resource profile of this slot
+	 *
+	 * @return The resource profile
+	 */
+	public ResourceProfile getProfiler() {
+		return profiler;
+	}
+
+	/**
+	 * Get the allocation id of this slot
+	 *
+	 * @return The allocation id if this slot is allocated, otherwise null
+	 */
+	public AllocationID getAllocationID() {
+		return allocationID;
+	}
+
+	/**
+	 * Get the job id of the slot allocated for
+	 *
+	 * @return The job id if this slot is allocated, otherwise null
+	 */
+	public JobID getJobID() {
+		return jobID;
+	}
+
+	@Override
+	public boolean equals(Object o) {
+		if (this == o) {
+			return true;
+		}
+		if (o == null || getClass() != o.getClass()) {
+			return false;
+		}
+
+		SlotStatus that = (SlotStatus) o;
+
+		if (!slotID.equals(that.slotID)) {
+			return false;
+		}
+		if (!profiler.equals(that.profiler)) {
+			return false;
+		}
+		if (allocationID != null ? !allocationID.equals(that.allocationID) : that.allocationID != null) {
+			return false;
+		}
+		return jobID != null ? jobID.equals(that.jobID) : that.jobID == null;
+
+	}
+
+	@Override
+	public int hashCode() {
+		int result = slotID.hashCode();
+		result = 31 * result + profiler.hashCode();
+		result = 31 * result + (allocationID != null ? allocationID.hashCode() : 0);
+		result = 31 * result + (jobID != null ? jobID.hashCode() : 0);
+		return result;
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/108f43f7/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/SlotManagerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/SlotManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/SlotManagerTest.java
new file mode 100644
index 0000000..2ee280f
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/SlotManagerTest.java
@@ -0,0 +1,540 @@
+/*
+ * 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.clusterframework;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.ResourceSlot;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.rpc.resourcemanager.ResourceManagerGateway;
+import org.apache.flink.runtime.rpc.resourcemanager.SlotRequest;
+import org.apache.flink.runtime.rpc.taskexecutor.SlotStatus;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+
+public class SlotManagerTest {
+
+	private static final double DEFAULT_TESTING_CPU_CORES = 1.0;
+
+	private static final long DEFAULT_TESTING_MEMORY = 512;
+
+	private static final ResourceProfile DEFAULT_TESTING_PROFILE =
+		new ResourceProfile(DEFAULT_TESTING_CPU_CORES, DEFAULT_TESTING_MEMORY);
+
+	private static final ResourceProfile DEFAULT_TESTING_BIG_PROFILE =
+		new ResourceProfile(DEFAULT_TESTING_CPU_CORES * 2, DEFAULT_TESTING_MEMORY * 2);
+
+	private ResourceManagerGateway resourceManagerGateway;
+
+	@Before
+	public void setUp() {
+		resourceManagerGateway = mock(ResourceManagerGateway.class);
+	}
+
+	/**
+	 * Tests that there are no free slots when we request, need to allocate from cluster manager master
+	 */
+	@Test
+	public void testRequestSlotWithoutFreeSlot() {
+		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
+		slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE));
+
+		assertEquals(0, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertEquals(1, slotManager.getPendingRequestCount());
+		assertEquals(1, slotManager.getAllocatedContainers().size());
+		assertEquals(DEFAULT_TESTING_PROFILE, slotManager.getAllocatedContainers().get(0));
+	}
+
+	/**
+	 * Tests that there are some free slots when we request, and the request is fulfilled immediately
+	 */
+	@Test
+	public void testRequestSlotWithFreeSlot() {
+		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
+
+		directlyProvideFreeSlots(slotManager, DEFAULT_TESTING_PROFILE, 1);
+		assertEquals(1, slotManager.getFreeSlotCount());
+
+		slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE));
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertEquals(0, slotManager.getPendingRequestCount());
+		assertEquals(0, slotManager.getAllocatedContainers().size());
+	}
+
+	/**
+	 * Tests that there are some free slots when we request, but none of them are suitable
+	 */
+	@Test
+	public void testRequestSlotWithoutSuitableSlot() {
+		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
+
+		directlyProvideFreeSlots(slotManager, DEFAULT_TESTING_PROFILE, 2);
+		assertEquals(2, slotManager.getFreeSlotCount());
+
+		slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_BIG_PROFILE));
+		assertEquals(0, slotManager.getAllocatedSlotCount());
+		assertEquals(2, slotManager.getFreeSlotCount());
+		assertEquals(1, slotManager.getPendingRequestCount());
+		assertEquals(1, slotManager.getAllocatedContainers().size());
+		assertEquals(DEFAULT_TESTING_BIG_PROFILE, slotManager.getAllocatedContainers().get(0));
+	}
+
+	/**
+	 * Tests that we send duplicated slot request
+	 */
+	@Test
+	public void testDuplicatedSlotRequest() {
+		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
+		directlyProvideFreeSlots(slotManager, DEFAULT_TESTING_PROFILE, 1);
+
+		SlotRequest request1 = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE);
+		SlotRequest request2 = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_BIG_PROFILE);
+
+		slotManager.requestSlot(request1);
+		slotManager.requestSlot(request2);
+		slotManager.requestSlot(request2);
+		slotManager.requestSlot(request1);
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertEquals(1, slotManager.getPendingRequestCount());
+		assertEquals(1, slotManager.getAllocatedContainers().size());
+		assertEquals(DEFAULT_TESTING_BIG_PROFILE, slotManager.getAllocatedContainers().get(0));
+	}
+
+	/**
+	 * Tests that we send multiple slot requests
+	 */
+	@Test
+	public void testRequestMultipleSlots() {
+		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
+		directlyProvideFreeSlots(slotManager, DEFAULT_TESTING_PROFILE, 5);
+
+		// request 3 normal slots
+		for (int i = 0; i < 3; ++i) {
+			slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE));
+		}
+
+		// request 2 big slots
+		for (int i = 0; i < 2; ++i) {
+			slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_BIG_PROFILE));
+		}
+
+		// request 1 normal slot again
+		slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE));
+
+		assertEquals(4, slotManager.getAllocatedSlotCount());
+		assertEquals(1, slotManager.getFreeSlotCount());
+		assertEquals(2, slotManager.getPendingRequestCount());
+		assertEquals(2, slotManager.getAllocatedContainers().size());
+		assertEquals(DEFAULT_TESTING_BIG_PROFILE, slotManager.getAllocatedContainers().get(0));
+		assertEquals(DEFAULT_TESTING_BIG_PROFILE, slotManager.getAllocatedContainers().get(1));
+	}
+
+	/**
+	 * Tests that a new slot appeared in SlotReport, and we used it to fulfill a pending request
+	 */
+	@Test
+	public void testNewlyAppearedFreeSlotFulfillPendingRequest() {
+		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
+		slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE));
+		assertEquals(1, slotManager.getPendingRequestCount());
+
+		SlotID slotId = SlotID.generate();
+		SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE);
+		slotManager.updateSlotStatus(slotStatus);
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertEquals(0, slotManager.getPendingRequestCount());
+		assertTrue(slotManager.isAllocated(slotId));
+	}
+
+	/**
+	 * Tests that a new slot appeared in SlotReport, but we have no pending request
+	 */
+	@Test
+	public void testNewlyAppearedFreeSlot() {
+		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
+		SlotID slotId = SlotID.generate();
+		SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE);
+		slotManager.updateSlotStatus(slotStatus);
+
+		assertEquals(0, slotManager.getAllocatedSlotCount());
+		assertEquals(1, slotManager.getFreeSlotCount());
+	}
+
+	/**
+	 * Tests that a new slot appeared in SlotReport, but it't not suitable for all the pending requests
+	 */
+	@Test
+	public void testNewlyAppearedFreeSlotNotMatchPendingRequests() {
+		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
+		slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_BIG_PROFILE));
+		assertEquals(1, slotManager.getPendingRequestCount());
+
+		SlotID slotId = SlotID.generate();
+		SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE);
+		slotManager.updateSlotStatus(slotStatus);
+
+		assertEquals(0, slotManager.getAllocatedSlotCount());
+		assertEquals(1, slotManager.getFreeSlotCount());
+		assertEquals(1, slotManager.getPendingRequestCount());
+		assertFalse(slotManager.isAllocated(slotId));
+	}
+
+	/**
+	 * Tests that a new slot appeared in SlotReport, and it's been reported using by some job
+	 */
+	@Test
+	public void testNewlyAppearedInUseSlot() {
+		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
+
+		SlotID slotId = SlotID.generate();
+		SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE, new AllocationID(), new JobID());
+		slotManager.updateSlotStatus(slotStatus);
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertTrue(slotManager.isAllocated(slotId));
+	}
+
+	/**
+	 * Tests that we had a slot in-use, and it's confirmed by SlotReport
+	 */
+	@Test
+	public void testExistingInUseSlotUpdateStatus() {
+		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
+		SlotRequest request = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE);
+		slotManager.requestSlot(request);
+
+		// make this slot in use
+		SlotID slotId = SlotID.generate();
+		SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE);
+		slotManager.updateSlotStatus(slotStatus);
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertTrue(slotManager.isAllocated(slotId));
+
+		// slot status is confirmed
+		SlotStatus slotStatus2 = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE,
+			request.getAllocationId(), request.getJobId());
+		slotManager.updateSlotStatus(slotStatus2);
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertTrue(slotManager.isAllocated(slotId));
+	}
+
+	/**
+	 * Tests that we had a slot in-use, but it's empty according to the SlotReport
+	 */
+	@Test
+	public void testExistingInUseSlotAdjustedToEmpty() {
+		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
+		SlotRequest request1 = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE);
+		slotManager.requestSlot(request1);
+
+		// make this slot in use
+		SlotID slotId = SlotID.generate();
+		SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE);
+		slotManager.updateSlotStatus(slotStatus);
+
+		// another request pending
+		SlotRequest request2 = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE);
+		slotManager.requestSlot(request2);
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertEquals(1, slotManager.getPendingRequestCount());
+		assertTrue(slotManager.isAllocated(slotId));
+		assertTrue(slotManager.isAllocated(request1.getAllocationId()));
+
+
+		// but slot is reported empty again, request2 will be fulfilled, request1 will be missing
+		slotManager.updateSlotStatus(slotStatus);
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertEquals(0, slotManager.getPendingRequestCount());
+		assertTrue(slotManager.isAllocated(slotId));
+		assertTrue(slotManager.isAllocated(request2.getAllocationId()));
+	}
+
+	/**
+	 * Tests that we had a slot in use, and it's also reported in use by TaskManager, but the allocation
+	 * information didn't match.
+	 */
+	@Test
+	public void testExistingInUseSlotWithDifferentAllocationInfo() {
+		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
+		SlotRequest request = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE);
+		slotManager.requestSlot(request);
+
+		// make this slot in use
+		SlotID slotId = SlotID.generate();
+		SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE);
+		slotManager.updateSlotStatus(slotStatus);
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertEquals(0, slotManager.getPendingRequestCount());
+		assertTrue(slotManager.isAllocated(slotId));
+		assertTrue(slotManager.isAllocated(request.getAllocationId()));
+
+		SlotStatus slotStatus2 = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE, new AllocationID(), new JobID());
+		// update slot status with different allocation info
+		slotManager.updateSlotStatus(slotStatus2);
+
+		// original request is missing and won't be allocated
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertEquals(0, slotManager.getPendingRequestCount());
+		assertTrue(slotManager.isAllocated(slotId));
+		assertFalse(slotManager.isAllocated(request.getAllocationId()));
+		assertTrue(slotManager.isAllocated(slotStatus2.getAllocationID()));
+	}
+
+	/**
+	 * Tests that we had a free slot, and it's confirmed by SlotReport
+	 */
+	@Test
+	public void testExistingEmptySlotUpdateStatus() {
+		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
+		ResourceSlot slot = new ResourceSlot(SlotID.generate(), DEFAULT_TESTING_PROFILE);
+		slotManager.addFreeSlot(slot);
+
+		SlotStatus slotStatus = new SlotStatus(slot.getSlotId(), DEFAULT_TESTING_PROFILE);
+		slotManager.updateSlotStatus(slotStatus);
+
+		assertEquals(0, slotManager.getAllocatedSlotCount());
+		assertEquals(1, slotManager.getFreeSlotCount());
+		assertEquals(0, slotManager.getPendingRequestCount());
+	}
+
+	/**
+	 * Tests that we had a free slot, and it's reported in-use by TaskManager
+	 */
+	@Test
+	public void testExistingEmptySlotAdjustedToInUse() {
+		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
+		ResourceSlot slot = new ResourceSlot(SlotID.generate(), DEFAULT_TESTING_PROFILE);
+		slotManager.addFreeSlot(slot);
+
+		SlotStatus slotStatus = new SlotStatus(slot.getSlotId(), DEFAULT_TESTING_PROFILE,
+			new AllocationID(), new JobID());
+		slotManager.updateSlotStatus(slotStatus);
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertEquals(0, slotManager.getPendingRequestCount());
+		assertTrue(slotManager.isAllocated(slot.getSlotId()));
+	}
+
+	/**
+	 * Tests that we did some allocation but failed / rejected by TaskManager, request will retry
+	 */
+	@Test
+	public void testSlotAllocationFailedAtTaskManager() {
+		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
+		ResourceSlot slot = new ResourceSlot(SlotID.generate(), DEFAULT_TESTING_PROFILE);
+		slotManager.addFreeSlot(slot);
+
+		SlotRequest request = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE);
+		slotManager.requestSlot(request);
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertEquals(0, slotManager.getPendingRequestCount());
+		assertTrue(slotManager.isAllocated(slot.getSlotId()));
+
+		slotManager.handleSlotRequestFailedAtTaskManager(request, slot.getSlotId());
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertEquals(0, slotManager.getPendingRequestCount());
+	}
+
+
+	/**
+	 * Tests that we did some allocation but failed / rejected by TaskManager, and slot is occupied by another request
+	 */
+	@Test
+	public void testSlotAllocationFailedAtTaskManagerOccupiedByOther() {
+		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
+		ResourceSlot slot = new ResourceSlot(SlotID.generate(), DEFAULT_TESTING_PROFILE);
+		slotManager.addFreeSlot(slot);
+
+		SlotRequest request = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE);
+		slotManager.requestSlot(request);
+
+		// slot is set empty by heartbeat
+		SlotStatus slotStatus = new SlotStatus(slot.getSlotId(), slot.getResourceProfile());
+		slotManager.updateSlotStatus(slotStatus);
+
+		// another request took this slot
+		SlotRequest request2 = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE);
+		slotManager.requestSlot(request2);
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertEquals(0, slotManager.getPendingRequestCount());
+		assertFalse(slotManager.isAllocated(request.getAllocationId()));
+		assertTrue(slotManager.isAllocated(request2.getAllocationId()));
+
+		// original request should be pended
+		slotManager.handleSlotRequestFailedAtTaskManager(request, slot.getSlotId());
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertEquals(1, slotManager.getPendingRequestCount());
+		assertFalse(slotManager.isAllocated(request.getAllocationId()));
+		assertTrue(slotManager.isAllocated(request2.getAllocationId()));
+	}
+
+	@Test
+	public void testNotifyTaskManagerFailure() {
+		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
+
+		ResourceID resource1 = ResourceID.generate();
+		ResourceID resource2 = ResourceID.generate();
+
+		ResourceSlot slot11 = new ResourceSlot(new SlotID(resource1, 1), DEFAULT_TESTING_PROFILE);
+		ResourceSlot slot12 = new ResourceSlot(new SlotID(resource1, 2), DEFAULT_TESTING_PROFILE);
+		ResourceSlot slot21 = new ResourceSlot(new SlotID(resource2, 1), DEFAULT_TESTING_PROFILE);
+		ResourceSlot slot22 = new ResourceSlot(new SlotID(resource2, 2), DEFAULT_TESTING_PROFILE);
+
+		slotManager.addFreeSlot(slot11);
+		slotManager.addFreeSlot(slot21);
+
+		slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE));
+		slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE));
+
+		assertEquals(2, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertEquals(0, slotManager.getPendingRequestCount());
+
+		slotManager.addFreeSlot(slot12);
+		slotManager.addFreeSlot(slot22);
+
+		assertEquals(2, slotManager.getAllocatedSlotCount());
+		assertEquals(2, slotManager.getFreeSlotCount());
+		assertEquals(0, slotManager.getPendingRequestCount());
+
+		slotManager.notifyTaskManagerFailure(resource2);
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(1, slotManager.getFreeSlotCount());
+		assertEquals(0, slotManager.getPendingRequestCount());
+
+		// notify an not exist resource failure
+		slotManager.notifyTaskManagerFailure(ResourceID.generate());
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(1, slotManager.getFreeSlotCount());
+		assertEquals(0, slotManager.getPendingRequestCount());
+	}
+
+	// ------------------------------------------------------------------------
+	//  testing utilities
+	// ------------------------------------------------------------------------
+
+	private void directlyProvideFreeSlots(
+		final SlotManager slotManager,
+		final ResourceProfile resourceProfile,
+		final int freeSlotNum)
+	{
+		for (int i = 0; i < freeSlotNum; ++i) {
+			slotManager.addFreeSlot(new ResourceSlot(SlotID.generate(), new ResourceProfile(resourceProfile)));
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  testing classes
+	// ------------------------------------------------------------------------
+
+	private static class TestingSlotManager extends SlotManager {
+
+		private final List<ResourceProfile> allocatedContainers;
+
+		TestingSlotManager(ResourceManagerGateway resourceManagerGateway) {
+			super(resourceManagerGateway);
+			this.allocatedContainers = new LinkedList<>();
+		}
+
+		/**
+		 * Choose slot randomly if it matches requirement
+		 *
+		 * @param request   The slot request
+		 * @param freeSlots All slots which can be used
+		 * @return The chosen slot or null if cannot find a match
+		 */
+		@Override
+		protected ResourceSlot chooseSlotToUse(SlotRequest request, Map<SlotID, ResourceSlot> freeSlots) {
+			for (ResourceSlot slot : freeSlots.values()) {
+				if (slot.isMatchingRequirement(request.getResourceProfile())) {
+					return slot;
+				}
+			}
+			return null;
+		}
+
+		/**
+		 * Choose request randomly if offered slot can match its requirement
+		 *
+		 * @param offeredSlot     The free slot
+		 * @param pendingRequests All the pending slot requests
+		 * @return The chosen request's AllocationID or null if cannot find a match
+		 */
+		@Override
+		protected SlotRequest chooseRequestToFulfill(ResourceSlot offeredSlot,
+			Map<AllocationID, SlotRequest> pendingRequests)
+		{
+			for (Map.Entry<AllocationID, SlotRequest> pendingRequest : pendingRequests.entrySet()) {
+				if (offeredSlot.isMatchingRequirement(pendingRequest.getValue().getResourceProfile())) {
+					return pendingRequest.getValue();
+				}
+			}
+			return null;
+		}
+
+		@Override
+		protected void allocateContainer(ResourceProfile resourceProfile) {
+			allocatedContainers.add(resourceProfile);
+		}
+
+		List<ResourceProfile> getAllocatedContainers() {
+			return allocatedContainers;
+		}
+	}
+}


[34/50] [abbrv] flink git commit: [hotfix] Remove RecoveryMode from JobMaster

Posted by tr...@apache.org.
[hotfix] Remove RecoveryMode from JobMaster

The recovery mode is not used any more by the latest CheckpointCoordinator.

All difference in recovery logic between high-availability and non-high-availability
is encapsulated in the HighAvailabilityServices.


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

Branch: refs/heads/flip-6
Commit: 282566da305144ae34207c0238b3accd89a58163
Parents: 0482489
Author: Stephan Ewen <se...@apache.org>
Authored: Thu Aug 25 20:37:15 2016 +0200
Committer: Till Rohrmann <tr...@apache.org>
Committed: Thu Sep 8 17:26:59 2016 +0200

----------------------------------------------------------------------
 .../java/org/apache/flink/runtime/rpc/jobmaster/JobMaster.java    | 3 ---
 1 file changed, 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/282566da/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/jobmaster/JobMaster.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/jobmaster/JobMaster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/jobmaster/JobMaster.java
index 49b200b..a046cb8 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/jobmaster/JobMaster.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/jobmaster/JobMaster.java
@@ -22,7 +22,6 @@ import org.apache.flink.api.common.JobID;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
 import org.apache.flink.runtime.jobgraph.JobGraph;
-import org.apache.flink.runtime.jobmanager.RecoveryMode;
 import org.apache.flink.runtime.leaderelection.LeaderContender;
 import org.apache.flink.runtime.leaderelection.LeaderElectionService;
 import org.apache.flink.runtime.messages.Acknowledge;
@@ -57,7 +56,6 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 
 	/** Configuration of the job */
 	private final Configuration configuration;
-	private final RecoveryMode recoveryMode;
 
 	/** Service to contend for and retrieve the leadership of JM and RM */
 	private final HighAvailabilityServices highAvailabilityServices;
@@ -86,7 +84,6 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 		this.jobID = Preconditions.checkNotNull(jobGraph.getJobID());
 
 		this.configuration = Preconditions.checkNotNull(configuration);
-		this.recoveryMode = RecoveryMode.fromConfig(configuration);
 
 		this.highAvailabilityServices = Preconditions.checkNotNull(highAvailabilityService);
 	}


[40/50] [abbrv] flink git commit: [FLINK-4516] leader election of resourcemanager

Posted by tr...@apache.org.
[FLINK-4516] leader election of resourcemanager

- add serial rpc service
- add a special rpcService implementation which directly executes the asynchronous calls serially one by one, it is just for testcase
- Change ResourceManagerLeaderContender code and TestingSerialRpcService code
- override shutdown logic to stop leadershipService
- use a mocked RpcService rather than TestingSerialRpcService for resourceManager HA test

This closes #2427


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

Branch: refs/heads/flip-6
Commit: 26082bd33937c3e1a503c9f3150b0db67567c9db
Parents: 108f43f
Author: beyond1920 <be...@126.com>
Authored: Sat Aug 27 14:14:28 2016 +0800
Committer: Till Rohrmann <tr...@apache.org>
Committed: Thu Sep 8 17:27:00 2016 +0200

----------------------------------------------------------------------
 .../HighAvailabilityServices.java               |   7 +
 .../runtime/highavailability/NonHaServices.java |   5 +
 .../rpc/resourcemanager/ResourceManager.java    | 111 +++++-
 .../TestingHighAvailabilityServices.java        |  19 +-
 .../runtime/rpc/TestingSerialRpcService.java    | 369 +++++++++++++++++++
 .../resourcemanager/ResourceManagerHATest.java  |  76 ++++
 6 files changed, 578 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/26082bd3/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServices.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServices.java
index 73e4f1f..298147c 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServices.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServices.java
@@ -40,6 +40,13 @@ public interface HighAvailabilityServices {
 	LeaderRetrievalService getResourceManagerLeaderRetriever() throws Exception;
 
 	/**
+	 * Gets the leader election service for the cluster's resource manager.
+	 * @return
+	 * @throws Exception
+	 */
+	LeaderElectionService getResourceManagerLeaderElectionService() throws Exception;
+
+	/**
 	 * Gets the leader election service for the given job.
 	 *
 	 * @param jobID The identifier of the job running the election.

http://git-wip-us.apache.org/repos/asf/flink/blob/26082bd3/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/NonHaServices.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/NonHaServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/NonHaServices.java
index 3d2769b..292a404 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/NonHaServices.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/NonHaServices.java
@@ -61,6 +61,11 @@ public class NonHaServices implements HighAvailabilityServices {
 	}
 
 	@Override
+	public LeaderElectionService getResourceManagerLeaderElectionService() throws Exception {
+		return new StandaloneLeaderElectionService();
+	}
+
+	@Override
 	public LeaderElectionService getJobMasterLeaderElectionService(JobID jobID) throws Exception {
 		return new StandaloneLeaderElectionService();
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/26082bd3/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManager.java
index 6f34465..f7147c9 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManager.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManager.java
@@ -20,24 +20,26 @@ package org.apache.flink.runtime.rpc.resourcemanager;
 
 import akka.dispatch.Mapper;
 
+import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
 import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.leaderelection.LeaderContender;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
 import org.apache.flink.runtime.rpc.RpcMethod;
 import org.apache.flink.runtime.rpc.RpcEndpoint;
 import org.apache.flink.runtime.rpc.RpcService;
 import org.apache.flink.runtime.rpc.jobmaster.JobMaster;
 import org.apache.flink.runtime.rpc.jobmaster.JobMasterGateway;
 import org.apache.flink.runtime.rpc.taskexecutor.TaskExecutorRegistrationSuccess;
-import org.apache.flink.util.Preconditions;
 
-import scala.concurrent.ExecutionContext;
-import scala.concurrent.ExecutionContext$;
 import scala.concurrent.Future;
 
 import java.util.HashMap;
 import java.util.Map;
 import java.util.UUID;
-import java.util.concurrent.ExecutorService;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**
  * ResourceManager implementation. The resource manager is responsible for resource de-/allocation
@@ -50,16 +52,51 @@ import java.util.concurrent.ExecutorService;
  * </ul>
  */
 public class ResourceManager extends RpcEndpoint<ResourceManagerGateway> {
-	private final ExecutionContext executionContext;
 	private final Map<JobMasterGateway, InstanceID> jobMasterGateways;
+	private final HighAvailabilityServices highAvailabilityServices;
+	private LeaderElectionService leaderElectionService = null;
+	private UUID leaderSessionID = null;
 
-	public ResourceManager(RpcService rpcService, ExecutorService executorService) {
+	public ResourceManager(RpcService rpcService, HighAvailabilityServices highAvailabilityServices) {
 		super(rpcService);
-		this.executionContext = ExecutionContext$.MODULE$.fromExecutor(
-			Preconditions.checkNotNull(executorService));
+		this.highAvailabilityServices = checkNotNull(highAvailabilityServices);
 		this.jobMasterGateways = new HashMap<>();
 	}
 
+	@Override
+	public void start() {
+		// start a leader
+		try {
+			super.start();
+			leaderElectionService = highAvailabilityServices.getResourceManagerLeaderElectionService();
+			leaderElectionService.start(new ResourceManagerLeaderContender());
+		} catch (Throwable e) {
+			log.error("A fatal error happened when starting the ResourceManager", e);
+			throw new RuntimeException("A fatal error happened when starting the ResourceManager", e);
+		}
+	}
+
+	@Override
+	public void shutDown() {
+		try {
+			leaderElectionService.stop();
+			super.shutDown();
+		} catch(Throwable e) {
+			log.error("A fatal error happened when shutdown the ResourceManager", e);
+			throw new RuntimeException("A fatal error happened when shutdown the ResourceManager", e);
+		}
+	}
+
+	/**
+	 * Gets the leader session id of current resourceManager.
+	 *
+	 * @return return the leaderSessionId of current resourceManager, this returns null until the current resourceManager is granted leadership.
+	 */
+	@VisibleForTesting
+	UUID getLeaderSessionID() {
+		return leaderSessionID;
+	}
+
 	/**
 	 * Register a {@link JobMaster} at the resource manager.
 	 *
@@ -116,4 +153,62 @@ public class ResourceManager extends RpcEndpoint<ResourceManagerGateway> {
 
 		return new TaskExecutorRegistrationSuccess(new InstanceID(), 5000);
 	}
+
+	private class ResourceManagerLeaderContender implements LeaderContender {
+
+		/**
+		 * Callback method when current resourceManager is granted leadership
+		 *
+		 * @param leaderSessionID unique leadershipID
+		 */
+		@Override
+		public void grantLeadership(final UUID leaderSessionID) {
+			runAsync(new Runnable() {
+				@Override
+				public void run() {
+					log.info("ResourceManager {} was granted leadership with leader session ID {}", getAddress(), leaderSessionID);
+					ResourceManager.this.leaderSessionID = leaderSessionID;
+					// confirming the leader session ID might be blocking,
+					leaderElectionService.confirmLeaderSessionID(leaderSessionID);
+				}
+			});
+		}
+
+		/**
+		 * Callback method when current resourceManager lose leadership.
+		 */
+		@Override
+		public void revokeLeadership() {
+			runAsync(new Runnable() {
+				@Override
+				public void run() {
+					log.info("ResourceManager {} was revoked leadership.", getAddress());
+					jobMasterGateways.clear();
+					leaderSessionID = null;
+				}
+			});
+		}
+
+		@Override
+		public String getAddress() {
+			return ResourceManager.this.getAddress();
+		}
+
+		/**
+		 * Handles error occurring in the leader election service
+		 *
+		 * @param exception Exception being thrown in the leader election service
+		 */
+		@Override
+		public void handleError(final Exception exception) {
+			runAsync(new Runnable() {
+				@Override
+				public void run() {
+					log.error("ResourceManager received an error from the LeaderElectionService.", exception);
+					// terminate ResourceManager in case of an error
+					shutDown();
+				}
+			});
+		}
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/26082bd3/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingHighAvailabilityServices.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingHighAvailabilityServices.java b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingHighAvailabilityServices.java
index 4d654a3..3162f40 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingHighAvailabilityServices.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingHighAvailabilityServices.java
@@ -32,6 +32,8 @@ public class TestingHighAvailabilityServices implements HighAvailabilityServices
 
 	private volatile LeaderElectionService jobMasterLeaderElectionService;
 
+	private volatile LeaderElectionService resourceManagerLeaderElectionService;
+
 
 	// ------------------------------------------------------------------------
 	//  Setters for mock / testing implementations
@@ -44,7 +46,11 @@ public class TestingHighAvailabilityServices implements HighAvailabilityServices
 	public void setJobMasterLeaderElectionService(LeaderElectionService leaderElectionService) {
 		this.jobMasterLeaderElectionService = leaderElectionService;
 	}
-	
+
+	public void setResourceManagerLeaderElectionService(LeaderElectionService leaderElectionService) {
+		this.resourceManagerLeaderElectionService = leaderElectionService;
+	}
+
 	// ------------------------------------------------------------------------
 	//  HA Services Methods
 	// ------------------------------------------------------------------------
@@ -69,4 +75,15 @@ public class TestingHighAvailabilityServices implements HighAvailabilityServices
 			throw new IllegalStateException("JobMasterLeaderElectionService has not been set");
 		}
 	}
+
+	@Override
+	public LeaderElectionService getResourceManagerLeaderElectionService() throws Exception {
+		LeaderElectionService service = resourceManagerLeaderElectionService;
+
+		if (service != null) {
+			return service;
+		} else {
+			throw new IllegalStateException("ResourceManagerLeaderElectionService has not been set");
+		}
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/26082bd3/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingSerialRpcService.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingSerialRpcService.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingSerialRpcService.java
new file mode 100644
index 0000000..7bdbb99
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingSerialRpcService.java
@@ -0,0 +1,369 @@
+/*
+ * 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.rpc;
+
+import akka.dispatch.ExecutionContexts;
+import akka.dispatch.Futures;
+import akka.util.Timeout;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.runtime.util.DirectExecutorService;
+import org.apache.flink.util.Preconditions;
+import scala.concurrent.Await;
+import scala.concurrent.ExecutionContext;
+import scala.concurrent.Future;
+import scala.concurrent.duration.FiniteDuration;
+
+import java.lang.annotation.Annotation;
+import java.lang.reflect.InvocationHandler;
+import java.lang.reflect.Method;
+import java.lang.reflect.Proxy;
+import java.util.BitSet;
+import java.util.UUID;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+
+/**
+ * An RPC Service implementation for testing. This RPC service directly executes all asynchronous calls one by one in the main thread.
+ */
+public class TestingSerialRpcService implements RpcService {
+
+	private final DirectExecutorService executorService;
+	private final ConcurrentHashMap<String, RpcGateway> registeredConnections;
+
+	public TestingSerialRpcService() {
+		executorService = new DirectExecutorService();
+		this.registeredConnections = new ConcurrentHashMap<>();
+	}
+
+	@Override
+	public void scheduleRunnable(final Runnable runnable, final long delay, final TimeUnit unit) {
+		try {
+			unit.sleep(delay);
+			runnable.run();
+		} catch (Throwable e) {
+			throw new RuntimeException(e);
+		}
+	}
+
+	@Override
+	public ExecutionContext getExecutionContext() {
+		return ExecutionContexts.fromExecutorService(executorService);
+	}
+
+	@Override
+	public void stopService() {
+		executorService.shutdown();
+		registeredConnections.clear();
+	}
+
+	@Override
+	public void stopServer(RpcGateway selfGateway) {
+
+	}
+
+	@Override
+	public <C extends RpcGateway, S extends RpcEndpoint<C>> C startServer(S rpcEndpoint) {
+		final String address = UUID.randomUUID().toString();
+
+		InvocationHandler akkaInvocationHandler = new TestingSerialInvocationHandler(address, rpcEndpoint);
+		ClassLoader classLoader = getClass().getClassLoader();
+
+		@SuppressWarnings("unchecked")
+		C self = (C) Proxy.newProxyInstance(
+			classLoader,
+			new Class<?>[]{
+				rpcEndpoint.getSelfGatewayType(),
+				MainThreadExecutor.class,
+				StartStoppable.class,
+				RpcGateway.class
+			},
+			akkaInvocationHandler);
+
+		return self;
+	}
+
+	@Override
+	public <C extends RpcGateway> Future<C> connect(String address, Class<C> clazz) {
+		RpcGateway gateway = registeredConnections.get(address);
+
+		if (gateway != null) {
+			if (clazz.isAssignableFrom(gateway.getClass())) {
+				@SuppressWarnings("unchecked")
+				C typedGateway = (C) gateway;
+				return Futures.successful(typedGateway);
+			} else {
+				return Futures.failed(
+					new Exception("Gateway registered under " + address + " is not of type " + clazz));
+			}
+		} else {
+			return Futures.failed(new Exception("No gateway registered under that name"));
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	// connections
+	// ------------------------------------------------------------------------
+
+	public void registerGateway(String address, RpcGateway gateway) {
+		checkNotNull(address);
+		checkNotNull(gateway);
+
+		if (registeredConnections.putIfAbsent(address, gateway) != null) {
+			throw new IllegalStateException("a gateway is already registered under " + address);
+		}
+	}
+
+	private static class TestingSerialInvocationHandler<C extends RpcGateway, T extends RpcEndpoint<C>> implements InvocationHandler, RpcGateway, MainThreadExecutor, StartStoppable {
+
+		private final T rpcEndpoint;
+
+		/** default timeout for asks */
+		private final Timeout timeout;
+
+		private final String address;
+
+		private TestingSerialInvocationHandler(String address, T rpcEndpoint) {
+			this(address, rpcEndpoint, new Timeout(new FiniteDuration(10, TimeUnit.SECONDS)));
+		}
+
+		private TestingSerialInvocationHandler(String address, T rpcEndpoint, Timeout timeout) {
+			this.rpcEndpoint = rpcEndpoint;
+			this.timeout = timeout;
+			this.address = address;
+		}
+
+		@Override
+		public Object invoke(Object proxy, Method method, Object[] args) throws Throwable {
+			Class<?> declaringClass = method.getDeclaringClass();
+			if (declaringClass.equals(MainThreadExecutor.class) ||
+				declaringClass.equals(Object.class) || declaringClass.equals(StartStoppable.class) ||
+				declaringClass.equals(RpcGateway.class)) {
+				return method.invoke(this, args);
+			} else {
+				final String methodName = method.getName();
+				Class<?>[] parameterTypes = method.getParameterTypes();
+				Annotation[][] parameterAnnotations = method.getParameterAnnotations();
+				Timeout futureTimeout = extractRpcTimeout(parameterAnnotations, args, timeout);
+
+				final Tuple2<Class<?>[], Object[]> filteredArguments = filterArguments(
+					parameterTypes,
+					parameterAnnotations,
+					args);
+
+				Class<?> returnType = method.getReturnType();
+
+				if (returnType.equals(Future.class)) {
+					try {
+						Object result = handleRpcInvocationSync(methodName, filteredArguments.f0, filteredArguments.f1, futureTimeout);
+						return Futures.successful(result);
+					} catch (Throwable e) {
+						return Futures.failed(e);
+					}
+				} else {
+					return handleRpcInvocationSync(methodName, filteredArguments.f0, filteredArguments.f1, futureTimeout);
+				}
+			}
+		}
+
+		/**
+		 * Handle rpc invocations by looking up the rpc method on the rpc endpoint and calling this
+		 * method with the provided method arguments. If the method has a return value, it is returned
+		 * to the sender of the call.
+		 */
+		private Object handleRpcInvocationSync(final String methodName,
+			final Class<?>[] parameterTypes,
+			final Object[] args,
+			final Timeout futureTimeout) throws Exception {
+			final Method rpcMethod = lookupRpcMethod(methodName, parameterTypes);
+			Object result = rpcMethod.invoke(rpcEndpoint, args);
+
+			if (result != null && result instanceof Future) {
+				Future<?> future = (Future<?>) result;
+				return Await.result(future, futureTimeout.duration());
+			} else {
+				return result;
+			}
+		}
+
+		@Override
+		public void runAsync(Runnable runnable) {
+			runnable.run();
+		}
+
+		@Override
+		public <V> Future<V> callAsync(Callable<V> callable, Timeout callTimeout) {
+			try {
+				return Futures.successful(callable.call());
+			} catch (Throwable e) {
+				return Futures.failed(e);
+			}
+		}
+
+		@Override
+		public void scheduleRunAsync(final Runnable runnable, final long delay) {
+			try {
+				TimeUnit.MILLISECONDS.sleep(delay);
+				runnable.run();
+			} catch (Throwable e) {
+				throw new RuntimeException(e);
+			}
+		}
+
+		@Override
+		public String getAddress() {
+			return address;
+		}
+
+		@Override
+		public void start() {
+			// do nothing
+		}
+
+		@Override
+		public void stop() {
+			// do nothing
+		}
+
+		/**
+		 * Look up the rpc method on the given {@link RpcEndpoint} instance.
+		 *
+		 * @param methodName     Name of the method
+		 * @param parameterTypes Parameter types of the method
+		 * @return Method of the rpc endpoint
+		 * @throws NoSuchMethodException Thrown if the method with the given name and parameter types
+		 *                               cannot be found at the rpc endpoint
+		 */
+		private Method lookupRpcMethod(final String methodName,
+			final Class<?>[] parameterTypes) throws NoSuchMethodException {
+			return rpcEndpoint.getClass().getMethod(methodName, parameterTypes);
+		}
+
+		// ------------------------------------------------------------------------
+		//  Helper methods
+		// ------------------------------------------------------------------------
+
+		/**
+		 * Extracts the {@link RpcTimeout} annotated rpc timeout value from the list of given method
+		 * arguments. If no {@link RpcTimeout} annotated parameter could be found, then the default
+		 * timeout is returned.
+		 *
+		 * @param parameterAnnotations Parameter annotations
+		 * @param args                 Array of arguments
+		 * @param defaultTimeout       Default timeout to return if no {@link RpcTimeout} annotated parameter
+		 *                             has been found
+		 * @return Timeout extracted from the array of arguments or the default timeout
+		 */
+		private static Timeout extractRpcTimeout(Annotation[][] parameterAnnotations, Object[] args,
+			Timeout defaultTimeout) {
+			if (args != null) {
+				Preconditions.checkArgument(parameterAnnotations.length == args.length);
+
+				for (int i = 0; i < parameterAnnotations.length; i++) {
+					if (isRpcTimeout(parameterAnnotations[i])) {
+						if (args[i] instanceof FiniteDuration) {
+							return new Timeout((FiniteDuration) args[i]);
+						} else {
+							throw new RuntimeException("The rpc timeout parameter must be of type " +
+								FiniteDuration.class.getName() + ". The type " + args[i].getClass().getName() +
+								" is not supported.");
+						}
+					}
+				}
+			}
+
+			return defaultTimeout;
+		}
+
+		/**
+		 * Removes all {@link RpcTimeout} annotated parameters from the parameter type and argument
+		 * list.
+		 *
+		 * @param parameterTypes       Array of parameter types
+		 * @param parameterAnnotations Array of parameter annotations
+		 * @param args                 Arary of arguments
+		 * @return Tuple of filtered parameter types and arguments which no longer contain the
+		 * {@link RpcTimeout} annotated parameter types and arguments
+		 */
+		private static Tuple2<Class<?>[], Object[]> filterArguments(
+			Class<?>[] parameterTypes,
+			Annotation[][] parameterAnnotations,
+			Object[] args) {
+
+			Class<?>[] filteredParameterTypes;
+			Object[] filteredArgs;
+
+			if (args == null) {
+				filteredParameterTypes = parameterTypes;
+				filteredArgs = null;
+			} else {
+				Preconditions.checkArgument(parameterTypes.length == parameterAnnotations.length);
+				Preconditions.checkArgument(parameterAnnotations.length == args.length);
+
+				BitSet isRpcTimeoutParameter = new BitSet(parameterTypes.length);
+				int numberRpcParameters = parameterTypes.length;
+
+				for (int i = 0; i < parameterTypes.length; i++) {
+					if (isRpcTimeout(parameterAnnotations[i])) {
+						isRpcTimeoutParameter.set(i);
+						numberRpcParameters--;
+					}
+				}
+
+				if (numberRpcParameters == parameterTypes.length) {
+					filteredParameterTypes = parameterTypes;
+					filteredArgs = args;
+				} else {
+					filteredParameterTypes = new Class<?>[numberRpcParameters];
+					filteredArgs = new Object[numberRpcParameters];
+					int counter = 0;
+
+					for (int i = 0; i < parameterTypes.length; i++) {
+						if (!isRpcTimeoutParameter.get(i)) {
+							filteredParameterTypes[counter] = parameterTypes[i];
+							filteredArgs[counter] = args[i];
+							counter++;
+						}
+					}
+				}
+			}
+			return Tuple2.of(filteredParameterTypes, filteredArgs);
+		}
+
+		/**
+		 * Checks whether any of the annotations is of type {@link RpcTimeout}
+		 *
+		 * @param annotations Array of annotations
+		 * @return True if {@link RpcTimeout} was found; otherwise false
+		 */
+		private static boolean isRpcTimeout(Annotation[] annotations) {
+			for (Annotation annotation : annotations) {
+				if (annotation.annotationType().equals(RpcTimeout.class)) {
+					return true;
+				}
+			}
+
+			return false;
+		}
+
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/26082bd3/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManagerHATest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManagerHATest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManagerHATest.java
new file mode 100644
index 0000000..dfffeda
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManagerHATest.java
@@ -0,0 +1,76 @@
+/*
+ * 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.rpc.resourcemanager;
+
+import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices;
+import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService;
+import org.apache.flink.runtime.rpc.MainThreadExecutor;
+import org.apache.flink.runtime.rpc.RpcEndpoint;
+import org.apache.flink.runtime.rpc.RpcGateway;
+import org.apache.flink.runtime.rpc.RpcService;
+import org.apache.flink.runtime.rpc.StartStoppable;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.UUID;
+
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.doCallRealMethod;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+/**
+ * resourceManager HA test, including grant leadership and revoke leadership
+ */
+public class ResourceManagerHATest {
+
+	@Test
+	public void testGrantAndRevokeLeadership() throws Exception {
+		// mock a RpcService which will return a special RpcGateway when call its startServer method, the returned RpcGateway directly execute runAsync call
+		TestingResourceManagerGatewayProxy gateway = mock(TestingResourceManagerGatewayProxy.class);
+		doCallRealMethod().when(gateway).runAsync(any(Runnable.class));
+
+		RpcService rpcService = mock(RpcService.class);
+		when(rpcService.startServer(any(RpcEndpoint.class))).thenReturn(gateway);
+
+		TestingLeaderElectionService leaderElectionService = new TestingLeaderElectionService();
+		TestingHighAvailabilityServices highAvailabilityServices = new TestingHighAvailabilityServices();
+		highAvailabilityServices.setResourceManagerLeaderElectionService(leaderElectionService);
+
+		final ResourceManager resourceManager = new ResourceManager(rpcService, highAvailabilityServices);
+		resourceManager.start();
+		// before grant leadership, resourceManager's leaderId is null
+		Assert.assertNull(resourceManager.getLeaderSessionID());
+		final UUID leaderId = UUID.randomUUID();
+		leaderElectionService.isLeader(leaderId);
+		// after grant leadership, resourceManager's leaderId has value
+		Assert.assertEquals(leaderId, resourceManager.getLeaderSessionID());
+		// then revoke leadership, resourceManager's leaderId is null again
+		leaderElectionService.notLeader();
+		Assert.assertNull(resourceManager.getLeaderSessionID());
+	}
+
+	private static abstract class TestingResourceManagerGatewayProxy implements MainThreadExecutor, StartStoppable, RpcGateway {
+		@Override
+		public void runAsync(Runnable runnable) {
+			runnable.run();
+		}
+	}
+
+}


[43/50] [abbrv] flink git commit: [FLINK-4529] [flip-6] Move TaskExecutor, JobMaster and ResourceManager out of the rpc package

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/72468d14/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/SlotManagerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/SlotManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/SlotManagerTest.java
new file mode 100644
index 0000000..52d9d06
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/resourcemanager/SlotManagerTest.java
@@ -0,0 +1,538 @@
+/*
+ * 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.resourcemanager;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
+import org.apache.flink.runtime.clusterframework.types.ResourceSlot;
+import org.apache.flink.runtime.clusterframework.types.SlotID;
+import org.apache.flink.runtime.taskexecutor.SlotStatus;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+
+public class SlotManagerTest {
+
+	private static final double DEFAULT_TESTING_CPU_CORES = 1.0;
+
+	private static final long DEFAULT_TESTING_MEMORY = 512;
+
+	private static final ResourceProfile DEFAULT_TESTING_PROFILE =
+		new ResourceProfile(DEFAULT_TESTING_CPU_CORES, DEFAULT_TESTING_MEMORY);
+
+	private static final ResourceProfile DEFAULT_TESTING_BIG_PROFILE =
+		new ResourceProfile(DEFAULT_TESTING_CPU_CORES * 2, DEFAULT_TESTING_MEMORY * 2);
+
+	private ResourceManagerGateway resourceManagerGateway;
+
+	@Before
+	public void setUp() {
+		resourceManagerGateway = mock(ResourceManagerGateway.class);
+	}
+
+	/**
+	 * Tests that there are no free slots when we request, need to allocate from cluster manager master
+	 */
+	@Test
+	public void testRequestSlotWithoutFreeSlot() {
+		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
+		slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE));
+
+		assertEquals(0, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertEquals(1, slotManager.getPendingRequestCount());
+		assertEquals(1, slotManager.getAllocatedContainers().size());
+		assertEquals(DEFAULT_TESTING_PROFILE, slotManager.getAllocatedContainers().get(0));
+	}
+
+	/**
+	 * Tests that there are some free slots when we request, and the request is fulfilled immediately
+	 */
+	@Test
+	public void testRequestSlotWithFreeSlot() {
+		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
+
+		directlyProvideFreeSlots(slotManager, DEFAULT_TESTING_PROFILE, 1);
+		assertEquals(1, slotManager.getFreeSlotCount());
+
+		slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE));
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertEquals(0, slotManager.getPendingRequestCount());
+		assertEquals(0, slotManager.getAllocatedContainers().size());
+	}
+
+	/**
+	 * Tests that there are some free slots when we request, but none of them are suitable
+	 */
+	@Test
+	public void testRequestSlotWithoutSuitableSlot() {
+		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
+
+		directlyProvideFreeSlots(slotManager, DEFAULT_TESTING_PROFILE, 2);
+		assertEquals(2, slotManager.getFreeSlotCount());
+
+		slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_BIG_PROFILE));
+		assertEquals(0, slotManager.getAllocatedSlotCount());
+		assertEquals(2, slotManager.getFreeSlotCount());
+		assertEquals(1, slotManager.getPendingRequestCount());
+		assertEquals(1, slotManager.getAllocatedContainers().size());
+		assertEquals(DEFAULT_TESTING_BIG_PROFILE, slotManager.getAllocatedContainers().get(0));
+	}
+
+	/**
+	 * Tests that we send duplicated slot request
+	 */
+	@Test
+	public void testDuplicatedSlotRequest() {
+		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
+		directlyProvideFreeSlots(slotManager, DEFAULT_TESTING_PROFILE, 1);
+
+		SlotRequest request1 = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE);
+		SlotRequest request2 = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_BIG_PROFILE);
+
+		slotManager.requestSlot(request1);
+		slotManager.requestSlot(request2);
+		slotManager.requestSlot(request2);
+		slotManager.requestSlot(request1);
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertEquals(1, slotManager.getPendingRequestCount());
+		assertEquals(1, slotManager.getAllocatedContainers().size());
+		assertEquals(DEFAULT_TESTING_BIG_PROFILE, slotManager.getAllocatedContainers().get(0));
+	}
+
+	/**
+	 * Tests that we send multiple slot requests
+	 */
+	@Test
+	public void testRequestMultipleSlots() {
+		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
+		directlyProvideFreeSlots(slotManager, DEFAULT_TESTING_PROFILE, 5);
+
+		// request 3 normal slots
+		for (int i = 0; i < 3; ++i) {
+			slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE));
+		}
+
+		// request 2 big slots
+		for (int i = 0; i < 2; ++i) {
+			slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_BIG_PROFILE));
+		}
+
+		// request 1 normal slot again
+		slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE));
+
+		assertEquals(4, slotManager.getAllocatedSlotCount());
+		assertEquals(1, slotManager.getFreeSlotCount());
+		assertEquals(2, slotManager.getPendingRequestCount());
+		assertEquals(2, slotManager.getAllocatedContainers().size());
+		assertEquals(DEFAULT_TESTING_BIG_PROFILE, slotManager.getAllocatedContainers().get(0));
+		assertEquals(DEFAULT_TESTING_BIG_PROFILE, slotManager.getAllocatedContainers().get(1));
+	}
+
+	/**
+	 * Tests that a new slot appeared in SlotReport, and we used it to fulfill a pending request
+	 */
+	@Test
+	public void testNewlyAppearedFreeSlotFulfillPendingRequest() {
+		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
+		slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE));
+		assertEquals(1, slotManager.getPendingRequestCount());
+
+		SlotID slotId = SlotID.generate();
+		SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE);
+		slotManager.updateSlotStatus(slotStatus);
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertEquals(0, slotManager.getPendingRequestCount());
+		assertTrue(slotManager.isAllocated(slotId));
+	}
+
+	/**
+	 * Tests that a new slot appeared in SlotReport, but we have no pending request
+	 */
+	@Test
+	public void testNewlyAppearedFreeSlot() {
+		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
+		SlotID slotId = SlotID.generate();
+		SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE);
+		slotManager.updateSlotStatus(slotStatus);
+
+		assertEquals(0, slotManager.getAllocatedSlotCount());
+		assertEquals(1, slotManager.getFreeSlotCount());
+	}
+
+	/**
+	 * Tests that a new slot appeared in SlotReport, but it't not suitable for all the pending requests
+	 */
+	@Test
+	public void testNewlyAppearedFreeSlotNotMatchPendingRequests() {
+		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
+		slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_BIG_PROFILE));
+		assertEquals(1, slotManager.getPendingRequestCount());
+
+		SlotID slotId = SlotID.generate();
+		SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE);
+		slotManager.updateSlotStatus(slotStatus);
+
+		assertEquals(0, slotManager.getAllocatedSlotCount());
+		assertEquals(1, slotManager.getFreeSlotCount());
+		assertEquals(1, slotManager.getPendingRequestCount());
+		assertFalse(slotManager.isAllocated(slotId));
+	}
+
+	/**
+	 * Tests that a new slot appeared in SlotReport, and it's been reported using by some job
+	 */
+	@Test
+	public void testNewlyAppearedInUseSlot() {
+		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
+
+		SlotID slotId = SlotID.generate();
+		SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE, new AllocationID(), new JobID());
+		slotManager.updateSlotStatus(slotStatus);
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertTrue(slotManager.isAllocated(slotId));
+	}
+
+	/**
+	 * Tests that we had a slot in-use, and it's confirmed by SlotReport
+	 */
+	@Test
+	public void testExistingInUseSlotUpdateStatus() {
+		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
+		SlotRequest request = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE);
+		slotManager.requestSlot(request);
+
+		// make this slot in use
+		SlotID slotId = SlotID.generate();
+		SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE);
+		slotManager.updateSlotStatus(slotStatus);
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertTrue(slotManager.isAllocated(slotId));
+
+		// slot status is confirmed
+		SlotStatus slotStatus2 = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE,
+			request.getAllocationId(), request.getJobId());
+		slotManager.updateSlotStatus(slotStatus2);
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertTrue(slotManager.isAllocated(slotId));
+	}
+
+	/**
+	 * Tests that we had a slot in-use, but it's empty according to the SlotReport
+	 */
+	@Test
+	public void testExistingInUseSlotAdjustedToEmpty() {
+		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
+		SlotRequest request1 = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE);
+		slotManager.requestSlot(request1);
+
+		// make this slot in use
+		SlotID slotId = SlotID.generate();
+		SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE);
+		slotManager.updateSlotStatus(slotStatus);
+
+		// another request pending
+		SlotRequest request2 = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE);
+		slotManager.requestSlot(request2);
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertEquals(1, slotManager.getPendingRequestCount());
+		assertTrue(slotManager.isAllocated(slotId));
+		assertTrue(slotManager.isAllocated(request1.getAllocationId()));
+
+
+		// but slot is reported empty again, request2 will be fulfilled, request1 will be missing
+		slotManager.updateSlotStatus(slotStatus);
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertEquals(0, slotManager.getPendingRequestCount());
+		assertTrue(slotManager.isAllocated(slotId));
+		assertTrue(slotManager.isAllocated(request2.getAllocationId()));
+	}
+
+	/**
+	 * Tests that we had a slot in use, and it's also reported in use by TaskManager, but the allocation
+	 * information didn't match.
+	 */
+	@Test
+	public void testExistingInUseSlotWithDifferentAllocationInfo() {
+		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
+		SlotRequest request = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE);
+		slotManager.requestSlot(request);
+
+		// make this slot in use
+		SlotID slotId = SlotID.generate();
+		SlotStatus slotStatus = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE);
+		slotManager.updateSlotStatus(slotStatus);
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertEquals(0, slotManager.getPendingRequestCount());
+		assertTrue(slotManager.isAllocated(slotId));
+		assertTrue(slotManager.isAllocated(request.getAllocationId()));
+
+		SlotStatus slotStatus2 = new SlotStatus(slotId, DEFAULT_TESTING_PROFILE, new AllocationID(), new JobID());
+		// update slot status with different allocation info
+		slotManager.updateSlotStatus(slotStatus2);
+
+		// original request is missing and won't be allocated
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertEquals(0, slotManager.getPendingRequestCount());
+		assertTrue(slotManager.isAllocated(slotId));
+		assertFalse(slotManager.isAllocated(request.getAllocationId()));
+		assertTrue(slotManager.isAllocated(slotStatus2.getAllocationID()));
+	}
+
+	/**
+	 * Tests that we had a free slot, and it's confirmed by SlotReport
+	 */
+	@Test
+	public void testExistingEmptySlotUpdateStatus() {
+		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
+		ResourceSlot slot = new ResourceSlot(SlotID.generate(), DEFAULT_TESTING_PROFILE);
+		slotManager.addFreeSlot(slot);
+
+		SlotStatus slotStatus = new SlotStatus(slot.getSlotId(), DEFAULT_TESTING_PROFILE);
+		slotManager.updateSlotStatus(slotStatus);
+
+		assertEquals(0, slotManager.getAllocatedSlotCount());
+		assertEquals(1, slotManager.getFreeSlotCount());
+		assertEquals(0, slotManager.getPendingRequestCount());
+	}
+
+	/**
+	 * Tests that we had a free slot, and it's reported in-use by TaskManager
+	 */
+	@Test
+	public void testExistingEmptySlotAdjustedToInUse() {
+		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
+		ResourceSlot slot = new ResourceSlot(SlotID.generate(), DEFAULT_TESTING_PROFILE);
+		slotManager.addFreeSlot(slot);
+
+		SlotStatus slotStatus = new SlotStatus(slot.getSlotId(), DEFAULT_TESTING_PROFILE,
+			new AllocationID(), new JobID());
+		slotManager.updateSlotStatus(slotStatus);
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertEquals(0, slotManager.getPendingRequestCount());
+		assertTrue(slotManager.isAllocated(slot.getSlotId()));
+	}
+
+	/**
+	 * Tests that we did some allocation but failed / rejected by TaskManager, request will retry
+	 */
+	@Test
+	public void testSlotAllocationFailedAtTaskManager() {
+		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
+		ResourceSlot slot = new ResourceSlot(SlotID.generate(), DEFAULT_TESTING_PROFILE);
+		slotManager.addFreeSlot(slot);
+
+		SlotRequest request = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE);
+		slotManager.requestSlot(request);
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertEquals(0, slotManager.getPendingRequestCount());
+		assertTrue(slotManager.isAllocated(slot.getSlotId()));
+
+		slotManager.handleSlotRequestFailedAtTaskManager(request, slot.getSlotId());
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertEquals(0, slotManager.getPendingRequestCount());
+	}
+
+
+	/**
+	 * Tests that we did some allocation but failed / rejected by TaskManager, and slot is occupied by another request
+	 */
+	@Test
+	public void testSlotAllocationFailedAtTaskManagerOccupiedByOther() {
+		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
+		ResourceSlot slot = new ResourceSlot(SlotID.generate(), DEFAULT_TESTING_PROFILE);
+		slotManager.addFreeSlot(slot);
+
+		SlotRequest request = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE);
+		slotManager.requestSlot(request);
+
+		// slot is set empty by heartbeat
+		SlotStatus slotStatus = new SlotStatus(slot.getSlotId(), slot.getResourceProfile());
+		slotManager.updateSlotStatus(slotStatus);
+
+		// another request took this slot
+		SlotRequest request2 = new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE);
+		slotManager.requestSlot(request2);
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertEquals(0, slotManager.getPendingRequestCount());
+		assertFalse(slotManager.isAllocated(request.getAllocationId()));
+		assertTrue(slotManager.isAllocated(request2.getAllocationId()));
+
+		// original request should be pended
+		slotManager.handleSlotRequestFailedAtTaskManager(request, slot.getSlotId());
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertEquals(1, slotManager.getPendingRequestCount());
+		assertFalse(slotManager.isAllocated(request.getAllocationId()));
+		assertTrue(slotManager.isAllocated(request2.getAllocationId()));
+	}
+
+	@Test
+	public void testNotifyTaskManagerFailure() {
+		TestingSlotManager slotManager = new TestingSlotManager(resourceManagerGateway);
+
+		ResourceID resource1 = ResourceID.generate();
+		ResourceID resource2 = ResourceID.generate();
+
+		ResourceSlot slot11 = new ResourceSlot(new SlotID(resource1, 1), DEFAULT_TESTING_PROFILE);
+		ResourceSlot slot12 = new ResourceSlot(new SlotID(resource1, 2), DEFAULT_TESTING_PROFILE);
+		ResourceSlot slot21 = new ResourceSlot(new SlotID(resource2, 1), DEFAULT_TESTING_PROFILE);
+		ResourceSlot slot22 = new ResourceSlot(new SlotID(resource2, 2), DEFAULT_TESTING_PROFILE);
+
+		slotManager.addFreeSlot(slot11);
+		slotManager.addFreeSlot(slot21);
+
+		slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE));
+		slotManager.requestSlot(new SlotRequest(new JobID(), new AllocationID(), DEFAULT_TESTING_PROFILE));
+
+		assertEquals(2, slotManager.getAllocatedSlotCount());
+		assertEquals(0, slotManager.getFreeSlotCount());
+		assertEquals(0, slotManager.getPendingRequestCount());
+
+		slotManager.addFreeSlot(slot12);
+		slotManager.addFreeSlot(slot22);
+
+		assertEquals(2, slotManager.getAllocatedSlotCount());
+		assertEquals(2, slotManager.getFreeSlotCount());
+		assertEquals(0, slotManager.getPendingRequestCount());
+
+		slotManager.notifyTaskManagerFailure(resource2);
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(1, slotManager.getFreeSlotCount());
+		assertEquals(0, slotManager.getPendingRequestCount());
+
+		// notify an not exist resource failure
+		slotManager.notifyTaskManagerFailure(ResourceID.generate());
+
+		assertEquals(1, slotManager.getAllocatedSlotCount());
+		assertEquals(1, slotManager.getFreeSlotCount());
+		assertEquals(0, slotManager.getPendingRequestCount());
+	}
+
+	// ------------------------------------------------------------------------
+	//  testing utilities
+	// ------------------------------------------------------------------------
+
+	private void directlyProvideFreeSlots(
+		final SlotManager slotManager,
+		final ResourceProfile resourceProfile,
+		final int freeSlotNum)
+	{
+		for (int i = 0; i < freeSlotNum; ++i) {
+			slotManager.addFreeSlot(new ResourceSlot(SlotID.generate(), new ResourceProfile(resourceProfile)));
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  testing classes
+	// ------------------------------------------------------------------------
+
+	private static class TestingSlotManager extends SlotManager {
+
+		private final List<ResourceProfile> allocatedContainers;
+
+		TestingSlotManager(ResourceManagerGateway resourceManagerGateway) {
+			super(resourceManagerGateway);
+			this.allocatedContainers = new LinkedList<>();
+		}
+
+		/**
+		 * Choose slot randomly if it matches requirement
+		 *
+		 * @param request   The slot request
+		 * @param freeSlots All slots which can be used
+		 * @return The chosen slot or null if cannot find a match
+		 */
+		@Override
+		protected ResourceSlot chooseSlotToUse(SlotRequest request, Map<SlotID, ResourceSlot> freeSlots) {
+			for (ResourceSlot slot : freeSlots.values()) {
+				if (slot.isMatchingRequirement(request.getResourceProfile())) {
+					return slot;
+				}
+			}
+			return null;
+		}
+
+		/**
+		 * Choose request randomly if offered slot can match its requirement
+		 *
+		 * @param offeredSlot     The free slot
+		 * @param pendingRequests All the pending slot requests
+		 * @return The chosen request's AllocationID or null if cannot find a match
+		 */
+		@Override
+		protected SlotRequest chooseRequestToFulfill(ResourceSlot offeredSlot,
+			Map<AllocationID, SlotRequest> pendingRequests)
+		{
+			for (Map.Entry<AllocationID, SlotRequest> pendingRequest : pendingRequests.entrySet()) {
+				if (offeredSlot.isMatchingRequirement(pendingRequest.getValue().getResourceProfile())) {
+					return pendingRequest.getValue();
+				}
+			}
+			return null;
+		}
+
+		@Override
+		protected void allocateContainer(ResourceProfile resourceProfile) {
+			allocatedContainers.add(resourceProfile);
+		}
+
+		List<ResourceProfile> getAllocatedContainers() {
+			return allocatedContainers;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/72468d14/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java
index 2790cf8..f55069e 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java
@@ -21,28 +21,14 @@ package org.apache.flink.runtime.rpc.akka;
 import akka.actor.ActorSystem;
 import akka.util.Timeout;
 import org.apache.flink.core.testutils.OneShotLatch;
-import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.akka.AkkaUtils;
-import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
-import org.apache.flink.runtime.highavailability.NonHaServices;
-import org.apache.flink.runtime.jobgraph.JobGraph;
-import org.apache.flink.runtime.rpc.jobmaster.JobMaster;
-import org.apache.flink.runtime.rpc.resourcemanager.ResourceManagerGateway;
-import org.apache.flink.runtime.rpc.resourcemanager.ResourceManager;
 import org.apache.flink.util.TestLogger;
 
 import org.junit.AfterClass;
 import org.junit.Test;
 
-import org.mockito.Mockito;
-import scala.concurrent.duration.Deadline;
-import scala.concurrent.duration.FiniteDuration;
-
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.ForkJoinPool;
 import java.util.concurrent.TimeUnit;
 
-import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
 public class AkkaRpcServiceTest extends TestLogger {

http://git-wip-us.apache.org/repos/asf/flink/blob/72468d14/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/registration/RetryingRegistrationTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/registration/RetryingRegistrationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/registration/RetryingRegistrationTest.java
deleted file mode 100644
index 9508825..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/registration/RetryingRegistrationTest.java
+++ /dev/null
@@ -1,336 +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.rpc.registration;
-
-import akka.dispatch.Futures;
-
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.runtime.rpc.RpcService;
-import org.apache.flink.runtime.rpc.TestingRpcService;
-import org.apache.flink.util.TestLogger;
-
-import org.junit.Test;
-
-import org.slf4j.LoggerFactory;
-
-import scala.concurrent.Await;
-import scala.concurrent.ExecutionContext$;
-import scala.concurrent.Future;
-import scala.concurrent.Promise;
-import scala.concurrent.duration.FiniteDuration;
-
-import java.util.UUID;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.TimeoutException;
-
-import static java.util.concurrent.TimeUnit.SECONDS;
-import static org.junit.Assert.*;
-import static org.mockito.Mockito.*;
-
-/**
- * Tests for the generic retrying registration class, validating the failure, retry, and back-off behavior.
- */
-public class RetryingRegistrationTest extends TestLogger {
-
-	@Test
-	public void testSimpleSuccessfulRegistration() throws Exception {
-		final String testId = "laissez les bon temps roulez";
-		final String testEndpointAddress = "<test-address>";
-		final UUID leaderId = UUID.randomUUID();
-
-		// an endpoint that immediately returns success
-		TestRegistrationGateway testGateway = new TestRegistrationGateway(new TestRegistrationSuccess(testId));
-		TestingRpcService rpc = new TestingRpcService();
-
-		try {
-			rpc.registerGateway(testEndpointAddress, testGateway);
-
-			TestRetryingRegistration registration = new TestRetryingRegistration(rpc, testEndpointAddress, leaderId);
-			registration.startRegistration();
-
-			Future<Tuple2<TestRegistrationGateway, TestRegistrationSuccess>> future = registration.getFuture();
-			assertNotNull(future);
-
-			// multiple accesses return the same future
-			assertEquals(future, registration.getFuture());
-
-			Tuple2<TestRegistrationGateway, TestRegistrationSuccess> success = 
-					Await.result(future, new FiniteDuration(10, SECONDS));
-
-			// validate correct invocation and result
-			assertEquals(testId, success.f1.getCorrelationId());
-			assertEquals(leaderId, testGateway.getInvocations().take().leaderId());
-		}
-		finally {
-			testGateway.stop();
-			rpc.stopService();
-		}
-	}
-	
-	@Test
-	public void testPropagateFailures() throws Exception {
-		final String testExceptionMessage = "testExceptionMessage";
-
-		// RPC service that fails with exception upon the connection
-		RpcService rpc = mock(RpcService.class);
-		when(rpc.connect(anyString(), any(Class.class))).thenThrow(new RuntimeException(testExceptionMessage));
-
-		TestRetryingRegistration registration = new TestRetryingRegistration(rpc, "testaddress", UUID.randomUUID());
-		registration.startRegistration();
-
-		Future<?> future = registration.getFuture();
-		assertTrue(future.failed().isCompleted());
-
-		assertEquals(testExceptionMessage, future.failed().value().get().get().getMessage());
-	}
-
-	@Test
-	public void testRetryConnectOnFailure() throws Exception {
-		final String testId = "laissez les bon temps roulez";
-		final UUID leaderId = UUID.randomUUID();
-
-		ExecutorService executor = Executors.newCachedThreadPool();
-		TestRegistrationGateway testGateway = new TestRegistrationGateway(new TestRegistrationSuccess(testId));
-
-		try {
-			// RPC service that fails upon the first connection, but succeeds on the second
-			RpcService rpc = mock(RpcService.class);
-			when(rpc.connect(anyString(), any(Class.class))).thenReturn(
-					Futures.failed(new Exception("test connect failure")),  // first connection attempt fails
-					Futures.successful(testGateway)                         // second connection attempt succeeds
-			);
-			when(rpc.getExecutionContext()).thenReturn(ExecutionContext$.MODULE$.fromExecutor(executor));
-
-			TestRetryingRegistration registration = new TestRetryingRegistration(rpc, "foobar address", leaderId);
-			registration.startRegistration();
-
-			Tuple2<TestRegistrationGateway, TestRegistrationSuccess> success =
-					Await.result(registration.getFuture(), new FiniteDuration(10, SECONDS));
-
-			// validate correct invocation and result
-			assertEquals(testId, success.f1.getCorrelationId());
-			assertEquals(leaderId, testGateway.getInvocations().take().leaderId());
-		}
-		finally {
-			testGateway.stop();
-			executor.shutdown();
-		}
-	}
-
-	@Test
-	public void testRetriesOnTimeouts() throws Exception {
-		final String testId = "rien ne va plus";
-		final String testEndpointAddress = "<test-address>";
-		final UUID leaderId = UUID.randomUUID();
-
-		// an endpoint that immediately returns futures with timeouts before returning a successful future
-		TestRegistrationGateway testGateway = new TestRegistrationGateway(
-				null, // timeout
-				null, // timeout
-				new TestRegistrationSuccess(testId) // success
-		);
-
-		TestingRpcService rpc = new TestingRpcService();
-
-		try {
-			rpc.registerGateway(testEndpointAddress, testGateway);
-	
-			TestRetryingRegistration registration = new TestRetryingRegistration(rpc, testEndpointAddress, leaderId);
-	
-			long started = System.nanoTime();
-			registration.startRegistration();
-	
-			Future<Tuple2<TestRegistrationGateway, TestRegistrationSuccess>> future = registration.getFuture();
-			Tuple2<TestRegistrationGateway, TestRegistrationSuccess> success =
-					Await.result(future, new FiniteDuration(10, SECONDS));
-	
-			long finished = System.nanoTime();
-			long elapsedMillis = (finished - started) / 1000000;
-	
-			// validate correct invocation and result
-			assertEquals(testId, success.f1.getCorrelationId());
-			assertEquals(leaderId, testGateway.getInvocations().take().leaderId());
-	
-			// validate that some retry-delay / back-off behavior happened
-			assertTrue("retries did not properly back off", elapsedMillis >= 3 * TestRetryingRegistration.INITIAL_TIMEOUT);
-		}
-		finally {
-			rpc.stopService();
-			testGateway.stop();
-		}
-	}
-
-	@Test
-	public void testDecline() throws Exception {
-		final String testId = "qui a coupe le fromage";
-		final String testEndpointAddress = "<test-address>";
-		final UUID leaderId = UUID.randomUUID();
-
-		TestingRpcService rpc = new TestingRpcService();
-
-		TestRegistrationGateway testGateway = new TestRegistrationGateway(
-				null, // timeout
-				new RegistrationResponse.Decline("no reason "),
-				null, // timeout
-				new TestRegistrationSuccess(testId) // success
-		);
-
-		try {
-			rpc.registerGateway(testEndpointAddress, testGateway);
-
-			TestRetryingRegistration registration = new TestRetryingRegistration(rpc, testEndpointAddress, leaderId);
-
-			long started = System.nanoTime();
-			registration.startRegistration();
-	
-			Future<Tuple2<TestRegistrationGateway, TestRegistrationSuccess>> future = registration.getFuture();
-			Tuple2<TestRegistrationGateway, TestRegistrationSuccess> success =
-					Await.result(future, new FiniteDuration(10, SECONDS));
-
-			long finished = System.nanoTime();
-			long elapsedMillis = (finished - started) / 1000000;
-
-			// validate correct invocation and result
-			assertEquals(testId, success.f1.getCorrelationId());
-			assertEquals(leaderId, testGateway.getInvocations().take().leaderId());
-
-			// validate that some retry-delay / back-off behavior happened
-			assertTrue("retries did not properly back off", elapsedMillis >= 
-					2 * TestRetryingRegistration.INITIAL_TIMEOUT + TestRetryingRegistration.DELAY_ON_DECLINE);
-		}
-		finally {
-			testGateway.stop();
-			rpc.stopService();
-		}
-	}
-	
-	@Test
-	@SuppressWarnings("unchecked")
-	public void testRetryOnError() throws Exception {
-		final String testId = "Petit a petit, l'oiseau fait son nid";
-		final String testEndpointAddress = "<test-address>";
-		final UUID leaderId = UUID.randomUUID();
-
-		TestingRpcService rpc = new TestingRpcService();
-
-		try {
-			// gateway that upon calls first responds with a failure, then with a success
-			TestRegistrationGateway testGateway = mock(TestRegistrationGateway.class);
-
-			when(testGateway.registrationCall(any(UUID.class), anyLong())).thenReturn(
-					Futures.<RegistrationResponse>failed(new Exception("test exception")),
-					Futures.<RegistrationResponse>successful(new TestRegistrationSuccess(testId)));
-			
-			rpc.registerGateway(testEndpointAddress, testGateway);
-
-			TestRetryingRegistration registration = new TestRetryingRegistration(rpc, testEndpointAddress, leaderId);
-
-			long started = System.nanoTime();
-			registration.startRegistration();
-
-			Future<Tuple2<TestRegistrationGateway, TestRegistrationSuccess>> future = registration.getFuture();
-			Tuple2<TestRegistrationGateway, TestRegistrationSuccess> success =
-					Await.result(future, new FiniteDuration(10, SECONDS));
-
-			long finished = System.nanoTime();
-			long elapsedMillis = (finished - started) / 1000000;
-			
-			assertEquals(testId, success.f1.getCorrelationId());
-
-			// validate that some retry-delay / back-off behavior happened
-			assertTrue("retries did not properly back off",
-					elapsedMillis >= TestRetryingRegistration.DELAY_ON_ERROR);
-		}
-		finally {
-			rpc.stopService();
-		}
-	}
-
-	@Test
-	public void testCancellation() throws Exception {
-		final String testEndpointAddress = "my-test-address";
-		final UUID leaderId = UUID.randomUUID();
-
-		TestingRpcService rpc = new TestingRpcService();
-
-		try {
-			Promise<RegistrationResponse> result = Futures.promise();
-
-			TestRegistrationGateway testGateway = mock(TestRegistrationGateway.class);
-			when(testGateway.registrationCall(any(UUID.class), anyLong())).thenReturn(result.future());
-
-			rpc.registerGateway(testEndpointAddress, testGateway);
-
-			TestRetryingRegistration registration = new TestRetryingRegistration(rpc, testEndpointAddress, leaderId);
-			registration.startRegistration();
-
-			// cancel and fail the current registration attempt
-			registration.cancel();
-			result.failure(new TimeoutException());
-
-			// there should not be a second registration attempt
-			verify(testGateway, atMost(1)).registrationCall(any(UUID.class), anyLong());
-		}
-		finally {
-			rpc.stopService();
-		}
-	}
-
-	// ------------------------------------------------------------------------
-	//  test registration
-	// ------------------------------------------------------------------------
-
-	private static class TestRegistrationSuccess extends RegistrationResponse.Success {
-		private static final long serialVersionUID = 5542698790917150604L;
-
-		private final String correlationId;
-
-		private TestRegistrationSuccess(String correlationId) {
-			this.correlationId = correlationId;
-		}
-
-		public String getCorrelationId() {
-			return correlationId;
-		}
-	}
-
-	private static class TestRetryingRegistration extends RetryingRegistration<TestRegistrationGateway, TestRegistrationSuccess> {
-
-		// we use shorter timeouts here to speed up the tests
-		static final long INITIAL_TIMEOUT = 20;
-		static final long MAX_TIMEOUT = 200;
-		static final long DELAY_ON_ERROR = 200;
-		static final long DELAY_ON_DECLINE = 200;
-
-		public TestRetryingRegistration(RpcService rpc, String targetAddress, UUID leaderId) {
-			super(LoggerFactory.getLogger(RetryingRegistrationTest.class),
-					rpc, "TestEndpoint",
-					TestRegistrationGateway.class,
-					targetAddress, leaderId,
-					INITIAL_TIMEOUT, MAX_TIMEOUT, DELAY_ON_ERROR, DELAY_ON_DECLINE);
-		}
-
-		@Override
-		protected Future<RegistrationResponse> invokeRegistration(
-				TestRegistrationGateway gateway, UUID leaderId, long timeoutMillis) {
-			return gateway.registrationCall(leaderId, timeoutMillis);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/72468d14/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/registration/TestRegistrationGateway.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/registration/TestRegistrationGateway.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/registration/TestRegistrationGateway.java
deleted file mode 100644
index a049e48..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/registration/TestRegistrationGateway.java
+++ /dev/null
@@ -1,85 +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.rpc.registration;
-
-import akka.dispatch.Futures;
-
-import org.apache.flink.runtime.rpc.TestingGatewayBase;
-import org.apache.flink.util.Preconditions;
-
-import scala.concurrent.Future;
-
-import java.util.UUID;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.LinkedBlockingQueue;
-
-public class TestRegistrationGateway extends TestingGatewayBase {
-
-	private final BlockingQueue<RegistrationCall> invocations;
-
-	private final RegistrationResponse[] responses;
-
-	private int pos;
-
-	public TestRegistrationGateway(RegistrationResponse... responses) {
-		Preconditions.checkArgument(responses != null && responses.length > 0);
-
-		this.invocations = new LinkedBlockingQueue<>();
-		this.responses = responses;
-		
-	}
-
-	// ------------------------------------------------------------------------
-
-	public Future<RegistrationResponse> registrationCall(UUID leaderId, long timeout) {
-		invocations.add(new RegistrationCall(leaderId, timeout));
-
-		RegistrationResponse response = responses[pos];
-		if (pos < responses.length - 1) {
-			pos++;
-		}
-
-		// return a completed future (for a proper value), or one that never completes and will time out (for null)
-		return response != null ? Futures.successful(response) : this.<RegistrationResponse>futureWithTimeout(timeout);
-	}
-
-	public BlockingQueue<RegistrationCall> getInvocations() {
-		return invocations;
-	}
-
-	// ------------------------------------------------------------------------
-
-	public static class RegistrationCall {
-		private final UUID leaderId;
-		private final long timeout;
-
-		public RegistrationCall(UUID leaderId, long timeout) {
-			this.leaderId = leaderId;
-			this.timeout = timeout;
-		}
-
-		public UUID leaderId() {
-			return leaderId;
-		}
-
-		public long timeout() {
-			return timeout;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/72468d14/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManagerHATest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManagerHATest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManagerHATest.java
deleted file mode 100644
index dfffeda..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManagerHATest.java
+++ /dev/null
@@ -1,76 +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.rpc.resourcemanager;
-
-import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices;
-import org.apache.flink.runtime.leaderelection.TestingLeaderElectionService;
-import org.apache.flink.runtime.rpc.MainThreadExecutor;
-import org.apache.flink.runtime.rpc.RpcEndpoint;
-import org.apache.flink.runtime.rpc.RpcGateway;
-import org.apache.flink.runtime.rpc.RpcService;
-import org.apache.flink.runtime.rpc.StartStoppable;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.util.UUID;
-
-import static org.mockito.Matchers.any;
-import static org.mockito.Mockito.doCallRealMethod;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-/**
- * resourceManager HA test, including grant leadership and revoke leadership
- */
-public class ResourceManagerHATest {
-
-	@Test
-	public void testGrantAndRevokeLeadership() throws Exception {
-		// mock a RpcService which will return a special RpcGateway when call its startServer method, the returned RpcGateway directly execute runAsync call
-		TestingResourceManagerGatewayProxy gateway = mock(TestingResourceManagerGatewayProxy.class);
-		doCallRealMethod().when(gateway).runAsync(any(Runnable.class));
-
-		RpcService rpcService = mock(RpcService.class);
-		when(rpcService.startServer(any(RpcEndpoint.class))).thenReturn(gateway);
-
-		TestingLeaderElectionService leaderElectionService = new TestingLeaderElectionService();
-		TestingHighAvailabilityServices highAvailabilityServices = new TestingHighAvailabilityServices();
-		highAvailabilityServices.setResourceManagerLeaderElectionService(leaderElectionService);
-
-		final ResourceManager resourceManager = new ResourceManager(rpcService, highAvailabilityServices);
-		resourceManager.start();
-		// before grant leadership, resourceManager's leaderId is null
-		Assert.assertNull(resourceManager.getLeaderSessionID());
-		final UUID leaderId = UUID.randomUUID();
-		leaderElectionService.isLeader(leaderId);
-		// after grant leadership, resourceManager's leaderId has value
-		Assert.assertEquals(leaderId, resourceManager.getLeaderSessionID());
-		// then revoke leadership, resourceManager's leaderId is null again
-		leaderElectionService.notLeader();
-		Assert.assertNull(resourceManager.getLeaderSessionID());
-	}
-
-	private static abstract class TestingResourceManagerGatewayProxy implements MainThreadExecutor, StartStoppable, RpcGateway {
-		@Override
-		public void runAsync(Runnable runnable) {
-			runnable.run();
-		}
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/72468d14/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorTest.java
deleted file mode 100644
index 25a670c..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorTest.java
+++ /dev/null
@@ -1,117 +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.rpc.taskexecutor;
-
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.clusterframework.types.ResourceID;
-import org.apache.flink.runtime.highavailability.NonHaServices;
-import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices;
-import org.apache.flink.runtime.leaderelection.TestingLeaderRetrievalService;
-import org.apache.flink.runtime.rpc.TestingRpcService;
-import org.apache.flink.runtime.rpc.resourcemanager.ResourceManagerGateway;
-import org.apache.flink.util.TestLogger;
-
-import org.junit.Test;
-
-import scala.concurrent.duration.FiniteDuration;
-
-import java.util.UUID;
-
-import static org.junit.Assert.*;
-import static org.mockito.Mockito.*;
-
-public class TaskExecutorTest extends TestLogger {
-
-	@Test
-	public void testImmediatelyRegistersIfLeaderIsKnown() throws Exception {
-		final ResourceID resourceID = ResourceID.generate();
-		final String resourceManagerAddress = "/resource/manager/address/one";
-
-		final TestingRpcService rpc = new TestingRpcService();
-		try {
-			// register a mock resource manager gateway
-			ResourceManagerGateway rmGateway = mock(ResourceManagerGateway.class);
-			rpc.registerGateway(resourceManagerAddress, rmGateway);
-
-			NonHaServices haServices = new NonHaServices(resourceManagerAddress);
-			TaskExecutor taskManager = TaskExecutor.startTaskManagerComponentsAndActor(
-				new Configuration(), resourceID, rpc, "localhost", haServices, true);
-			String taskManagerAddress = taskManager.getAddress();
-			taskManager.start();
-
-			verify(rmGateway, timeout(5000)).registerTaskExecutor(
-					any(UUID.class), eq(taskManagerAddress), eq(resourceID), any(FiniteDuration.class));
-		}
-		finally {
-			rpc.stopService();
-		}
-	}
-
-	@Test
-	public void testTriggerRegistrationOnLeaderChange() throws Exception {
-		final ResourceID resourceID = ResourceID.generate();
-
-		final String address1 = "/resource/manager/address/one";
-		final String address2 = "/resource/manager/address/two";
-		final UUID leaderId1 = UUID.randomUUID();
-		final UUID leaderId2 = UUID.randomUUID();
-
-		final TestingRpcService rpc = new TestingRpcService();
-		try {
-			// register the mock resource manager gateways
-			ResourceManagerGateway rmGateway1 = mock(ResourceManagerGateway.class);
-			ResourceManagerGateway rmGateway2 = mock(ResourceManagerGateway.class);
-			rpc.registerGateway(address1, rmGateway1);
-			rpc.registerGateway(address2, rmGateway2);
-
-			TestingLeaderRetrievalService testLeaderService = new TestingLeaderRetrievalService();
-
-			TestingHighAvailabilityServices haServices = new TestingHighAvailabilityServices();
-			haServices.setResourceManagerLeaderRetriever(testLeaderService);
-
-			TaskExecutor taskManager = TaskExecutor.startTaskManagerComponentsAndActor(
-				new Configuration(), resourceID, rpc, "localhost", haServices, true);
-			String taskManagerAddress = taskManager.getAddress();
-			taskManager.start();
-
-			// no connection initially, since there is no leader
-			assertNull(taskManager.getResourceManagerConnection());
-
-			// define a leader and see that a registration happens
-			testLeaderService.notifyListener(address1, leaderId1);
-
-			verify(rmGateway1, timeout(5000)).registerTaskExecutor(
-					eq(leaderId1), eq(taskManagerAddress), eq(resourceID), any(FiniteDuration.class));
-			assertNotNull(taskManager.getResourceManagerConnection());
-
-			// cancel the leader 
-			testLeaderService.notifyListener(null, null);
-
-			// set a new leader, see that a registration happens 
-			testLeaderService.notifyListener(address2, leaderId2);
-
-			verify(rmGateway2, timeout(5000)).registerTaskExecutor(
-					eq(leaderId2), eq(taskManagerAddress), eq(resourceID), any(FiniteDuration.class));
-			assertNotNull(taskManager.getResourceManagerConnection());
-		}
-		finally {
-			rpc.stopService();
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/72468d14/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java
new file mode 100644
index 0000000..a8d5bd7
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java
@@ -0,0 +1,117 @@
+/*
+ * 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.taskexecutor;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.highavailability.NonHaServices;
+import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices;
+import org.apache.flink.runtime.leaderelection.TestingLeaderRetrievalService;
+import org.apache.flink.runtime.rpc.TestingRpcService;
+import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Test;
+
+import scala.concurrent.duration.FiniteDuration;
+
+import java.util.UUID;
+
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
+
+public class TaskExecutorTest extends TestLogger {
+
+	@Test
+	public void testImmediatelyRegistersIfLeaderIsKnown() throws Exception {
+		final ResourceID resourceID = ResourceID.generate();
+		final String resourceManagerAddress = "/resource/manager/address/one";
+
+		final TestingRpcService rpc = new TestingRpcService();
+		try {
+			// register a mock resource manager gateway
+			ResourceManagerGateway rmGateway = mock(ResourceManagerGateway.class);
+			rpc.registerGateway(resourceManagerAddress, rmGateway);
+
+			NonHaServices haServices = new NonHaServices(resourceManagerAddress);
+			TaskExecutor taskManager = TaskExecutor.startTaskManagerComponentsAndActor(
+				new Configuration(), resourceID, rpc, "localhost", haServices, true);
+			String taskManagerAddress = taskManager.getAddress();
+			taskManager.start();
+
+			verify(rmGateway, timeout(5000)).registerTaskExecutor(
+					any(UUID.class), eq(taskManagerAddress), eq(resourceID), any(FiniteDuration.class));
+		}
+		finally {
+			rpc.stopService();
+		}
+	}
+
+	@Test
+	public void testTriggerRegistrationOnLeaderChange() throws Exception {
+		final ResourceID resourceID = ResourceID.generate();
+
+		final String address1 = "/resource/manager/address/one";
+		final String address2 = "/resource/manager/address/two";
+		final UUID leaderId1 = UUID.randomUUID();
+		final UUID leaderId2 = UUID.randomUUID();
+
+		final TestingRpcService rpc = new TestingRpcService();
+		try {
+			// register the mock resource manager gateways
+			ResourceManagerGateway rmGateway1 = mock(ResourceManagerGateway.class);
+			ResourceManagerGateway rmGateway2 = mock(ResourceManagerGateway.class);
+			rpc.registerGateway(address1, rmGateway1);
+			rpc.registerGateway(address2, rmGateway2);
+
+			TestingLeaderRetrievalService testLeaderService = new TestingLeaderRetrievalService();
+
+			TestingHighAvailabilityServices haServices = new TestingHighAvailabilityServices();
+			haServices.setResourceManagerLeaderRetriever(testLeaderService);
+
+			TaskExecutor taskManager = TaskExecutor.startTaskManagerComponentsAndActor(
+				new Configuration(), resourceID, rpc, "localhost", haServices, true);
+			String taskManagerAddress = taskManager.getAddress();
+			taskManager.start();
+
+			// no connection initially, since there is no leader
+			assertNull(taskManager.getResourceManagerConnection());
+
+			// define a leader and see that a registration happens
+			testLeaderService.notifyListener(address1, leaderId1);
+
+			verify(rmGateway1, timeout(5000)).registerTaskExecutor(
+					eq(leaderId1), eq(taskManagerAddress), eq(resourceID), any(FiniteDuration.class));
+			assertNotNull(taskManager.getResourceManagerConnection());
+
+			// cancel the leader 
+			testLeaderService.notifyListener(null, null);
+
+			// set a new leader, see that a registration happens 
+			testLeaderService.notifyListener(address2, leaderId2);
+
+			verify(rmGateway2, timeout(5000)).registerTaskExecutor(
+					eq(leaderId2), eq(taskManagerAddress), eq(resourceID), any(FiniteDuration.class));
+			assertNotNull(taskManager.getResourceManagerConnection());
+		}
+		finally {
+			rpc.stopService();
+		}
+	}
+}


[29/50] [abbrv] flink git commit: [FLINK-4403] [rpc] Use relative classloader for proxies, rather than system class loader.

Posted by tr...@apache.org.
[FLINK-4403] [rpc] Use relative classloader for proxies, rather than system class loader.


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

Branch: refs/heads/flip-6
Commit: a7e3579b2f1a88db3cde13e324e11ee0dbc0a9e4
Parents: f94ae53
Author: Stephan Ewen <se...@apache.org>
Authored: Tue Aug 16 21:11:01 2016 +0200
Committer: Till Rohrmann <tr...@apache.org>
Committed: Thu Sep 8 17:26:58 2016 +0200

----------------------------------------------------------------------
 .../org/apache/flink/runtime/rpc/akka/AkkaRpcService.java     | 7 ++++++-
 1 file changed, 6 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/a7e3579b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java
index b647bbd..d987c2f 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java
@@ -104,9 +104,14 @@ public class AkkaRpcService implements RpcService {
 
 				InvocationHandler akkaInvocationHandler = new AkkaInvocationHandler(actorRef, timeout, maximumFramesize);
 
+				// Rather than using the System ClassLoader directly, we derive the ClassLoader
+				// from this class . That works better in cases where Flink runs embedded and all Flink
+				// code is loaded dynamically (for example from an OSGI bundle) through a custom ClassLoader
+				ClassLoader classLoader = AkkaRpcService.this.getClass().getClassLoader();
+				
 				@SuppressWarnings("unchecked")
 				C proxy = (C) Proxy.newProxyInstance(
-					ClassLoader.getSystemClassLoader(),
+					classLoader,
 					new Class<?>[] {clazz},
 					akkaInvocationHandler);
 


[42/50] [abbrv] flink git commit: [FLINK-4451] [rpc] Throw RpcConnectionException when rpc endpoint is not reachable

Posted by tr...@apache.org.
[FLINK-4451] [rpc] Throw RpcConnectionException when rpc endpoint is not reachable

This PR introduces a RpcConnectionException which is thrown if the rpc endpoint
is not reachable when calling RpcService.connect.

This closes #2405.


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

Branch: refs/heads/flip-6
Commit: 1f3256b593295e7d4945379e8323c53e2b243a9a
Parents: 562b293
Author: Till Rohrmann <tr...@apache.org>
Authored: Tue Aug 23 17:59:54 2016 +0200
Committer: Till Rohrmann <tr...@apache.org>
Committed: Thu Sep 8 17:27:01 2016 +0200

----------------------------------------------------------------------
 .../registration/RetryingRegistration.java      |  2 +-
 .../apache/flink/runtime/rpc/RpcService.java    |  7 +++-
 .../flink/runtime/rpc/akka/AkkaRpcService.java  | 38 +++++++++++-------
 .../rpc/exceptions/RpcConnectionException.java  | 41 ++++++++++++++++++++
 .../runtime/rpc/akka/AkkaRpcActorTest.java      | 18 +++++++++
 5 files changed, 88 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/1f3256b5/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RetryingRegistration.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RetryingRegistration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RetryingRegistration.java
index 88fe9b5..ea49e42 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RetryingRegistration.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/registration/RetryingRegistration.java
@@ -197,7 +197,7 @@ public abstract class RetryingRegistration<Gateway extends RpcGateway, Success e
 				@Override
 				public void onFailure(Throwable failure) {
 					if (!isCanceled()) {
-						log.warn("Could not resolve {} address {}, retrying...", targetName, targetAddress);
+						log.warn("Could not resolve {} address {}, retrying...", targetName, targetAddress, failure);
 						startRegistration();
 					}
 				}

http://git-wip-us.apache.org/repos/asf/flink/blob/1f3256b5/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java
index bc0f5cb..78c1cec 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.runtime.rpc;
 
+import org.apache.flink.runtime.rpc.exceptions.RpcConnectionException;
 import scala.concurrent.ExecutionContext;
 import scala.concurrent.Future;
 
@@ -32,12 +33,14 @@ public interface RpcService {
 
 	/**
 	 * Connect to a remote rpc server under the provided address. Returns a rpc gateway which can
-	 * be used to communicate with the rpc server.
+	 * be used to communicate with the rpc server. If the connection failed, then the returned
+	 * future is failed with a {@link RpcConnectionException}.
 	 *
 	 * @param address Address of the remote rpc server
 	 * @param clazz Class of the rpc gateway to return
 	 * @param <C> Type of the rpc gateway to return
-	 * @return Future containing the rpc gateway
+	 * @return Future containing the rpc gateway or an {@link RpcConnectionException} if the
+	 * connection attempt failed
 	 */
 	<C extends RpcGateway> Future<C> connect(String address, Class<C> clazz);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/1f3256b5/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java
index 00a6932..060a1ef 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java
@@ -35,6 +35,7 @@ import org.apache.flink.runtime.rpc.RpcGateway;
 import org.apache.flink.runtime.rpc.RpcEndpoint;
 import org.apache.flink.runtime.rpc.RpcService;
 import org.apache.flink.runtime.rpc.StartStoppable;
+import org.apache.flink.runtime.rpc.exceptions.RpcConnectionException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -99,25 +100,32 @@ public class AkkaRpcService implements RpcService {
 		final Future<Object> identify = asker.ask(new Identify(42), timeout);
 		return identify.map(new Mapper<Object, C>(){
 			@Override
-			public C apply(Object obj) {
-				ActorRef actorRef = ((ActorIdentity) obj).getRef();
+			public C checkedApply(Object obj) throws Exception {
 
-				final String address = AkkaUtils.getAkkaURL(actorSystem, actorRef);
+				ActorIdentity actorIdentity = (ActorIdentity) obj;
 
-				InvocationHandler akkaInvocationHandler = new AkkaInvocationHandler(address, actorRef, timeout, maximumFramesize);
+				if (actorIdentity.getRef() == null) {
+					throw new RpcConnectionException("Could not connect to rpc endpoint under address " + address + '.');
+				} else {
+					ActorRef actorRef = actorIdentity.getRef();
+
+					final String address = AkkaUtils.getAkkaURL(actorSystem, actorRef);
+
+					InvocationHandler akkaInvocationHandler = new AkkaInvocationHandler(address, actorRef, timeout, maximumFramesize);
 
-				// Rather than using the System ClassLoader directly, we derive the ClassLoader
-				// from this class . That works better in cases where Flink runs embedded and all Flink
-				// code is loaded dynamically (for example from an OSGI bundle) through a custom ClassLoader
-				ClassLoader classLoader = AkkaRpcService.this.getClass().getClassLoader();
-				
-				@SuppressWarnings("unchecked")
-				C proxy = (C) Proxy.newProxyInstance(
-					classLoader,
-					new Class<?>[] {clazz},
-					akkaInvocationHandler);
+					// Rather than using the System ClassLoader directly, we derive the ClassLoader
+					// from this class . That works better in cases where Flink runs embedded and all Flink
+					// code is loaded dynamically (for example from an OSGI bundle) through a custom ClassLoader
+					ClassLoader classLoader = AkkaRpcService.this.getClass().getClassLoader();
 
-				return proxy;
+					@SuppressWarnings("unchecked")
+					C proxy = (C) Proxy.newProxyInstance(
+						classLoader,
+						new Class<?>[]{clazz},
+						akkaInvocationHandler);
+
+					return proxy;
+				}
 			}
 		}, actorSystem.dispatcher());
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/1f3256b5/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/exceptions/RpcConnectionException.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/exceptions/RpcConnectionException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/exceptions/RpcConnectionException.java
new file mode 100644
index 0000000..a22ebe7
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/exceptions/RpcConnectionException.java
@@ -0,0 +1,41 @@
+/*
+ * 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.rpc.exceptions;
+
+import java.util.concurrent.ExecutionException;
+
+/**
+ * Exception class which is thrown if a rpc connection failed. Usually this happens if the remote
+ * host cannot be reached.
+ */
+public class RpcConnectionException extends ExecutionException {
+	private static final long serialVersionUID = -5500560405481142472L;
+
+	public RpcConnectionException(String message) {
+		super(message);
+	}
+
+	public RpcConnectionException(String message, Throwable cause) {
+		super(message, cause);
+	}
+
+	public RpcConnectionException(Throwable cause) {
+		super(cause);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/1f3256b5/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java
index 82d13f0..a6ceb91 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java
@@ -25,6 +25,7 @@ import org.apache.flink.runtime.rpc.RpcEndpoint;
 import org.apache.flink.runtime.rpc.RpcGateway;
 import org.apache.flink.runtime.rpc.RpcMethod;
 import org.apache.flink.runtime.rpc.RpcService;
+import org.apache.flink.runtime.rpc.exceptions.RpcConnectionException;
 import org.apache.flink.util.TestLogger;
 import org.hamcrest.core.Is;
 import org.junit.AfterClass;
@@ -36,6 +37,7 @@ import java.util.concurrent.TimeUnit;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
 
 public class AkkaRpcActorTest extends TestLogger {
 
@@ -73,6 +75,22 @@ public class AkkaRpcActorTest extends TestLogger {
 	}
 
 	/**
+	 * Tests that a {@link RpcConnectionException} is thrown if the rpc endpoint cannot be connected to.
+	 */
+	@Test
+	public void testFailingAddressResolution() throws Exception {
+		Future<DummyRpcGateway> futureRpcGateway = akkaRpcService.connect("foobar", DummyRpcGateway.class);
+
+		try {
+			DummyRpcGateway gateway = Await.result(futureRpcGateway, timeout.duration());
+
+			fail("The rpc connection resolution should have failed.");
+		} catch (RpcConnectionException exception) {
+			// we're expecting a RpcConnectionException
+		}
+	}
+
+	/**
 	 * Tests that the {@link AkkaRpcActor} stashes messages until the corresponding
 	 * {@link RpcEndpoint} has been started.
 	 */


[23/50] [abbrv] flink git commit: [FLINK-4362] [rpc] Auto generate rpc gateways via Java proxies

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/67e6f574/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java
index 642a380..a4e1d7f 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java
@@ -61,10 +61,10 @@ public class AkkaRpcServiceTest extends TestLogger {
 		AkkaGateway akkaClient = (AkkaGateway) rm;
 
 		
-		jobMaster.registerAtResourceManager(AkkaUtils.getAkkaURL(actorSystem, akkaClient.getActorRef()));
+		jobMaster.registerAtResourceManager(AkkaUtils.getAkkaURL(actorSystem, akkaClient.getRpcServer()));
 
 		// wait for successful registration
-		FiniteDuration timeout = new FiniteDuration(20, TimeUnit.SECONDS);
+		FiniteDuration timeout = new FiniteDuration(200, TimeUnit.SECONDS);
 		Deadline deadline = timeout.fromNow();
 
 		while (deadline.hasTimeLeft() && !jobMaster.isConnected()) {

http://git-wip-us.apache.org/repos/asf/flink/blob/67e6f574/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorTest.java
index c143527..33c9cb6 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorTest.java
@@ -48,7 +48,7 @@ public class TaskExecutorTest extends TestLogger {
 	@Test
 	public void testTaskExecution() throws Exception {
 		RpcService testingRpcService = mock(RpcService.class);
-		DirectExecutorService directExecutorService = null;
+		DirectExecutorService directExecutorService = new DirectExecutorService();
 		TaskExecutor taskExecutor = new TaskExecutor(testingRpcService, directExecutorService);
 
 		TaskDeploymentDescriptor tdd = new TaskDeploymentDescriptor(


[16/50] [abbrv] flink git commit: [FLINK-4458] Replace ForkableFlinkMiniCluster by LocalFlinkMiniCluster

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingTaskManagerLike.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingTaskManagerLike.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingTaskManagerLike.scala
deleted file mode 100644
index a6963fe..0000000
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingTaskManagerLike.scala
+++ /dev/null
@@ -1,248 +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.testingUtils
-
-import akka.actor.{ActorRef, Terminated}
-import org.apache.flink.api.common.JobID
-import org.apache.flink.runtime.FlinkActor
-import org.apache.flink.runtime.execution.ExecutionState
-import org.apache.flink.runtime.executiongraph.ExecutionAttemptID
-import org.apache.flink.runtime.messages.JobManagerMessages.{RequestLeaderSessionID, ResponseLeaderSessionID}
-import org.apache.flink.runtime.messages.Messages.{Acknowledge, Disconnect}
-import org.apache.flink.runtime.messages.RegistrationMessages.{AcknowledgeRegistration, AlreadyRegistered}
-import org.apache.flink.runtime.messages.TaskMessages.{SubmitTask, TaskInFinalState, UpdateTaskExecutionState}
-import org.apache.flink.runtime.taskmanager.TaskManager
-import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages.NotifyWhenJobRemoved
-import org.apache.flink.runtime.testingUtils.TestingMessages._
-import org.apache.flink.runtime.testingUtils.TestingTaskManagerMessages._
-
-import scala.concurrent.duration._
-import scala.language.postfixOps
-
-/** This mixin can be used to decorate a TaskManager with messages for testing purposes. */
-trait TestingTaskManagerLike extends FlinkActor {
-  that: TaskManager =>
-
-  import scala.collection.JavaConverters._
-
-  val waitForRemoval = scala.collection.mutable.HashMap[ExecutionAttemptID, Set[ActorRef]]()
-  val waitForJobManagerToBeTerminated = scala.collection.mutable.HashMap[String, Set[ActorRef]]()
-  val waitForRegisteredAtResourceManager =
-    scala.collection.mutable.HashMap[ActorRef, Set[ActorRef]]()
-  val waitForRunning = scala.collection.mutable.HashMap[ExecutionAttemptID, Set[ActorRef]]()
-  val unregisteredTasks = scala.collection.mutable.HashSet[ExecutionAttemptID]()
-
-  /** Map of registered task submit listeners */
-  val registeredSubmitTaskListeners = scala.collection.mutable.HashMap[JobID, ActorRef]()
-
-  val waitForShutdown = scala.collection.mutable.HashSet[ActorRef]()
-
-  var disconnectDisabled = false
-
-  /**
-   * Handler for testing related messages
-   */
-  abstract override def handleMessage: Receive = {
-    handleTestingMessage orElse super.handleMessage
-  }
-
-  def handleTestingMessage: Receive = {
-    case Alive => sender() ! Acknowledge
-
-    case NotifyWhenTaskIsRunning(executionID) =>
-      Option(runningTasks.get(executionID)) match {
-        case Some(task) if task.getExecutionState == ExecutionState.RUNNING =>
-          sender ! decorateMessage(true)
-
-        case _ =>
-          val listeners = waitForRunning.getOrElse(executionID, Set())
-          waitForRunning += (executionID -> (listeners + sender))
-      }
-
-    case RequestRunningTasks =>
-      sender ! decorateMessage(ResponseRunningTasks(runningTasks.asScala.toMap))
-
-    case NotifyWhenTaskRemoved(executionID) =>
-      Option(runningTasks.get(executionID)) match {
-        case Some(_) =>
-          val set = waitForRemoval.getOrElse(executionID, Set())
-          waitForRemoval += (executionID -> (set + sender))
-        case None =>
-          if(unregisteredTasks.contains(executionID)) {
-            sender ! decorateMessage(true)
-          } else {
-            val set = waitForRemoval.getOrElse(executionID, Set())
-            waitForRemoval += (executionID -> (set + sender))
-          }
-      }
-
-    case TaskInFinalState(executionID) =>
-      super.handleMessage(TaskInFinalState(executionID))
-      waitForRemoval.remove(executionID) match {
-        case Some(actors) => for(actor <- actors) actor ! decorateMessage(true)
-        case None =>
-      }
-
-      unregisteredTasks += executionID
-
-    case RequestBroadcastVariablesWithReferences =>
-      sender ! decorateMessage(
-        ResponseBroadcastVariablesWithReferences(
-          bcVarManager.getNumberOfVariablesWithReferences)
-      )
-
-    case RequestNumActiveConnections =>
-      val numActive = if (!network.isShutdown) {
-        network.getConnectionManager.getNumberOfActiveConnections
-      } else {
-        0
-      }
-      sender ! decorateMessage(ResponseNumActiveConnections(numActive))
-
-    case NotifyWhenJobRemoved(jobID) =>
-      if(runningTasks.values.asScala.exists(_.getJobID == jobID)){
-        context.system.scheduler.scheduleOnce(
-          200 milliseconds,
-          self,
-          decorateMessage(CheckIfJobRemoved(jobID)))(
-            context.dispatcher,
-            sender()
-          )
-      }else{
-        sender ! decorateMessage(true)
-      }
-
-    case CheckIfJobRemoved(jobID) =>
-      if(runningTasks.values.asScala.forall(_.getJobID != jobID)){
-        sender ! decorateMessage(true)
-      } else {
-        context.system.scheduler.scheduleOnce(
-          200 milliseconds,
-          self,
-          decorateMessage(CheckIfJobRemoved(jobID)))(
-            context.dispatcher,
-            sender()
-          )
-      }
-
-    case NotifyWhenJobManagerTerminated(jobManager) =>
-      val waiting = waitForJobManagerToBeTerminated.getOrElse(jobManager.path.name, Set())
-      waitForJobManagerToBeTerminated += jobManager.path.name -> (waiting + sender)
-
-    case RegisterSubmitTaskListener(jobId) =>
-      registeredSubmitTaskListeners.put(jobId, sender())
-
-    case msg@SubmitTask(tdd) =>
-      registeredSubmitTaskListeners.get(tdd.getJobID) match {
-        case Some(listenerRef) =>
-          listenerRef ! ResponseSubmitTaskListener(tdd)
-        case None =>
-        // Nothing to do
-      }
-
-      super.handleMessage(msg)
-
-    /**
-     * Message from task manager that accumulator values changed and need to be reported immediately
-     * instead of lazily through the
-     * [[org.apache.flink.runtime.messages.TaskManagerMessages.Heartbeat]] message. We forward this
-     * message to the job manager that it knows it should report to the listeners.
-     */
-    case msg: AccumulatorsChanged =>
-      currentJobManager match {
-        case Some(jobManager) =>
-          jobManager.forward(msg)
-          sendHeartbeatToJobManager()
-          sender ! true
-        case None =>
-      }
-
-    case msg@Terminated(jobManager) =>
-      super.handleMessage(msg)
-
-      waitForJobManagerToBeTerminated.remove(jobManager.path.name) foreach {
-        _ foreach {
-          _ ! decorateMessage(JobManagerTerminated(jobManager))
-        }
-      }
-
-    case msg:Disconnect =>
-      if (!disconnectDisabled) {
-        super.handleMessage(msg)
-
-        val jobManager = sender()
-
-        waitForJobManagerToBeTerminated.remove(jobManager.path.name) foreach {
-          _ foreach {
-            _ ! decorateMessage(JobManagerTerminated(jobManager))
-          }
-        }
-      }
-
-    case DisableDisconnect =>
-      disconnectDisabled = true
-
-    case NotifyOfComponentShutdown =>
-      waitForShutdown += sender()
-
-    case msg @ UpdateTaskExecutionState(taskExecutionState) =>
-      super.handleMessage(msg)
-
-      if(taskExecutionState.getExecutionState == ExecutionState.RUNNING) {
-        waitForRunning.get(taskExecutionState.getID) foreach {
-          _ foreach (_ ! decorateMessage(true))
-        }
-      }
-
-    case RequestLeaderSessionID =>
-      sender() ! ResponseLeaderSessionID(leaderSessionID.orNull)
-
-    case NotifyWhenRegisteredAtJobManager(jobManager: ActorRef) =>
-      if(isConnected && jobManager == currentJobManager.get) {
-        sender() ! true
-      } else {
-        val list = waitForRegisteredAtResourceManager.getOrElse(
-          jobManager,
-          Set[ActorRef]())
-
-        waitForRegisteredAtResourceManager += jobManager -> (list + sender())
-      }
-
-    case msg @ (_: AcknowledgeRegistration | _: AlreadyRegistered) =>
-      super.handleMessage(msg)
-
-      val jm = sender()
-
-      waitForRegisteredAtResourceManager.remove(jm).foreach {
-        listeners => listeners.foreach{
-          listener =>
-            listener ! true
-        }
-      }
-  }
-
-  /**
-    * No killing of the VM for testing.
-    */
-  override protected def shutdown(): Unit = {
-    log.info("Shutting down TestingJobManager.")
-    waitForShutdown.foreach(_ ! ComponentShutdown(self))
-    waitForShutdown.clear()
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingTaskManagerMessages.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingTaskManagerMessages.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingTaskManagerMessages.scala
deleted file mode 100644
index 974e4e8..0000000
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/testingUtils/TestingTaskManagerMessages.scala
+++ /dev/null
@@ -1,94 +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.testingUtils
-
-import akka.actor.ActorRef
-import org.apache.flink.api.common.JobID
-import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor
-import org.apache.flink.runtime.executiongraph.ExecutionAttemptID
-import org.apache.flink.runtime.taskmanager.Task
-
-/**
- * Additional messages that the [[TestingTaskManager]] understands.
- */
-object TestingTaskManagerMessages {
-  
-  case class NotifyWhenTaskRemoved(executionID: ExecutionAttemptID)
-
-  case class NotifyWhenTaskIsRunning(executionID: ExecutionAttemptID)
-  
-  case class ResponseRunningTasks(tasks: Map[ExecutionAttemptID, Task]){
-    import collection.JavaConverters._
-    def asJava: java.util.Map[ExecutionAttemptID, Task] = tasks.asJava
-  }
-  
-  case class ResponseBroadcastVariablesWithReferences(number: Int)
-
-  case object RequestNumActiveConnections
-  case class ResponseNumActiveConnections(number: Int)
-  
-  case object RequestRunningTasks
-  
-  case object RequestBroadcastVariablesWithReferences
-
-  case class NotifyWhenJobManagerTerminated(jobManager: ActorRef)
-
-  case class JobManagerTerminated(jobManager: ActorRef)
-
-  case class NotifyWhenRegisteredAtJobManager(resourceManager: ActorRef)
-
-  /**
-   * Message to give a hint to the task manager that accumulator values were updated in the task.
-   * This message is forwarded to the job manager which knows that it needs to notify listeners
-   * of accumulator updates.
-   */
-  case class AccumulatorsChanged(jobID: JobID)
-
-  /**
-    * Registers a listener for all [[org.apache.flink.runtime.messages.TaskMessages.SubmitTask]]
-    * messages of the given job.
-    *
-    * If a task is submitted with the given job ID the task deployment
-    * descriptor is forwarded to the listener.
-    *
-    * @param jobId The job ID to listen for.
-    */
-  case class RegisterSubmitTaskListener(jobId: JobID)
-
-  /**
-    * A response to a listened job ID containing the submitted task deployment descriptor.
-    *
-    * @param tdd The submitted task deployment descriptor.
-    */
-  case class ResponseSubmitTaskListener(tdd: TaskDeploymentDescriptor)
-
-  // --------------------------------------------------------------------------
-  // Utility methods to allow simpler case object access from Java
-  // --------------------------------------------------------------------------
-  
-  def getRequestRunningTasksMessage: AnyRef = {
-    RequestRunningTasks
-  }
-  
-  def getRequestBroadcastVariablesWithReferencesMessage: AnyRef = {
-    RequestBroadcastVariablesWithReferences
-  }
-
-}
-

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderElectionRetrievalTestingCluster.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderElectionRetrievalTestingCluster.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderElectionRetrievalTestingCluster.java
index e596166..c143fe2 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderElectionRetrievalTestingCluster.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderElectionRetrievalTestingCluster.java
@@ -20,7 +20,6 @@ package org.apache.flink.runtime.leaderelection;
 
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.executiongraph.restart.RestartStrategy;
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
 import org.apache.flink.runtime.testingUtils.TestingCluster;
 import scala.Option;
@@ -86,7 +85,7 @@ public class LeaderElectionRetrievalTestingCluster extends TestingCluster {
 
 	@Override
 	public int getNumberOfJobManagers() {
-		return this.configuration().getInteger(
+		return this.originalConfiguration().getInteger(
 				ConfigConstants.LOCAL_NUMBER_JOB_MANAGER,
 				ConfigConstants.DEFAULT_LOCAL_NUMBER_JOB_MANAGER);
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/TestingResourceManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/TestingResourceManager.java b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/TestingResourceManager.java
new file mode 100644
index 0000000..495cacd
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/TestingResourceManager.java
@@ -0,0 +1,137 @@
+/*
+ * 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.testutils;
+
+import akka.actor.ActorRef;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.clusterframework.ApplicationStatus;
+import org.apache.flink.runtime.clusterframework.messages.RegisterResourceManagerSuccessful;
+import org.apache.flink.runtime.clusterframework.standalone.StandaloneResourceManager;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.runtime.messages.Messages;
+import org.apache.flink.runtime.testingUtils.TestingMessages;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Set;
+
+
+/**
+ * A testing resource manager which may alter the default standalone resource master's behavior.
+ */
+public class TestingResourceManager extends StandaloneResourceManager {
+
+	/** Set of Actors which want to be informed of a connection to the job manager */
+	private Set<ActorRef> waitForResourceManagerConnected = new HashSet<>();
+
+	/** Set of Actors which want to be informed of a shutdown */
+	private Set<ActorRef> waitForShutdown = new HashSet<>();
+
+	/** Flag to signal a connection to the JobManager */
+	private boolean isConnected = false;
+
+	public TestingResourceManager(Configuration flinkConfig, LeaderRetrievalService leaderRetriever) {
+		super(flinkConfig, leaderRetriever);
+	}
+
+	/**
+	 * Overwrite messages here if desired
+	 */
+	@Override
+	protected void handleMessage(Object message) {
+
+		if (message instanceof GetRegisteredResources) {
+			sender().tell(new GetRegisteredResourcesReply(getStartedTaskManagers()), self());
+		} else if (message instanceof FailResource) {
+			ResourceID resourceID = ((FailResource) message).resourceID;
+			notifyWorkerFailed(resourceID, "Failed for test case.");
+
+		} else if (message instanceof NotifyWhenResourceManagerConnected) {
+			if (isConnected) {
+				sender().tell(
+					Messages.getAcknowledge(),
+					self());
+			} else {
+				waitForResourceManagerConnected.add(sender());
+			}
+		} else if (message instanceof RegisterResourceManagerSuccessful) {
+			super.handleMessage(message);
+
+			isConnected = true;
+
+			for (ActorRef ref : waitForResourceManagerConnected) {
+				ref.tell(
+					Messages.getAcknowledge(),
+					self());
+			}
+			waitForResourceManagerConnected.clear();
+
+		} else if (message instanceof TestingMessages.NotifyOfComponentShutdown$) {
+			waitForShutdown.add(sender());
+		} else if (message instanceof TestingMessages.Alive$) {
+			sender().tell(Messages.getAcknowledge(), self());
+		} else {
+			super.handleMessage(message);
+		}
+	}
+
+	/**
+	 * Testing messages
+	 */
+	public static class GetRegisteredResources {}
+
+	public static class GetRegisteredResourcesReply {
+
+		public Collection<ResourceID> resources;
+
+		public GetRegisteredResourcesReply(Collection<ResourceID> resources) {
+			this.resources = resources;
+		}
+
+	}
+
+	/**
+	 * Fails all resources that the resource manager has registered
+	 */
+	public static class FailResource {
+
+		public ResourceID resourceID;
+
+		public FailResource(ResourceID resourceID) {
+			this.resourceID = resourceID;
+		}
+	}
+
+	/**
+	 * The sender of this message will be informed of a connection to the Job Manager
+	 */
+	public static class NotifyWhenResourceManagerConnected {}
+
+	/**
+	 * Inform registered listeners about a shutdown of the application.
+     */
+	@Override
+	protected void shutdownApplication(ApplicationStatus finalStatus, String optionalDiagnostics) {
+		for (ActorRef listener : waitForShutdown) {
+			listener.tell(new TestingMessages.ComponentShutdown(self()), self());
+		}
+		waitForShutdown.clear();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingCluster.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingCluster.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingCluster.scala
index b4ba40b..c01a321 100644
--- a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingCluster.scala
+++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingCluster.scala
@@ -18,22 +18,32 @@
 
 package org.apache.flink.runtime.testingUtils
 
-import java.util.concurrent.TimeoutException
+import java.util.concurrent.{ExecutorService, TimeUnit, TimeoutException}
 
 import akka.pattern.ask
-import akka.actor.{ActorRef, Props, ActorSystem}
+import akka.actor.{ActorRef, ActorSystem, Props}
+import akka.pattern.Patterns._
 import akka.testkit.CallingThreadDispatcher
 import org.apache.flink.configuration.{ConfigConstants, Configuration}
+import org.apache.flink.runtime.akka.AkkaUtils
+import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory
+import org.apache.flink.runtime.checkpoint.savepoint.SavepointStore
 import org.apache.flink.runtime.clusterframework.FlinkResourceManager
-import org.apache.flink.runtime.clusterframework.types.ResourceID
-import org.apache.flink.runtime.executiongraph.restart.RestartStrategy
-import org.apache.flink.runtime.jobmanager.JobManager
+import org.apache.flink.runtime.clusterframework.types.ResourceIDRetrievable
+import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager
+import org.apache.flink.runtime.executiongraph.restart.RestartStrategyFactory
+import org.apache.flink.runtime.instance.InstanceManager
+import org.apache.flink.runtime.jobmanager.scheduler.Scheduler
+import org.apache.flink.runtime.jobmanager.{JobManager, MemoryArchivist, SubmittedJobGraphStore}
 import org.apache.flink.runtime.leaderelection.LeaderElectionService
-import org.apache.flink.runtime.minicluster.FlinkMiniCluster
+import org.apache.flink.runtime.metrics.MetricRegistry
+import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster
 import org.apache.flink.runtime.testutils.TestingResourceManager
 import org.apache.flink.runtime.taskmanager.TaskManager
 import org.apache.flink.runtime.testingUtils.TestingMessages.Alive
+import org.apache.flink.runtime.testingUtils.TestingTaskManagerMessages.NotifyWhenRegisteredAtJobManager
 
+import scala.concurrent.duration.FiniteDuration
 import scala.concurrent.{Await, Future}
 
 /**
@@ -48,7 +58,7 @@ class TestingCluster(
     userConfiguration: Configuration,
     singleActorSystem: Boolean,
     synchronousDispatcher: Boolean)
-  extends FlinkMiniCluster(
+  extends LocalFlinkMiniCluster(
     userConfiguration,
     singleActorSystem) {
 
@@ -59,133 +69,54 @@ class TestingCluster(
 
   // --------------------------------------------------------------------------
 
-  override def generateConfiguration(userConfig: Configuration): Configuration = {
-    val cfg = new Configuration()
-    cfg.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, "localhost")
-    cfg.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, 0)
-    cfg.setInteger(ConfigConstants.RESOURCE_MANAGER_IPC_PORT_KEY, 0)
-    cfg.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 10)
-    cfg.setInteger(ConfigConstants.JOB_MANAGER_WEB_PORT_KEY, -1)
-
-    setDefaultCiConfig(cfg)
-
-    cfg.addAll(userConfig)
-    cfg
-  }
-
-  override def startJobManager(index: Int, actorSystem: ActorSystem): ActorRef = {
-    val config = configuration.clone()
-
-    val jobManagerName = if(singleActorSystem) {
-      JobManager.JOB_MANAGER_NAME + "_" + (index + 1)
-    } else {
-      JobManager.JOB_MANAGER_NAME
-    }
-
-    val archiveName = if(singleActorSystem) {
-      JobManager.ARCHIVE_NAME + "_" + (index + 1)
-    } else {
-      JobManager.ARCHIVE_NAME
-    }
-
-    val jobManagerPort = config.getInteger(
-      ConfigConstants.JOB_MANAGER_IPC_PORT_KEY,
-      ConfigConstants.DEFAULT_JOB_MANAGER_IPC_PORT)
-
-    if(jobManagerPort > 0) {
-      config.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, jobManagerPort + index)
-    }
-
-    val (executionContext,
-    instanceManager,
-    scheduler,
-    libraryCacheManager,
-    restartStrategyFactory,
-    timeout,
-    archiveCount,
-    leaderElectionService,
-    submittedJobsGraphs,
-    checkpointRecoveryFactory,
-    savepointStore,
-    jobRecoveryTimeout,
-    metricRegistry) = JobManager.createJobManagerComponents(
-      config,
-      createLeaderElectionService())
-
-    val testArchiveProps = Props(new TestingMemoryArchivist(archiveCount))
-    val archive = actorSystem.actorOf(testArchiveProps, archiveName)
-
-    val jobManagerProps = Props(
-      new TestingJobManager(
-        configuration,
-        executionContext,
-        instanceManager,
-        scheduler,
-        libraryCacheManager,
-        archive,
-        restartStrategyFactory,
-        timeout,
-        leaderElectionService,
-        submittedJobsGraphs,
-        checkpointRecoveryFactory,
-        savepointStore,
-        jobRecoveryTimeout,
-        metricRegistry))
-
-    val dispatcherJobManagerProps = if (synchronousDispatcher) {
-      // disable asynchronous futures (e.g. accumulator update in Heartbeat)
-      jobManagerProps.withDispatcher(CallingThreadDispatcher.Id)
-    } else {
-      jobManagerProps
-    }
-
-    actorSystem.actorOf(dispatcherJobManagerProps, jobManagerName)
-  }
-
-  override def startResourceManager(index: Int, system: ActorSystem): ActorRef = {
-    val config = configuration.clone()
-
-    val resourceManagerName = if(singleActorSystem) {
-      FlinkResourceManager.RESOURCE_MANAGER_NAME + "_" + (index + 1)
+  override val jobManagerClass: Class[_ <: JobManager] = classOf[TestingJobManager]
+
+  override val resourceManagerClass: Class[_ <: FlinkResourceManager[_ <: ResourceIDRetrievable]] =
+    classOf[TestingResourceManager]
+
+  override val taskManagerClass: Class[_ <: TaskManager] = classOf[TestingTaskManager]
+
+  override val memoryArchivistClass: Class[_ <: MemoryArchivist] = classOf[TestingMemoryArchivist]
+
+  override def getJobManagerProps(
+    jobManagerClass: Class[_ <: JobManager],
+    configuration: Configuration,
+    executorService: ExecutorService,
+    instanceManager: InstanceManager,
+    scheduler: Scheduler,
+    libraryCacheManager: BlobLibraryCacheManager,
+    archive: ActorRef,
+    restartStrategyFactory: RestartStrategyFactory,
+    timeout: FiniteDuration,
+    leaderElectionService: LeaderElectionService,
+    submittedJobGraphStore: SubmittedJobGraphStore,
+    checkpointRecoveryFactory: CheckpointRecoveryFactory,
+    savepointStore: SavepointStore,
+    jobRecoveryTimeout: FiniteDuration,
+    metricsRegistry: Option[MetricRegistry]): Props = {
+
+    val props = super.getJobManagerProps(
+      jobManagerClass,
+      configuration,
+      executorService,
+      instanceManager,
+      scheduler,
+      libraryCacheManager,
+      archive,
+      restartStrategyFactory,
+      timeout,
+      leaderElectionService,
+      submittedJobGraphStore,
+      checkpointRecoveryFactory,
+      savepointStore,
+      jobRecoveryTimeout,
+      metricsRegistry)
+
+    if (synchronousDispatcher) {
+      props.withDispatcher(CallingThreadDispatcher.Id)
     } else {
-      FlinkResourceManager.RESOURCE_MANAGER_NAME
+      props
     }
-
-    val resourceManagerPort = config.getInteger(
-      ConfigConstants.RESOURCE_MANAGER_IPC_PORT_KEY,
-      ConfigConstants.DEFAULT_RESOURCE_MANAGER_IPC_PORT)
-
-    if(resourceManagerPort > 0) {
-      config.setInteger(ConfigConstants.RESOURCE_MANAGER_IPC_PORT_KEY, resourceManagerPort + index)
-    }
-
-    val testResourceManagerProps = Props(
-      new TestingResourceManager(
-        config,
-        createLeaderRetrievalService()
-      ))
-
-    system.actorOf(testResourceManagerProps, resourceManagerName)
-  }
-
-  override def startTaskManager(index: Int, system: ActorSystem) = {
-
-    val tmActorName = TaskManager.TASK_MANAGER_NAME + "_" + (index + 1)
-
-    TaskManager.startTaskManagerComponentsAndActor(
-      configuration,
-      ResourceID.generate(),
-      system,
-      hostname,
-      Some(tmActorName),
-      Some(createLeaderRetrievalService()),
-      numTaskManagers == 1,
-      classOf[TestingTaskManager])
-  }
-
-
-  def createLeaderElectionService(): Option[LeaderElectionService] = {
-    None
   }
 
   @throws(classOf[TimeoutException])
@@ -228,4 +159,131 @@ class TestingCluster(
 
     Await.ready(combinedFuture, timeout)
   }
+
+  def waitForTaskManagersToBeRegisteredAtJobManager(jobManager: ActorRef): Unit = {
+    val futures = taskManagerActors.map {
+      _.map {
+              tm => (tm ? NotifyWhenRegisteredAtJobManager(jobManager))(timeout)
+            }
+    }.getOrElse(Seq())
+
+    try {
+      Await.ready(Future.sequence(futures), timeout)
+    } catch {
+      case t: TimeoutException =>
+        throw new Exception("Timeout while waiting for TaskManagers to register at " +
+                              s"${jobManager.path}")
+    }
+
+  }
+
+  def restartLeadingJobManager(): Unit = {
+    this.synchronized {
+      (jobManagerActorSystems, jobManagerActors) match {
+        case (Some(jmActorSystems), Some(jmActors)) =>
+          val leader = getLeaderGateway(AkkaUtils.getTimeout(originalConfiguration))
+          val index = getLeaderIndex(AkkaUtils.getTimeout(originalConfiguration))
+
+          // restart the leading job manager with the same port
+          val port = getLeaderRPCPort
+          val oldPort = originalConfiguration.getInteger(
+            ConfigConstants.JOB_MANAGER_IPC_PORT_KEY,
+            0)
+
+          // we have to set the old port in the configuration file because this is used for startup
+          originalConfiguration.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, port)
+
+          clearLeader()
+
+          val stopped = gracefulStop(leader.actor(), TestingCluster.MAX_RESTART_DURATION)
+          Await.result(stopped, TestingCluster.MAX_RESTART_DURATION)
+
+          if(!singleActorSystem) {
+            jmActorSystems(index).shutdown()
+            jmActorSystems(index).awaitTermination()
+          }
+
+          val newJobManagerActorSystem = if(!singleActorSystem) {
+            startJobManagerActorSystem(index)
+          } else {
+            jmActorSystems.head
+          }
+
+          // reset the original configuration
+          originalConfiguration.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, oldPort)
+
+          val newJobManagerActor = startJobManager(index, newJobManagerActorSystem)
+
+          jobManagerActors = Some(jmActors.patch(index, Seq(newJobManagerActor), 1))
+          jobManagerActorSystems = Some(jmActorSystems.patch(
+            index,
+            Seq(newJobManagerActorSystem),
+            1))
+
+          val lrs = createLeaderRetrievalService()
+
+          jobManagerLeaderRetrievalService = Some(lrs)
+          lrs.start(this)
+
+        case _ => throw new Exception("The JobManager of the TestingCluster have not " +
+                                        "been started properly.")
+      }
+    }
+  }
+
+  def restartTaskManager(index: Int): Unit = {
+    (taskManagerActorSystems, taskManagerActors) match {
+      case (Some(tmActorSystems), Some(tmActors)) =>
+        val stopped = gracefulStop(tmActors(index), TestingCluster.MAX_RESTART_DURATION)
+        Await.result(stopped, TestingCluster.MAX_RESTART_DURATION)
+
+        if(!singleActorSystem) {
+          tmActorSystems(index).shutdown()
+          tmActorSystems(index).awaitTermination()
+        }
+
+        val taskManagerActorSystem  = if(!singleActorSystem) {
+          startTaskManagerActorSystem(index)
+        } else {
+          tmActorSystems.head
+        }
+
+        val taskManagerActor = startTaskManager(index, taskManagerActorSystem)
+
+        taskManagerActors = Some(tmActors.patch(index, Seq(taskManagerActor), 1))
+        taskManagerActorSystems = Some(tmActorSystems.patch(index, Seq(taskManagerActorSystem), 1))
+
+      case _ => throw new Exception("The TaskManager of the TestingCluster have not " +
+                                      "been started properly.")
+    }
+  }
+
+  def addTaskManager(): Unit = {
+    if (useSingleActorSystem) {
+      (jobManagerActorSystems, taskManagerActors) match {
+        case (Some(jmSystems), Some(tmActors)) =>
+          val index = numTaskManagers
+          taskManagerActors = Some(tmActors :+ startTaskManager(index, jmSystems(0)))
+          numTaskManagers += 1
+        case _ => throw new IllegalStateException("Cluster has not been started properly.")
+      }
+    } else {
+      (taskManagerActorSystems, taskManagerActors) match {
+        case (Some(tmSystems), Some(tmActors)) =>
+          val index = numTaskManagers
+          val newTmSystem = startTaskManagerActorSystem(index)
+          val newTmActor = startTaskManager(index, newTmSystem)
+
+          taskManagerActorSystems = Some(tmSystems :+ newTmSystem)
+          taskManagerActors = Some(tmActors :+ newTmActor)
+
+          numTaskManagers += 1
+        case _ => throw new IllegalStateException("Cluster has not been started properly.")
+      }
+    }
+  }
+}
+
+object TestingCluster {
+  val MAX_RESTART_DURATION = new FiniteDuration(2, TimeUnit.MINUTES)
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/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
new file mode 100644
index 0000000..62349db
--- /dev/null
+++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManager.scala
@@ -0,0 +1,71 @@
+/*
+ * 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.testingUtils
+
+import java.util.concurrent.ExecutorService
+
+import akka.actor.ActorRef
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory
+import org.apache.flink.runtime.checkpoint.savepoint.SavepointStore
+import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager
+import org.apache.flink.runtime.executiongraph.restart.RestartStrategyFactory
+import org.apache.flink.runtime.instance.InstanceManager
+import org.apache.flink.runtime.jobmanager.scheduler.Scheduler
+import org.apache.flink.runtime.jobmanager.{JobManager, SubmittedJobGraphStore}
+import org.apache.flink.runtime.leaderelection.LeaderElectionService
+import org.apache.flink.runtime.metrics.MetricRegistry
+
+import scala.concurrent.duration._
+import scala.language.postfixOps
+
+/** JobManager implementation extended by testing messages
+  *
+  */
+class TestingJobManager(
+    flinkConfiguration: Configuration,
+    executorService: ExecutorService,
+    instanceManager: InstanceManager,
+    scheduler: Scheduler,
+    libraryCacheManager: BlobLibraryCacheManager,
+    archive: ActorRef,
+    restartStrategyFactory: RestartStrategyFactory,
+    timeout: FiniteDuration,
+    leaderElectionService: LeaderElectionService,
+    submittedJobGraphs : SubmittedJobGraphStore,
+    checkpointRecoveryFactory : CheckpointRecoveryFactory,
+    savepointStore : SavepointStore,
+    jobRecoveryTimeout : FiniteDuration,
+    metricRegistry : Option[MetricRegistry])
+  extends JobManager(
+    flinkConfiguration,
+      executorService,
+    instanceManager,
+    scheduler,
+    libraryCacheManager,
+    archive,
+    restartStrategyFactory,
+    timeout,
+    leaderElectionService,
+    submittedJobGraphs,
+    checkpointRecoveryFactory,
+    savepointStore,
+    jobRecoveryTimeout,
+    metricRegistry)
+  with TestingJobManagerLike {}

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManagerLike.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManagerLike.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManagerLike.scala
new file mode 100644
index 0000000..5ba2790
--- /dev/null
+++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManagerLike.scala
@@ -0,0 +1,417 @@
+/*
+ * 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.testingUtils
+
+import akka.actor.{ActorRef, Cancellable, Terminated}
+import akka.pattern.{ask, pipe}
+import org.apache.flink.api.common.JobID
+import org.apache.flink.runtime.FlinkActor
+import org.apache.flink.runtime.execution.ExecutionState
+import org.apache.flink.runtime.jobgraph.JobStatus
+import org.apache.flink.runtime.jobmanager.JobManager
+import org.apache.flink.runtime.messages.ExecutionGraphMessages.JobStatusChanged
+import org.apache.flink.runtime.messages.JobManagerMessages.{GrantLeadership, RegisterJobClient, RequestClassloadingProps}
+import org.apache.flink.runtime.messages.Messages.{Acknowledge, Disconnect}
+import org.apache.flink.runtime.messages.RegistrationMessages.RegisterTaskManager
+import org.apache.flink.runtime.messages.TaskManagerMessages.Heartbeat
+import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages._
+import org.apache.flink.runtime.testingUtils.TestingMessages._
+import org.apache.flink.runtime.testingUtils.TestingTaskManagerMessages.AccumulatorsChanged
+
+import scala.collection.mutable
+import scala.concurrent.Future
+import scala.concurrent.duration._
+import scala.language.postfixOps
+
+/** This mixin can be used to decorate a JobManager with messages for testing purpose.  */
+trait TestingJobManagerLike extends FlinkActor {
+  that: JobManager =>
+
+  import context._
+
+  import scala.collection.JavaConverters._
+
+  val waitForAllVerticesToBeRunning = scala.collection.mutable.HashMap[JobID, Set[ActorRef]]()
+  val waitForTaskManagerToBeTerminated = scala.collection.mutable.HashMap[String, Set[ActorRef]]()
+
+  val waitForAllVerticesToBeRunningOrFinished =
+    scala.collection.mutable.HashMap[JobID, Set[ActorRef]]()
+
+  var periodicCheck: Option[Cancellable] = None
+
+  val waitForJobStatus = scala.collection.mutable.HashMap[JobID,
+    collection.mutable.HashMap[JobStatus, Set[ActorRef]]]()
+
+  val waitForAccumulatorUpdate = scala.collection.mutable.HashMap[JobID, (Boolean, Set[ActorRef])]()
+
+  val waitForLeader = scala.collection.mutable.HashSet[ActorRef]()
+
+  val waitForNumRegisteredTaskManagers = mutable.PriorityQueue.newBuilder(
+    new Ordering[(Int, ActorRef)] {
+      override def compare(x: (Int, ActorRef), y: (Int, ActorRef)): Int = y._1 - x._1
+    })
+
+  val waitForClient = scala.collection.mutable.HashSet[ActorRef]()
+
+  val waitForShutdown = scala.collection.mutable.HashSet[ActorRef]()
+
+  var disconnectDisabled = false
+
+  var postStopEnabled = true
+
+  abstract override def postStop(): Unit = {
+    if (postStopEnabled) {
+      super.postStop()
+    } else {
+      // only stop leader election service to revoke the leadership of this JM so that a new JM
+      // can be elected leader
+      leaderElectionService.stop()
+    }
+  }
+
+  abstract override def handleMessage: Receive = {
+    handleTestingMessage orElse super.handleMessage
+  }
+
+  def handleTestingMessage: Receive = {
+    case Alive => sender() ! Acknowledge
+
+    case RequestExecutionGraph(jobID) =>
+      currentJobs.get(jobID) match {
+        case Some((executionGraph, jobInfo)) => sender() ! decorateMessage(
+          ExecutionGraphFound(
+            jobID,
+            executionGraph)
+        )
+
+        case None => archive.tell(decorateMessage(RequestExecutionGraph(jobID)), sender())
+      }
+
+    case WaitForAllVerticesToBeRunning(jobID) =>
+      if(checkIfAllVerticesRunning(jobID)){
+        sender() ! decorateMessage(AllVerticesRunning(jobID))
+      }else{
+        val waiting = waitForAllVerticesToBeRunning.getOrElse(jobID, Set[ActorRef]())
+        waitForAllVerticesToBeRunning += jobID -> (waiting + sender())
+
+        if(periodicCheck.isEmpty){
+          periodicCheck =
+            Some(
+              context.system.scheduler.schedule(
+                0 seconds,
+                200 millis,
+                self,
+                decorateMessage(NotifyListeners)
+              )
+            )
+        }
+      }
+    case WaitForAllVerticesToBeRunningOrFinished(jobID) =>
+      if(checkIfAllVerticesRunningOrFinished(jobID)){
+        sender() ! decorateMessage(AllVerticesRunning(jobID))
+      }else{
+        val waiting = waitForAllVerticesToBeRunningOrFinished.getOrElse(jobID, Set[ActorRef]())
+        waitForAllVerticesToBeRunningOrFinished += jobID -> (waiting + sender())
+
+        if(periodicCheck.isEmpty){
+          periodicCheck =
+            Some(
+              context.system.scheduler.schedule(
+                0 seconds,
+                200 millis,
+                self,
+                decorateMessage(NotifyListeners)
+              )
+            )
+        }
+      }
+
+    case NotifyListeners =>
+      for(jobID <- currentJobs.keySet){
+        notifyListeners(jobID)
+      }
+
+      if(waitForAllVerticesToBeRunning.isEmpty && waitForAllVerticesToBeRunningOrFinished.isEmpty) {
+        periodicCheck foreach { _.cancel() }
+        periodicCheck = None
+      }
+
+
+    case NotifyWhenJobRemoved(jobID) =>
+      val gateways = instanceManager.getAllRegisteredInstances.asScala.map(_.getActorGateway)
+
+      val responses = gateways.map{
+        gateway => gateway.ask(NotifyWhenJobRemoved(jobID), timeout).mapTo[Boolean]
+      }
+
+      val jobRemovedOnJobManager = (self ? CheckIfJobRemoved(jobID))(timeout).mapTo[Boolean]
+
+      val allFutures = responses ++ Seq(jobRemovedOnJobManager)
+
+      import context.dispatcher
+      Future.fold(allFutures)(true)(_ & _) map(decorateMessage(_)) pipeTo sender()
+
+    case CheckIfJobRemoved(jobID) =>
+      if(currentJobs.contains(jobID)) {
+        context.system.scheduler.scheduleOnce(
+          200 milliseconds,
+          self,
+          decorateMessage(CheckIfJobRemoved(jobID))
+        )(context.dispatcher, sender())
+      } else {
+        sender() ! decorateMessage(true)
+      }
+
+    case NotifyWhenTaskManagerTerminated(taskManager) =>
+      val waiting = waitForTaskManagerToBeTerminated.getOrElse(taskManager.path.name, Set())
+      waitForTaskManagerToBeTerminated += taskManager.path.name -> (waiting + sender)
+
+    case msg@Terminated(taskManager) =>
+      super.handleMessage(msg)
+
+      waitForTaskManagerToBeTerminated.remove(taskManager.path.name) foreach {
+        _ foreach {
+          listener =>
+            listener ! decorateMessage(TaskManagerTerminated(taskManager))
+        }
+      }
+
+    // see shutdown method for reply
+    case NotifyOfComponentShutdown =>
+      waitForShutdown += sender()
+
+    case NotifyWhenAccumulatorChange(jobID) =>
+
+      val (updated, registered) = waitForAccumulatorUpdate.
+        getOrElse(jobID, (false, Set[ActorRef]()))
+      waitForAccumulatorUpdate += jobID -> (updated, registered + sender)
+      sender ! true
+
+    /**
+     * Notification from the task manager that changed accumulator are transferred on next
+     * Hearbeat. We need to keep this state to notify the listeners on next Heartbeat report.
+     */
+    case AccumulatorsChanged(jobID: JobID) =>
+      waitForAccumulatorUpdate.get(jobID) match {
+        case Some((updated, registered)) =>
+          waitForAccumulatorUpdate.put(jobID, (true, registered))
+        case None =>
+      }
+
+    /**
+     * Disabled async processing of accumulator values and send accumulators to the listeners if
+     * we previously received an [[AccumulatorsChanged]] message.
+     */
+    case msg : Heartbeat =>
+      super.handleMessage(msg)
+
+      waitForAccumulatorUpdate foreach {
+        case (jobID, (updated, actors)) if updated =>
+          currentJobs.get(jobID) match {
+            case Some((graph, jobInfo)) =>
+              val flinkAccumulators = graph.getFlinkAccumulators
+              val userAccumulators = graph.aggregateUserAccumulators
+              actors foreach {
+                 actor => actor ! UpdatedAccumulators(jobID, flinkAccumulators, userAccumulators)
+              }
+            case None =>
+          }
+          waitForAccumulatorUpdate.put(jobID, (false, actors))
+        case _ =>
+      }
+
+    case RequestWorkingTaskManager(jobID) =>
+      currentJobs.get(jobID) match {
+        case Some((eg, _)) =>
+          if(eg.getAllExecutionVertices.asScala.isEmpty){
+            sender ! decorateMessage(WorkingTaskManager(None))
+          } else {
+            val resource = eg.getAllExecutionVertices.asScala.head.getCurrentAssignedResource
+
+            if(resource == null){
+              sender ! decorateMessage(WorkingTaskManager(None))
+            } else {
+              sender ! decorateMessage(
+                WorkingTaskManager(
+                  Some(resource.getTaskManagerActorGateway())
+                )
+              )
+            }
+          }
+        case None => sender ! decorateMessage(WorkingTaskManager(None))
+      }
+
+    case NotifyWhenJobStatus(jobID, state) =>
+      val jobStatusListener = waitForJobStatus.getOrElseUpdate(jobID,
+        scala.collection.mutable.HashMap[JobStatus, Set[ActorRef]]())
+
+      val listener = jobStatusListener.getOrElse(state, Set[ActorRef]())
+
+      jobStatusListener += state -> (listener + sender)
+
+    case msg@JobStatusChanged(jobID, newJobStatus, _, _) =>
+      super.handleMessage(msg)
+
+      val cleanup = waitForJobStatus.get(jobID) match {
+        case Some(stateListener) =>
+          stateListener.remove(newJobStatus) match {
+            case Some(listeners) =>
+              listeners foreach {
+                _ ! decorateMessage(JobStatusIs(jobID, newJobStatus))
+              }
+            case _ =>
+          }
+          stateListener.isEmpty
+
+        case _ => false
+      }
+
+      if (cleanup) {
+        waitForJobStatus.remove(jobID)
+      }
+
+    case DisableDisconnect =>
+      disconnectDisabled = true
+
+    case DisablePostStop =>
+      postStopEnabled = false
+
+    case RequestSavepoint(savepointPath) =>
+      try {
+        val savepoint = savepointStore.loadSavepoint(savepointPath)
+        sender ! ResponseSavepoint(savepoint)
+      }
+      catch {
+        case e: Exception =>
+          sender ! ResponseSavepoint(null)
+      }
+
+    case msg: Disconnect =>
+      if (!disconnectDisabled) {
+        super.handleMessage(msg)
+
+        val taskManager = sender()
+
+        waitForTaskManagerToBeTerminated.remove(taskManager.path.name) foreach {
+          _ foreach {
+            listener =>
+              listener ! decorateMessage(TaskManagerTerminated(taskManager))
+          }
+        }
+      }
+
+    case NotifyWhenLeader =>
+      if (leaderElectionService.hasLeadership) {
+        sender() ! true
+      } else {
+        waitForLeader += sender()
+      }
+
+    case msg: GrantLeadership =>
+      super.handleMessage(msg)
+
+      waitForLeader.foreach(_ ! true)
+
+      waitForLeader.clear()
+
+    case NotifyWhenClientConnects =>
+      waitForClient += sender()
+      sender() ! true
+
+    case msg: RegisterJobClient =>
+      super.handleMessage(msg)
+      waitForClient.foreach(_ ! ClientConnected)
+    case msg: RequestClassloadingProps =>
+      super.handleMessage(msg)
+      waitForClient.foreach(_ ! ClassLoadingPropsDelivered)
+
+    case NotifyWhenAtLeastNumTaskManagerAreRegistered(numRegisteredTaskManager) =>
+      if (that.instanceManager.getNumberOfRegisteredTaskManagers >= numRegisteredTaskManager) {
+        // there are already at least numRegisteredTaskManager registered --> send Acknowledge
+        sender() ! Acknowledge
+      } else {
+        // wait until we see at least numRegisteredTaskManager being registered at the JobManager
+        waitForNumRegisteredTaskManagers += ((numRegisteredTaskManager, sender()))
+      }
+
+    // TaskManager may be registered on these two messages
+    case msg @ (_: RegisterTaskManager) =>
+      super.handleMessage(msg)
+
+      // dequeue all senders which wait for instanceManager.getNumberOfStartedTaskManagers or
+      // fewer registered TaskManagers
+      while (waitForNumRegisteredTaskManagers.nonEmpty &&
+        waitForNumRegisteredTaskManagers.head._1 <=
+          instanceManager.getNumberOfRegisteredTaskManagers) {
+        val receiver = waitForNumRegisteredTaskManagers.dequeue()._2
+        receiver ! Acknowledge
+      }
+  }
+
+  def checkIfAllVerticesRunning(jobID: JobID): Boolean = {
+    currentJobs.get(jobID) match {
+      case Some((eg, _)) =>
+        eg.getAllExecutionVertices.asScala.forall( _.getExecutionState == ExecutionState.RUNNING)
+      case None => false
+    }
+  }
+
+  def checkIfAllVerticesRunningOrFinished(jobID: JobID): Boolean = {
+    currentJobs.get(jobID) match {
+      case Some((eg, _)) =>
+        eg.getAllExecutionVertices.asScala.forall {
+          case vertex =>
+            (vertex.getExecutionState == ExecutionState.RUNNING
+              || vertex.getExecutionState == ExecutionState.FINISHED)
+        }
+      case None => false
+    }
+  }
+
+  def notifyListeners(jobID: JobID): Unit = {
+    if(checkIfAllVerticesRunning(jobID)) {
+      waitForAllVerticesToBeRunning.remove(jobID) match {
+        case Some(listeners) =>
+          for (listener <- listeners) {
+            listener ! decorateMessage(AllVerticesRunning(jobID))
+          }
+        case _ =>
+      }
+    }
+
+    if(checkIfAllVerticesRunningOrFinished(jobID)) {
+      waitForAllVerticesToBeRunningOrFinished.remove(jobID) match {
+        case Some(listeners) =>
+          for (listener <- listeners) {
+            listener ! decorateMessage(AllVerticesRunning(jobID))
+          }
+        case _ =>
+      }
+    }
+  }
+
+  /**
+    * No killing of the VM for testing.
+    */
+  override protected def shutdown(): Unit = {
+    log.info("Shutting down TestingJobManager.")
+    waitForShutdown.foreach(_ ! ComponentShutdown(self))
+    waitForShutdown.clear()
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/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
new file mode 100644
index 0000000..d07c48f
--- /dev/null
+++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingJobManagerMessages.scala
@@ -0,0 +1,132 @@
+/*
+ * 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.testingUtils
+
+import java.util.Map
+
+import akka.actor.ActorRef
+import org.apache.flink.api.common.JobID
+import org.apache.flink.api.common.accumulators.Accumulator
+import org.apache.flink.runtime.accumulators.AccumulatorRegistry
+import org.apache.flink.runtime.checkpoint.savepoint.Savepoint
+import org.apache.flink.runtime.executiongraph.{ExecutionAttemptID, ExecutionGraph}
+import org.apache.flink.runtime.instance.ActorGateway
+import org.apache.flink.runtime.jobgraph.JobStatus
+
+object TestingJobManagerMessages {
+
+  case class RequestExecutionGraph(jobID: JobID)
+
+  sealed trait ResponseExecutionGraph {
+    def jobID: JobID
+  }
+
+  case class ExecutionGraphFound(jobID: JobID, executionGraph: ExecutionGraph) extends
+  ResponseExecutionGraph
+
+  case class ExecutionGraphNotFound(jobID: JobID) extends ResponseExecutionGraph
+
+  case class WaitForAllVerticesToBeRunning(jobID: JobID)
+  case class WaitForAllVerticesToBeRunningOrFinished(jobID: JobID)
+  case class AllVerticesRunning(jobID: JobID)
+
+  case class NotifyWhenJobRemoved(jobID: JobID)
+
+  case class RequestWorkingTaskManager(jobID: JobID)
+  case class WorkingTaskManager(gatewayOption: Option[ActorGateway])
+
+  case class NotifyWhenJobStatus(jobID: JobID, state: JobStatus)
+  case class JobStatusIs(jobID: JobID, state: JobStatus)
+
+  case object NotifyListeners
+
+  case class NotifyWhenTaskManagerTerminated(taskManager: ActorRef)
+  case class TaskManagerTerminated(taskManager: ActorRef)
+
+  /**
+   * Registers a listener to receive a message when accumulators changed.
+   * The change must be explicitly triggered by the TestingTaskManager which can receive an
+   * [[org.apache.flink.runtime.testingUtils.TestingTaskManagerMessages.AccumulatorsChanged]]
+   * message by a task that changed the accumulators. This message is then
+   * forwarded to the JobManager which will send the accumulators in the [[UpdatedAccumulators]]
+   * message when the next Heartbeat occurs.
+   */
+  case class NotifyWhenAccumulatorChange(jobID: JobID)
+
+  /**
+   * Reports updated accumulators back to the listener.
+   */
+  case class UpdatedAccumulators(jobID: JobID,
+    flinkAccumulators: Map[ExecutionAttemptID, Map[AccumulatorRegistry.Metric, Accumulator[_,_]]],
+    userAccumulators: Map[String, Accumulator[_,_]])
+
+  /** Notifies the sender when the [[TestingJobManager]] has been elected as the leader
+   *
+   */
+  case object NotifyWhenLeader
+
+  /**
+    * Notifies the sender when the [[TestingJobManager]] receives new clients for jobs
+    */
+  case object NotifyWhenClientConnects
+  /**
+    * Notifes of client connect
+    */
+  case object ClientConnected
+  /**
+    * Notifies when the client has requested class loading information
+    */
+  case object ClassLoadingPropsDelivered
+
+  /**
+   * Registers to be notified by an [[org.apache.flink.runtime.messages.Messages.Acknowledge]]
+   * message when at least numRegisteredTaskManager have registered at the JobManager.
+   *
+   * @param numRegisteredTaskManager minimum number of registered TMs before the sender is notified
+   */
+  case class NotifyWhenAtLeastNumTaskManagerAreRegistered(numRegisteredTaskManager: Int)
+
+  /** Disables the post stop method of the [[TestingJobManager]].
+    *
+    * Only the leaderElectionService is stopped in the postStop method call to revoke the leadership
+    */
+  case object DisablePostStop
+
+  /**
+    * Requests a savepoint from the job manager.
+    *
+    * @param savepointPath The path of the savepoint to request.
+    */
+  case class RequestSavepoint(savepointPath: String)
+
+  /**
+    * Response to a savepoint request.
+    *
+    * @param savepoint The requested savepoint or null if none available.
+    */
+  case class ResponseSavepoint(savepoint: Savepoint)
+
+  def getNotifyWhenLeader(): AnyRef = NotifyWhenLeader
+  def getNotifyWhenClientConnects(): AnyRef = NotifyWhenClientConnects
+  def getDisablePostStop(): AnyRef = DisablePostStop
+
+  def getClientConnected(): AnyRef = ClientConnected
+  def getClassLoadingPropsDelivered(): AnyRef = ClassLoadingPropsDelivered
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingMemoryArchivist.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingMemoryArchivist.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingMemoryArchivist.scala
new file mode 100644
index 0000000..48a1ddd
--- /dev/null
+++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingMemoryArchivist.scala
@@ -0,0 +1,43 @@
+/*
+ * 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.testingUtils
+
+import org.apache.flink.runtime.jobmanager.MemoryArchivist
+import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages.{ExecutionGraphFound, ExecutionGraphNotFound, RequestExecutionGraph}
+
+/** Memory archivist extended by testing messages
+  *
+  * @param maxEntries number of maximum number of archived jobs
+  */
+class TestingMemoryArchivist(maxEntries: Int) extends MemoryArchivist(maxEntries) {
+
+  override def handleMessage: Receive = {
+    handleTestingMessage orElse super.handleMessage
+  }
+
+  def handleTestingMessage: Receive = {
+    case RequestExecutionGraph(jobID) =>
+      val executionGraph = graphs.get(jobID)
+      
+      executionGraph match {
+        case Some(graph) => sender ! decorateMessage(ExecutionGraphFound(jobID, graph))
+        case None => sender ! decorateMessage(ExecutionGraphNotFound(jobID))
+      }
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingMessages.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingMessages.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingMessages.scala
new file mode 100644
index 0000000..91d169a
--- /dev/null
+++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingMessages.scala
@@ -0,0 +1,40 @@
+/*
+ * 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.testingUtils
+
+import akka.actor.ActorRef
+import org.apache.flink.api.common.JobID
+
+object TestingMessages {
+
+  case class CheckIfJobRemoved(jobID: JobID)
+
+  case object DisableDisconnect
+
+  case object Alive
+
+  def getAlive: AnyRef = Alive
+
+  def getDisableDisconnect: AnyRef = DisableDisconnect
+
+  case object NotifyOfComponentShutdown
+  case class ComponentShutdown(ref: ActorRef)
+
+  def getNotifyOfComponentShutdown(): AnyRef = NotifyOfComponentShutdown
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/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
new file mode 100644
index 0000000..9b5a147
--- /dev/null
+++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingTaskManager.scala
@@ -0,0 +1,70 @@
+/*
+ * 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.testingUtils
+
+import org.apache.flink.runtime.clusterframework.types.ResourceID
+import org.apache.flink.runtime.io.disk.iomanager.IOManager
+import org.apache.flink.runtime.io.network.NetworkEnvironment
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService
+import org.apache.flink.runtime.memory.MemoryManager
+import org.apache.flink.runtime.taskmanager.{TaskManagerLocation, TaskManager, TaskManagerConfiguration}
+
+import scala.language.postfixOps
+
+/** Subclass of the [[TaskManager]] to support testing messages
+ */
+class TestingTaskManager(
+                          config: TaskManagerConfiguration,
+                          resourceID: ResourceID,
+                          connectionInfo: TaskManagerLocation,
+                          memoryManager: MemoryManager,
+                          ioManager: IOManager,
+                          network: NetworkEnvironment,
+                          numberOfSlots: Int,
+                          leaderRetrievalService: LeaderRetrievalService)
+  extends TaskManager(
+    config,
+    resourceID,
+    connectionInfo,
+    memoryManager,
+    ioManager,
+    network,
+    numberOfSlots,
+    leaderRetrievalService)
+  with TestingTaskManagerLike {
+
+  def this(
+            config: TaskManagerConfiguration,
+            connectionInfo: TaskManagerLocation,
+            memoryManager: MemoryManager,
+            ioManager: IOManager,
+            network: NetworkEnvironment,
+            numberOfSlots: Int,
+            leaderRetrievalService: LeaderRetrievalService) {
+    this(
+      config,
+      ResourceID.generate(),
+      connectionInfo,
+      memoryManager,
+      ioManager,
+      network,
+      numberOfSlots,
+      leaderRetrievalService)
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingTaskManagerLike.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingTaskManagerLike.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingTaskManagerLike.scala
new file mode 100644
index 0000000..2498dbe
--- /dev/null
+++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingTaskManagerLike.scala
@@ -0,0 +1,234 @@
+/*
+ * 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.testingUtils
+
+import akka.actor.{ActorRef, Terminated}
+import org.apache.flink.api.common.JobID
+import org.apache.flink.runtime.FlinkActor
+import org.apache.flink.runtime.execution.ExecutionState
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID
+import org.apache.flink.runtime.messages.JobManagerMessages.{RequestLeaderSessionID, ResponseLeaderSessionID}
+import org.apache.flink.runtime.messages.Messages.{Acknowledge, Disconnect}
+import org.apache.flink.runtime.messages.RegistrationMessages.{AcknowledgeRegistration, AlreadyRegistered}
+import org.apache.flink.runtime.messages.TaskMessages.{SubmitTask, TaskInFinalState, UpdateTaskExecutionState}
+import org.apache.flink.runtime.taskmanager.TaskManager
+import org.apache.flink.runtime.testingUtils.TestingJobManagerMessages.NotifyWhenJobRemoved
+import org.apache.flink.runtime.testingUtils.TestingMessages._
+import org.apache.flink.runtime.testingUtils.TestingTaskManagerMessages._
+
+import scala.concurrent.duration._
+import scala.language.postfixOps
+
+/** This mixin can be used to decorate a TaskManager with messages for testing purposes. */
+trait TestingTaskManagerLike extends FlinkActor {
+  that: TaskManager =>
+
+  import scala.collection.JavaConverters._
+
+  val waitForRemoval = scala.collection.mutable.HashMap[ExecutionAttemptID, Set[ActorRef]]()
+  val waitForJobManagerToBeTerminated = scala.collection.mutable.HashMap[String, Set[ActorRef]]()
+  val waitForRegisteredAtResourceManager =
+    scala.collection.mutable.HashMap[ActorRef, Set[ActorRef]]()
+  val waitForRunning = scala.collection.mutable.HashMap[ExecutionAttemptID, Set[ActorRef]]()
+  val unregisteredTasks = scala.collection.mutable.HashSet[ExecutionAttemptID]()
+
+  /** Map of registered task submit listeners */
+  val registeredSubmitTaskListeners = scala.collection.mutable.HashMap[JobID, ActorRef]()
+
+  val waitForShutdown = scala.collection.mutable.HashSet[ActorRef]()
+
+  var disconnectDisabled = false
+
+  /**
+   * Handler for testing related messages
+   */
+  abstract override def handleMessage: Receive = {
+    handleTestingMessage orElse super.handleMessage
+  }
+
+  def handleTestingMessage: Receive = {
+    case Alive => sender() ! Acknowledge
+
+    case NotifyWhenTaskIsRunning(executionID) =>
+      Option(runningTasks.get(executionID)) match {
+        case Some(task) if task.getExecutionState == ExecutionState.RUNNING =>
+          sender ! decorateMessage(true)
+
+        case _ =>
+          val listeners = waitForRunning.getOrElse(executionID, Set())
+          waitForRunning += (executionID -> (listeners + sender))
+      }
+
+    case RequestRunningTasks =>
+      sender ! decorateMessage(ResponseRunningTasks(runningTasks.asScala.toMap))
+
+    case NotifyWhenTaskRemoved(executionID) =>
+      Option(runningTasks.get(executionID)) match {
+        case Some(_) =>
+          val set = waitForRemoval.getOrElse(executionID, Set())
+          waitForRemoval += (executionID -> (set + sender))
+        case None =>
+          if(unregisteredTasks.contains(executionID)) {
+            sender ! decorateMessage(true)
+          } else {
+            val set = waitForRemoval.getOrElse(executionID, Set())
+            waitForRemoval += (executionID -> (set + sender))
+          }
+      }
+
+    case TaskInFinalState(executionID) =>
+      super.handleMessage(TaskInFinalState(executionID))
+      waitForRemoval.remove(executionID) match {
+        case Some(actors) => for(actor <- actors) actor ! decorateMessage(true)
+        case None =>
+      }
+
+      unregisteredTasks += executionID
+
+    case NotifyWhenJobRemoved(jobID) =>
+      if(runningTasks.values.asScala.exists(_.getJobID == jobID)){
+        context.system.scheduler.scheduleOnce(
+          200 milliseconds,
+          self,
+          decorateMessage(CheckIfJobRemoved(jobID)))(
+            context.dispatcher,
+            sender()
+          )
+      }else{
+        sender ! decorateMessage(true)
+      }
+
+    case CheckIfJobRemoved(jobID) =>
+      if(runningTasks.values.asScala.forall(_.getJobID != jobID)){
+        sender ! decorateMessage(true)
+      } else {
+        context.system.scheduler.scheduleOnce(
+          200 milliseconds,
+          self,
+          decorateMessage(CheckIfJobRemoved(jobID)))(
+            context.dispatcher,
+            sender()
+          )
+      }
+
+    case NotifyWhenJobManagerTerminated(jobManager) =>
+      val waiting = waitForJobManagerToBeTerminated.getOrElse(jobManager.path.name, Set())
+      waitForJobManagerToBeTerminated += jobManager.path.name -> (waiting + sender)
+
+    case RegisterSubmitTaskListener(jobId) =>
+      registeredSubmitTaskListeners.put(jobId, sender())
+
+    case msg@SubmitTask(tdd) =>
+      registeredSubmitTaskListeners.get(tdd.getJobID) match {
+        case Some(listenerRef) =>
+          listenerRef ! ResponseSubmitTaskListener(tdd)
+        case None =>
+        // Nothing to do
+      }
+
+      super.handleMessage(msg)
+
+    /**
+     * Message from task manager that accumulator values changed and need to be reported immediately
+     * instead of lazily through the
+     * [[org.apache.flink.runtime.messages.TaskManagerMessages.Heartbeat]] message. We forward this
+     * message to the job manager that it knows it should report to the listeners.
+     */
+    case msg: AccumulatorsChanged =>
+      currentJobManager match {
+        case Some(jobManager) =>
+          jobManager.forward(msg)
+          sendHeartbeatToJobManager()
+          sender ! true
+        case None =>
+      }
+
+    case msg@Terminated(jobManager) =>
+      super.handleMessage(msg)
+
+      waitForJobManagerToBeTerminated.remove(jobManager.path.name) foreach {
+        _ foreach {
+          _ ! decorateMessage(JobManagerTerminated(jobManager))
+        }
+      }
+
+    case msg:Disconnect =>
+      if (!disconnectDisabled) {
+        super.handleMessage(msg)
+
+        val jobManager = sender()
+
+        waitForJobManagerToBeTerminated.remove(jobManager.path.name) foreach {
+          _ foreach {
+            _ ! decorateMessage(JobManagerTerminated(jobManager))
+          }
+        }
+      }
+
+    case DisableDisconnect =>
+      disconnectDisabled = true
+
+    case NotifyOfComponentShutdown =>
+      waitForShutdown += sender()
+
+    case msg @ UpdateTaskExecutionState(taskExecutionState) =>
+      super.handleMessage(msg)
+
+      if(taskExecutionState.getExecutionState == ExecutionState.RUNNING) {
+        waitForRunning.get(taskExecutionState.getID) foreach {
+          _ foreach (_ ! decorateMessage(true))
+        }
+      }
+
+    case RequestLeaderSessionID =>
+      sender() ! ResponseLeaderSessionID(leaderSessionID.orNull)
+
+    case NotifyWhenRegisteredAtJobManager(jobManager: ActorRef) =>
+      if(isConnected && jobManager == currentJobManager.get) {
+        sender() ! true
+      } else {
+        val list = waitForRegisteredAtResourceManager.getOrElse(
+          jobManager,
+          Set[ActorRef]())
+
+        waitForRegisteredAtResourceManager += jobManager -> (list + sender())
+      }
+
+    case msg @ (_: AcknowledgeRegistration | _: AlreadyRegistered) =>
+      super.handleMessage(msg)
+
+      val jm = sender()
+
+      waitForRegisteredAtResourceManager.remove(jm).foreach {
+        listeners => listeners.foreach{
+          listener =>
+            listener ! true
+        }
+      }
+  }
+
+  /**
+    * No killing of the VM for testing.
+    */
+  override protected def shutdown(): Unit = {
+    log.info("Shutting down TestingJobManager.")
+    waitForShutdown.foreach(_ ! ComponentShutdown(self))
+    waitForShutdown.clear()
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/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
new file mode 100644
index 0000000..32c3c55
--- /dev/null
+++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingTaskManagerMessages.scala
@@ -0,0 +1,82 @@
+/*
+ * 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.testingUtils
+
+import akka.actor.ActorRef
+import org.apache.flink.api.common.JobID
+import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID
+import org.apache.flink.runtime.taskmanager.Task
+
+/**
+ * Additional messages that the [[TestingTaskManager]] understands.
+ */
+object TestingTaskManagerMessages {
+  
+  case class NotifyWhenTaskRemoved(executionID: ExecutionAttemptID)
+
+  case class NotifyWhenTaskIsRunning(executionID: ExecutionAttemptID)
+  
+  case class ResponseRunningTasks(tasks: Map[ExecutionAttemptID, Task]){
+    import collection.JavaConverters._
+    def asJava: java.util.Map[ExecutionAttemptID, Task] = tasks.asJava
+  }
+  
+  case object RequestRunningTasks
+
+  case class NotifyWhenJobManagerTerminated(jobManager: ActorRef)
+
+  case class JobManagerTerminated(jobManager: ActorRef)
+
+  case class NotifyWhenRegisteredAtJobManager(resourceManager: ActorRef)
+
+  /**
+   * Message to give a hint to the task manager that accumulator values were updated in the task.
+   * This message is forwarded to the job manager which knows that it needs to notify listeners
+   * of accumulator updates.
+   */
+  case class AccumulatorsChanged(jobID: JobID)
+
+  /**
+    * Registers a listener for all [[org.apache.flink.runtime.messages.TaskMessages.SubmitTask]]
+    * messages of the given job.
+    *
+    * If a task is submitted with the given job ID the task deployment
+    * descriptor is forwarded to the listener.
+    *
+    * @param jobId The job ID to listen for.
+    */
+  case class RegisterSubmitTaskListener(jobId: JobID)
+
+  /**
+    * A response to a listened job ID containing the submitted task deployment descriptor.
+    *
+    * @param tdd The submitted task deployment descriptor.
+    */
+  case class ResponseSubmitTaskListener(tdd: TaskDeploymentDescriptor)
+
+  // --------------------------------------------------------------------------
+  // Utility methods to allow simpler case object access from Java
+  // --------------------------------------------------------------------------
+
+  def getRequestRunningTasksMessage: AnyRef = {
+    RequestRunningTasks
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/flink-scala-shell/src/test/scala/org/apache/flink/api/scala/ScalaShellITCase.scala
----------------------------------------------------------------------
diff --git a/flink-scala-shell/src/test/scala/org/apache/flink/api/scala/ScalaShellITCase.scala b/flink-scala-shell/src/test/scala/org/apache/flink/api/scala/ScalaShellITCase.scala
index ee1b264..00410cc 100644
--- a/flink-scala-shell/src/test/scala/org/apache/flink/api/scala/ScalaShellITCase.scala
+++ b/flink-scala-shell/src/test/scala/org/apache/flink/api/scala/ScalaShellITCase.scala
@@ -22,9 +22,10 @@ import java.io._
 import java.util.concurrent.TimeUnit
 
 import org.apache.flink.configuration.GlobalConfiguration
-import org.apache.flink.test.util.{ForkableFlinkMiniCluster, TestBaseUtils}
+import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster
+import org.apache.flink.test.util.TestBaseUtils
 import org.apache.flink.util.TestLogger
-import org.junit.{AfterClass, BeforeClass, Test, Assert}
+import org.junit.{AfterClass, Assert, BeforeClass, Test}
 
 import scala.concurrent.duration.FiniteDuration
 import scala.tools.nsc.Settings
@@ -334,7 +335,7 @@ class ScalaShellITCase extends TestLogger {
 }
 
 object ScalaShellITCase {
-  var cluster: Option[ForkableFlinkMiniCluster] = None
+  var cluster: Option[LocalFlinkMiniCluster] = None
   val parallelism = 4
 
   @BeforeClass

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/flink-streaming-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/CassandraConnectorITCase.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/CassandraConnectorITCase.java b/flink-streaming-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/CassandraConnectorITCase.java
index 8bb440c..f94ff68 100644
--- a/flink-streaming-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/CassandraConnectorITCase.java
+++ b/flink-streaming-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/CassandraConnectorITCase.java
@@ -40,6 +40,7 @@ import org.apache.flink.batch.connectors.cassandra.CassandraOutputFormat;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter;
+import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
 import org.apache.flink.runtime.testutils.CommonTestUtils;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.datastream.DataStreamSource;
@@ -49,7 +50,6 @@ import org.apache.flink.streaming.runtime.operators.WriteAheadSinkTestBase;
 
 import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
 import org.apache.flink.streaming.util.TestStreamEnvironment;
-import org.apache.flink.test.util.ForkableFlinkMiniCluster;
 import org.apache.flink.test.util.TestEnvironment;
 import org.junit.After;
 import org.junit.AfterClass;
@@ -134,7 +134,7 @@ public class CassandraConnectorITCase extends WriteAheadSinkTestBase<Tuple3<Stri
 		}
 	}
 
-	private static ForkableFlinkMiniCluster flinkCluster;
+	private static LocalFlinkMiniCluster flinkCluster;
 
 	// ------------------------------------------------------------------------
 	//  Cluster Setup (Cassandra & Flink)
@@ -205,7 +205,7 @@ public class CassandraConnectorITCase extends WriteAheadSinkTestBase<Tuple3<Stri
 		Configuration config = new Configuration();
 		config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 4);
 
-		flinkCluster = new ForkableFlinkMiniCluster(config);
+		flinkCluster = new LocalFlinkMiniCluster(config);
 		flinkCluster.start();
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetentionTestBase.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetentionTestBase.java b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetentionTestBase.java
index 9e3c33b..c4949ff 100644
--- a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetentionTestBase.java
+++ b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetentionTestBase.java
@@ -22,6 +22,7 @@ import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.java.typeutils.TypeInfoParser;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.datastream.DataStreamSource;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
@@ -30,7 +31,6 @@ import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunctio
 import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema;
 import org.apache.flink.streaming.util.serialization.KeyedSerializationSchemaWrapper;
 import org.apache.flink.streaming.util.serialization.SimpleStringSchema;
-import org.apache.flink.test.util.ForkableFlinkMiniCluster;
 import org.apache.flink.util.InstantiationUtil;
 
 import org.junit.AfterClass;
@@ -58,7 +58,7 @@ public class KafkaShortRetentionTestBase implements Serializable {
 	
 	private static KafkaTestEnvironment kafkaServer;
 	private static Properties standardProps;
-	private static ForkableFlinkMiniCluster flink;
+	private static LocalFlinkMiniCluster flink;
 
 	@BeforeClass
 	public static void prepare() throws IOException, ClassNotFoundException {
@@ -88,7 +88,7 @@ public class KafkaShortRetentionTestBase implements Serializable {
 		flinkConfig.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 16);
 		flinkConfig.setString(ConfigConstants.RESTART_STRATEGY_FIXED_DELAY_DELAY, "0 s");
 
-		flink = new ForkableFlinkMiniCluster(flinkConfig, false);
+		flink = new LocalFlinkMiniCluster(flinkConfig, false);
 		flink.start();
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java
index eddb57c..771db17 100644
--- a/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java
+++ b/flink-streaming-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java
@@ -22,8 +22,8 @@ import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.metrics.jmx.JMXReporter;
 import org.apache.flink.runtime.client.JobExecutionException;
+import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.test.util.ForkableFlinkMiniCluster;
 import org.apache.flink.test.util.SuccessException;
 import org.apache.flink.util.InstantiationUtil;
 import org.apache.flink.util.TestLogger;
@@ -65,7 +65,7 @@ public abstract class KafkaTestBase extends TestLogger {
 
 	protected static Properties standardProps;
 	
-	protected static ForkableFlinkMiniCluster flink;
+	protected static LocalFlinkMiniCluster flink;
 
 	protected static int flinkPort;
 
@@ -105,7 +105,7 @@ public abstract class KafkaTestBase extends TestLogger {
 		flinkConfig.setString(ConfigConstants.METRICS_REPORTERS_LIST, "my_reporter");
 		flinkConfig.setString(ConfigConstants.METRICS_REPORTER_PREFIX + "my_reporter." + ConfigConstants.METRICS_REPORTER_CLASS_SUFFIX, JMXReporter.class.getName());
 
-		flink = new ForkableFlinkMiniCluster(flinkConfig, false);
+		flink = new LocalFlinkMiniCluster(flinkConfig, false);
 		flink.start();
 
 		flinkPort = flink.getLeaderRPCPort();

http://git-wip-us.apache.org/repos/asf/flink/blob/02b852e3/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualExactlyOnceTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualExactlyOnceTest.java b/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualExactlyOnceTest.java
index 3705943..2e452c1 100644
--- a/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualExactlyOnceTest.java
+++ b/flink-streaming-connectors/flink-connector-kinesis/src/test/java/org/apache/flink/streaming/connectors/kinesis/manualtests/ManualExactlyOnceTest.java
@@ -21,11 +21,11 @@ import com.amazonaws.services.kinesis.model.DescribeStreamResult;
 import org.apache.flink.api.java.utils.ParameterTool;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.minicluster.LocalFlinkMiniCluster;
 import org.apache.flink.streaming.connectors.kinesis.config.AWSConfigConstants;
 import org.apache.flink.streaming.connectors.kinesis.testutils.ExactlyOnceValidatingConsumerThread;
 import org.apache.flink.streaming.connectors.kinesis.testutils.KinesisEventsGeneratorProducerThread;
 import org.apache.flink.streaming.connectors.kinesis.util.AWSUtil;
-import org.apache.flink.test.util.ForkableFlinkMiniCluster;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -80,7 +80,7 @@ public class ManualExactlyOnceTest {
 		flinkConfig.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 16);
 		flinkConfig.setString(ConfigConstants.RESTART_STRATEGY_FIXED_DELAY_DELAY, "0 s");
 
-		ForkableFlinkMiniCluster flink = new ForkableFlinkMiniCluster(flinkConfig, false);
+		LocalFlinkMiniCluster flink = new LocalFlinkMiniCluster(flinkConfig, false);
 		flink.start();
 
 		final int flinkPort = flink.getLeaderRPCPort();


[37/50] [abbrv] flink git commit: [FLINK-4443] [rpc] Add support for rpc gateway and rpc endpoint inheritance

Posted by tr...@apache.org.
[FLINK-4443] [rpc] Add support for rpc gateway and rpc endpoint inheritance

This commit extends the RpcCompletenessTest such that it can now check for inherited
remote procedure calls. All methods defined at the RpcGateway are considered native.
This means that they need no RpcEndpoint counterpart because they are implemented by
the RpcGateway implementation.

This closes #2401.

update comments

remove native method annotation

add line break


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

Branch: refs/heads/flip-6
Commit: 048248988ccc55db4a793d16f34f73427bf7cdea
Parents: 81a35c1
Author: wenlong.lwl <we...@alibaba-inc.com>
Authored: Sun Aug 21 00:46:51 2016 +0800
Committer: Till Rohrmann <tr...@apache.org>
Committed: Thu Sep 8 17:26:59 2016 +0200

----------------------------------------------------------------------
 .../org/apache/flink/runtime/rpc/RpcMethod.java |  2 ++
 .../TestingHighAvailabilityServices.java        | 19 +++++++++++
 .../flink/runtime/rpc/RpcCompletenessTest.java  | 33 ++++++++++++++++++--
 3 files changed, 52 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/04824898/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcMethod.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcMethod.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcMethod.java
index 875e557..e4b0e94 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcMethod.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcMethod.java
@@ -19,6 +19,7 @@
 package org.apache.flink.runtime.rpc;
 
 import java.lang.annotation.ElementType;
+import java.lang.annotation.Inherited;
 import java.lang.annotation.Retention;
 import java.lang.annotation.RetentionPolicy;
 import java.lang.annotation.Target;
@@ -29,6 +30,7 @@ import java.lang.annotation.Target;
  * RpcCompletenessTest makes sure that the set of rpc methods in a rpc server and the set of
  * gateway methods in the corresponding gateway implementation are identical.
  */
+@Inherited
 @Target(ElementType.METHOD)
 @Retention(RetentionPolicy.RUNTIME)
 public @interface RpcMethod {

http://git-wip-us.apache.org/repos/asf/flink/blob/04824898/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingHighAvailabilityServices.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingHighAvailabilityServices.java b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingHighAvailabilityServices.java
index 3a9f943..4d654a3 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingHighAvailabilityServices.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingHighAvailabilityServices.java
@@ -18,6 +18,8 @@
 
 package org.apache.flink.runtime.highavailability;
 
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
 
 /**
@@ -28,6 +30,8 @@ public class TestingHighAvailabilityServices implements HighAvailabilityServices
 
 	private volatile LeaderRetrievalService resourceManagerLeaderRetriever;
 
+	private volatile LeaderElectionService jobMasterLeaderElectionService;
+
 
 	// ------------------------------------------------------------------------
 	//  Setters for mock / testing implementations
@@ -36,6 +40,10 @@ public class TestingHighAvailabilityServices implements HighAvailabilityServices
 	public void setResourceManagerLeaderRetriever(LeaderRetrievalService resourceManagerLeaderRetriever) {
 		this.resourceManagerLeaderRetriever = resourceManagerLeaderRetriever;
 	}
+
+	public void setJobMasterLeaderElectionService(LeaderElectionService leaderElectionService) {
+		this.jobMasterLeaderElectionService = leaderElectionService;
+	}
 	
 	// ------------------------------------------------------------------------
 	//  HA Services Methods
@@ -50,4 +58,15 @@ public class TestingHighAvailabilityServices implements HighAvailabilityServices
 			throw new IllegalStateException("ResourceManagerLeaderRetriever has not been set");
 		}
 	}
+
+	@Override
+	public LeaderElectionService getJobMasterLeaderElectionService(JobID jobID) throws Exception {
+		LeaderElectionService service = jobMasterLeaderElectionService;
+
+		if (service != null) {
+			return service;
+		} else {
+			throw new IllegalStateException("JobMasterLeaderElectionService has not been set");
+		}
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/04824898/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java
index b8aad62..b431eb9 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java
@@ -68,8 +68,8 @@ public class RpcCompletenessTest extends TestLogger {
 
 	@SuppressWarnings("rawtypes")
 	private void checkCompleteness(Class<? extends RpcEndpoint> rpcEndpoint, Class<? extends RpcGateway> rpcGateway) {
-		Method[] gatewayMethods = rpcGateway.getDeclaredMethods();
-		Method[] serverMethods = rpcEndpoint.getDeclaredMethods();
+		Method[] gatewayMethods = getRpcMethodsFromGateway(rpcGateway).toArray(new Method[0]);
+		Method[] serverMethods = rpcEndpoint.getMethods();
 
 		Map<String, Set<Method>> rpcMethods = new HashMap<>();
 		Set<Method> unmatchedRpcMethods = new HashSet<>();
@@ -340,4 +340,33 @@ public class RpcCompletenessTest extends TestLogger {
 			throw new RuntimeException("Could not retrive basic type information for primitive type " + primitveType + '.');
 		}
 	}
+
+	/**
+	 * Extract all rpc methods defined by the gateway interface
+	 *
+	 * @param interfaceClass the given rpc gateway interface
+	 * @return all methods defined by the given interface
+	 */
+	private List<Method> getRpcMethodsFromGateway(Class<? extends RpcGateway> interfaceClass) {
+		if(!interfaceClass.isInterface()) {
+			fail(interfaceClass.getName() + "is not a interface");
+		}
+
+		ArrayList<Method> allMethods = new ArrayList<>();
+		// Methods defined in RpcGateway are native method
+		if(interfaceClass.equals(RpcGateway.class)) {
+			return allMethods;
+		}
+
+		// Get all methods declared in current interface
+		for(Method method : interfaceClass.getDeclaredMethods()) {
+			allMethods.add(method);
+		}
+
+		// Get all method inherited from super interface
+		for(Class superClass : interfaceClass.getInterfaces()) {
+			allMethods.addAll(getRpcMethodsFromGateway(superClass));
+		}
+		return allMethods;
+	}
 }


[08/50] [abbrv] flink git commit: [FLINK-4073] fix stability of TaskManagerFailure test

Posted by tr...@apache.org.
[FLINK-4073] fix stability of TaskManagerFailure test


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

Branch: refs/heads/flip-6
Commit: 2f87f61d34414074bc09ba8584d345bd400ed3cd
Parents: 9bbb8fa
Author: Maximilian Michels <mx...@apache.org>
Authored: Mon Sep 5 16:05:50 2016 +0200
Committer: Maximilian Michels <mx...@apache.org>
Committed: Mon Sep 5 16:14:03 2016 +0200

----------------------------------------------------------------------
 .../yarn/YARNSessionCapacitySchedulerITCase.java    | 16 +++++++++++-----
 1 file changed, 11 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/2f87f61d/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java
----------------------------------------------------------------------
diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java
index 513a9fc..886ec32 100644
--- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java
+++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java
@@ -29,12 +29,13 @@ import org.apache.flink.test.testdata.WordCountData;
 import org.apache.flink.test.util.TestBaseUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.protocolrecords.StopContainersRequest;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.NodeReport;
-import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.api.records.NodeState;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.client.api.YarnClient;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.security.NMTokenIdentifier;
@@ -249,11 +250,16 @@ public class YARNSessionCapacitySchedulerITCase extends YarnTestBase {
 			List<NodeReport> nodeReports = yc.getNodeReports(NodeState.RUNNING);
 
 			// we asked for one node with 2 vcores so we expect 2 vcores
-			int userVcores = 0;
+			// note that the JobManager may also run on the NodeManager
+			boolean foundVCoresSetting = false;
 			for (NodeReport rep: nodeReports) {
-				userVcores += rep.getUsed().getVirtualCores();
+				Resource resource = rep.getUsed();
+				if (resource != null && resource.getVirtualCores() == 2) {
+					foundVCoresSetting = true;
+					break;
+				}
 			}
-			Assert.assertEquals(2, userVcores);
+			Assert.assertTrue(foundVCoresSetting);
 		} catch (Exception e) {
 			Assert.fail("Test failed: " + e.getMessage());
 		}


[02/50] [abbrv] flink git commit: [FLINK-4567] [runtime] Enhance SerializedThrowable to properly mimic Exception causes

Posted by tr...@apache.org.
[FLINK-4567] [runtime] Enhance SerializedThrowable to properly mimic Exception causes


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

Branch: refs/heads/flip-6
Commit: 761d0a02505c7eaef7a566f978145b187c89cbf8
Parents: c251efc
Author: Stephan Ewen <se...@apache.org>
Authored: Fri Sep 2 11:38:53 2016 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Sep 2 17:32:57 2016 +0200

----------------------------------------------------------------------
 .../flink/runtime/util/SerializedThrowable.java | 88 ++++++++++----------
 .../runtime/util/SerializedThrowableTest.java   | 40 ++++++++-
 2 files changed, 83 insertions(+), 45 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/761d0a02/flink-runtime/src/main/java/org/apache/flink/runtime/util/SerializedThrowable.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/SerializedThrowable.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/SerializedThrowable.java
index a7739ef..4dea59c 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/SerializedThrowable.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/SerializedThrowable.java
@@ -21,18 +21,19 @@ package org.apache.flink.runtime.util;
 import org.apache.flink.util.ExceptionUtils;
 import org.apache.flink.util.InstantiationUtil;
 
-import java.io.IOException;
 import java.io.PrintStream;
 import java.io.PrintWriter;
 import java.io.Serializable;
 import java.lang.ref.WeakReference;
+import java.util.HashSet;
+import java.util.Set;
 
 /**
  * Utility class for dealing with user-defined Throwable types that are serialized (for
  * example during RPC/Actor communication), but cannot be resolved with the default
  * class loader.
- * <p>
- * This exception mimics the original exception with respect to message and stack trace,
+ * 
+ * <p>This exception mimics the original exception with respect to message and stack trace,
  * and contains the original exception in serialized form. The original exception
  * can be re-obtained by supplying the appropriate class loader.
  */
@@ -49,10 +50,6 @@ public class SerializedThrowable extends Exception implements Serializable {
 	/** The original stack trace, to be printed */
 	private final String fullStingifiedStackTrace;
 
-	/** A guaranteed serializable placeholder exception that will be used as
-	 * cause and to capture the original stack trace */
-	private final Exception placeholder;
-	
 	/** The original exception, not transported via serialization, 
 	 * because the class may not be part of the system class loader.
 	 * In addition, we make sure our cached references to not prevent
@@ -66,33 +63,43 @@ public class SerializedThrowable extends Exception implements Serializable {
 	 * @param exception The exception to serialize.
 	 */
 	public SerializedThrowable(Throwable exception) {
+		this(exception, new HashSet<Throwable>());
+	}
+
+	private SerializedThrowable(Throwable exception, Set<Throwable> alreadySeen) {
 		super(getMessageOrError(exception));
 
 		if (!(exception instanceof SerializedThrowable)) {
-			this.cachedException = new WeakReference<Throwable>(exception);
-			
-			this.originalErrorClassName = exception.getClass().getName();
-			this.fullStingifiedStackTrace = ExceptionUtils.stringifyException(exception);
-			this.placeholder = new Exception(
-					"Serialized representation of " + originalErrorClassName + ": " + getMessage());
-			this.placeholder.setStackTrace(exception.getStackTrace());
-			initCause(this.placeholder);
-			
+			// serialize and memoize the original message
 			byte[] serialized;
 			try {
 				serialized = InstantiationUtil.serializeObject(exception);
 			}
 			catch (Throwable t) {
-				// could not serialize exception. send the stringified version instead
-				try {
-					serialized = InstantiationUtil.serializeObject(placeholder);
-				}
-				catch (IOException e) {
-					// this should really never happen, as we only serialize a a standard exception
-					throw new RuntimeException(e.getMessage(), e);
-				}
+				serialized = null;
 			}
 			this.serializedException = serialized;
+			this.cachedException = new WeakReference<Throwable>(exception);
+
+			// record the original exception's properties (name, stack prints)
+			this.originalErrorClassName = exception.getClass().getName();
+			this.fullStingifiedStackTrace = ExceptionUtils.stringifyException(exception);
+
+			// mimic the original exception's stack trace
+			setStackTrace(exception.getStackTrace());
+
+			// mimic the original exception's cause
+			if (exception.getCause() == null) {
+				initCause(null);
+			}
+			else {
+				// exception causes may by cyclic, so we truncate the cycle when we find it 
+				if (alreadySeen.add(exception)) {
+					// we are not in a cycle, yet
+					initCause(new SerializedThrowable(exception.getCause(), alreadySeen));
+				}
+			}
+
 		}
 		else {
 			// copy from that serialized throwable
@@ -100,39 +107,37 @@ public class SerializedThrowable extends Exception implements Serializable {
 			this.serializedException = other.serializedException;
 			this.originalErrorClassName = other.originalErrorClassName;
 			this.fullStingifiedStackTrace = other.fullStingifiedStackTrace;
-			this.placeholder = other.placeholder;
 			this.cachedException = other.cachedException;
 		}
 	}
 
 	public Throwable deserializeError(ClassLoader classloader) {
+		if (serializedException == null) {
+			// failed to serialize the original exception
+			// return this SerializedThrowable as a stand in
+			return this;
+		}
+
 		Throwable cached = cachedException == null ? null : cachedException.get();
 		if (cached == null) {
 			try {
 				cached = InstantiationUtil.deserializeObject(serializedException, classloader);
 				cachedException = new WeakReference<Throwable>(cached);
 			}
-			catch (Exception e) {
-				return placeholder;
+			catch (Throwable t) {
+				// something went wrong
+				// return this SerializedThrowable as a stand in
+				return this;
 			}
 		}
 		return cached;
 	}
-	
-	public String getStrigifiedStackTrace() {
-		return fullStingifiedStackTrace;
-	}
-	
+
 	// ------------------------------------------------------------------------
 	//  Override the behavior of Throwable
 	// ------------------------------------------------------------------------
 
 	@Override
-	public Throwable getCause() {
-		return placeholder;
-	}
-
-	@Override
 	public void printStackTrace(PrintStream s) {
 		s.print(fullStingifiedStackTrace);
 		s.flush();
@@ -150,15 +155,10 @@ public class SerializedThrowable extends Exception implements Serializable {
 		return (message != null) ? (originalErrorClassName + ": " + message) : originalErrorClassName;
 	}
 
-	@Override
-	public StackTraceElement[] getStackTrace() {
-		return placeholder.getStackTrace();
-	}
-
 	// ------------------------------------------------------------------------
 	//  Static utilities
 	// ------------------------------------------------------------------------
-	
+
 	public static Throwable get(Throwable serThrowable, ClassLoader loader) {
 		if (serThrowable instanceof SerializedThrowable) {
 			return ((SerializedThrowable)serThrowable).deserializeError(loader);
@@ -166,7 +166,7 @@ public class SerializedThrowable extends Exception implements Serializable {
 			return serThrowable;
 		}
 	}
-	
+
 	private static String getMessageOrError(Throwable error) {
 		try {
 			return error.getMessage();

http://git-wip-us.apache.org/repos/asf/flink/blob/761d0a02/flink-runtime/src/test/java/org/apache/flink/runtime/util/SerializedThrowableTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/SerializedThrowableTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/SerializedThrowableTest.java
index 50efd52..4d57892 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/util/SerializedThrowableTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/SerializedThrowableTest.java
@@ -131,10 +131,48 @@ public class SerializedThrowableTest {
 			// deserialize the proper exception
 			Throwable deserialized = copy.deserializeError(loader); 
 			assertEquals(clazz, deserialized.getClass());
+
+			// deserialization with the wrong classloader does not lead to a failure
+			Throwable wronglyDeserialized = copy.deserializeError(getClass().getClassLoader());
+			assertEquals(ExceptionUtils.stringifyException(userException),
+					ExceptionUtils.stringifyException(wronglyDeserialized));
 		}
 		catch (Exception e) {
 			e.printStackTrace();
 			fail(e.getMessage());
 		}
-	} 
+	}
+
+	@Test
+	public void testCauseChaining() {
+		Exception cause2 = new Exception("level2");
+		Exception cause1 = new Exception("level1", cause2);
+		Exception root = new Exception("level0", cause1);
+
+		SerializedThrowable st = new SerializedThrowable(root);
+
+		assertEquals("level0", st.getMessage());
+
+		assertNotNull(st.getCause());
+		assertEquals("level1", st.getCause().getMessage());
+
+		assertNotNull(st.getCause().getCause());
+		assertEquals("level2", st.getCause().getCause().getMessage());
+	}
+
+	@Test
+	public void testCyclicCauseChaining() {
+		Exception cause3 = new Exception("level3");
+		Exception cause2 = new Exception("level2", cause3);
+		Exception cause1 = new Exception("level1", cause2);
+		Exception root = new Exception("level0", cause1);
+
+		// introduce a cyclic reference
+		cause3.initCause(cause1);
+
+		SerializedThrowable st = new SerializedThrowable(root);
+
+		assertArrayEquals(root.getStackTrace(), st.getStackTrace());
+		assertEquals(ExceptionUtils.stringifyException(root), ExceptionUtils.stringifyException(st));
+	}
 }


[31/50] [abbrv] flink git commit: [FLINK-4382] [rpc] Buffer rpc calls until the RpcEndpoint has been started

Posted by tr...@apache.org.
[FLINK-4382] [rpc] Buffer rpc calls until the RpcEndpoint has been started

This PR allows the AkkaRpcActor to stash messages until the corresponding RcpEndpoint
has been started. When receiving a Processing.START message, the AkkaRpcActor
unstashes all messages and starts processing rpcs. When receiving a Processing.STOP
message, it will stop processing messages and stash incoming messages again.

Add test case for message stashing

This closes #2358.


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

Branch: refs/heads/flip-6
Commit: 2c54a61b7e7f6c96834a7f41c39b4a3e4231b19a
Parents: e966f82
Author: Till Rohrmann <tr...@apache.org>
Authored: Thu Aug 11 18:13:25 2016 +0200
Committer: Till Rohrmann <tr...@apache.org>
Committed: Thu Sep 8 17:26:58 2016 +0200

----------------------------------------------------------------------
 .../apache/flink/runtime/rpc/RpcEndpoint.java   |  15 ++-
 .../flink/runtime/rpc/StartStoppable.java       |  35 ++++++
 .../runtime/rpc/akka/AkkaInvocationHandler.java |  21 +++-
 .../flink/runtime/rpc/akka/AkkaRpcActor.java    |  39 ++++++-
 .../flink/runtime/rpc/akka/AkkaRpcService.java  |   8 +-
 .../runtime/rpc/akka/messages/Processing.java   |  27 +++++
 .../flink/runtime/rpc/RpcCompletenessTest.java  |  45 +++++++-
 .../runtime/rpc/akka/AkkaRpcActorTest.java      | 108 +++++++++++++++++++
 .../runtime/rpc/akka/AkkaRpcServiceTest.java    |   3 +
 .../flink/runtime/rpc/akka/AsyncCallsTest.java  |   5 +-
 .../rpc/akka/MainThreadValidationTest.java      |   4 +-
 .../rpc/akka/MessageSerializationTest.java      |   4 +
 .../rpc/taskexecutor/TaskExecutorTest.java      |  18 ++++
 13 files changed, 315 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/2c54a61b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java
index d36a283..67ac182 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java
@@ -74,7 +74,7 @@ public abstract class RpcEndpoint<C extends RpcGateway> {
 
 	/** The main thread execution context to be used to execute future callbacks in the main thread
 	 * of the executing rpc server. */
-	private final MainThreadExecutionContext mainThreadExecutionContext;
+	private final ExecutionContext mainThreadExecutionContext;
 
 	/** A reference to the endpoint's main thread, if the current method is called by the main thread */
 	final AtomicReference<Thread> currentMainThread = new AtomicReference<>(null); 
@@ -106,10 +106,21 @@ public abstract class RpcEndpoint<C extends RpcGateway> {
 	}
 	
 	// ------------------------------------------------------------------------
-	//  Shutdown
+	//  Start & Shutdown
 	// ------------------------------------------------------------------------
 
 	/**
+	 * Starts the rpc endpoint. This tells the underlying rpc server that the rpc endpoint is ready
+	 * to process remote procedure calls.
+	 *
+	 * IMPORTANT: Whenever you override this method, call the parent implementation to enable
+	 * rpc processing. It is advised to make the parent call last.
+	 */
+	public void start() {
+		((StartStoppable) self).start();
+	}
+
+	/**
 	 * Shuts down the underlying RPC endpoint via the RPC service.
 	 * After this method was called, the RPC endpoint will no longer be reachable, neither remotely,
 	 * not via its {@link #getSelf() self gateway}. It will also not accepts executions in main thread

http://git-wip-us.apache.org/repos/asf/flink/blob/2c54a61b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/StartStoppable.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/StartStoppable.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/StartStoppable.java
new file mode 100644
index 0000000..dd5595f
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/StartStoppable.java
@@ -0,0 +1,35 @@
+/*
+ * 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.rpc;
+
+/**
+ * Interface to start and stop the processing of rpc calls in the rpc server.
+ */
+public interface StartStoppable {
+
+	/**
+	 * Starts the processing of remote procedure calls.
+	 */
+	void start();
+
+	/**
+	 * Stops the processing of remote procedure calls.
+	 */
+	void stop();
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/2c54a61b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaInvocationHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaInvocationHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaInvocationHandler.java
index 297104b..524bf74 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaInvocationHandler.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaInvocationHandler.java
@@ -24,8 +24,10 @@ import akka.util.Timeout;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.runtime.rpc.MainThreadExecutor;
 import org.apache.flink.runtime.rpc.RpcTimeout;
+import org.apache.flink.runtime.rpc.StartStoppable;
 import org.apache.flink.runtime.rpc.akka.messages.CallAsync;
 import org.apache.flink.runtime.rpc.akka.messages.LocalRpcInvocation;
+import org.apache.flink.runtime.rpc.akka.messages.Processing;
 import org.apache.flink.runtime.rpc.akka.messages.RemoteRpcInvocation;
 import org.apache.flink.runtime.rpc.akka.messages.RpcInvocation;
 import org.apache.flink.runtime.rpc.akka.messages.RunAsync;
@@ -50,7 +52,7 @@ import static org.apache.flink.util.Preconditions.checkArgument;
  * rpc in a {@link LocalRpcInvocation} message and then sends it to the {@link AkkaRpcActor} where it is
  * executed.
  */
-class AkkaInvocationHandler implements InvocationHandler, AkkaGateway, MainThreadExecutor {
+class AkkaInvocationHandler implements InvocationHandler, AkkaGateway, MainThreadExecutor, StartStoppable {
 	private static final Logger LOG = Logger.getLogger(AkkaInvocationHandler.class);
 
 	private final ActorRef rpcEndpoint;
@@ -76,7 +78,8 @@ class AkkaInvocationHandler implements InvocationHandler, AkkaGateway, MainThrea
 
 		Object result;
 
-		if (declaringClass.equals(AkkaGateway.class) || declaringClass.equals(MainThreadExecutor.class) || declaringClass.equals(Object.class)) {
+		if (declaringClass.equals(AkkaGateway.class) || declaringClass.equals(MainThreadExecutor.class) ||
+			declaringClass.equals(Object.class) || declaringClass.equals(StartStoppable.class)) {
 			result = method.invoke(this, args);
 		} else {
 			String methodName = method.getName();
@@ -171,6 +174,20 @@ class AkkaInvocationHandler implements InvocationHandler, AkkaGateway, MainThrea
 		}
 	}
 
+	@Override
+	public void start() {
+		rpcEndpoint.tell(Processing.START, ActorRef.noSender());
+	}
+
+	@Override
+	public void stop() {
+		rpcEndpoint.tell(Processing.STOP, ActorRef.noSender());
+	}
+
+	// ------------------------------------------------------------------------
+	//  Helper methods
+	// ------------------------------------------------------------------------
+
 	/**
 	 * Extracts the {@link RpcTimeout} annotated rpc timeout value from the list of given method
 	 * arguments. If no {@link RpcTimeout} annotated parameter could be found, then the default

http://git-wip-us.apache.org/repos/asf/flink/blob/2c54a61b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java
index dfcbcc3..2373be9 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java
@@ -20,13 +20,15 @@ package org.apache.flink.runtime.rpc.akka;
 
 import akka.actor.ActorRef;
 import akka.actor.Status;
-import akka.actor.UntypedActor;
+import akka.actor.UntypedActorWithStash;
+import akka.japi.Procedure;
 import akka.pattern.Patterns;
 import org.apache.flink.runtime.rpc.MainThreadValidatorUtil;
 import org.apache.flink.runtime.rpc.RpcEndpoint;
 import org.apache.flink.runtime.rpc.RpcGateway;
 import org.apache.flink.runtime.rpc.akka.messages.CallAsync;
 import org.apache.flink.runtime.rpc.akka.messages.LocalRpcInvocation;
+import org.apache.flink.runtime.rpc.akka.messages.Processing;
 import org.apache.flink.runtime.rpc.akka.messages.RpcInvocation;
 import org.apache.flink.runtime.rpc.akka.messages.RunAsync;
 
@@ -45,18 +47,23 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**
  * Akka rpc actor which receives {@link LocalRpcInvocation}, {@link RunAsync} and {@link CallAsync}
- * messages.
+ * {@link Processing} messages.
  * <p>
  * The {@link LocalRpcInvocation} designates a rpc and is dispatched to the given {@link RpcEndpoint}
  * instance.
  * <p>
  * The {@link RunAsync} and {@link CallAsync} messages contain executable code which is executed
  * in the context of the actor thread.
+ * <p>
+ * The {@link Processing} message controls the processing behaviour of the akka rpc actor. A
+ * {@link Processing#START} message unstashes all stashed messages and starts processing incoming
+ * messages. A {@link Processing#STOP} message stops processing messages and stashes incoming
+ * messages.
  *
  * @param <C> Type of the {@link RpcGateway} associated with the {@link RpcEndpoint}
  * @param <T> Type of the {@link RpcEndpoint}
  */
-class AkkaRpcActor<C extends RpcGateway, T extends RpcEndpoint<C>> extends UntypedActor {
+class AkkaRpcActor<C extends RpcGateway, T extends RpcEndpoint<C>> extends UntypedActorWithStash {
 	
 	private static final Logger LOG = LoggerFactory.getLogger(AkkaRpcActor.class);
 
@@ -73,6 +80,27 @@ class AkkaRpcActor<C extends RpcGateway, T extends RpcEndpoint<C>> extends Untyp
 
 	@Override
 	public void onReceive(final Object message) {
+		if (message.equals(Processing.START)) {
+			unstashAll();
+			getContext().become(new Procedure<Object>() {
+				@Override
+				public void apply(Object message) throws Exception {
+					if (message.equals(Processing.STOP)) {
+						getContext().unbecome();
+					} else {
+						handleMessage(message);
+					}
+				}
+			});
+		} else {
+			LOG.info("The rpc endpoint {} has not been started yet. Stashing message {} until processing is started.",
+				rpcEndpoint.getClass().getName(),
+				message.getClass().getName());
+			stash();
+		}
+	}
+
+	private void handleMessage(Object message) {
 		mainThreadValidator.enterMainThread();
 		try {
 			if (message instanceof RunAsync) {
@@ -82,7 +110,10 @@ class AkkaRpcActor<C extends RpcGateway, T extends RpcEndpoint<C>> extends Untyp
 			} else if (message instanceof RpcInvocation) {
 				handleRpcInvocation((RpcInvocation) message);
 			} else {
-				LOG.warn("Received message of unknown type {}. Dropping this message!", message.getClass());
+				LOG.warn(
+					"Received message of unknown type {} with value {}. Dropping this message!",
+					message.getClass().getName(),
+					message);
 			}
 		} finally {
 			mainThreadValidator.exitMainThread();

http://git-wip-us.apache.org/repos/asf/flink/blob/2c54a61b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java
index b963c53..7b33524 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java
@@ -34,7 +34,7 @@ import org.apache.flink.runtime.rpc.MainThreadExecutor;
 import org.apache.flink.runtime.rpc.RpcGateway;
 import org.apache.flink.runtime.rpc.RpcEndpoint;
 import org.apache.flink.runtime.rpc.RpcService;
-
+import org.apache.flink.runtime.rpc.StartStoppable;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -136,7 +136,11 @@ public class AkkaRpcService implements RpcService {
 		@SuppressWarnings("unchecked")
 		C self = (C) Proxy.newProxyInstance(
 			classLoader,
-			new Class<?>[]{rpcEndpoint.getSelfGatewayType(), MainThreadExecutor.class, AkkaGateway.class},
+			new Class<?>[]{
+				rpcEndpoint.getSelfGatewayType(),
+				MainThreadExecutor.class,
+				StartStoppable.class,
+				AkkaGateway.class},
 			akkaInvocationHandler);
 
 		return self;

http://git-wip-us.apache.org/repos/asf/flink/blob/2c54a61b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/Processing.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/Processing.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/Processing.java
new file mode 100644
index 0000000..5c7df5d
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/Processing.java
@@ -0,0 +1,27 @@
+/*
+ * 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.rpc.akka.messages;
+
+/**
+ * Controls the processing behaviour of the {@link org.apache.flink.runtime.rpc.akka.AkkaRpcActor}
+ */
+public enum Processing {
+	START, // Unstashes all stashed messages and starts processing incoming messages
+	STOP // Stop processing messages and stashes all incoming messages
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/2c54a61b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java
index e50533e..97cf0cb 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java
@@ -18,6 +18,8 @@
 
 package org.apache.flink.runtime.rpc;
 
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.util.ReflectionUtil;
 import org.apache.flink.util.TestLogger;
 import org.junit.Test;
@@ -140,7 +142,7 @@ public class RpcCompletenessTest extends TestLogger {
 		int rpcTimeoutParameters = 0;
 
 		for (int i = 0; i < parameterAnnotations.length; i++) {
-			if (isRpcTimeout(parameterAnnotations[i])) {
+			if (RpcCompletenessTest.isRpcTimeout(parameterAnnotations[i])) {
 				assertTrue(
 					"The rpc timeout has to be of type " + FiniteDuration.class.getName() + ".",
 					parameterTypes[i].equals(FiniteDuration.class));
@@ -185,7 +187,7 @@ public class RpcCompletenessTest extends TestLogger {
 
 		// filter out the RpcTimeout parameters
 		for (int i = 0; i < gatewayParameterTypes.length; i++) {
-			if (!isRpcTimeout(gatewayParameterAnnotations[i])) {
+			if (!RpcCompletenessTest.isRpcTimeout(gatewayParameterAnnotations[i])) {
 				filteredGatewayParameterTypes.add(gatewayParameterTypes[i]);
 			}
 		}
@@ -235,7 +237,22 @@ public class RpcCompletenessTest extends TestLogger {
 	}
 
 	private boolean checkType(Class<?> firstType, Class<?> secondType) {
-		return firstType.equals(secondType);
+		Class<?> firstResolvedType;
+		Class<?> secondResolvedType;
+
+		if (firstType.isPrimitive()) {
+			firstResolvedType = RpcCompletenessTest.resolvePrimitiveType(firstType);
+		} else {
+			firstResolvedType = firstType;
+		}
+
+		if (secondType.isPrimitive()) {
+			secondResolvedType = RpcCompletenessTest.resolvePrimitiveType(secondType);
+		} else {
+			secondResolvedType = secondType;
+		}
+
+		return firstResolvedType.equals(secondResolvedType);
 	}
 
 	/**
@@ -279,7 +296,7 @@ public class RpcCompletenessTest extends TestLogger {
 
 		for (int i = 0; i < parameterTypes.length; i++) {
 			// filter out the RpcTimeout parameters
-			if (!isRpcTimeout(parameterAnnotations[i])) {
+			if (!RpcCompletenessTest.isRpcTimeout(parameterAnnotations[i])) {
 				builder.append(parameterTypes[i].getName());
 
 				if (i < parameterTypes.length -1) {
@@ -293,7 +310,7 @@ public class RpcCompletenessTest extends TestLogger {
 		return builder.toString();
 	}
 
-	private boolean isRpcTimeout(Annotation[] annotations) {
+	private static boolean isRpcTimeout(Annotation[] annotations) {
 		for (Annotation annotation : annotations) {
 			if (annotation.annotationType().equals(RpcTimeout.class)) {
 				return true;
@@ -302,4 +319,22 @@ public class RpcCompletenessTest extends TestLogger {
 
 		return false;
 	}
+
+	/**
+	 * Returns the boxed type for a primitive type.
+	 *
+	 * @param primitveType Primitive type to resolve
+	 * @return Boxed type for the given primitive type
+	 */
+	private static Class<?> resolvePrimitiveType(Class<?> primitveType) {
+		assert primitveType.isPrimitive();
+
+		TypeInformation<?> typeInformation = BasicTypeInfo.getInfoFor(primitveType);
+
+		if (typeInformation != null) {
+			return typeInformation.getTypeClass();
+		} else {
+			throw new RuntimeException("Could not retrive basic type information for primitive type " + primitveType + '.');
+		}
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/2c54a61b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java
new file mode 100644
index 0000000..1653fac
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActorTest.java
@@ -0,0 +1,108 @@
+/*
+ * 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.rpc.akka;
+
+import akka.actor.ActorSystem;
+import akka.util.Timeout;
+import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.rpc.RpcEndpoint;
+import org.apache.flink.runtime.rpc.RpcGateway;
+import org.apache.flink.runtime.rpc.RpcMethod;
+import org.apache.flink.runtime.rpc.RpcService;
+import org.apache.flink.util.TestLogger;
+import org.hamcrest.core.Is;
+import org.junit.AfterClass;
+import org.junit.Test;
+import scala.concurrent.Await;
+import scala.concurrent.Future;
+
+import java.util.concurrent.TimeUnit;
+
+import static org.junit.Assert.assertThat;
+
+public class AkkaRpcActorTest extends TestLogger {
+
+	// ------------------------------------------------------------------------
+	//  shared test members
+	// ------------------------------------------------------------------------
+
+	private static ActorSystem actorSystem = AkkaUtils.createDefaultActorSystem();
+
+	private static Timeout timeout = new Timeout(10000, TimeUnit.MILLISECONDS);
+
+	private static AkkaRpcService akkaRpcService =
+		new AkkaRpcService(actorSystem, timeout);
+
+	@AfterClass
+	public static void shutdown() {
+		akkaRpcService.stopService();
+		actorSystem.shutdown();
+		actorSystem.awaitTermination();
+	}
+
+	/**
+	 * Tests that the {@link AkkaRpcActor} stashes messages until the corresponding
+	 * {@link RpcEndpoint} has been started.
+	 */
+	@Test
+	public void testMessageStashing() throws Exception {
+		int expectedValue = 1337;
+
+		DummyRpcEndpoint rpcEndpoint = new DummyRpcEndpoint(akkaRpcService);
+
+		DummyRpcGateway rpcGateway = rpcEndpoint.getSelf();
+
+		// this message should not be processed until we've started the rpc endpoint
+		Future<Integer> result = rpcGateway.foobar();
+
+		// set a new value which we expect to be returned
+		rpcEndpoint.setFoobar(expectedValue);
+
+		// now process the rpc
+		rpcEndpoint.start();
+
+		Integer actualValue = Await.result(result, timeout.duration());
+
+		assertThat("The new foobar value should have been returned.", actualValue, Is.is(expectedValue));
+
+		rpcEndpoint.shutDown();
+	}
+
+	private interface DummyRpcGateway extends RpcGateway {
+		Future<Integer> foobar();
+	}
+
+	private static class DummyRpcEndpoint extends RpcEndpoint<DummyRpcGateway> {
+
+		private volatile int _foobar = 42;
+
+		protected DummyRpcEndpoint(RpcService rpcService) {
+			super(rpcService);
+		}
+
+		@RpcMethod
+		public int foobar() {
+			return _foobar;
+		}
+
+		public void setFoobar(int value) {
+			_foobar = value;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/2c54a61b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java
index f26b40b..fd55904 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java
@@ -57,6 +57,9 @@ public class AkkaRpcServiceTest extends TestLogger {
 		ResourceManager resourceManager = new ResourceManager(akkaRpcService, executorService);
 		JobMaster jobMaster = new JobMaster(akkaRpcService2, executorService);
 
+		resourceManager.start();
+		jobMaster.start();
+
 		ResourceManagerGateway rm = resourceManager.getSelf();
 
 		assertTrue(rm instanceof AkkaGateway);

http://git-wip-us.apache.org/repos/asf/flink/blob/2c54a61b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AsyncCallsTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AsyncCallsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AsyncCallsTest.java
index f2ce52d..d33987c 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AsyncCallsTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AsyncCallsTest.java
@@ -28,6 +28,7 @@ import org.apache.flink.runtime.rpc.RpcGateway;
 import org.apache.flink.runtime.rpc.RpcMethod;
 import org.apache.flink.runtime.rpc.RpcService;
 
+import org.apache.flink.util.TestLogger;
 import org.junit.AfterClass;
 import org.junit.Test;
 
@@ -42,7 +43,7 @@ import java.util.concurrent.locks.ReentrantLock;
 
 import static org.junit.Assert.*;
 
-public class AsyncCallsTest {
+public class AsyncCallsTest extends TestLogger {
 
 	// ------------------------------------------------------------------------
 	//  shared test members
@@ -72,6 +73,7 @@ public class AsyncCallsTest {
 		final AtomicBoolean concurrentAccess = new AtomicBoolean(false);
 
 		TestEndpoint testEndpoint = new TestEndpoint(akkaRpcService, lock);
+		testEndpoint.start();
 		TestGateway gateway = testEndpoint.getSelf();
 
 		// a bunch of gateway calls
@@ -127,6 +129,7 @@ public class AsyncCallsTest {
 		final long delay = 200;
 
 		TestEndpoint testEndpoint = new TestEndpoint(akkaRpcService, lock);
+		testEndpoint.start();
 
 		// run something asynchronously
 		testEndpoint.runAsync(new Runnable() {

http://git-wip-us.apache.org/repos/asf/flink/blob/2c54a61b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MainThreadValidationTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MainThreadValidationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MainThreadValidationTest.java
index b854143..9ffafda 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MainThreadValidationTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MainThreadValidationTest.java
@@ -27,13 +27,14 @@ import org.apache.flink.runtime.rpc.RpcGateway;
 import org.apache.flink.runtime.rpc.RpcMethod;
 import org.apache.flink.runtime.rpc.RpcService;
 
+import org.apache.flink.util.TestLogger;
 import org.junit.Test;
 
 import java.util.concurrent.TimeUnit;
 
 import static org.junit.Assert.assertTrue;
 
-public class MainThreadValidationTest {
+public class MainThreadValidationTest extends TestLogger {
 
 	@Test
 	public void failIfNotInMainThread() {
@@ -51,6 +52,7 @@ public class MainThreadValidationTest {
 
 		try {
 			TestEndpoint testEndpoint = new TestEndpoint(akkaRpcService);
+			testEndpoint.start();
 
 			// this works, because it is executed as an RPC call
 			testEndpoint.getSelf().someConcurrencyCriticalFunction();

http://git-wip-us.apache.org/repos/asf/flink/blob/2c54a61b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MessageSerializationTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MessageSerializationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MessageSerializationTest.java
index ca8179c..9d2ed99 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MessageSerializationTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MessageSerializationTest.java
@@ -86,6 +86,7 @@ public class MessageSerializationTest extends TestLogger {
 	public void testNonSerializableLocalMessageTransfer() throws InterruptedException, IOException {
 		LinkedBlockingQueue<Object> linkedBlockingQueue = new LinkedBlockingQueue<>();
 		TestEndpoint testEndpoint = new TestEndpoint(akkaRpcService1, linkedBlockingQueue);
+		testEndpoint.start();
 
 		TestGateway testGateway = testEndpoint.getSelf();
 
@@ -106,6 +107,7 @@ public class MessageSerializationTest extends TestLogger {
 		LinkedBlockingQueue<Object> linkedBlockingQueue = new LinkedBlockingQueue<>();
 
 		TestEndpoint testEndpoint = new TestEndpoint(akkaRpcService1, linkedBlockingQueue);
+		testEndpoint.start();
 
 		String address = testEndpoint.getAddress();
 
@@ -126,6 +128,7 @@ public class MessageSerializationTest extends TestLogger {
 		LinkedBlockingQueue<Object> linkedBlockingQueue = new LinkedBlockingQueue<>();
 
 		TestEndpoint testEndpoint = new TestEndpoint(akkaRpcService1, linkedBlockingQueue);
+		testEndpoint.start();
 
 		String address = testEndpoint.getAddress();
 
@@ -149,6 +152,7 @@ public class MessageSerializationTest extends TestLogger {
 		LinkedBlockingQueue<Object> linkedBlockingQueue = new LinkedBlockingQueue<>();
 
 		TestEndpoint testEndpoint = new TestEndpoint(akkaRpcService1, linkedBlockingQueue);
+		testEndpoint.start();
 
 		String address = testEndpoint.getAddress();
 

http://git-wip-us.apache.org/repos/asf/flink/blob/2c54a61b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorTest.java
index 33c9cb6..c96f4f6 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorTest.java
@@ -28,17 +28,26 @@ import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.rpc.MainThreadExecutor;
+import org.apache.flink.runtime.rpc.RpcEndpoint;
+import org.apache.flink.runtime.rpc.RpcGateway;
 import org.apache.flink.runtime.rpc.RpcService;
+import org.apache.flink.runtime.rpc.StartStoppable;
 import org.apache.flink.runtime.util.DirectExecutorService;
 import org.apache.flink.util.SerializedValue;
 import org.apache.flink.util.TestLogger;
 import org.junit.Test;
+import org.mockito.Matchers;
+import org.mockito.cglib.proxy.InvocationHandler;
+import org.mockito.cglib.proxy.Proxy;
+import scala.concurrent.Future;
 
 import java.net.URL;
 import java.util.Collections;
 
 import static org.junit.Assert.fail;
 import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
 
 public class TaskExecutorTest extends TestLogger {
 
@@ -48,8 +57,13 @@ public class TaskExecutorTest extends TestLogger {
 	@Test
 	public void testTaskExecution() throws Exception {
 		RpcService testingRpcService = mock(RpcService.class);
+		InvocationHandler invocationHandler = mock(InvocationHandler.class);
+		Object selfGateway = Proxy.newProxyInstance(ClassLoader.getSystemClassLoader(), new Class<?>[] {TaskExecutorGateway.class, MainThreadExecutor.class, StartStoppable.class}, invocationHandler);
+		when(testingRpcService.startServer(Matchers.any(RpcEndpoint.class))).thenReturn((RpcGateway)selfGateway);
+
 		DirectExecutorService directExecutorService = new DirectExecutorService();
 		TaskExecutor taskExecutor = new TaskExecutor(testingRpcService, directExecutorService);
+		taskExecutor.start();
 
 		TaskDeploymentDescriptor tdd = new TaskDeploymentDescriptor(
 			new JobID(),
@@ -82,8 +96,12 @@ public class TaskExecutorTest extends TestLogger {
 	@Test(expected=Exception.class)
 	public void testWrongTaskCancellation() throws Exception {
 		RpcService testingRpcService = mock(RpcService.class);
+		InvocationHandler invocationHandler = mock(InvocationHandler.class);
+		Object selfGateway = Proxy.newProxyInstance(ClassLoader.getSystemClassLoader(), new Class<?>[] {TaskExecutorGateway.class, MainThreadExecutor.class, StartStoppable.class}, invocationHandler);
+		when(testingRpcService.startServer(Matchers.any(RpcEndpoint.class))).thenReturn((RpcGateway)selfGateway);
 		DirectExecutorService directExecutorService = null;
 		TaskExecutor taskExecutor = new TaskExecutor(testingRpcService, directExecutorService);
+		taskExecutor.start();
 
 		taskExecutor.cancelTask(new ExecutionAttemptID());
 


[25/50] [abbrv] flink git commit: [FLINK-4386] [rpc] Add a utility to verify calls happen in the Rpc Endpoint's main thread

Posted by tr...@apache.org.
[FLINK-4386] [rpc] Add a utility to verify calls happen in the Rpc Endpoint's main thread


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

Branch: refs/heads/flip-6
Commit: 2963e6c127ad3cb4eaf2007dd9d4aaa4a1bfd9f7
Parents: 518fb68
Author: Stephan Ewen <se...@apache.org>
Authored: Thu Aug 11 20:30:54 2016 +0200
Committer: Till Rohrmann <tr...@apache.org>
Committed: Thu Sep 8 17:26:57 2016 +0200

----------------------------------------------------------------------
 .../flink/runtime/rpc/MainThreadExecutor.java   |  2 +-
 .../runtime/rpc/MainThreadValidatorUtil.java    | 47 ++++++++++
 .../apache/flink/runtime/rpc/RpcEndpoint.java   | 38 +++++++-
 .../flink/runtime/rpc/akka/AkkaRpcActor.java    | 37 +++++---
 .../flink/runtime/rpc/akka/AkkaRpcService.java  |  2 +-
 .../rpc/akka/MainThreadValidationTest.java      | 97 ++++++++++++++++++++
 6 files changed, 205 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/2963e6c1/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/MainThreadExecutor.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/MainThreadExecutor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/MainThreadExecutor.java
index 4efb382..5e4fead 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/MainThreadExecutor.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/MainThreadExecutor.java
@@ -30,7 +30,7 @@ import java.util.concurrent.TimeoutException;
  *
  * <p>This interface is intended to be implemented by the self gateway in a {@link RpcEndpoint}
  * implementation which allows to dispatch local procedures to the main thread of the underlying
- * rpc server.
+ * RPC endpoint.
  */
 public interface MainThreadExecutor {
 

http://git-wip-us.apache.org/repos/asf/flink/blob/2963e6c1/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/MainThreadValidatorUtil.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/MainThreadValidatorUtil.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/MainThreadValidatorUtil.java
new file mode 100644
index 0000000..b3fea77
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/MainThreadValidatorUtil.java
@@ -0,0 +1,47 @@
+/*
+ * 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.rpc;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * This utility exists to bridge between the visibility of the
+ * {@code currentMainThread} field in the {@link RpcEndpoint}.
+ * 
+ * The {@code currentMainThread} can be hidden from {@code RpcEndpoint} implementations
+ * and only be accessed via this utility from other packages.
+ */
+public final class MainThreadValidatorUtil {
+
+	private final RpcEndpoint<?> endpoint;
+
+	public MainThreadValidatorUtil(RpcEndpoint<?> endpoint) {
+		this.endpoint = checkNotNull(endpoint);
+	}
+
+	public void enterMainThread() {
+		assert(endpoint.currentMainThread.compareAndSet(null, Thread.currentThread())) : 
+				"The RpcEndpoint has concurrent access from " + endpoint.currentMainThread.get();
+	}
+	
+	public void exitMainThread() {
+		assert(endpoint.currentMainThread.compareAndSet(Thread.currentThread(), null)) :
+				"The RpcEndpoint has concurrent access from " + endpoint.currentMainThread.get();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/2963e6c1/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java
index 44933d5..d36a283 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java
@@ -29,6 +29,7 @@ import scala.concurrent.Future;
 
 import java.util.concurrent.Callable;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
@@ -75,6 +76,9 @@ public abstract class RpcEndpoint<C extends RpcGateway> {
 	 * of the executing rpc server. */
 	private final MainThreadExecutionContext mainThreadExecutionContext;
 
+	/** A reference to the endpoint's main thread, if the current method is called by the main thread */
+	final AtomicReference<Thread> currentMainThread = new AtomicReference<>(null); 
+
 	/**
 	 * Initializes the RPC endpoint.
 	 * 
@@ -92,6 +96,15 @@ public abstract class RpcEndpoint<C extends RpcGateway> {
 		this.mainThreadExecutionContext = new MainThreadExecutionContext((MainThreadExecutor) self);
 	}
 
+	/**
+	 * Returns the class of the self gateway type.
+	 *
+	 * @return Class of the self gateway type
+	 */
+	public final Class<C> getSelfGatewayType() {
+		return selfGatewayType;
+	}
+	
 	// ------------------------------------------------------------------------
 	//  Shutdown
 	// ------------------------------------------------------------------------
@@ -193,13 +206,28 @@ public abstract class RpcEndpoint<C extends RpcGateway> {
 		return ((MainThreadExecutor) self).callAsync(callable, timeout);
 	}
 
+	// ------------------------------------------------------------------------
+	//  Main Thread Validation
+	// ------------------------------------------------------------------------
+
 	/**
-	 * Returns the class of the self gateway type.
-	 *
-	 * @return Class of the self gateway type
+	 * Validates that the method call happens in the RPC endpoint's main thread.
+	 * 
+	 * <p><b>IMPORTANT:</b> This check only happens when assertions are enabled,
+	 * such as when running tests.
+	 * 
+	 * <p>This can be used for additional checks, like
+	 * <pre>{@code
+	 * protected void concurrencyCriticalMethod() {
+	 *     validateRunsInMainThread();
+	 *     
+	 *     // some critical stuff
+	 * }
+	 * }</pre>
 	 */
-	public final Class<C> getSelfGatewayType() {
-		return selfGatewayType;
+	public void validateRunsInMainThread() {
+		// because the initialization is lazy, it can be that certain methods are
+		assert currentMainThread.get() == Thread.currentThread();
 	}
 
 	// ------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/flink/blob/2963e6c1/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java
index 18ccf1b..5e0a7da 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java
@@ -22,14 +22,16 @@ import akka.actor.ActorRef;
 import akka.actor.Status;
 import akka.actor.UntypedActor;
 import akka.pattern.Patterns;
+import org.apache.flink.runtime.rpc.MainThreadValidatorUtil;
 import org.apache.flink.runtime.rpc.RpcEndpoint;
 import org.apache.flink.runtime.rpc.RpcGateway;
 import org.apache.flink.runtime.rpc.akka.messages.CallAsync;
 import org.apache.flink.runtime.rpc.akka.messages.RpcInvocation;
 import org.apache.flink.runtime.rpc.akka.messages.RunAsync;
-import org.apache.flink.util.Preconditions;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import scala.concurrent.Future;
 import scala.concurrent.duration.FiniteDuration;
 
@@ -37,6 +39,8 @@ import java.lang.reflect.Method;
 import java.util.concurrent.Callable;
 import java.util.concurrent.TimeUnit;
 
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
 /**
  * Akka rpc actor which receives {@link RpcInvocation}, {@link RunAsync} and {@link CallAsync}
  * messages.
@@ -51,24 +55,35 @@ import java.util.concurrent.TimeUnit;
  * @param <T> Type of the {@link RpcEndpoint}
  */
 class AkkaRpcActor<C extends RpcGateway, T extends RpcEndpoint<C>> extends UntypedActor {
+	
 	private static final Logger LOG = LoggerFactory.getLogger(AkkaRpcActor.class);
 
+	/** the endpoint to invoke the methods on */
 	private final T rpcEndpoint;
 
+	/** the helper that tracks whether calls come from the main thread */
+	private final MainThreadValidatorUtil mainThreadValidator;
+
 	AkkaRpcActor(final T rpcEndpoint) {
-		this.rpcEndpoint = Preconditions.checkNotNull(rpcEndpoint, "rpc endpoint");
+		this.rpcEndpoint = checkNotNull(rpcEndpoint, "rpc endpoint");
+		this.mainThreadValidator = new MainThreadValidatorUtil(rpcEndpoint);
 	}
 
 	@Override
-	public void onReceive(final Object message)  {
-		if (message instanceof RunAsync) {
-			handleRunAsync((RunAsync) message);
-		} else if (message instanceof CallAsync) {
-			handleCallAsync((CallAsync) message);
-		} else if (message instanceof RpcInvocation) {
-			handleRpcInvocation((RpcInvocation) message);
-		} else {
-			LOG.warn("Received message of unknown type {}. Dropping this message!", message.getClass());
+	public void onReceive(final Object message) {
+		mainThreadValidator.enterMainThread();
+		try {
+			if (message instanceof RunAsync) {
+				handleRunAsync((RunAsync) message);
+			} else if (message instanceof CallAsync) {
+				handleCallAsync((CallAsync) message);
+			} else if (message instanceof RpcInvocation) {
+				handleRpcInvocation((RpcInvocation) message);
+			} else {
+				LOG.warn("Received message of unknown type {}. Dropping this message!", message.getClass());
+			}
+		} finally {
+			mainThreadValidator.exitMainThread();
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/2963e6c1/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java
index 448216c..db40f10 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java
@@ -174,7 +174,7 @@ public class AkkaRpcService implements RpcService {
 	}
 
 	@Override
-	public <C extends RpcGateway> String getAddress(C selfGateway) {
+	public String getAddress(RpcGateway selfGateway) {
 		checkState(!stopped, "RpcService is stopped");
 
 		if (selfGateway instanceof AkkaGateway) {

http://git-wip-us.apache.org/repos/asf/flink/blob/2963e6c1/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MainThreadValidationTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MainThreadValidationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MainThreadValidationTest.java
new file mode 100644
index 0000000..b854143
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MainThreadValidationTest.java
@@ -0,0 +1,97 @@
+/*
+ * 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.rpc.akka;
+
+import akka.util.Timeout;
+
+import org.apache.flink.runtime.akka.AkkaUtils;
+
+import org.apache.flink.runtime.rpc.RpcEndpoint;
+import org.apache.flink.runtime.rpc.RpcGateway;
+import org.apache.flink.runtime.rpc.RpcMethod;
+import org.apache.flink.runtime.rpc.RpcService;
+
+import org.junit.Test;
+
+import java.util.concurrent.TimeUnit;
+
+import static org.junit.Assert.assertTrue;
+
+public class MainThreadValidationTest {
+
+	@Test
+	public void failIfNotInMainThread() {
+		// test if assertions are activated. The test only works if assertions are loaded.
+		try {
+			assert false;
+			// apparently they are not activated
+			return;
+		} catch (AssertionError ignored) {}
+
+		// actual test
+		AkkaRpcService akkaRpcService = new AkkaRpcService(
+				AkkaUtils.createDefaultActorSystem(),
+				new Timeout(10000, TimeUnit.MILLISECONDS));
+
+		try {
+			TestEndpoint testEndpoint = new TestEndpoint(akkaRpcService);
+
+			// this works, because it is executed as an RPC call
+			testEndpoint.getSelf().someConcurrencyCriticalFunction();
+
+			// this fails, because it is executed directly
+			boolean exceptionThrown;
+			try {
+				testEndpoint.someConcurrencyCriticalFunction();
+				exceptionThrown = false;
+			}
+			catch (AssertionError e) {
+				exceptionThrown = true;
+			}
+			assertTrue("should fail with an assertion error", exceptionThrown);
+
+			akkaRpcService.stopServer(testEndpoint.getSelf());
+		}
+		finally {
+			akkaRpcService.stopService();
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  test RPC endpoint
+	// ------------------------------------------------------------------------
+
+	interface TestGateway extends RpcGateway {
+
+		void someConcurrencyCriticalFunction();
+	}
+
+	@SuppressWarnings("unused")
+	public static class TestEndpoint extends RpcEndpoint<TestGateway> {
+
+		public TestEndpoint(RpcService rpcService) {
+			super(rpcService);
+		}
+
+		@RpcMethod
+		public void someConcurrencyCriticalFunction() {
+			validateRunsInMainThread();
+		}
+	}
+}


[24/50] [abbrv] flink git commit: [FLINK-4362] [rpc] Auto generate rpc gateways via Java proxies

Posted by tr...@apache.org.
[FLINK-4362] [rpc] Auto generate rpc gateways via Java proxies

This PR introduces a generic AkkaRpcActor which receives rpc calls as a
RpcInvocation message. The RpcInvocation message is generated by the
AkkaInvocationHandler which gets them from automatically generated Java Proxies.

Add documentation for proxy based akka rpc service

Log unknown message type in AkkaRpcActor but do not fail actor

Use ReflectionUtil to extract RpcGateway type from RpcEndpoint

This closes #2357.


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

Branch: refs/heads/flip-6
Commit: 67e6f574fa3d54d0c3c8e414f04bc7732988f966
Parents: 506aac0
Author: Till Rohrmann <tr...@apache.org>
Authored: Wed Aug 10 18:42:26 2016 +0200
Committer: Till Rohrmann <tr...@apache.org>
Committed: Thu Sep 8 17:26:56 2016 +0200

----------------------------------------------------------------------
 .../org/apache/flink/util/ReflectionUtil.java   |  10 +-
 .../flink/runtime/rpc/MainThreadExecutor.java   |   4 +-
 .../apache/flink/runtime/rpc/RpcEndpoint.java   |  22 +-
 .../apache/flink/runtime/rpc/RpcService.java    |   2 +-
 .../flink/runtime/rpc/akka/AkkaGateway.java     |   4 +-
 .../runtime/rpc/akka/AkkaInvocationHandler.java | 226 +++++++++++++++++++
 .../flink/runtime/rpc/akka/AkkaRpcActor.java    | 175 ++++++++++++++
 .../flink/runtime/rpc/akka/AkkaRpcService.java  | 121 +++++-----
 .../flink/runtime/rpc/akka/BaseAkkaActor.java   |  50 ----
 .../flink/runtime/rpc/akka/BaseAkkaGateway.java |  41 ----
 .../rpc/akka/jobmaster/JobMasterAkkaActor.java  |  58 -----
 .../akka/jobmaster/JobMasterAkkaGateway.java    |  57 -----
 .../runtime/rpc/akka/messages/CallAsync.java    |  41 ++++
 .../rpc/akka/messages/CallableMessage.java      |  33 ---
 .../runtime/rpc/akka/messages/CancelTask.java   |  36 ---
 .../runtime/rpc/akka/messages/ExecuteTask.java  |  36 ---
 .../messages/RegisterAtResourceManager.java     |  36 ---
 .../rpc/akka/messages/RegisterJobMaster.java    |  36 ---
 .../runtime/rpc/akka/messages/RequestSlot.java  |  37 ---
 .../rpc/akka/messages/RpcInvocation.java        |  98 ++++++++
 .../runtime/rpc/akka/messages/RunAsync.java     |  40 ++++
 .../rpc/akka/messages/RunnableMessage.java      |  31 ---
 .../akka/messages/UpdateTaskExecutionState.java |  37 ---
 .../ResourceManagerAkkaActor.java               |  65 ------
 .../ResourceManagerAkkaGateway.java             |  67 ------
 .../taskexecutor/TaskExecutorAkkaActor.java     |  77 -------
 .../taskexecutor/TaskExecutorAkkaGateway.java   |  59 -----
 .../flink/runtime/rpc/jobmaster/JobMaster.java  |   4 +-
 .../rpc/resourcemanager/ResourceManager.java    |   4 +-
 .../runtime/rpc/taskexecutor/TaskExecutor.java  |   4 +-
 .../flink/runtime/rpc/RpcCompletenessTest.java  |  50 ++--
 .../runtime/rpc/akka/AkkaRpcServiceTest.java    |   4 +-
 .../rpc/taskexecutor/TaskExecutorTest.java      |   2 +-
 33 files changed, 700 insertions(+), 867 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/67e6f574/flink-core/src/main/java/org/apache/flink/util/ReflectionUtil.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/ReflectionUtil.java b/flink-core/src/main/java/org/apache/flink/util/ReflectionUtil.java
index fe2d4c0..b851eba 100644
--- a/flink-core/src/main/java/org/apache/flink/util/ReflectionUtil.java
+++ b/flink-core/src/main/java/org/apache/flink/util/ReflectionUtil.java
@@ -48,6 +48,14 @@ public final class ReflectionUtil {
 		return getTemplateType(clazz, 0);
 	}
 
+	public static <T> Class<T> getTemplateType1(Type type) {
+		if (type instanceof ParameterizedType) {
+			return (Class<T>) getTemplateTypes((ParameterizedType) type)[0];
+		} else {
+			throw new IllegalArgumentException();
+		}
+	}
+
 	public static <T> Class<T> getTemplateType2(Class<?> clazz) {
 		return getTemplateType(clazz, 1);
 	}
@@ -123,7 +131,7 @@ public final class ReflectionUtil {
 		Class<?>[] types = new Class<?>[paramterizedType.getActualTypeArguments().length];
 		int i = 0;
 		for (Type templateArgument : paramterizedType.getActualTypeArguments()) {
-			assert (templateArgument instanceof Class<?>);
+			assert templateArgument instanceof Class<?>;
 			types[i++] = (Class<?>) templateArgument;
 		}
 		return types;

http://git-wip-us.apache.org/repos/asf/flink/blob/67e6f574/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/MainThreadExecutor.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/MainThreadExecutor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/MainThreadExecutor.java
index 14b2997..882c1b7 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/MainThreadExecutor.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/MainThreadExecutor.java
@@ -47,9 +47,9 @@ public interface MainThreadExecutor {
 	 * future will throw a {@link TimeoutException}.
 	 *
 	 * @param callable Callable to be executed
-	 * @param timeout Timeout for the future to complete
+	 * @param callTimeout Timeout for the future to complete
 	 * @param <V> Return value of the callable
 	 * @return Future of the callable result
 	 */
-	<V> Future<V> callAsync(Callable<V> callable, Timeout timeout);
+	<V> Future<V> callAsync(Callable<V> callable, Timeout callTimeout);
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/67e6f574/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java
index 0d928a8..aef0803 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java
@@ -20,6 +20,7 @@ package org.apache.flink.runtime.rpc;
 
 import akka.util.Timeout;
 
+import org.apache.flink.util.ReflectionUtil;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -60,6 +61,9 @@ public abstract class RpcEndpoint<C extends RpcGateway> {
 	/** RPC service to be used to start the RPC server and to obtain rpc gateways */
 	private final RpcService rpcService;
 
+	/** Class of the self gateway */
+	private final Class<C> selfGatewayType;
+
 	/** Self gateway which can be used to schedule asynchronous calls on yourself */
 	private final C self;
 
@@ -70,15 +74,19 @@ public abstract class RpcEndpoint<C extends RpcGateway> {
 	 * of the executing rpc server. */
 	private final MainThreadExecutionContext mainThreadExecutionContext;
 
-
 	/**
 	 * Initializes the RPC endpoint.
 	 * 
 	 * @param rpcService The RPC server that dispatches calls to this RPC endpoint. 
 	 */
-	public RpcEndpoint(RpcService rpcService) {
+	protected RpcEndpoint(final RpcService rpcService) {
 		this.rpcService = checkNotNull(rpcService, "rpcService");
+
+		// IMPORTANT: Don't change order of selfGatewayType and self because rpcService.startServer
+		// requires that selfGatewayType has been initialized
+		this.selfGatewayType = ReflectionUtil.getTemplateType1(getClass());
 		this.self = rpcService.startServer(this);
+		
 		this.selfAddress = rpcService.getAddress(self);
 		this.mainThreadExecutionContext = new MainThreadExecutionContext((MainThreadExecutor) self);
 	}
@@ -149,6 +157,7 @@ public abstract class RpcEndpoint<C extends RpcGateway> {
 	//  Asynchronous executions
 	// ------------------------------------------------------------------------
 
+
 	/**
 	 * Execute the runnable in the main thread of the underlying RPC endpoint.
 	 *
@@ -172,6 +181,15 @@ public abstract class RpcEndpoint<C extends RpcGateway> {
 		return ((MainThreadExecutor) self).callAsync(callable, timeout);
 	}
 
+	/**
+	 * Returns the class of the self gateway type.
+	 *
+	 * @return Class of the self gateway type
+	 */
+	public final Class<C> getSelfGatewayType() {
+		return selfGatewayType;
+	}
+
 	// ------------------------------------------------------------------------
 	//  Utilities
 	// ------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/flink/blob/67e6f574/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java
index 90ff7b6..f93be83 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java
@@ -46,7 +46,7 @@ public interface RpcService {
 	 * @param <C> Type of the self rpc gateway associated with the rpc server
 	 * @return Self gateway to dispatch remote procedure calls to oneself
 	 */
-	<S extends RpcEndpoint, C extends RpcGateway> C startServer(S rpcEndpoint);
+	<C extends RpcGateway, S extends RpcEndpoint<C>> C startServer(S rpcEndpoint);
 
 	/**
 	 * Stop the underlying rpc server of the provided self gateway.

http://git-wip-us.apache.org/repos/asf/flink/blob/67e6f574/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaGateway.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaGateway.java
index a96a600..a826e7d 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaGateway.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaGateway.java
@@ -23,7 +23,7 @@ import akka.actor.ActorRef;
 /**
  * Interface for Akka based rpc gateways
  */
-public interface AkkaGateway {
+interface AkkaGateway {
 
-	ActorRef getActorRef();
+	ActorRef getRpcServer();
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/67e6f574/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaInvocationHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaInvocationHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaInvocationHandler.java
new file mode 100644
index 0000000..e8e383a
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaInvocationHandler.java
@@ -0,0 +1,226 @@
+/*
+ * 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.rpc.akka;
+
+import akka.actor.ActorRef;
+import akka.pattern.Patterns;
+import akka.util.Timeout;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.runtime.rpc.MainThreadExecutor;
+import org.apache.flink.runtime.rpc.RpcTimeout;
+import org.apache.flink.runtime.rpc.akka.messages.CallAsync;
+import org.apache.flink.runtime.rpc.akka.messages.RpcInvocation;
+import org.apache.flink.runtime.rpc.akka.messages.RunAsync;
+import org.apache.flink.util.Preconditions;
+import scala.concurrent.Await;
+import scala.concurrent.Future;
+import scala.concurrent.duration.FiniteDuration;
+
+import java.lang.annotation.Annotation;
+import java.lang.reflect.InvocationHandler;
+import java.lang.reflect.Method;
+import java.util.BitSet;
+import java.util.concurrent.Callable;
+
+/**
+ * Invocation handler to be used with a {@link AkkaRpcActor}. The invocation handler wraps the
+ * rpc in a {@link RpcInvocation} message and then sends it to the {@link AkkaRpcActor} where it is
+ * executed.
+ */
+class AkkaInvocationHandler implements InvocationHandler, AkkaGateway, MainThreadExecutor {
+	private final ActorRef rpcServer;
+
+	// default timeout for asks
+	private final Timeout timeout;
+
+	AkkaInvocationHandler(ActorRef rpcServer, Timeout timeout) {
+		this.rpcServer = Preconditions.checkNotNull(rpcServer);
+		this.timeout = Preconditions.checkNotNull(timeout);
+	}
+
+	@Override
+	public Object invoke(Object proxy, Method method, Object[] args) throws Throwable {
+		Class<?> declaringClass = method.getDeclaringClass();
+
+		Object result;
+
+		if (declaringClass.equals(AkkaGateway.class) || declaringClass.equals(MainThreadExecutor.class) || declaringClass.equals(Object.class)) {
+			result = method.invoke(this, args);
+		} else {
+			String methodName = method.getName();
+			Class<?>[] parameterTypes = method.getParameterTypes();
+			Annotation[][] parameterAnnotations = method.getParameterAnnotations();
+			Timeout futureTimeout = extractRpcTimeout(parameterAnnotations, args, timeout);
+
+			Tuple2<Class<?>[], Object[]> filteredArguments = filterArguments(
+				parameterTypes,
+				parameterAnnotations,
+				args);
+
+			RpcInvocation rpcInvocation = new RpcInvocation(
+				methodName,
+				filteredArguments.f0,
+				filteredArguments.f1);
+
+			Class<?> returnType = method.getReturnType();
+
+			if (returnType.equals(Void.TYPE)) {
+				rpcServer.tell(rpcInvocation, ActorRef.noSender());
+
+				result = null;
+			} else if (returnType.equals(Future.class)) {
+				// execute an asynchronous call
+				result = Patterns.ask(rpcServer, rpcInvocation, futureTimeout);
+			} else {
+				// execute a synchronous call
+				Future<?> futureResult = Patterns.ask(rpcServer, rpcInvocation, futureTimeout);
+				FiniteDuration duration = timeout.duration();
+
+				result = Await.result(futureResult, duration);
+			}
+		}
+
+		return result;
+	}
+
+	@Override
+	public ActorRef getRpcServer() {
+		return rpcServer;
+	}
+
+	@Override
+	public void runAsync(Runnable runnable) {
+		// Unfortunately I couldn't find a way to allow only local communication. Therefore, the
+		// runnable field is transient transient
+		rpcServer.tell(new RunAsync(runnable), ActorRef.noSender());
+	}
+
+	@Override
+	public <V> Future<V> callAsync(Callable<V> callable, Timeout callTimeout) {
+		// Unfortunately I couldn't find a way to allow only local communication. Therefore, the
+		// callable field is declared transient
+		@SuppressWarnings("unchecked")
+		Future<V> result = (Future<V>) Patterns.ask(rpcServer, new CallAsync(callable), callTimeout);
+
+		return result;
+	}
+
+	/**
+	 * Extracts the {@link RpcTimeout} annotated rpc timeout value from the list of given method
+	 * arguments. If no {@link RpcTimeout} annotated parameter could be found, then the default
+	 * timeout is returned.
+	 *
+	 * @param parameterAnnotations Parameter annotations
+	 * @param args Array of arguments
+	 * @param defaultTimeout Default timeout to return if no {@link RpcTimeout} annotated parameter
+	 *                       has been found
+	 * @return Timeout extracted from the array of arguments or the default timeout
+	 */
+	private static Timeout extractRpcTimeout(Annotation[][] parameterAnnotations, Object[] args, Timeout defaultTimeout) {
+		if (args != null) {
+			Preconditions.checkArgument(parameterAnnotations.length == args.length);
+
+			for (int i = 0; i < parameterAnnotations.length; i++) {
+				if (isRpcTimeout(parameterAnnotations[i])) {
+					if (args[i] instanceof FiniteDuration) {
+						return new Timeout((FiniteDuration) args[i]);
+					} else {
+						throw new RuntimeException("The rpc timeout parameter must be of type " +
+							FiniteDuration.class.getName() + ". The type " + args[i].getClass().getName() +
+							" is not supported.");
+					}
+				}
+			}
+		}
+
+		return defaultTimeout;
+	}
+
+	/**
+	 * Removes all {@link RpcTimeout} annotated parameters from the parameter type and argument
+	 * list.
+	 *
+	 * @param parameterTypes Array of parameter types
+	 * @param parameterAnnotations Array of parameter annotations
+	 * @param args Arary of arguments
+	 * @return Tuple of filtered parameter types and arguments which no longer contain the
+	 * {@link RpcTimeout} annotated parameter types and arguments
+	 */
+	private static Tuple2<Class<?>[], Object[]> filterArguments(
+		Class<?>[] parameterTypes,
+		Annotation[][] parameterAnnotations,
+		Object[] args) {
+
+		Class<?>[] filteredParameterTypes;
+		Object[] filteredArgs;
+
+		if (args == null) {
+			filteredParameterTypes = parameterTypes;
+			filteredArgs = null;
+		} else {
+			Preconditions.checkArgument(parameterTypes.length == parameterAnnotations.length);
+			Preconditions.checkArgument(parameterAnnotations.length == args.length);
+
+			BitSet isRpcTimeoutParameter = new BitSet(parameterTypes.length);
+			int numberRpcParameters = parameterTypes.length;
+
+			for (int i = 0; i < parameterTypes.length; i++) {
+				if (isRpcTimeout(parameterAnnotations[i])) {
+					isRpcTimeoutParameter.set(i);
+					numberRpcParameters--;
+				}
+			}
+
+			if (numberRpcParameters == parameterTypes.length) {
+				filteredParameterTypes = parameterTypes;
+				filteredArgs = args;
+			} else {
+				filteredParameterTypes = new Class<?>[numberRpcParameters];
+				filteredArgs = new Object[numberRpcParameters];
+				int counter = 0;
+
+				for (int i = 0; i < parameterTypes.length; i++) {
+					if (!isRpcTimeoutParameter.get(i)) {
+						filteredParameterTypes[counter] = parameterTypes[i];
+						filteredArgs[counter] = args[i];
+						counter++;
+					}
+				}
+			}
+		}
+
+		return Tuple2.of(filteredParameterTypes, filteredArgs);
+	}
+
+	/**
+	 * Checks whether any of the annotations is of type {@link RpcTimeout}
+	 *
+	 * @param annotations Array of annotations
+	 * @return True if {@link RpcTimeout} was found; otherwise false
+	 */
+	private static boolean isRpcTimeout(Annotation[] annotations) {
+		for (Annotation annotation : annotations) {
+			if (annotation.annotationType().equals(RpcTimeout.class)) {
+				return true;
+			}
+		}
+
+		return false;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/67e6f574/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java
new file mode 100644
index 0000000..57da38a
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java
@@ -0,0 +1,175 @@
+/*
+ * 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.rpc.akka;
+
+import akka.actor.Status;
+import akka.actor.UntypedActor;
+import akka.pattern.Patterns;
+import org.apache.flink.runtime.rpc.RpcEndpoint;
+import org.apache.flink.runtime.rpc.RpcGateway;
+import org.apache.flink.runtime.rpc.akka.messages.CallAsync;
+import org.apache.flink.runtime.rpc.akka.messages.RpcInvocation;
+import org.apache.flink.runtime.rpc.akka.messages.RunAsync;
+import org.apache.flink.util.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.concurrent.Future;
+
+import java.lang.reflect.Method;
+import java.util.concurrent.Callable;
+
+/**
+ * Akka rpc actor which receives {@link RpcInvocation}, {@link RunAsync} and {@link CallAsync}
+ * messages.
+ * <p>
+ * The {@link RpcInvocation} designates a rpc and is dispatched to the given {@link RpcEndpoint}
+ * instance.
+ * <p>
+ * The {@link RunAsync} and {@link CallAsync} messages contain executable code which is executed
+ * in the context of the actor thread.
+ *
+ * @param <C> Type of the {@link RpcGateway} associated with the {@link RpcEndpoint}
+ * @param <T> Type of the {@link RpcEndpoint}
+ */
+class AkkaRpcActor<C extends RpcGateway, T extends RpcEndpoint<C>> extends UntypedActor {
+	private static final Logger LOG = LoggerFactory.getLogger(AkkaRpcActor.class);
+
+	private final T rpcEndpoint;
+
+	AkkaRpcActor(final T rpcEndpoint) {
+		this.rpcEndpoint = Preconditions.checkNotNull(rpcEndpoint, "rpc endpoint");
+	}
+
+	@Override
+	public void onReceive(final Object message)  {
+		if (message instanceof RunAsync) {
+			handleRunAsync((RunAsync) message);
+		} else if (message instanceof CallAsync) {
+			handleCallAsync((CallAsync) message);
+		} else if (message instanceof RpcInvocation) {
+			handleRpcInvocation((RpcInvocation) message);
+		} else {
+			LOG.warn("Received message of unknown type {}. Dropping this message!", message.getClass());
+		}
+	}
+
+	/**
+	 * Handle rpc invocations by looking up the rpc method on the rpc endpoint and calling this
+	 * method with the provided method arguments. If the method has a return value, it is returned
+	 * to the sender of the call.
+	 *
+	 * @param rpcInvocation Rpc invocation message
+	 */
+	private void handleRpcInvocation(RpcInvocation rpcInvocation) {
+		Method rpcMethod = null;
+
+		try {
+			rpcMethod = lookupRpcMethod(rpcInvocation.getMethodName(), rpcInvocation.getParameterTypes());
+		} catch (final NoSuchMethodException e) {
+			LOG.error("Could not find rpc method for rpc invocation: {}.", rpcInvocation, e);
+		}
+
+		if (rpcMethod != null) {
+			if (rpcMethod.getReturnType().equals(Void.TYPE)) {
+				// No return value to send back
+				try {
+					rpcMethod.invoke(rpcEndpoint, rpcInvocation.getArgs());
+				} catch (Throwable e) {
+					LOG.error("Error while executing remote procedure call {}.", rpcMethod, e);
+				}
+			} else {
+				try {
+					Object result = rpcMethod.invoke(rpcEndpoint, rpcInvocation.getArgs());
+
+					if (result instanceof Future) {
+						// pipe result to sender
+						Patterns.pipe((Future<?>) result, getContext().dispatcher()).to(getSender());
+					} else {
+						// tell the sender the result of the computation
+						getSender().tell(new Status.Success(result), getSelf());
+					}
+				} catch (Throwable e) {
+					// tell the sender about the failure
+					getSender().tell(new Status.Failure(e), getSelf());
+				}
+			}
+		}
+	}
+
+	/**
+	 * Handle asynchronous {@link Callable}. This method simply executes the given {@link Callable}
+	 * in the context of the actor thread.
+	 *
+	 * @param callAsync Call async message
+	 */
+	private void handleCallAsync(CallAsync callAsync) {
+		if (callAsync.getCallable() == null) {
+			final String result = "Received a " + callAsync.getClass().getName() + " message with an empty " +
+				"callable field. This indicates that this message has been serialized " +
+				"prior to sending the message. The " + callAsync.getClass().getName() +
+				" is only supported with local communication.";
+
+			LOG.warn(result);
+
+			getSender().tell(new Status.Failure(new Exception(result)), getSelf());
+		} else {
+			try {
+				Object result = callAsync.getCallable().call();
+
+				getSender().tell(new Status.Success(result), getSelf());
+			} catch (Throwable e) {
+				getSender().tell(new Status.Failure(e), getSelf());
+			}
+		}
+	}
+
+	/**
+	 * Handle asynchronous {@link Runnable}. This method simply executes the given {@link Runnable}
+	 * in the context of the actor thread.
+	 *
+	 * @param runAsync Run async message
+	 */
+	private void handleRunAsync(RunAsync runAsync) {
+		if (runAsync.getRunnable() == null) {
+			LOG.warn("Received a {} message with an empty runnable field. This indicates " +
+				"that this message has been serialized prior to sending the message. The " +
+				"{} is only supported with local communication.",
+				runAsync.getClass().getName(),
+				runAsync.getClass().getName());
+		} else {
+			try {
+				runAsync.getRunnable().run();
+			} catch (final Throwable e) {
+				LOG.error("Caught exception while executing runnable in main thread.", e);
+			}
+		}
+	}
+
+	/**
+	 * Look up the rpc method on the given {@link RpcEndpoint} instance.
+	 *
+	 * @param methodName Name of the method
+	 * @param parameterTypes Parameter types of the method
+	 * @return Method of the rpc endpoint
+	 * @throws NoSuchMethodException
+	 */
+	private Method lookupRpcMethod(final String methodName, final Class<?>[] parameterTypes) throws NoSuchMethodException {
+		return rpcEndpoint.getClass().getMethod(methodName, parameterTypes);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/67e6f574/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java
index d55bd13..17983d0 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java
@@ -29,88 +29,82 @@ import akka.dispatch.Mapper;
 import akka.pattern.AskableActorSelection;
 import akka.util.Timeout;
 import org.apache.flink.runtime.akka.AkkaUtils;
-import org.apache.flink.runtime.rpc.jobmaster.JobMaster;
-import org.apache.flink.runtime.rpc.jobmaster.JobMasterGateway;
-import org.apache.flink.runtime.rpc.resourcemanager.ResourceManager;
-import org.apache.flink.runtime.rpc.resourcemanager.ResourceManagerGateway;
+import org.apache.flink.runtime.rpc.MainThreadExecutor;
 import org.apache.flink.runtime.rpc.RpcGateway;
 import org.apache.flink.runtime.rpc.RpcEndpoint;
 import org.apache.flink.runtime.rpc.RpcService;
-import org.apache.flink.runtime.rpc.akka.jobmaster.JobMasterAkkaActor;
-import org.apache.flink.runtime.rpc.akka.jobmaster.JobMasterAkkaGateway;
-import org.apache.flink.runtime.rpc.akka.resourcemanager.ResourceManagerAkkaActor;
-import org.apache.flink.runtime.rpc.akka.resourcemanager.ResourceManagerAkkaGateway;
-import org.apache.flink.runtime.rpc.akka.taskexecutor.TaskExecutorAkkaActor;
-import org.apache.flink.runtime.rpc.akka.taskexecutor.TaskExecutorAkkaGateway;
-import org.apache.flink.runtime.rpc.taskexecutor.TaskExecutorGateway;
-import org.apache.flink.runtime.rpc.taskexecutor.TaskExecutor;
+import org.apache.flink.util.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import scala.concurrent.Future;
 
+import java.lang.reflect.InvocationHandler;
+import java.lang.reflect.Proxy;
+import java.util.Collection;
 import java.util.HashSet;
-import java.util.Set;
 
+/**
+ * Akka based {@link RpcService} implementation. The rpc service starts an Akka actor to receive
+ * rpcs from a {@link RpcGateway}.
+ */
 public class AkkaRpcService implements RpcService {
+	private static final Logger LOG = LoggerFactory.getLogger(AkkaRpcService.class);
+
 	private final ActorSystem actorSystem;
 	private final Timeout timeout;
-	private final Set<ActorRef> actors = new HashSet<>();
+	private final Collection<ActorRef> actors = new HashSet<>(4);
 
-	public AkkaRpcService(ActorSystem actorSystem, Timeout timeout) {
-		this.actorSystem = actorSystem;
-		this.timeout = timeout;
+	public AkkaRpcService(final ActorSystem actorSystem, final Timeout timeout) {
+		this.actorSystem = Preconditions.checkNotNull(actorSystem, "actor system");
+		this.timeout = Preconditions.checkNotNull(timeout, "timeout");
 	}
 
 	@Override
-	public <C extends RpcGateway> Future<C> connect(String address, final Class<C> clazz) {
-		ActorSelection actorSel = actorSystem.actorSelection(address);
+	public <C extends RpcGateway> Future<C> connect(final String address, final Class<C> clazz) {
+		LOG.info("Try to connect to remote rpc server with address {}. Returning a {} gateway.", address, clazz.getName());
 
-		AskableActorSelection asker = new AskableActorSelection(actorSel);
+		final ActorSelection actorSel = actorSystem.actorSelection(address);
 
-		Future<Object> identify = asker.ask(new Identify(42), timeout);
+		final AskableActorSelection asker = new AskableActorSelection(actorSel);
+
+		final Future<Object> identify = asker.ask(new Identify(42), timeout);
 
 		return identify.map(new Mapper<Object, C>(){
+			@Override
 			public C apply(Object obj) {
 				ActorRef actorRef = ((ActorIdentity) obj).getRef();
 
-				if (clazz == TaskExecutorGateway.class) {
-					return (C) new TaskExecutorAkkaGateway(actorRef, timeout);
-				} else if (clazz == ResourceManagerGateway.class) {
-					return (C) new ResourceManagerAkkaGateway(actorRef, timeout);
-				} else if (clazz == JobMasterGateway.class) {
-					return (C) new JobMasterAkkaGateway(actorRef, timeout);
-				} else {
-					throw new RuntimeException("Could not find remote endpoint " + clazz);
-				}
+				InvocationHandler akkaInvocationHandler = new AkkaInvocationHandler(actorRef, timeout);
+
+				@SuppressWarnings("unchecked")
+				C proxy = (C) Proxy.newProxyInstance(
+					ClassLoader.getSystemClassLoader(),
+					new Class<?>[] {clazz},
+					akkaInvocationHandler);
+
+				return proxy;
 			}
 		}, actorSystem.dispatcher());
 	}
 
 	@Override
-	public <S extends RpcEndpoint, C extends RpcGateway> C startServer(S rpcEndpoint) {
-		ActorRef ref;
-		C self;
-		if (rpcEndpoint instanceof TaskExecutor) {
-			ref = actorSystem.actorOf(
-				Props.create(TaskExecutorAkkaActor.class, rpcEndpoint)
-			);
-
-			self = (C) new TaskExecutorAkkaGateway(ref, timeout);
-		} else if (rpcEndpoint instanceof ResourceManager) {
-			ref = actorSystem.actorOf(
-				Props.create(ResourceManagerAkkaActor.class, rpcEndpoint)
-			);
-
-			self = (C) new ResourceManagerAkkaGateway(ref, timeout);
-		} else if (rpcEndpoint instanceof JobMaster) {
-			ref = actorSystem.actorOf(
-				Props.create(JobMasterAkkaActor.class, rpcEndpoint)
-			);
-
-			self = (C) new JobMasterAkkaGateway(ref, timeout);
-		} else {
-			throw new RuntimeException("Could not start RPC server for class " + rpcEndpoint.getClass());
-		}
+	public <C extends RpcGateway, S extends RpcEndpoint<C>> C startServer(S rpcEndpoint) {
+		Preconditions.checkNotNull(rpcEndpoint, "rpc endpoint");
 
-		actors.add(ref);
+		LOG.info("Start Akka rpc actor to handle rpcs for {}.", rpcEndpoint.getClass().getName());
+
+		Props akkaRpcActorProps = Props.create(AkkaRpcActor.class, rpcEndpoint);
+
+		ActorRef actorRef = actorSystem.actorOf(akkaRpcActorProps);
+		actors.add(actorRef);
+
+		InvocationHandler akkaInvocationHandler = new AkkaInvocationHandler(actorRef, timeout);
+
+		@SuppressWarnings("unchecked")
+		C self = (C) Proxy.newProxyInstance(
+			ClassLoader.getSystemClassLoader(),
+			new Class<?>[]{rpcEndpoint.getSelfGatewayType(), MainThreadExecutor.class, AkkaGateway.class},
+			akkaInvocationHandler);
 
 		return self;
 	}
@@ -120,16 +114,19 @@ public class AkkaRpcService implements RpcService {
 		if (selfGateway instanceof AkkaGateway) {
 			AkkaGateway akkaClient = (AkkaGateway) selfGateway;
 
-			if (actors.contains(akkaClient.getActorRef())) {
-				akkaClient.getActorRef().tell(PoisonPill.getInstance(), ActorRef.noSender());
-			} else {
-				// don't stop this actor since it was not started by this RPC service
+			if (actors.contains(akkaClient.getRpcServer())) {
+				ActorRef selfActorRef = akkaClient.getRpcServer();
+
+				LOG.info("Stop Akka rpc actor {}.", selfActorRef.path());
+
+				selfActorRef.tell(PoisonPill.getInstance(), ActorRef.noSender());
 			}
 		}
 	}
 
 	@Override
 	public void stopService() {
+		LOG.info("Stop Akka rpc service.");
 		actorSystem.shutdown();
 		actorSystem.awaitTermination();
 	}
@@ -137,9 +134,11 @@ public class AkkaRpcService implements RpcService {
 	@Override
 	public <C extends RpcGateway> String getAddress(C selfGateway) {
 		if (selfGateway instanceof AkkaGateway) {
-			return AkkaUtils.getAkkaURL(actorSystem, ((AkkaGateway) selfGateway).getActorRef());
+			ActorRef actorRef = ((AkkaGateway) selfGateway).getRpcServer();
+			return AkkaUtils.getAkkaURL(actorSystem, actorRef);
 		} else {
-			throw new RuntimeException("Cannot get address for non " + AkkaGateway.class.getName() + ".");
+			String className = AkkaGateway.class.getName();
+			throw new RuntimeException("Cannot get address for non " + className + '.');
 		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/67e6f574/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/BaseAkkaActor.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/BaseAkkaActor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/BaseAkkaActor.java
deleted file mode 100644
index 3cb499c..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/BaseAkkaActor.java
+++ /dev/null
@@ -1,50 +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.rpc.akka;
-
-import akka.actor.Status;
-import akka.actor.UntypedActor;
-import org.apache.flink.runtime.rpc.akka.messages.CallableMessage;
-import org.apache.flink.runtime.rpc.akka.messages.RunnableMessage;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class BaseAkkaActor extends UntypedActor {
-	private static final Logger LOG = LoggerFactory.getLogger(BaseAkkaActor.class);
-
-	@Override
-	public void onReceive(Object message) throws Exception {
-		if (message instanceof RunnableMessage) {
-			try {
-				((RunnableMessage) message).getRunnable().run();
-			} catch (Exception e) {
-				LOG.error("Encountered error while executing runnable.", e);
-			}
-		} else if (message instanceof CallableMessage<?>) {
-			try {
-				Object result = ((CallableMessage<?>) message).getCallable().call();
-				sender().tell(new Status.Success(result), getSelf());
-			} catch (Exception e) {
-				sender().tell(new Status.Failure(e), getSelf());
-			}
-		} else {
-			throw new RuntimeException("Unknown message " + message);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/67e6f574/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/BaseAkkaGateway.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/BaseAkkaGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/BaseAkkaGateway.java
deleted file mode 100644
index 512790d..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/BaseAkkaGateway.java
+++ /dev/null
@@ -1,41 +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.rpc.akka;
-
-import akka.actor.ActorRef;
-import akka.pattern.Patterns;
-import akka.util.Timeout;
-import org.apache.flink.runtime.rpc.MainThreadExecutor;
-import org.apache.flink.runtime.rpc.akka.messages.CallableMessage;
-import org.apache.flink.runtime.rpc.akka.messages.RunnableMessage;
-import scala.concurrent.Future;
-
-import java.util.concurrent.Callable;
-
-public abstract class BaseAkkaGateway implements MainThreadExecutor, AkkaGateway {
-	@Override
-	public void runAsync(Runnable runnable) {
-		getActorRef().tell(new RunnableMessage(runnable), ActorRef.noSender());
-	}
-
-	@Override
-	public <V> Future<V> callAsync(Callable<V> callable, Timeout timeout) {
-		return (Future<V>) Patterns.ask(getActorRef(), new CallableMessage(callable), timeout);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/67e6f574/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/jobmaster/JobMasterAkkaActor.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/jobmaster/JobMasterAkkaActor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/jobmaster/JobMasterAkkaActor.java
deleted file mode 100644
index 9e04ea9..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/jobmaster/JobMasterAkkaActor.java
+++ /dev/null
@@ -1,58 +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.rpc.akka.jobmaster;
-
-import akka.actor.ActorRef;
-import akka.actor.Status;
-import org.apache.flink.runtime.rpc.akka.BaseAkkaActor;
-import org.apache.flink.runtime.rpc.akka.messages.RegisterAtResourceManager;
-import org.apache.flink.runtime.rpc.jobmaster.JobMaster;
-import org.apache.flink.runtime.messages.Acknowledge;
-import org.apache.flink.runtime.rpc.akka.messages.UpdateTaskExecutionState;
-
-public class JobMasterAkkaActor extends BaseAkkaActor {
-	private final JobMaster jobMaster;
-
-	public JobMasterAkkaActor(JobMaster jobMaster) {
-		this.jobMaster = jobMaster;
-	}
-
-	@Override
-	public void onReceive(Object message) throws Exception {
-		if (message instanceof UpdateTaskExecutionState) {
-
-			final ActorRef sender = getSender();
-
-			UpdateTaskExecutionState updateTaskExecutionState = (UpdateTaskExecutionState) message;
-
-			try {
-				Acknowledge result = jobMaster.updateTaskExecutionState(updateTaskExecutionState.getTaskExecutionState());
-				sender.tell(new Status.Success(result), getSelf());
-			} catch (Exception e) {
-				sender.tell(new Status.Failure(e), getSelf());
-			}
-		} else if (message instanceof RegisterAtResourceManager) {
-			RegisterAtResourceManager registerAtResourceManager = (RegisterAtResourceManager) message;
-
-			jobMaster.registerAtResourceManager(registerAtResourceManager.getAddress());
-		} else {
-			super.onReceive(message);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/67e6f574/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/jobmaster/JobMasterAkkaGateway.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/jobmaster/JobMasterAkkaGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/jobmaster/JobMasterAkkaGateway.java
deleted file mode 100644
index e6bf061..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/jobmaster/JobMasterAkkaGateway.java
+++ /dev/null
@@ -1,57 +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.rpc.akka.jobmaster;
-
-import akka.actor.ActorRef;
-import akka.pattern.AskableActorRef;
-import akka.util.Timeout;
-import org.apache.flink.runtime.rpc.akka.BaseAkkaGateway;
-import org.apache.flink.runtime.rpc.akka.messages.RegisterAtResourceManager;
-import org.apache.flink.runtime.rpc.jobmaster.JobMasterGateway;
-import org.apache.flink.runtime.messages.Acknowledge;
-import org.apache.flink.runtime.rpc.akka.messages.UpdateTaskExecutionState;
-import org.apache.flink.runtime.taskmanager.TaskExecutionState;
-import scala.concurrent.Future;
-import scala.reflect.ClassTag$;
-
-public class JobMasterAkkaGateway extends BaseAkkaGateway implements JobMasterGateway {
-	private final AskableActorRef actorRef;
-	private final Timeout timeout;
-
-	public JobMasterAkkaGateway(ActorRef actorRef, Timeout timeout) {
-		this.actorRef = new AskableActorRef(actorRef);
-		this.timeout = timeout;
-	}
-
-	@Override
-	public Future<Acknowledge> updateTaskExecutionState(TaskExecutionState taskExecutionState) {
-		return actorRef.ask(new UpdateTaskExecutionState(taskExecutionState), timeout)
-			.mapTo(ClassTag$.MODULE$.<Acknowledge>apply(Acknowledge.class));
-	}
-
-	@Override
-	public void registerAtResourceManager(String address) {
-		actorRef.actorRef().tell(new RegisterAtResourceManager(address), actorRef.actorRef());
-	}
-
-	@Override
-	public ActorRef getActorRef() {
-		return actorRef.actorRef();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/67e6f574/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/CallAsync.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/CallAsync.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/CallAsync.java
new file mode 100644
index 0000000..79b7825
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/CallAsync.java
@@ -0,0 +1,41 @@
+/*
+ * 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.rpc.akka.messages;
+
+import org.apache.flink.util.Preconditions;
+
+import java.io.Serializable;
+import java.util.concurrent.Callable;
+
+/**
+ * Message for asynchronous callable invocations
+ */
+public final class CallAsync implements Serializable {
+	private static final long serialVersionUID = 2834204738928484060L;
+
+	private transient Callable<?> callable;
+
+	public CallAsync(Callable<?> callable) {
+		this.callable = Preconditions.checkNotNull(callable);
+	}
+
+	public Callable<?> getCallable() {
+		return callable;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/67e6f574/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/CallableMessage.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/CallableMessage.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/CallableMessage.java
deleted file mode 100644
index f0e555f..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/CallableMessage.java
+++ /dev/null
@@ -1,33 +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.rpc.akka.messages;
-
-import java.util.concurrent.Callable;
-
-public class CallableMessage<V> {
-	private final Callable<V> callable;
-
-	public CallableMessage(Callable<V> callable) {
-		this.callable = callable;
-	}
-
-	public Callable<V> getCallable() {
-		return callable;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/67e6f574/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/CancelTask.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/CancelTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/CancelTask.java
deleted file mode 100644
index 0b9e9dc..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/CancelTask.java
+++ /dev/null
@@ -1,36 +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.rpc.akka.messages;
-
-import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
-
-import java.io.Serializable;
-
-public class CancelTask implements Serializable {
-	private static final long serialVersionUID = -2998176874447950595L;
-	private final ExecutionAttemptID executionAttemptID;
-
-	public CancelTask(ExecutionAttemptID executionAttemptID) {
-		this.executionAttemptID = executionAttemptID;
-	}
-
-	public ExecutionAttemptID getExecutionAttemptID() {
-		return executionAttemptID;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/67e6f574/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/ExecuteTask.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/ExecuteTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/ExecuteTask.java
deleted file mode 100644
index a83d539..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/ExecuteTask.java
+++ /dev/null
@@ -1,36 +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.rpc.akka.messages;
-
-import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor;
-
-import java.io.Serializable;
-
-public class ExecuteTask implements Serializable {
-	private static final long serialVersionUID = -6769958430967048348L;
-	private final TaskDeploymentDescriptor taskDeploymentDescriptor;
-
-	public ExecuteTask(TaskDeploymentDescriptor taskDeploymentDescriptor) {
-		this.taskDeploymentDescriptor = taskDeploymentDescriptor;
-	}
-
-	public TaskDeploymentDescriptor getTaskDeploymentDescriptor() {
-		return taskDeploymentDescriptor;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/67e6f574/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RegisterAtResourceManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RegisterAtResourceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RegisterAtResourceManager.java
deleted file mode 100644
index 3ade082..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RegisterAtResourceManager.java
+++ /dev/null
@@ -1,36 +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.rpc.akka.messages;
-
-import java.io.Serializable;
-
-public class RegisterAtResourceManager implements Serializable {
-
-	private static final long serialVersionUID = -4175905742620903602L;
-
-	private final String address;
-
-	public RegisterAtResourceManager(String address) {
-		this.address = address;
-	}
-
-	public String getAddress() {
-		return address;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/67e6f574/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RegisterJobMaster.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RegisterJobMaster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RegisterJobMaster.java
deleted file mode 100644
index b35ea38..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RegisterJobMaster.java
+++ /dev/null
@@ -1,36 +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.rpc.akka.messages;
-
-import org.apache.flink.runtime.rpc.resourcemanager.JobMasterRegistration;
-
-import java.io.Serializable;
-
-public class RegisterJobMaster implements Serializable{
-	private static final long serialVersionUID = -4616879574192641507L;
-	private final JobMasterRegistration jobMasterRegistration;
-
-	public RegisterJobMaster(JobMasterRegistration jobMasterRegistration) {
-		this.jobMasterRegistration = jobMasterRegistration;
-	}
-
-	public JobMasterRegistration getJobMasterRegistration() {
-		return jobMasterRegistration;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/67e6f574/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RequestSlot.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RequestSlot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RequestSlot.java
deleted file mode 100644
index 85ceeec..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RequestSlot.java
+++ /dev/null
@@ -1,37 +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.rpc.akka.messages;
-
-import org.apache.flink.runtime.rpc.resourcemanager.SlotRequest;
-
-import java.io.Serializable;
-
-public class RequestSlot implements Serializable {
-	private static final long serialVersionUID = 7207463889348525866L;
-
-	private final SlotRequest slotRequest;
-
-	public RequestSlot(SlotRequest slotRequest) {
-		this.slotRequest = slotRequest;
-	}
-
-	public SlotRequest getSlotRequest() {
-		return slotRequest;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/67e6f574/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RpcInvocation.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RpcInvocation.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RpcInvocation.java
new file mode 100644
index 0000000..5d52ef1
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RpcInvocation.java
@@ -0,0 +1,98 @@
+/*
+ * 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.rpc.akka.messages;
+
+import org.apache.flink.util.Preconditions;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.Serializable;
+
+/**
+ * Rpc invocation message containing the remote procedure name, its parameter types and the
+ * corresponding call arguments.
+ */
+public final class RpcInvocation implements Serializable {
+	private static final long serialVersionUID = -7058254033460536037L;
+
+	private final String methodName;
+	private final Class<?>[] parameterTypes;
+	private transient Object[] args;
+
+	public RpcInvocation(String methodName, Class<?>[] parameterTypes, Object[] args) {
+		this.methodName = Preconditions.checkNotNull(methodName);
+		this.parameterTypes = Preconditions.checkNotNull(parameterTypes);
+		this.args = args;
+	}
+
+	public String getMethodName() {
+		return methodName;
+	}
+
+	public Class<?>[] getParameterTypes() {
+		return parameterTypes;
+	}
+
+	public Object[] getArgs() {
+		return args;
+	}
+
+	private void writeObject(ObjectOutputStream oos) throws IOException {
+		oos.defaultWriteObject();
+
+		if (args != null) {
+			// write has args true
+			oos.writeBoolean(true);
+
+			for (int i = 0; i < args.length; i++) {
+				try {
+					oos.writeObject(args[i]);
+				} catch (IOException e) {
+					Class<?> argClass = args[i].getClass();
+
+					throw new IOException("Could not write " + i + "th argument of method " +
+						methodName + ". The argument type is " + argClass + ". " +
+						"Make sure that this type is serializable.", e);
+				}
+			}
+		} else {
+			// write has args false
+			oos.writeBoolean(false);
+		}
+	}
+
+	private void readObject(ObjectInputStream ois) throws IOException, ClassNotFoundException {
+		ois.defaultReadObject();
+
+		boolean hasArgs = ois.readBoolean();
+
+		if (hasArgs) {
+			int numberArguments = parameterTypes.length;
+
+			args = new Object[numberArguments];
+
+			for (int i = 0; i < numberArguments; i++) {
+				args[i] = ois.readObject();
+			}
+		} else {
+			args = null;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/67e6f574/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RunAsync.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RunAsync.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RunAsync.java
new file mode 100644
index 0000000..fb95852
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RunAsync.java
@@ -0,0 +1,40 @@
+/*
+ * 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.rpc.akka.messages;
+
+import org.apache.flink.util.Preconditions;
+
+import java.io.Serializable;
+
+/**
+ * Message for asynchronous runnable invocations
+ */
+public final class RunAsync implements Serializable {
+	private static final long serialVersionUID = -3080595100695371036L;
+
+	private final transient Runnable runnable;
+
+	public RunAsync(Runnable runnable) {
+		this.runnable = Preconditions.checkNotNull(runnable);
+	}
+
+	public Runnable getRunnable() {
+		return runnable;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/67e6f574/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RunnableMessage.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RunnableMessage.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RunnableMessage.java
deleted file mode 100644
index 3556738..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RunnableMessage.java
+++ /dev/null
@@ -1,31 +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.rpc.akka.messages;
-
-public class RunnableMessage {
-	private final Runnable runnable;
-
-	public RunnableMessage(Runnable runnable) {
-		this.runnable = runnable;
-	}
-
-	public Runnable getRunnable() {
-		return runnable;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/67e6f574/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/UpdateTaskExecutionState.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/UpdateTaskExecutionState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/UpdateTaskExecutionState.java
deleted file mode 100644
index f89cd2f..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/UpdateTaskExecutionState.java
+++ /dev/null
@@ -1,37 +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.rpc.akka.messages;
-
-import org.apache.flink.runtime.taskmanager.TaskExecutionState;
-
-import java.io.Serializable;
-
-public class UpdateTaskExecutionState implements Serializable{
-	private static final long serialVersionUID = -6662229114427331436L;
-
-	private final TaskExecutionState taskExecutionState;
-
-	public UpdateTaskExecutionState(TaskExecutionState taskExecutionState) {
-		this.taskExecutionState = taskExecutionState;
-	}
-
-	public TaskExecutionState getTaskExecutionState() {
-		return taskExecutionState;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/67e6f574/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/resourcemanager/ResourceManagerAkkaActor.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/resourcemanager/ResourceManagerAkkaActor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/resourcemanager/ResourceManagerAkkaActor.java
deleted file mode 100644
index 13101f9..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/resourcemanager/ResourceManagerAkkaActor.java
+++ /dev/null
@@ -1,65 +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.rpc.akka.resourcemanager;
-
-import akka.actor.ActorRef;
-import akka.actor.Status;
-import akka.pattern.Patterns;
-import org.apache.flink.runtime.rpc.akka.BaseAkkaActor;
-import org.apache.flink.runtime.rpc.resourcemanager.RegistrationResponse;
-import org.apache.flink.runtime.rpc.resourcemanager.ResourceManager;
-import org.apache.flink.runtime.rpc.resourcemanager.SlotAssignment;
-import org.apache.flink.runtime.rpc.akka.messages.RegisterJobMaster;
-import org.apache.flink.runtime.rpc.akka.messages.RequestSlot;
-import scala.concurrent.Future;
-
-public class ResourceManagerAkkaActor extends BaseAkkaActor {
-	private final ResourceManager resourceManager;
-
-	public ResourceManagerAkkaActor(ResourceManager resourceManager) {
-		this.resourceManager = resourceManager;
-	}
-
-	@Override
-	public void onReceive(Object message) throws Exception {
-		final ActorRef sender = getSender();
-
-		if (message instanceof RegisterJobMaster) {
-			RegisterJobMaster registerJobMaster = (RegisterJobMaster) message;
-
-			try {
-				Future<RegistrationResponse> response = resourceManager.registerJobMaster(registerJobMaster.getJobMasterRegistration());
-				Patterns.pipe(response, getContext().dispatcher()).to(sender());
-			} catch (Exception e) {
-				sender.tell(new Status.Failure(e), getSelf());
-			}
-		} else if (message instanceof RequestSlot) {
-			RequestSlot requestSlot = (RequestSlot) message;
-
-			try {
-				SlotAssignment response = resourceManager.requestSlot(requestSlot.getSlotRequest());
-				sender.tell(new Status.Success(response), getSelf());
-			} catch (Exception e) {
-				sender.tell(new Status.Failure(e), getSelf());
-			}
-		} else {
-			super.onReceive(message);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/67e6f574/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/resourcemanager/ResourceManagerAkkaGateway.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/resourcemanager/ResourceManagerAkkaGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/resourcemanager/ResourceManagerAkkaGateway.java
deleted file mode 100644
index 1304707..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/resourcemanager/ResourceManagerAkkaGateway.java
+++ /dev/null
@@ -1,67 +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.rpc.akka.resourcemanager;
-
-import akka.actor.ActorRef;
-import akka.pattern.AskableActorRef;
-import akka.util.Timeout;
-import org.apache.flink.runtime.rpc.akka.BaseAkkaGateway;
-import org.apache.flink.runtime.rpc.resourcemanager.JobMasterRegistration;
-import org.apache.flink.runtime.rpc.resourcemanager.RegistrationResponse;
-import org.apache.flink.runtime.rpc.resourcemanager.ResourceManagerGateway;
-import org.apache.flink.runtime.rpc.resourcemanager.SlotAssignment;
-import org.apache.flink.runtime.rpc.resourcemanager.SlotRequest;
-import org.apache.flink.runtime.rpc.akka.messages.RegisterJobMaster;
-import org.apache.flink.runtime.rpc.akka.messages.RequestSlot;
-import scala.concurrent.Future;
-import scala.concurrent.duration.FiniteDuration;
-import scala.reflect.ClassTag$;
-
-public class ResourceManagerAkkaGateway extends BaseAkkaGateway implements ResourceManagerGateway {
-	private final AskableActorRef actorRef;
-	private final Timeout timeout;
-
-	public ResourceManagerAkkaGateway(ActorRef actorRef, Timeout timeout) {
-		this.actorRef = new AskableActorRef(actorRef);
-		this.timeout = timeout;
-	}
-
-	@Override
-	public Future<RegistrationResponse> registerJobMaster(JobMasterRegistration jobMasterRegistration, FiniteDuration timeout) {
-		return actorRef.ask(new RegisterJobMaster(jobMasterRegistration), new Timeout(timeout))
-			.mapTo(ClassTag$.MODULE$.<RegistrationResponse>apply(RegistrationResponse.class));
-	}
-
-	@Override
-	public Future<RegistrationResponse> registerJobMaster(JobMasterRegistration jobMasterRegistration) {
-		return actorRef.ask(new RegisterJobMaster(jobMasterRegistration), timeout)
-			.mapTo(ClassTag$.MODULE$.<RegistrationResponse>apply(RegistrationResponse.class));
-	}
-
-	@Override
-	public Future<SlotAssignment> requestSlot(SlotRequest slotRequest) {
-		return actorRef.ask(new RequestSlot(slotRequest), timeout)
-			.mapTo(ClassTag$.MODULE$.<SlotAssignment>apply(SlotAssignment.class));
-	}
-
-	@Override
-	public ActorRef getActorRef() {
-		return actorRef.actorRef();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/67e6f574/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/taskexecutor/TaskExecutorAkkaActor.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/taskexecutor/TaskExecutorAkkaActor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/taskexecutor/TaskExecutorAkkaActor.java
deleted file mode 100644
index ed522cc..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/taskexecutor/TaskExecutorAkkaActor.java
+++ /dev/null
@@ -1,77 +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.rpc.akka.taskexecutor;
-
-import akka.actor.ActorRef;
-import akka.actor.Status;
-import akka.dispatch.OnComplete;
-import org.apache.flink.runtime.messages.Acknowledge;
-import org.apache.flink.runtime.rpc.akka.BaseAkkaActor;
-import org.apache.flink.runtime.rpc.akka.messages.CancelTask;
-import org.apache.flink.runtime.rpc.akka.messages.ExecuteTask;
-import org.apache.flink.runtime.rpc.taskexecutor.TaskExecutorGateway;
-
-public class TaskExecutorAkkaActor extends BaseAkkaActor {
-	private final TaskExecutorGateway taskExecutor;
-
-	public TaskExecutorAkkaActor(TaskExecutorGateway taskExecutor) {
-		this.taskExecutor = taskExecutor;
-	}
-
-	@Override
-	public void onReceive(Object message) throws Exception {
-		final ActorRef sender = getSender();
-
-		if (message instanceof ExecuteTask) {
-			ExecuteTask executeTask = (ExecuteTask) message;
-
-			taskExecutor.executeTask(executeTask.getTaskDeploymentDescriptor()).onComplete(
-				new OnComplete<Acknowledge>() {
-					@Override
-					public void onComplete(Throwable failure, Acknowledge success) throws Throwable {
-						if (failure != null) {
-							sender.tell(new Status.Failure(failure), getSelf());
-						} else {
-							sender.tell(new Status.Success(Acknowledge.get()), getSelf());
-						}
-					}
-				},
-				getContext().dispatcher()
-			);
-		} else if (message instanceof CancelTask) {
-			CancelTask cancelTask = (CancelTask) message;
-
-			taskExecutor.cancelTask(cancelTask.getExecutionAttemptID()).onComplete(
-				new OnComplete<Acknowledge>() {
-					@Override
-					public void onComplete(Throwable failure, Acknowledge success) throws Throwable {
-						if (failure != null) {
-							sender.tell(new Status.Failure(failure), getSelf());
-						} else {
-							sender.tell(new Status.Success(Acknowledge.get()), getSelf());
-						}
-					}
-				},
-				getContext().dispatcher()
-			);
-		} else {
-			super.onReceive(message);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/67e6f574/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/taskexecutor/TaskExecutorAkkaGateway.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/taskexecutor/TaskExecutorAkkaGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/taskexecutor/TaskExecutorAkkaGateway.java
deleted file mode 100644
index 7f0a522..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/taskexecutor/TaskExecutorAkkaGateway.java
+++ /dev/null
@@ -1,59 +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.rpc.akka.taskexecutor;
-
-import akka.actor.ActorRef;
-import akka.pattern.AskableActorRef;
-import akka.util.Timeout;
-import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor;
-import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
-import org.apache.flink.runtime.messages.Acknowledge;
-import org.apache.flink.runtime.rpc.akka.BaseAkkaGateway;
-import org.apache.flink.runtime.rpc.akka.messages.CancelTask;
-import org.apache.flink.runtime.rpc.akka.messages.ExecuteTask;
-import org.apache.flink.runtime.rpc.taskexecutor.TaskExecutorGateway;
-import scala.concurrent.Future;
-import scala.reflect.ClassTag$;
-
-public class TaskExecutorAkkaGateway extends BaseAkkaGateway implements TaskExecutorGateway {
-	private final AskableActorRef actorRef;
-	private final Timeout timeout;
-
-	public TaskExecutorAkkaGateway(ActorRef actorRef, Timeout timeout) {
-		this.actorRef = new AskableActorRef(actorRef);
-		this.timeout = timeout;
-	}
-
-	@Override
-	public Future<Acknowledge> executeTask(TaskDeploymentDescriptor taskDeploymentDescriptor) {
-		return actorRef.ask(new ExecuteTask(taskDeploymentDescriptor), timeout)
-			.mapTo(ClassTag$.MODULE$.<Acknowledge>apply(Acknowledge.class));
-	}
-
-	@Override
-	public Future<Acknowledge> cancelTask(ExecutionAttemptID executionAttemptId) {
-		return actorRef.ask(new CancelTask(executionAttemptId), timeout)
-			.mapTo(ClassTag$.MODULE$.<Acknowledge>apply(Acknowledge.class));
-	}
-
-	@Override
-	public ActorRef getActorRef() {
-		return actorRef.actorRef();
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/67e6f574/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/jobmaster/JobMaster.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/jobmaster/JobMaster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/jobmaster/JobMaster.java
index b81b19c..e53cd68 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/jobmaster/JobMaster.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/jobmaster/JobMaster.java
@@ -30,6 +30,7 @@ import org.apache.flink.runtime.rpc.resourcemanager.ResourceManagerGateway;
 import org.apache.flink.runtime.rpc.RpcEndpoint;
 import org.apache.flink.runtime.rpc.RpcService;
 import org.apache.flink.runtime.taskmanager.TaskExecutionState;
+import org.apache.flink.util.Preconditions;
 import scala.Tuple2;
 import scala.concurrent.ExecutionContext;
 import scala.concurrent.ExecutionContext$;
@@ -76,7 +77,8 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 
 	public JobMaster(RpcService rpcService, ExecutorService executorService) {
 		super(rpcService);
-		executionContext = ExecutionContext$.MODULE$.fromExecutor(executorService);
+		executionContext = ExecutionContext$.MODULE$.fromExecutor(
+			Preconditions.checkNotNull(executorService));
 		scheduledExecutorService = new ScheduledThreadPoolExecutor(1);
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/67e6f574/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManager.java
index c7e8def..729ef0c 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManager.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManager.java
@@ -25,6 +25,7 @@ import org.apache.flink.runtime.rpc.RpcEndpoint;
 import org.apache.flink.runtime.rpc.RpcService;
 import org.apache.flink.runtime.rpc.jobmaster.JobMaster;
 import org.apache.flink.runtime.rpc.jobmaster.JobMasterGateway;
+import org.apache.flink.util.Preconditions;
 import scala.concurrent.ExecutionContext;
 import scala.concurrent.ExecutionContext$;
 import scala.concurrent.Future;
@@ -49,7 +50,8 @@ public class ResourceManager extends RpcEndpoint<ResourceManagerGateway> {
 
 	public ResourceManager(RpcService rpcService, ExecutorService executorService) {
 		super(rpcService);
-		this.executionContext = ExecutionContext$.MODULE$.fromExecutor(executorService);
+		this.executionContext = ExecutionContext$.MODULE$.fromExecutor(
+			Preconditions.checkNotNull(executorService));
 		this.jobMasterGateways = new HashMap<>();
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/67e6f574/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java
index cdfc3bd..3a7dd9f 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java
@@ -25,6 +25,7 @@ import org.apache.flink.runtime.messages.Acknowledge;
 import org.apache.flink.runtime.rpc.RpcMethod;
 import org.apache.flink.runtime.rpc.RpcEndpoint;
 import org.apache.flink.runtime.rpc.RpcService;
+import org.apache.flink.util.Preconditions;
 import scala.concurrent.ExecutionContext;
 
 import java.util.HashSet;
@@ -47,7 +48,8 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 
 	public TaskExecutor(RpcService rpcService, ExecutorService executorService) {
 		super(rpcService);
-		this.executionContext = ExecutionContexts$.MODULE$.fromExecutor(executorService);
+		this.executionContext = ExecutionContexts$.MODULE$.fromExecutor(
+			Preconditions.checkNotNull(executorService));
 	}
 
 	/**

http://git-wip-us.apache.org/repos/asf/flink/blob/67e6f574/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java
index 0ded25e..e50533e 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java
@@ -18,15 +18,15 @@
 
 package org.apache.flink.runtime.rpc;
 
+import org.apache.flink.util.ReflectionUtil;
 import org.apache.flink.util.TestLogger;
 import org.junit.Test;
 import org.reflections.Reflections;
 import scala.concurrent.Future;
+import scala.concurrent.duration.FiniteDuration;
 
 import java.lang.annotation.Annotation;
 import java.lang.reflect.Method;
-import java.lang.reflect.ParameterizedType;
-import java.lang.reflect.Type;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -51,9 +51,8 @@ public class RpcCompletenessTest extends TestLogger {
 
 		for (Class<? extends RpcEndpoint> rpcEndpoint :classes){
 			c = rpcEndpoint;
-			Type superClass = c.getGenericSuperclass();
 
-			Class<?> rpcGatewayType = extractTypeParameter(superClass, 0);
+			Class<?> rpcGatewayType = ReflectionUtil.getTemplateType1(c);
 
 			if (rpcGatewayType != null) {
 				checkCompleteness(rpcEndpoint, (Class<? extends RpcGateway>) rpcGatewayType);
@@ -137,13 +136,16 @@ public class RpcCompletenessTest extends TestLogger {
 		}
 
 		Annotation[][] parameterAnnotations = gatewayMethod.getParameterAnnotations();
+		Class<?>[] parameterTypes = gatewayMethod.getParameterTypes();
 		int rpcTimeoutParameters = 0;
 
-		for (Annotation[] parameterAnnotation : parameterAnnotations) {
-			for (Annotation annotation : parameterAnnotation) {
-				if (annotation.equals(RpcTimeout.class)) {
-					rpcTimeoutParameters++;
-				}
+		for (int i = 0; i < parameterAnnotations.length; i++) {
+			if (isRpcTimeout(parameterAnnotations[i])) {
+				assertTrue(
+					"The rpc timeout has to be of type " + FiniteDuration.class.getName() + ".",
+					parameterTypes[i].equals(FiniteDuration.class));
+
+				rpcTimeoutParameters++;
 			}
 		}
 
@@ -211,10 +213,10 @@ public class RpcCompletenessTest extends TestLogger {
 				if (!futureClass.equals(RpcCompletenessTest.futureClass)) {
 					return false;
 				} else {
-					Class<?> valueClass = extractTypeParameter(futureClass, 0);
+					Class<?> valueClass = ReflectionUtil.getTemplateType1(gatewayMethod.getGenericReturnType());
 
 					if (endpointMethod.getReturnType().equals(futureClass)) {
-						Class<?> rpcEndpointValueClass = extractTypeParameter(endpointMethod.getReturnType(), 0);
+						Class<?> rpcEndpointValueClass = ReflectionUtil.getTemplateType1(endpointMethod.getGenericReturnType());
 
 						// check if we have the same future value types
 						if (valueClass != null && rpcEndpointValueClass != null && !checkType(valueClass, rpcEndpointValueClass)) {
@@ -251,7 +253,7 @@ public class RpcCompletenessTest extends TestLogger {
 		if (method.getReturnType().equals(Void.TYPE)) {
 			builder.append("void").append(" ");
 		} else if (method.getReturnType().equals(futureClass)) {
-			Class<?> valueClass = extractTypeParameter(method.getGenericReturnType(), 0);
+			Class<?> valueClass = ReflectionUtil.getTemplateType1(method.getGenericReturnType());
 
 			builder
 				.append(futureClass.getSimpleName())
@@ -291,30 +293,6 @@ public class RpcCompletenessTest extends TestLogger {
 		return builder.toString();
 	}
 
-	private Class<?> extractTypeParameter(Type genericType, int position) {
-		if (genericType instanceof ParameterizedType) {
-			ParameterizedType parameterizedType = (ParameterizedType) genericType;
-
-			Type[] typeArguments = parameterizedType.getActualTypeArguments();
-
-			if (position < 0 || position >= typeArguments.length) {
-				throw new IndexOutOfBoundsException("The generic type " +
-					parameterizedType.getRawType() + " only has " + typeArguments.length +
-					" type arguments.");
-			} else {
-				Type typeArgument = typeArguments[position];
-
-				if (typeArgument instanceof Class<?>) {
-					return (Class<?>) typeArgument;
-				} else {
-					return null;
-				}
-			}
-		} else {
-			return null;
-		}
-	}
-
 	private boolean isRpcTimeout(Annotation[] annotations) {
 		for (Annotation annotation : annotations) {
 			if (annotation.annotationType().equals(RpcTimeout.class)) {


[26/50] [abbrv] flink git commit: [FLINK-4383] [rpc] Eagerly serialize remote rpc invocation messages

Posted by tr...@apache.org.
[FLINK-4383] [rpc] Eagerly serialize remote rpc invocation messages

This PR introduces an eager serialization for remote rpc invocation messages.
That way it is possible to check whether the message is serializable and
whether it exceeds the maximum allowed akka frame size. If either of these
constraints is violated, a proper exception is thrown instead of simply
swallowing the exception as Akka does it.

Address PR comments

This closes #2365.


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

Branch: refs/heads/flip-6
Commit: dfbbe55d989e8617d4361b28bfc200714284a5cd
Parents: 2963e6c
Author: Till Rohrmann <tr...@apache.org>
Authored: Fri Aug 12 10:32:30 2016 +0200
Committer: Till Rohrmann <tr...@apache.org>
Committed: Thu Sep 8 17:26:57 2016 +0200

----------------------------------------------------------------------
 .../flink/runtime/rpc/akka/AkkaGateway.java     |   2 +-
 .../runtime/rpc/akka/AkkaInvocationHandler.java |  83 ++++++--
 .../flink/runtime/rpc/akka/AkkaRpcActor.java    |  26 ++-
 .../flink/runtime/rpc/akka/AkkaRpcService.java  |  20 +-
 .../rpc/akka/messages/LocalRpcInvocation.java   |  54 +++++
 .../rpc/akka/messages/RemoteRpcInvocation.java  | 206 ++++++++++++++++++
 .../rpc/akka/messages/RpcInvocation.java        | 106 +++-------
 .../runtime/rpc/akka/AkkaRpcServiceTest.java    |   2 +-
 .../rpc/akka/MessageSerializationTest.java      | 210 +++++++++++++++++++
 9 files changed, 597 insertions(+), 112 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/dfbbe55d/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaGateway.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaGateway.java
index ec3091c..f6125dc 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaGateway.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaGateway.java
@@ -26,5 +26,5 @@ import org.apache.flink.runtime.rpc.RpcGateway;
  */
 interface AkkaGateway extends RpcGateway {
 
-	ActorRef getRpcServer();
+	ActorRef getRpcEndpoint();
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/dfbbe55d/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaInvocationHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaInvocationHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaInvocationHandler.java
index 580b161..297104b 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaInvocationHandler.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaInvocationHandler.java
@@ -25,13 +25,17 @@ import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.runtime.rpc.MainThreadExecutor;
 import org.apache.flink.runtime.rpc.RpcTimeout;
 import org.apache.flink.runtime.rpc.akka.messages.CallAsync;
+import org.apache.flink.runtime.rpc.akka.messages.LocalRpcInvocation;
+import org.apache.flink.runtime.rpc.akka.messages.RemoteRpcInvocation;
 import org.apache.flink.runtime.rpc.akka.messages.RpcInvocation;
 import org.apache.flink.runtime.rpc.akka.messages.RunAsync;
 import org.apache.flink.util.Preconditions;
+import org.apache.log4j.Logger;
 import scala.concurrent.Await;
 import scala.concurrent.Future;
 import scala.concurrent.duration.FiniteDuration;
 
+import java.io.IOException;
 import java.lang.annotation.Annotation;
 import java.lang.reflect.InvocationHandler;
 import java.lang.reflect.Method;
@@ -42,19 +46,28 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
 import static org.apache.flink.util.Preconditions.checkArgument;
 
 /**
- * Invocation handler to be used with a {@link AkkaRpcActor}. The invocation handler wraps the
- * rpc in a {@link RpcInvocation} message and then sends it to the {@link AkkaRpcActor} where it is
+ * Invocation handler to be used with an {@link AkkaRpcActor}. The invocation handler wraps the
+ * rpc in a {@link LocalRpcInvocation} message and then sends it to the {@link AkkaRpcActor} where it is
  * executed.
  */
 class AkkaInvocationHandler implements InvocationHandler, AkkaGateway, MainThreadExecutor {
-	private final ActorRef rpcServer;
+	private static final Logger LOG = Logger.getLogger(AkkaInvocationHandler.class);
+
+	private final ActorRef rpcEndpoint;
+
+	// whether the actor ref is local and thus no message serialization is needed
+	private final boolean isLocal;
 
 	// default timeout for asks
 	private final Timeout timeout;
 
-	AkkaInvocationHandler(ActorRef rpcServer, Timeout timeout) {
-		this.rpcServer = Preconditions.checkNotNull(rpcServer);
+	private final long maximumFramesize;
+
+	AkkaInvocationHandler(ActorRef rpcEndpoint, Timeout timeout, long maximumFramesize) {
+		this.rpcEndpoint = Preconditions.checkNotNull(rpcEndpoint);
+		this.isLocal = this.rpcEndpoint.path().address().hasLocalScope();
 		this.timeout = Preconditions.checkNotNull(timeout);
+		this.maximumFramesize = maximumFramesize;
 	}
 
 	@Override
@@ -76,23 +89,43 @@ class AkkaInvocationHandler implements InvocationHandler, AkkaGateway, MainThrea
 				parameterAnnotations,
 				args);
 
-			RpcInvocation rpcInvocation = new RpcInvocation(
-				methodName,
-				filteredArguments.f0,
-				filteredArguments.f1);
+			RpcInvocation rpcInvocation;
+
+			if (isLocal) {
+				rpcInvocation = new LocalRpcInvocation(
+					methodName,
+					filteredArguments.f0,
+					filteredArguments.f1);
+			} else {
+				try {
+					RemoteRpcInvocation remoteRpcInvocation = new RemoteRpcInvocation(
+						methodName,
+						filteredArguments.f0,
+						filteredArguments.f1);
+
+					if (remoteRpcInvocation.getSize() > maximumFramesize) {
+						throw new IOException("The rpc invocation size exceeds the maximum akka framesize.");
+					} else {
+						rpcInvocation = remoteRpcInvocation;
+					}
+				} catch (IOException e) {
+					LOG.warn("Could not create remote rpc invocation message. Failing rpc invocation because...", e);
+					throw e;
+				}
+			}
 
 			Class<?> returnType = method.getReturnType();
 
 			if (returnType.equals(Void.TYPE)) {
-				rpcServer.tell(rpcInvocation, ActorRef.noSender());
+				rpcEndpoint.tell(rpcInvocation, ActorRef.noSender());
 
 				result = null;
 			} else if (returnType.equals(Future.class)) {
 				// execute an asynchronous call
-				result = Patterns.ask(rpcServer, rpcInvocation, futureTimeout);
+				result = Patterns.ask(rpcEndpoint, rpcInvocation, futureTimeout);
 			} else {
 				// execute a synchronous call
-				Future<?> futureResult = Patterns.ask(rpcServer, rpcInvocation, futureTimeout);
+				Future<?> futureResult = Patterns.ask(rpcEndpoint, rpcInvocation, futureTimeout);
 				FiniteDuration duration = timeout.duration();
 
 				result = Await.result(futureResult, duration);
@@ -103,8 +136,8 @@ class AkkaInvocationHandler implements InvocationHandler, AkkaGateway, MainThrea
 	}
 
 	@Override
-	public ActorRef getRpcServer() {
-		return rpcServer;
+	public ActorRef getRpcEndpoint() {
+		return rpcEndpoint;
 	}
 
 	@Override
@@ -117,19 +150,25 @@ class AkkaInvocationHandler implements InvocationHandler, AkkaGateway, MainThrea
 		checkNotNull(runnable, "runnable");
 		checkArgument(delay >= 0, "delay must be zero or greater");
 		
-		// Unfortunately I couldn't find a way to allow only local communication. Therefore, the
-		// runnable field is transient transient
-		rpcServer.tell(new RunAsync(runnable, delay), ActorRef.noSender());
+		if (isLocal) {
+			rpcEndpoint.tell(new RunAsync(runnable, delay), ActorRef.noSender());
+		} else {
+			throw new RuntimeException("Trying to send a Runnable to a remote actor at " +
+				rpcEndpoint.path() + ". This is not supported.");
+		}
 	}
 
 	@Override
 	public <V> Future<V> callAsync(Callable<V> callable, Timeout callTimeout) {
-		// Unfortunately I couldn't find a way to allow only local communication. Therefore, the
-		// callable field is declared transient
-		@SuppressWarnings("unchecked")
-		Future<V> result = (Future<V>) Patterns.ask(rpcServer, new CallAsync(callable), callTimeout);
+		if(isLocal) {
+			@SuppressWarnings("unchecked")
+			Future<V> result = (Future<V>) Patterns.ask(rpcEndpoint, new CallAsync(callable), callTimeout);
 
-		return result;
+			return result;
+		} else {
+			throw new RuntimeException("Trying to send a Callable to a remote actor at " +
+				rpcEndpoint.path() + ". This is not supported.");
+		}
 	}
 
 	/**

http://git-wip-us.apache.org/repos/asf/flink/blob/dfbbe55d/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java
index 5e0a7da..dfcbcc3 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java
@@ -26,6 +26,7 @@ import org.apache.flink.runtime.rpc.MainThreadValidatorUtil;
 import org.apache.flink.runtime.rpc.RpcEndpoint;
 import org.apache.flink.runtime.rpc.RpcGateway;
 import org.apache.flink.runtime.rpc.akka.messages.CallAsync;
+import org.apache.flink.runtime.rpc.akka.messages.LocalRpcInvocation;
 import org.apache.flink.runtime.rpc.akka.messages.RpcInvocation;
 import org.apache.flink.runtime.rpc.akka.messages.RunAsync;
 
@@ -35,6 +36,7 @@ import org.slf4j.LoggerFactory;
 import scala.concurrent.Future;
 import scala.concurrent.duration.FiniteDuration;
 
+import java.io.IOException;
 import java.lang.reflect.Method;
 import java.util.concurrent.Callable;
 import java.util.concurrent.TimeUnit;
@@ -42,10 +44,10 @@ import java.util.concurrent.TimeUnit;
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**
- * Akka rpc actor which receives {@link RpcInvocation}, {@link RunAsync} and {@link CallAsync}
+ * Akka rpc actor which receives {@link LocalRpcInvocation}, {@link RunAsync} and {@link CallAsync}
  * messages.
  * <p>
- * The {@link RpcInvocation} designates a rpc and is dispatched to the given {@link RpcEndpoint}
+ * The {@link LocalRpcInvocation} designates a rpc and is dispatched to the given {@link RpcEndpoint}
  * instance.
  * <p>
  * The {@link RunAsync} and {@link CallAsync} messages contain executable code which is executed
@@ -95,15 +97,12 @@ class AkkaRpcActor<C extends RpcGateway, T extends RpcEndpoint<C>> extends Untyp
 	 * @param rpcInvocation Rpc invocation message
 	 */
 	private void handleRpcInvocation(RpcInvocation rpcInvocation) {
-		Method rpcMethod = null;
-
 		try {
-			rpcMethod = lookupRpcMethod(rpcInvocation.getMethodName(), rpcInvocation.getParameterTypes());
-		} catch (final NoSuchMethodException e) {
-			LOG.error("Could not find rpc method for rpc invocation: {}.", rpcInvocation, e);
-		}
+			String methodName = rpcInvocation.getMethodName();
+			Class<?>[] parameterTypes = rpcInvocation.getParameterTypes();
+
+			Method rpcMethod = lookupRpcMethod(methodName, parameterTypes);
 
-		if (rpcMethod != null) {
 			if (rpcMethod.getReturnType().equals(Void.TYPE)) {
 				// No return value to send back
 				try {
@@ -127,6 +126,12 @@ class AkkaRpcActor<C extends RpcGateway, T extends RpcEndpoint<C>> extends Untyp
 					getSender().tell(new Status.Failure(e), getSelf());
 				}
 			}
+		} catch(ClassNotFoundException e) {
+			LOG.error("Could not load method arguments.", e);
+		} catch (IOException e) {
+			LOG.error("Could not deserialize rpc invocation message.", e);
+		} catch (final NoSuchMethodException e) {
+			LOG.error("Could not find rpc method for rpc invocation: {}.", rpcInvocation, e);
 		}
 	}
 
@@ -195,7 +200,8 @@ class AkkaRpcActor<C extends RpcGateway, T extends RpcEndpoint<C>> extends Untyp
 	 * @param methodName Name of the method
 	 * @param parameterTypes Parameter types of the method
 	 * @return Method of the rpc endpoint
-	 * @throws NoSuchMethodException
+	 * @throws NoSuchMethodException Thrown if the method with the given name and parameter types
+	 * 									cannot be found at the rpc endpoint
 	 */
 	private Method lookupRpcMethod(final String methodName, final Class<?>[] parameterTypes) throws NoSuchMethodException {
 		return rpcEndpoint.getClass().getMethod(methodName, parameterTypes);

http://git-wip-us.apache.org/repos/asf/flink/blob/dfbbe55d/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java
index db40f10..b963c53 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java
@@ -58,17 +58,27 @@ public class AkkaRpcService implements RpcService {
 
 	private static final Logger LOG = LoggerFactory.getLogger(AkkaRpcService.class);
 
+	static final String MAXIMUM_FRAME_SIZE_PATH = "akka.remote.netty.tcp.maximum-frame-size";
+
 	private final Object lock = new Object();
 
 	private final ActorSystem actorSystem;
 	private final Timeout timeout;
 	private final Set<ActorRef> actors = new HashSet<>(4);
+	private final long maximumFramesize;
 
 	private volatile boolean stopped;
 
 	public AkkaRpcService(final ActorSystem actorSystem, final Timeout timeout) {
 		this.actorSystem = checkNotNull(actorSystem, "actor system");
 		this.timeout = checkNotNull(timeout, "timeout");
+
+		if (actorSystem.settings().config().hasPath(MAXIMUM_FRAME_SIZE_PATH)) {
+			maximumFramesize = actorSystem.settings().config().getBytes(MAXIMUM_FRAME_SIZE_PATH);
+		} else {
+			// only local communication
+			maximumFramesize = Long.MAX_VALUE;
+		}
 	}
 
 	// this method does not mutate state and is thus thread-safe
@@ -88,7 +98,7 @@ public class AkkaRpcService implements RpcService {
 			public C apply(Object obj) {
 				ActorRef actorRef = ((ActorIdentity) obj).getRef();
 
-				InvocationHandler akkaInvocationHandler = new AkkaInvocationHandler(actorRef, timeout);
+				InvocationHandler akkaInvocationHandler = new AkkaInvocationHandler(actorRef, timeout, maximumFramesize);
 
 				@SuppressWarnings("unchecked")
 				C proxy = (C) Proxy.newProxyInstance(
@@ -116,7 +126,7 @@ public class AkkaRpcService implements RpcService {
 
 		LOG.info("Starting RPC endpoint for {} at {} .", rpcEndpoint.getClass().getName(), actorRef.path());
 
-		InvocationHandler akkaInvocationHandler = new AkkaInvocationHandler(actorRef, timeout);
+		InvocationHandler akkaInvocationHandler = new AkkaInvocationHandler(actorRef, timeout, maximumFramesize);
 
 		// Rather than using the System ClassLoader directly, we derive the ClassLoader
 		// from this class . That works better in cases where Flink runs embedded and all Flink
@@ -142,12 +152,12 @@ public class AkkaRpcService implements RpcService {
 				if (stopped) {
 					return;
 				} else {
-					fromThisService = actors.remove(akkaClient.getRpcServer());
+					fromThisService = actors.remove(akkaClient.getRpcEndpoint());
 				}
 			}
 
 			if (fromThisService) {
-				ActorRef selfActorRef = akkaClient.getRpcServer();
+				ActorRef selfActorRef = akkaClient.getRpcEndpoint();
 				LOG.info("Stopping RPC endpoint {}.", selfActorRef.path());
 				selfActorRef.tell(PoisonPill.getInstance(), ActorRef.noSender());
 			} else {
@@ -178,7 +188,7 @@ public class AkkaRpcService implements RpcService {
 		checkState(!stopped, "RpcService is stopped");
 
 		if (selfGateway instanceof AkkaGateway) {
-			ActorRef actorRef = ((AkkaGateway) selfGateway).getRpcServer();
+			ActorRef actorRef = ((AkkaGateway) selfGateway).getRpcEndpoint();
 			return AkkaUtils.getAkkaURL(actorSystem, actorRef);
 		} else {
 			String className = AkkaGateway.class.getName();

http://git-wip-us.apache.org/repos/asf/flink/blob/dfbbe55d/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/LocalRpcInvocation.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/LocalRpcInvocation.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/LocalRpcInvocation.java
new file mode 100644
index 0000000..97c10d7
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/LocalRpcInvocation.java
@@ -0,0 +1,54 @@
+/*
+ * 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.rpc.akka.messages;
+
+import org.apache.flink.util.Preconditions;
+
+/**
+ * Local rpc invocation message containing the remote procedure name, its parameter types and the
+ * corresponding call arguments. This message will only be sent if the communication is local and,
+ * thus, the message does not have to be serialized.
+ */
+public final class LocalRpcInvocation implements RpcInvocation {
+
+	private final String methodName;
+	private final Class<?>[] parameterTypes;
+	private final Object[] args;
+
+	public LocalRpcInvocation(String methodName, Class<?>[] parameterTypes, Object[] args) {
+		this.methodName = Preconditions.checkNotNull(methodName);
+		this.parameterTypes = Preconditions.checkNotNull(parameterTypes);
+		this.args = args;
+	}
+
+	@Override
+	public String getMethodName() {
+		return methodName;
+	}
+
+	@Override
+	public Class<?>[] getParameterTypes() {
+		return parameterTypes;
+	}
+
+	@Override
+	public Object[] getArgs() {
+		return args;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/dfbbe55d/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RemoteRpcInvocation.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RemoteRpcInvocation.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RemoteRpcInvocation.java
new file mode 100644
index 0000000..bc26a29
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RemoteRpcInvocation.java
@@ -0,0 +1,206 @@
+/*
+ * 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.rpc.akka.messages;
+
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.SerializedValue;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.Serializable;
+
+/**
+ * Remote rpc invocation message which is used when the actor communication is remote and, thus, the
+ * message has to be serialized.
+ * <p>
+ * In order to fail fast and report an appropriate error message to the user, the method name, the
+ * parameter types and the arguments are eagerly serialized. In case the the invocation call
+ * contains a non-serializable object, then an {@link IOException} is thrown.
+ */
+public class RemoteRpcInvocation implements RpcInvocation, Serializable {
+	private static final long serialVersionUID = 6179354390913843809L;
+
+	// Serialized invocation data
+	private SerializedValue<RemoteRpcInvocation.MethodInvocation> serializedMethodInvocation;
+
+	// Transient field which is lazily initialized upon first access to the invocation data
+	private transient RemoteRpcInvocation.MethodInvocation methodInvocation;
+
+	public  RemoteRpcInvocation(
+		final String methodName,
+		final Class<?>[] parameterTypes,
+		final Object[] args) throws IOException {
+
+		serializedMethodInvocation = new SerializedValue<>(new RemoteRpcInvocation.MethodInvocation(methodName, parameterTypes, args));
+		methodInvocation = null;
+	}
+
+	@Override
+	public String getMethodName() throws IOException, ClassNotFoundException {
+		deserializeMethodInvocation();
+
+		return methodInvocation.getMethodName();
+	}
+
+	@Override
+	public Class<?>[] getParameterTypes() throws IOException, ClassNotFoundException {
+		deserializeMethodInvocation();
+
+		return methodInvocation.getParameterTypes();
+	}
+
+	@Override
+	public Object[] getArgs() throws IOException, ClassNotFoundException {
+		deserializeMethodInvocation();
+
+		return methodInvocation.getArgs();
+	}
+
+	/**
+	 * Size (#bytes of the serialized data) of the rpc invocation message.
+	 *
+	 * @return Size of the remote rpc invocation message
+	 */
+	public long getSize() {
+		return serializedMethodInvocation.getByteArray().length;
+	}
+
+	private void deserializeMethodInvocation() throws IOException, ClassNotFoundException {
+		if (methodInvocation == null) {
+			methodInvocation = serializedMethodInvocation.deserializeValue(ClassLoader.getSystemClassLoader());
+		}
+	}
+
+	// -------------------------------------------------------------------
+	// Serialization methods
+	// -------------------------------------------------------------------
+
+	private void writeObject(ObjectOutputStream oos) throws IOException {
+		oos.writeObject(serializedMethodInvocation);
+	}
+
+	@SuppressWarnings("unchecked")
+	private void readObject(ObjectInputStream ois) throws IOException, ClassNotFoundException {
+		serializedMethodInvocation = (SerializedValue<RemoteRpcInvocation.MethodInvocation>) ois.readObject();
+		methodInvocation = null;
+	}
+
+	// -------------------------------------------------------------------
+	// Utility classes
+	// -------------------------------------------------------------------
+
+	/**
+	 * Wrapper class for the method invocation information
+	 */
+	private static final class MethodInvocation implements Serializable {
+		private static final long serialVersionUID = 9187962608946082519L;
+
+		private String methodName;
+		private Class<?>[] parameterTypes;
+		private Object[] args;
+
+		private MethodInvocation(final String methodName, final Class<?>[] parameterTypes, final Object[] args) {
+			this.methodName = methodName;
+			this.parameterTypes = Preconditions.checkNotNull(parameterTypes);
+			this.args = args;
+		}
+
+		String getMethodName() {
+			return methodName;
+		}
+
+		Class<?>[] getParameterTypes() {
+			return parameterTypes;
+		}
+
+		Object[] getArgs() {
+			return args;
+		}
+
+		private void writeObject(ObjectOutputStream oos) throws IOException {
+			oos.writeUTF(methodName);
+
+			oos.writeInt(parameterTypes.length);
+
+			for (Class<?> parameterType : parameterTypes) {
+				oos.writeObject(parameterType);
+			}
+
+			if (args != null) {
+				oos.writeBoolean(true);
+
+				for (int i = 0; i < args.length; i++) {
+					try {
+						oos.writeObject(args[i]);
+					} catch (IOException e) {
+						throw new IOException("Could not serialize " + i + "th argument of method " +
+							methodName + ". This indicates that the argument type " +
+							args.getClass().getName() + " is not serializable. Arguments have to " +
+							"be serializable for remote rpc calls.", e);
+					}
+				}
+			} else {
+				oos.writeBoolean(false);
+			}
+		}
+
+		private void readObject(ObjectInputStream ois) throws IOException, ClassNotFoundException {
+			methodName = ois.readUTF();
+
+			int length = ois.readInt();
+
+			parameterTypes = new Class<?>[length];
+
+			for (int i = 0; i < length; i++) {
+				try {
+					parameterTypes[i] = (Class<?>) ois.readObject();
+				} catch (IOException e) {
+					throw new IOException("Could not deserialize " + i + "th parameter type of method " +
+						methodName + '.', e);
+				} catch (ClassNotFoundException e) {
+					throw new ClassNotFoundException("Could not deserialize " + i + "th " +
+						"parameter type of method " + methodName + ". This indicates that the parameter " +
+						"type is not part of the system class loader.", e);
+				}
+			}
+
+			boolean hasArgs = ois.readBoolean();
+
+			if (hasArgs) {
+				args = new Object[length];
+
+				for (int i = 0; i < length; i++) {
+					try {
+						args[i] = ois.readObject();
+					} catch (IOException e) {
+						throw new IOException("Could not deserialize " + i + "th argument of method " +
+							methodName + '.', e);
+					} catch (ClassNotFoundException e) {
+						throw new ClassNotFoundException("Could not deserialize " + i + "th " +
+							"argument of method " + methodName + ". This indicates that the argument " +
+							"type is not part of the system class loader.", e);
+					}
+				}
+			} else {
+				args = null;
+			}
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/dfbbe55d/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RpcInvocation.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RpcInvocation.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RpcInvocation.java
index 5d52ef1..b174c99 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RpcInvocation.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RpcInvocation.java
@@ -18,81 +18,41 @@
 
 package org.apache.flink.runtime.rpc.akka.messages;
 
-import org.apache.flink.util.Preconditions;
-
 import java.io.IOException;
-import java.io.ObjectInputStream;
-import java.io.ObjectOutputStream;
-import java.io.Serializable;
 
 /**
- * Rpc invocation message containing the remote procedure name, its parameter types and the
- * corresponding call arguments.
+ * Interface for rpc invocation messages. The interface allows to request all necessary information
+ * to lookup a method and call it with the corresponding arguments.
  */
-public final class RpcInvocation implements Serializable {
-	private static final long serialVersionUID = -7058254033460536037L;
-
-	private final String methodName;
-	private final Class<?>[] parameterTypes;
-	private transient Object[] args;
-
-	public RpcInvocation(String methodName, Class<?>[] parameterTypes, Object[] args) {
-		this.methodName = Preconditions.checkNotNull(methodName);
-		this.parameterTypes = Preconditions.checkNotNull(parameterTypes);
-		this.args = args;
-	}
-
-	public String getMethodName() {
-		return methodName;
-	}
-
-	public Class<?>[] getParameterTypes() {
-		return parameterTypes;
-	}
-
-	public Object[] getArgs() {
-		return args;
-	}
-
-	private void writeObject(ObjectOutputStream oos) throws IOException {
-		oos.defaultWriteObject();
-
-		if (args != null) {
-			// write has args true
-			oos.writeBoolean(true);
-
-			for (int i = 0; i < args.length; i++) {
-				try {
-					oos.writeObject(args[i]);
-				} catch (IOException e) {
-					Class<?> argClass = args[i].getClass();
-
-					throw new IOException("Could not write " + i + "th argument of method " +
-						methodName + ". The argument type is " + argClass + ". " +
-						"Make sure that this type is serializable.", e);
-				}
-			}
-		} else {
-			// write has args false
-			oos.writeBoolean(false);
-		}
-	}
-
-	private void readObject(ObjectInputStream ois) throws IOException, ClassNotFoundException {
-		ois.defaultReadObject();
-
-		boolean hasArgs = ois.readBoolean();
-
-		if (hasArgs) {
-			int numberArguments = parameterTypes.length;
-
-			args = new Object[numberArguments];
-
-			for (int i = 0; i < numberArguments; i++) {
-				args[i] = ois.readObject();
-			}
-		} else {
-			args = null;
-		}
-	}
+public interface RpcInvocation {
+
+	/**
+	 * Returns the method's name.
+	 *
+	 * @return Method name
+	 * @throws IOException if the rpc invocation message is a remote message and could not be deserialized
+	 * @throws ClassNotFoundException if the rpc invocation message is a remote message and contains
+	 * serialized classes which cannot be found on the receiving side
+	 */
+	String getMethodName() throws IOException, ClassNotFoundException;
+
+	/**
+	 * Returns the method's parameter types
+	 *
+	 * @return Method's parameter types
+	 * @throws IOException if the rpc invocation message is a remote message and could not be deserialized
+	 * @throws ClassNotFoundException if the rpc invocation message is a remote message and contains
+	 * serialized classes which cannot be found on the receiving side
+	 */
+	Class<?>[] getParameterTypes() throws IOException, ClassNotFoundException;
+
+	/**
+	 * Returns the arguments of the remote procedure call
+	 *
+	 * @return Arguments of the remote procedure call
+	 * @throws IOException if the rpc invocation message is a remote message and could not be deserialized
+	 * @throws ClassNotFoundException if the rpc invocation message is a remote message and contains
+	 * serialized classes which cannot be found on the receiving side
+	 */
+	Object[] getArgs() throws IOException, ClassNotFoundException;
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/dfbbe55d/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java
index 5e37e10..f26b40b 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java
@@ -64,7 +64,7 @@ public class AkkaRpcServiceTest extends TestLogger {
 		AkkaGateway akkaClient = (AkkaGateway) rm;
 
 		
-		jobMaster.registerAtResourceManager(AkkaUtils.getAkkaURL(actorSystem, akkaClient.getRpcServer()));
+		jobMaster.registerAtResourceManager(AkkaUtils.getAkkaURL(actorSystem, akkaClient.getRpcEndpoint()));
 
 		// wait for successful registration
 		FiniteDuration timeout = new FiniteDuration(200, TimeUnit.SECONDS);

http://git-wip-us.apache.org/repos/asf/flink/blob/dfbbe55d/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MessageSerializationTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MessageSerializationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MessageSerializationTest.java
new file mode 100644
index 0000000..ca8179c
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/MessageSerializationTest.java
@@ -0,0 +1,210 @@
+/*
+ * 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.rpc.akka;
+
+import akka.actor.ActorSystem;
+import akka.util.Timeout;
+import com.typesafe.config.Config;
+import com.typesafe.config.ConfigValueFactory;
+import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.rpc.RpcEndpoint;
+import org.apache.flink.runtime.rpc.RpcGateway;
+import org.apache.flink.runtime.rpc.RpcMethod;
+import org.apache.flink.runtime.rpc.RpcService;
+import org.apache.flink.util.TestLogger;
+import org.hamcrest.core.Is;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import scala.concurrent.Await;
+import scala.concurrent.Future;
+import scala.concurrent.duration.FiniteDuration;
+
+import java.io.IOException;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
+
+/**
+ * Tests that akka rpc invocation messages are properly serialized and errors reported
+ */
+public class MessageSerializationTest extends TestLogger {
+	private static ActorSystem actorSystem1;
+	private static ActorSystem actorSystem2;
+	private static AkkaRpcService akkaRpcService1;
+	private static AkkaRpcService akkaRpcService2;
+
+	private static final FiniteDuration timeout = new FiniteDuration(10L, TimeUnit.SECONDS);
+	private static final int maxFrameSize = 32000;
+
+	@BeforeClass
+	public static void setup() {
+		Config akkaConfig = AkkaUtils.getDefaultAkkaConfig();
+		Config modifiedAkkaConfig = akkaConfig.withValue(AkkaRpcService.MAXIMUM_FRAME_SIZE_PATH, ConfigValueFactory.fromAnyRef(maxFrameSize + "b"));
+
+		actorSystem1 = AkkaUtils.createActorSystem(modifiedAkkaConfig);
+		actorSystem2 = AkkaUtils.createActorSystem(modifiedAkkaConfig);
+
+		akkaRpcService1 = new AkkaRpcService(actorSystem1, new Timeout(timeout));
+		akkaRpcService2 = new AkkaRpcService(actorSystem2, new Timeout(timeout));
+	}
+
+	@AfterClass
+	public static void teardown() {
+		akkaRpcService1.stopService();
+		akkaRpcService2.stopService();
+
+		actorSystem1.shutdown();
+		actorSystem2.shutdown();
+
+		actorSystem1.awaitTermination();
+		actorSystem2.awaitTermination();
+	}
+
+	/**
+	 * Tests that a local rpc call with a non serializable argument can be executed.
+	 */
+	@Test
+	public void testNonSerializableLocalMessageTransfer() throws InterruptedException, IOException {
+		LinkedBlockingQueue<Object> linkedBlockingQueue = new LinkedBlockingQueue<>();
+		TestEndpoint testEndpoint = new TestEndpoint(akkaRpcService1, linkedBlockingQueue);
+
+		TestGateway testGateway = testEndpoint.getSelf();
+
+		NonSerializableObject expected = new NonSerializableObject(42);
+
+		testGateway.foobar(expected);
+
+		assertThat(linkedBlockingQueue.take(), Is.<Object>is(expected));
+	}
+
+	/**
+	 * Tests that a remote rpc call with a non-serializable argument fails with an
+	 * {@link IOException} (or an {@link java.lang.reflect.UndeclaredThrowableException} if the
+	 * the method declaration does not include the {@link IOException} as throwable).
+	 */
+	@Test(expected = IOException.class)
+	public void testNonSerializableRemoteMessageTransfer() throws Exception {
+		LinkedBlockingQueue<Object> linkedBlockingQueue = new LinkedBlockingQueue<>();
+
+		TestEndpoint testEndpoint = new TestEndpoint(akkaRpcService1, linkedBlockingQueue);
+
+		String address = testEndpoint.getAddress();
+
+		Future<TestGateway> remoteGatewayFuture = akkaRpcService2.connect(address, TestGateway.class);
+
+		TestGateway remoteGateway = Await.result(remoteGatewayFuture, timeout);
+
+		remoteGateway.foobar(new Object());
+
+		fail("Should have failed because Object is not serializable.");
+	}
+
+	/**
+	 * Tests that a remote rpc call with a serializable argument can be successfully executed.
+	 */
+	@Test
+	public void testSerializableRemoteMessageTransfer() throws Exception {
+		LinkedBlockingQueue<Object> linkedBlockingQueue = new LinkedBlockingQueue<>();
+
+		TestEndpoint testEndpoint = new TestEndpoint(akkaRpcService1, linkedBlockingQueue);
+
+		String address = testEndpoint.getAddress();
+
+		Future<TestGateway> remoteGatewayFuture = akkaRpcService2.connect(address, TestGateway.class);
+
+		TestGateway remoteGateway = Await.result(remoteGatewayFuture, timeout);
+
+		int expected = 42;
+
+		remoteGateway.foobar(expected);
+
+		assertThat(linkedBlockingQueue.take(), Is.<Object>is(expected));
+	}
+
+	/**
+	 * Tests that a message which exceeds the maximum frame size is detected and a corresponding
+	 * exception is thrown.
+	 */
+	@Test(expected = IOException.class)
+	public void testMaximumFramesizeRemoteMessageTransfer() throws Exception {
+		LinkedBlockingQueue<Object> linkedBlockingQueue = new LinkedBlockingQueue<>();
+
+		TestEndpoint testEndpoint = new TestEndpoint(akkaRpcService1, linkedBlockingQueue);
+
+		String address = testEndpoint.getAddress();
+
+		Future<TestGateway> remoteGatewayFuture = akkaRpcService2.connect(address, TestGateway.class);
+
+		TestGateway remoteGateway = Await.result(remoteGatewayFuture, timeout);
+
+		int bufferSize = maxFrameSize + 1;
+		byte[] buffer = new byte[bufferSize];
+
+		remoteGateway.foobar(buffer);
+
+		fail("Should have failed due to exceeding the maximum framesize.");
+	}
+
+	private interface TestGateway extends RpcGateway {
+		void foobar(Object object) throws IOException, InterruptedException;
+	}
+
+	private static class TestEndpoint extends RpcEndpoint<TestGateway> {
+
+		private final LinkedBlockingQueue<Object> queue;
+
+		protected TestEndpoint(RpcService rpcService, LinkedBlockingQueue<Object> queue) {
+			super(rpcService);
+			this.queue = queue;
+		}
+
+		@RpcMethod
+		public void foobar(Object object) throws InterruptedException {
+			queue.put(object);
+		}
+	}
+
+	private static class NonSerializableObject {
+		private final Object object = new Object();
+		private final int value;
+
+		NonSerializableObject(int value) {
+			this.value = value;
+		}
+
+		@Override
+		public boolean equals(Object obj) {
+			if (obj instanceof NonSerializableObject) {
+				NonSerializableObject nonSerializableObject = (NonSerializableObject) obj;
+
+				return value == nonSerializableObject.value;
+			} else {
+				return false;
+			}
+		}
+
+		@Override
+		public int hashCode() {
+			return value * 41;
+		}
+	}
+}


[20/50] [abbrv] flink git commit: [FLINK-4346] [rpc] Add new RPC abstraction

Posted by tr...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/0f36fb7f/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManagerGateway.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManagerGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManagerGateway.java
new file mode 100644
index 0000000..464a261
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManagerGateway.java
@@ -0,0 +1,58 @@
+/*
+ * 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.rpc.resourcemanager;
+
+import org.apache.flink.runtime.rpc.RpcGateway;
+import org.apache.flink.runtime.rpc.RpcTimeout;
+import org.apache.flink.runtime.rpc.jobmaster.JobMaster;
+import scala.concurrent.Future;
+import scala.concurrent.duration.FiniteDuration;
+
+/**
+ * {@link ResourceManager} rpc gateway interface.
+ */
+public interface ResourceManagerGateway extends RpcGateway {
+
+	/**
+	 * Register a {@link JobMaster} at the resource manager.
+	 *
+	 * @param jobMasterRegistration Job master registration information
+	 * @param timeout Timeout for the future to complete
+	 * @return Future registration response
+	 */
+	Future<RegistrationResponse> registerJobMaster(
+		JobMasterRegistration jobMasterRegistration,
+		@RpcTimeout FiniteDuration timeout);
+
+	/**
+	 * Register a {@link JobMaster} at the resource manager.
+	 *
+	 * @param jobMasterRegistration Job master registration information
+	 * @return Future registration response
+	 */
+	Future<RegistrationResponse> registerJobMaster(JobMasterRegistration jobMasterRegistration);
+
+	/**
+	 * Requests a slot from the resource manager.
+	 *
+	 * @param slotRequest Slot request
+	 * @return Future slot assignment
+	 */
+	Future<SlotAssignment> requestSlot(SlotRequest slotRequest);
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0f36fb7f/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/SlotAssignment.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/SlotAssignment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/SlotAssignment.java
new file mode 100644
index 0000000..86cd8b7
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/SlotAssignment.java
@@ -0,0 +1,25 @@
+/*
+ * 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.rpc.resourcemanager;
+
+import java.io.Serializable;
+
+public class SlotAssignment implements Serializable{
+	private static final long serialVersionUID = -6990813455942742322L;
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0f36fb7f/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/SlotRequest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/SlotRequest.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/SlotRequest.java
new file mode 100644
index 0000000..d8fe268
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/SlotRequest.java
@@ -0,0 +1,25 @@
+/*
+ * 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.rpc.resourcemanager;
+
+import java.io.Serializable;
+
+public class SlotRequest implements Serializable{
+	private static final long serialVersionUID = -6586877187990445986L;
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0f36fb7f/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java
new file mode 100644
index 0000000..cdfc3bd
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java
@@ -0,0 +1,82 @@
+/*
+ * 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.rpc.taskexecutor;
+
+import akka.dispatch.ExecutionContexts$;
+import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.rpc.RpcMethod;
+import org.apache.flink.runtime.rpc.RpcEndpoint;
+import org.apache.flink.runtime.rpc.RpcService;
+import scala.concurrent.ExecutionContext;
+
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ExecutorService;
+
+/**
+ * TaskExecutor implementation. The task executor is responsible for the execution of multiple
+ * {@link org.apache.flink.runtime.taskmanager.Task}.
+ *
+ * It offers the following methods as part of its rpc interface to interact with him remotely:
+ * <ul>
+ *     <li>{@link #executeTask(TaskDeploymentDescriptor)} executes a given task on the TaskExecutor</li>
+ *     <li>{@link #cancelTask(ExecutionAttemptID)} cancels a given task identified by the {@link ExecutionAttemptID}</li>
+ * </ul>
+ */
+public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
+	private final ExecutionContext executionContext;
+	private final Set<ExecutionAttemptID> tasks = new HashSet<>();
+
+	public TaskExecutor(RpcService rpcService, ExecutorService executorService) {
+		super(rpcService);
+		this.executionContext = ExecutionContexts$.MODULE$.fromExecutor(executorService);
+	}
+
+	/**
+	 * Execute the given task on the task executor. The task is described by the provided
+	 * {@link TaskDeploymentDescriptor}.
+	 *
+	 * @param taskDeploymentDescriptor Descriptor for the task to be executed
+	 * @return Acknowledge the start of the task execution
+	 */
+	@RpcMethod
+	public Acknowledge executeTask(TaskDeploymentDescriptor taskDeploymentDescriptor) {
+		tasks.add(taskDeploymentDescriptor.getExecutionId());
+		return Acknowledge.get();
+	}
+
+	/**
+	 * Cancel a task identified by it {@link ExecutionAttemptID}. If the task cannot be found, then
+	 * the method throws an {@link Exception}.
+	 *
+	 * @param executionAttemptId Execution attempt ID identifying the task to be canceled.
+	 * @return Acknowledge the task canceling
+	 * @throws Exception if the task with the given execution attempt id could not be found
+	 */
+	@RpcMethod
+	public Acknowledge cancelTask(ExecutionAttemptID executionAttemptId) throws Exception {
+		if (tasks.contains(executionAttemptId)) {
+			return Acknowledge.get();
+		} else {
+			throw new Exception("Could not find task.");
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0f36fb7f/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorGateway.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorGateway.java
new file mode 100644
index 0000000..450423e
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorGateway.java
@@ -0,0 +1,48 @@
+/*
+ * 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.rpc.taskexecutor;
+
+import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.rpc.RpcGateway;
+import scala.concurrent.Future;
+
+/**
+ * {@link TaskExecutor} rpc gateway interface
+ */
+public interface TaskExecutorGateway extends RpcGateway {
+	/**
+	 * Execute the given task on the task executor. The task is described by the provided
+	 * {@link TaskDeploymentDescriptor}.
+	 *
+	 * @param taskDeploymentDescriptor Descriptor for the task to be executed
+	 * @return Future acknowledge of the start of the task execution
+	 */
+	Future<Acknowledge> executeTask(TaskDeploymentDescriptor taskDeploymentDescriptor);
+
+	/**
+	 * Cancel a task identified by it {@link ExecutionAttemptID}. If the task cannot be found, then
+	 * the method throws an {@link Exception}.
+	 *
+	 * @param executionAttemptId Execution attempt ID identifying the task to be canceled.
+	 * @return Future acknowledge of the task canceling
+	 */
+	Future<Acknowledge> cancelTask(ExecutionAttemptID executionAttemptId);
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0f36fb7f/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java
new file mode 100644
index 0000000..0ded25e
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java
@@ -0,0 +1,327 @@
+/*
+ * 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.rpc;
+
+import org.apache.flink.util.TestLogger;
+import org.junit.Test;
+import org.reflections.Reflections;
+import scala.concurrent.Future;
+
+import java.lang.annotation.Annotation;
+import java.lang.reflect.Method;
+import java.lang.reflect.ParameterizedType;
+import java.lang.reflect.Type;
+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 static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class RpcCompletenessTest extends TestLogger {
+	private static final Class<?> futureClass = Future.class;
+
+	@Test
+	public void testRpcCompleteness() {
+		Reflections reflections = new Reflections("org.apache.flink");
+
+		Set<Class<? extends RpcEndpoint>> classes = reflections.getSubTypesOf(RpcEndpoint.class);
+
+		Class<? extends RpcEndpoint> c;
+
+		for (Class<? extends RpcEndpoint> rpcEndpoint :classes){
+			c = rpcEndpoint;
+			Type superClass = c.getGenericSuperclass();
+
+			Class<?> rpcGatewayType = extractTypeParameter(superClass, 0);
+
+			if (rpcGatewayType != null) {
+				checkCompleteness(rpcEndpoint, (Class<? extends RpcGateway>) rpcGatewayType);
+			} else {
+				fail("Could not retrieve the rpc gateway class for the given rpc endpoint class " + rpcEndpoint.getName());
+			}
+		}
+	}
+
+	private void checkCompleteness(Class<? extends RpcEndpoint> rpcEndpoint, Class<? extends RpcGateway> rpcGateway) {
+		Method[] gatewayMethods = rpcGateway.getDeclaredMethods();
+		Method[] serverMethods = rpcEndpoint.getDeclaredMethods();
+
+		Map<String, Set<Method>> rpcMethods = new HashMap<>();
+		Set<Method> unmatchedRpcMethods = new HashSet<>();
+
+		for (Method serverMethod : serverMethods) {
+			if (serverMethod.isAnnotationPresent(RpcMethod.class)) {
+				if (rpcMethods.containsKey(serverMethod.getName())) {
+					Set<Method> methods = rpcMethods.get(serverMethod.getName());
+					methods.add(serverMethod);
+
+					rpcMethods.put(serverMethod.getName(), methods);
+				} else {
+					Set<Method> methods = new HashSet<>();
+					methods.add(serverMethod);
+
+					rpcMethods.put(serverMethod.getName(), methods);
+				}
+
+				unmatchedRpcMethods.add(serverMethod);
+			}
+		}
+
+		for (Method gatewayMethod : gatewayMethods) {
+			assertTrue(
+				"The rpc endpoint " + rpcEndpoint.getName() + " does not contain a RpcMethod " +
+					"annotated method with the same name and signature " +
+					generateEndpointMethodSignature(gatewayMethod) + ".",
+				rpcMethods.containsKey(gatewayMethod.getName()));
+
+			checkGatewayMethod(gatewayMethod);
+
+			if (!matchGatewayMethodWithEndpoint(gatewayMethod, rpcMethods.get(gatewayMethod.getName()), unmatchedRpcMethods)) {
+				fail("Could not find a RpcMethod annotated method in rpc endpoint " +
+					rpcEndpoint.getName() + " matching the rpc gateway method " +
+					generateEndpointMethodSignature(gatewayMethod) + " defined in the rpc gateway " +
+					rpcGateway.getName() + ".");
+			}
+		}
+
+		if (!unmatchedRpcMethods.isEmpty()) {
+			StringBuilder builder = new StringBuilder();
+
+			for (Method unmatchedRpcMethod : unmatchedRpcMethods) {
+				builder.append(unmatchedRpcMethod).append("\n");
+			}
+
+			fail("The rpc endpoint " + rpcEndpoint.getName() + " contains rpc methods which " +
+				"are not matched to gateway methods of " + rpcGateway.getName() + ":\n" +
+				builder.toString());
+		}
+	}
+
+	/**
+	 * Checks whether the gateway method fulfills the gateway method requirements.
+	 * <ul>
+	 *     <li>It checks whether the return type is void or a {@link Future} wrapping the actual result. </li>
+	 *     <li>It checks that the method's parameter list contains at most one parameter annotated with {@link RpcTimeout}.</li>
+	 * </ul>
+	 *
+	 * @param gatewayMethod Gateway method to check
+	 */
+	private void checkGatewayMethod(Method gatewayMethod) {
+		if (!gatewayMethod.getReturnType().equals(Void.TYPE)) {
+			assertTrue(
+				"The return type of method " + gatewayMethod.getName() + " in the rpc gateway " +
+					gatewayMethod.getDeclaringClass().getName() + " is non void and not a " +
+					"future. Non-void return types have to be returned as a future.",
+				gatewayMethod.getReturnType().equals(futureClass));
+		}
+
+		Annotation[][] parameterAnnotations = gatewayMethod.getParameterAnnotations();
+		int rpcTimeoutParameters = 0;
+
+		for (Annotation[] parameterAnnotation : parameterAnnotations) {
+			for (Annotation annotation : parameterAnnotation) {
+				if (annotation.equals(RpcTimeout.class)) {
+					rpcTimeoutParameters++;
+				}
+			}
+		}
+
+		assertTrue("The gateway method " + gatewayMethod + " must have at most one RpcTimeout " +
+			"annotated parameter.", rpcTimeoutParameters <= 1);
+	}
+
+	/**
+	 * Checks whether we find a matching overloaded version for the gateway method among the methods
+	 * with the same name in the rpc endpoint.
+	 *
+	 * @param gatewayMethod Gateway method
+	 * @param endpointMethods Set of rpc methods on the rpc endpoint with the same name as the gateway
+	 *                   method
+	 * @param unmatchedRpcMethods Set of unmatched rpc methods on the endpoint side (so far)
+	 */
+	private boolean matchGatewayMethodWithEndpoint(Method gatewayMethod, Set<Method> endpointMethods, Set<Method> unmatchedRpcMethods) {
+		for (Method endpointMethod : endpointMethods) {
+			if (checkMethod(gatewayMethod, endpointMethod)) {
+				unmatchedRpcMethods.remove(endpointMethod);
+				return true;
+			}
+		}
+
+		return false;
+	}
+
+	private boolean checkMethod(Method gatewayMethod, Method endpointMethod) {
+		Class<?>[] gatewayParameterTypes = gatewayMethod.getParameterTypes();
+		Annotation[][] gatewayParameterAnnotations = gatewayMethod.getParameterAnnotations();
+
+		Class<?>[] endpointParameterTypes = endpointMethod.getParameterTypes();
+
+		List<Class<?>> filteredGatewayParameterTypes = new ArrayList<>();
+
+		assertEquals(gatewayParameterTypes.length, gatewayParameterAnnotations.length);
+
+		// filter out the RpcTimeout parameters
+		for (int i = 0; i < gatewayParameterTypes.length; i++) {
+			if (!isRpcTimeout(gatewayParameterAnnotations[i])) {
+				filteredGatewayParameterTypes.add(gatewayParameterTypes[i]);
+			}
+		}
+
+		if (filteredGatewayParameterTypes.size() != endpointParameterTypes.length) {
+			return false;
+		} else {
+			// check the parameter types
+			for (int i = 0; i < filteredGatewayParameterTypes.size(); i++) {
+				if (!checkType(filteredGatewayParameterTypes.get(i), endpointParameterTypes[i])) {
+					return false;
+				}
+			}
+
+			// check the return types
+			if (endpointMethod.getReturnType() == void.class) {
+				if (gatewayMethod.getReturnType() != void.class) {
+					return false;
+				}
+			} else {
+				// has return value. The gateway method should be wrapped in a future
+				Class<?> futureClass = gatewayMethod.getReturnType();
+
+				// sanity check that the return type of a gateway method must be void or a future
+				if (!futureClass.equals(RpcCompletenessTest.futureClass)) {
+					return false;
+				} else {
+					Class<?> valueClass = extractTypeParameter(futureClass, 0);
+
+					if (endpointMethod.getReturnType().equals(futureClass)) {
+						Class<?> rpcEndpointValueClass = extractTypeParameter(endpointMethod.getReturnType(), 0);
+
+						// check if we have the same future value types
+						if (valueClass != null && rpcEndpointValueClass != null && !checkType(valueClass, rpcEndpointValueClass)) {
+							return false;
+						}
+					} else {
+						if (valueClass != null && !checkType(valueClass, endpointMethod.getReturnType())) {
+							return false;
+						}
+					}
+				}
+			}
+
+			return gatewayMethod.getName().equals(endpointMethod.getName());
+		}
+	}
+
+	private boolean checkType(Class<?> firstType, Class<?> secondType) {
+		return firstType.equals(secondType);
+	}
+
+	/**
+	 * Generates from a gateway rpc method signature the corresponding rpc endpoint signature.
+	 *
+	 * For example the {@link RpcTimeout} annotation adds an additional parameter to the gateway
+	 * signature which is not relevant on the server side.
+	 *
+	 * @param method Method to generate the signature string for
+	 * @return String of the respective server side rpc method signature
+	 */
+	private String generateEndpointMethodSignature(Method method) {
+		StringBuilder builder = new StringBuilder();
+
+		if (method.getReturnType().equals(Void.TYPE)) {
+			builder.append("void").append(" ");
+		} else if (method.getReturnType().equals(futureClass)) {
+			Class<?> valueClass = extractTypeParameter(method.getGenericReturnType(), 0);
+
+			builder
+				.append(futureClass.getSimpleName())
+				.append("<")
+				.append(valueClass != null ? valueClass.getSimpleName() : "")
+				.append(">");
+
+			if (valueClass != null) {
+				builder.append("/").append(valueClass.getSimpleName());
+			}
+
+			builder.append(" ");
+		} else {
+			return "Invalid rpc method signature.";
+		}
+
+		builder.append(method.getName()).append("(");
+
+		Class<?>[] parameterTypes = method.getParameterTypes();
+		Annotation[][] parameterAnnotations = method.getParameterAnnotations();
+
+		assertEquals(parameterTypes.length, parameterAnnotations.length);
+
+		for (int i = 0; i < parameterTypes.length; i++) {
+			// filter out the RpcTimeout parameters
+			if (!isRpcTimeout(parameterAnnotations[i])) {
+				builder.append(parameterTypes[i].getName());
+
+				if (i < parameterTypes.length -1) {
+					builder.append(", ");
+				}
+			}
+		}
+
+		builder.append(")");
+
+		return builder.toString();
+	}
+
+	private Class<?> extractTypeParameter(Type genericType, int position) {
+		if (genericType instanceof ParameterizedType) {
+			ParameterizedType parameterizedType = (ParameterizedType) genericType;
+
+			Type[] typeArguments = parameterizedType.getActualTypeArguments();
+
+			if (position < 0 || position >= typeArguments.length) {
+				throw new IndexOutOfBoundsException("The generic type " +
+					parameterizedType.getRawType() + " only has " + typeArguments.length +
+					" type arguments.");
+			} else {
+				Type typeArgument = typeArguments[position];
+
+				if (typeArgument instanceof Class<?>) {
+					return (Class<?>) typeArgument;
+				} else {
+					return null;
+				}
+			}
+		} else {
+			return null;
+		}
+	}
+
+	private boolean isRpcTimeout(Annotation[] annotations) {
+		for (Annotation annotation : annotations) {
+			if (annotation.annotationType().equals(RpcTimeout.class)) {
+				return true;
+			}
+		}
+
+		return false;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0f36fb7f/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java
new file mode 100644
index 0000000..c5bac94
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java
@@ -0,0 +1,81 @@
+/*
+ * 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.rpc.akka;
+
+import akka.actor.ActorSystem;
+import akka.util.Timeout;
+import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.rpc.jobmaster.JobMaster;
+import org.apache.flink.runtime.rpc.resourcemanager.ResourceManagerGateway;
+import org.apache.flink.runtime.rpc.resourcemanager.ResourceManager;
+import org.apache.flink.util.TestLogger;
+import org.junit.Test;
+import scala.concurrent.duration.Deadline;
+import scala.concurrent.duration.FiniteDuration;
+
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ForkJoinPool;
+import java.util.concurrent.TimeUnit;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+public class AkkaRpcServiceTest extends TestLogger {
+
+	/**
+	 * Tests that the {@link JobMaster} can connect to the {@link ResourceManager} using the
+	 * {@link AkkaRpcService}.
+	 */
+	@Test
+	public void testJobMasterResourceManagerRegistration() throws Exception {
+		Timeout akkaTimeout = new Timeout(10, TimeUnit.SECONDS);
+		ActorSystem actorSystem = AkkaUtils.createDefaultActorSystem();
+		ActorSystem actorSystem2 = AkkaUtils.createDefaultActorSystem();
+		AkkaRpcService akkaRpcService = new AkkaRpcService(actorSystem, akkaTimeout);
+		AkkaRpcService akkaRpcService2 = new AkkaRpcService(actorSystem2, akkaTimeout);
+		ExecutorService executorService = new ForkJoinPool();
+
+		ResourceManager resourceManager = new ResourceManager(akkaRpcService, executorService);
+		JobMaster jobMaster = new JobMaster(akkaRpcService2, executorService);
+
+		resourceManager.start();
+
+		ResourceManagerGateway rm = resourceManager.getSelf();
+
+		assertTrue(rm instanceof AkkaGateway);
+
+		AkkaGateway akkaClient = (AkkaGateway) rm;
+
+		jobMaster.start();
+		jobMaster.registerAtResourceManager(AkkaUtils.getAkkaURL(actorSystem, akkaClient.getActorRef()));
+
+		// wait for successful registration
+		FiniteDuration timeout = new FiniteDuration(20, TimeUnit.SECONDS);
+		Deadline deadline = timeout.fromNow();
+
+		while (deadline.hasTimeLeft() && !jobMaster.isConnected()) {
+			Thread.sleep(100);
+		}
+
+		assertFalse(deadline.isOverdue());
+
+		jobMaster.shutDown();
+		resourceManager.shutDown();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0f36fb7f/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorTest.java
new file mode 100644
index 0000000..c143527
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorTest.java
@@ -0,0 +1,92 @@
+/*
+ * 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.rpc.taskexecutor;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.blob.BlobKey;
+import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor;
+import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor;
+import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.rpc.RpcService;
+import org.apache.flink.runtime.util.DirectExecutorService;
+import org.apache.flink.util.SerializedValue;
+import org.apache.flink.util.TestLogger;
+import org.junit.Test;
+
+import java.net.URL;
+import java.util.Collections;
+
+import static org.junit.Assert.fail;
+import static org.mockito.Mockito.mock;
+
+public class TaskExecutorTest extends TestLogger {
+
+	/**
+	 * Tests that we can deploy and cancel a task on the TaskExecutor without exceptions
+	 */
+	@Test
+	public void testTaskExecution() throws Exception {
+		RpcService testingRpcService = mock(RpcService.class);
+		DirectExecutorService directExecutorService = null;
+		TaskExecutor taskExecutor = new TaskExecutor(testingRpcService, directExecutorService);
+
+		TaskDeploymentDescriptor tdd = new TaskDeploymentDescriptor(
+			new JobID(),
+			"Test job",
+			new JobVertexID(),
+			new ExecutionAttemptID(),
+			new SerializedValue<ExecutionConfig>(null),
+			"Test task",
+			0,
+			1,
+			0,
+			new Configuration(),
+			new Configuration(),
+			"Invokable",
+			Collections.<ResultPartitionDeploymentDescriptor>emptyList(),
+			Collections.<InputGateDeploymentDescriptor>emptyList(),
+			Collections.<BlobKey>emptyList(),
+			Collections.<URL>emptyList(),
+			0
+		);
+
+		Acknowledge ack = taskExecutor.executeTask(tdd);
+
+		ack = taskExecutor.cancelTask(tdd.getExecutionId());
+	}
+
+	/**
+	 * Tests that cancelling a non-existing task will return an exception
+	 */
+	@Test(expected=Exception.class)
+	public void testWrongTaskCancellation() throws Exception {
+		RpcService testingRpcService = mock(RpcService.class);
+		DirectExecutorService directExecutorService = null;
+		TaskExecutor taskExecutor = new TaskExecutor(testingRpcService, directExecutorService);
+
+		taskExecutor.cancelTask(new ExecutionAttemptID());
+
+		fail("The cancellation should have thrown an exception.");
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0f36fb7f/flink-runtime/src/test/java/org/apache/flink/runtime/util/DirectExecutorService.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/DirectExecutorService.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/DirectExecutorService.java
new file mode 100644
index 0000000..1d7c971
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/DirectExecutorService.java
@@ -0,0 +1,234 @@
+/*
+ * 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.util;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+public class DirectExecutorService implements ExecutorService {
+	private boolean _shutdown = false;
+
+	@Override
+	public void shutdown() {
+		_shutdown = true;
+	}
+
+	@Override
+	public List<Runnable> shutdownNow() {
+		_shutdown = true;
+		return Collections.emptyList();
+	}
+
+	@Override
+	public boolean isShutdown() {
+		return _shutdown;
+	}
+
+	@Override
+	public boolean isTerminated() {
+		return _shutdown;
+	}
+
+	@Override
+	public boolean awaitTermination(long timeout, TimeUnit unit) throws InterruptedException {
+		return _shutdown;
+	}
+
+	@Override
+	public <T> Future<T> submit(Callable<T> task) {
+		try {
+			T result = task.call();
+
+			return new CompletedFuture<>(result, null);
+		} catch (Exception e) {
+			return new CompletedFuture<>(null, e);
+		}
+	}
+
+	@Override
+	public <T> Future<T> submit(Runnable task, T result) {
+		task.run();
+
+		return new CompletedFuture<>(result, null);
+	}
+
+	@Override
+	public Future<?> submit(Runnable task) {
+		task.run();
+		return new CompletedFuture<>(null, null);
+	}
+
+	@Override
+	public <T> List<Future<T>> invokeAll(Collection<? extends Callable<T>> tasks) throws InterruptedException {
+		ArrayList<Future<T>> result = new ArrayList<>();
+
+		for (Callable<T> task : tasks) {
+			try {
+				result.add(new CompletedFuture<T>(task.call(), null));
+			} catch (Exception e) {
+				result.add(new CompletedFuture<T>(null, e));
+			}
+		}
+		return result;
+	}
+
+	@Override
+	public <T> List<Future<T>> invokeAll(Collection<? extends Callable<T>> tasks, long timeout, TimeUnit unit) throws InterruptedException {
+		long end = System.currentTimeMillis() + unit.toMillis(timeout);
+		Iterator<? extends Callable<T>> iterator = tasks.iterator();
+		ArrayList<Future<T>> result = new ArrayList<>();
+
+		while (end > System.currentTimeMillis() && iterator.hasNext()) {
+			Callable<T> callable = iterator.next();
+
+			try {
+				result.add(new CompletedFuture<T>(callable.call(), null));
+			} catch (Exception e) {
+				result.add(new CompletedFuture<T>(null, e));
+			}
+		}
+
+		while(iterator.hasNext()) {
+			iterator.next();
+			result.add(new Future<T>() {
+				@Override
+				public boolean cancel(boolean mayInterruptIfRunning) {
+					return false;
+				}
+
+				@Override
+				public boolean isCancelled() {
+					return true;
+				}
+
+				@Override
+				public boolean isDone() {
+					return false;
+				}
+
+				@Override
+				public T get() throws InterruptedException, ExecutionException {
+					throw new CancellationException("Task has been cancelled.");
+				}
+
+				@Override
+				public T get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException {
+					throw new CancellationException("Task has been cancelled.");
+				}
+			});
+		}
+
+		return result;
+	}
+
+	@Override
+	public <T> T invokeAny(Collection<? extends Callable<T>> tasks) throws InterruptedException, ExecutionException {
+		Exception exception = null;
+
+		for (Callable<T> task : tasks) {
+			try {
+				return task.call();
+			} catch (Exception e) {
+				// try next task
+				exception = e;
+			}
+		}
+
+		throw new ExecutionException("No tasks finished successfully.", exception);
+	}
+
+	@Override
+	public <T> T invokeAny(Collection<? extends Callable<T>> tasks, long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException {
+		long end = System.currentTimeMillis() + unit.toMillis(timeout);
+		Exception exception = null;
+
+		Iterator<? extends Callable<T>> iterator = tasks.iterator();
+
+		while (end > System.currentTimeMillis() && iterator.hasNext()) {
+			Callable<T> callable = iterator.next();
+
+			try {
+				return callable.call();
+			} catch (Exception e) {
+				// ignore exception and try next
+				exception = e;
+			}
+		}
+
+		if (iterator.hasNext()) {
+			throw new TimeoutException("Could not finish execution of tasks within time.");
+		} else {
+			throw new ExecutionException("No tasks finished successfully.", exception);
+		}
+	}
+
+	@Override
+	public void execute(Runnable command) {
+		command.run();
+	}
+
+	public static class CompletedFuture<V> implements Future<V> {
+		private final V value;
+		private final Exception exception;
+
+		public CompletedFuture(V value, Exception exception) {
+			this.value = value;
+			this.exception = exception;
+		}
+
+		@Override
+		public boolean cancel(boolean mayInterruptIfRunning) {
+			return false;
+		}
+
+		@Override
+		public boolean isCancelled() {
+			return false;
+		}
+
+		@Override
+		public boolean isDone() {
+			return true;
+		}
+
+		@Override
+		public V get() throws InterruptedException, ExecutionException {
+			if (exception != null) {
+				throw new ExecutionException(exception);
+			} else {
+				return value;
+			}
+		}
+
+		@Override
+		public V get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException {
+			return get();
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0f36fb7f/flink-tests/pom.xml
----------------------------------------------------------------------
diff --git a/flink-tests/pom.xml b/flink-tests/pom.xml
index b09db1f..3202a9f 100644
--- a/flink-tests/pom.xml
+++ b/flink-tests/pom.xml
@@ -202,7 +202,6 @@ under the License.
 		<dependency>
 			<groupId>org.reflections</groupId>
 			<artifactId>reflections</artifactId>
-			<version>0.9.10</version>
 		</dependency>
 
 	</dependencies>

http://git-wip-us.apache.org/repos/asf/flink/blob/0f36fb7f/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 02e868e..97b9d71 100644
--- a/pom.xml
+++ b/pom.xml
@@ -406,6 +406,13 @@ under the License.
 				<artifactId>jackson-annotations</artifactId>
 				<version>${jackson.version}</version>
 			</dependency>
+
+			<dependency>
+				<groupId>org.reflections</groupId>
+				<artifactId>reflections</artifactId>
+				<version>0.9.10</version>
+				<scope>test</scope>
+			</dependency>
 		</dependencies>
 	</dependencyManagement>
 


[07/50] [abbrv] flink git commit: [hotfix] Delete leftover (superseded) StreamTaskAsyncCheckpointTest

Posted by tr...@apache.org.
[hotfix] Delete leftover (superseded) StreamTaskAsyncCheckpointTest

There is RocksDBAsyncSnapshotTest which tests async snapshots for the
RocksDB state backend. Operators themselves cannot do asynchronous
checkpoints right now.


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

Branch: refs/heads/flip-6
Commit: 9bbb8fab38daff8eb5679e4aa7151f68e0b12226
Parents: 6e40f59
Author: Aljoscha Krettek <al...@gmail.com>
Authored: Mon Sep 5 12:25:28 2016 +0200
Committer: Aljoscha Krettek <al...@gmail.com>
Committed: Mon Sep 5 12:26:38 2016 +0200

----------------------------------------------------------------------
 .../tasks/StreamTaskAsyncCheckpointTest.java    | 234 -------------------
 1 file changed, 234 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/9bbb8fab/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskAsyncCheckpointTest.java
----------------------------------------------------------------------
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskAsyncCheckpointTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskAsyncCheckpointTest.java
deleted file mode 100644
index 66bc237..0000000
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskAsyncCheckpointTest.java
+++ /dev/null
@@ -1,234 +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.streaming.runtime.tasks;
-//
-//import org.apache.flink.api.common.ExecutionConfig;
-//import org.apache.flink.api.common.functions.MapFunction;
-//import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
-//import org.apache.flink.core.testutils.OneShotLatch;
-//import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter;
-//import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider;
-//import org.apache.flink.streaming.api.graph.StreamConfig;
-//import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
-//import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
-//import org.apache.flink.streaming.api.watermark.Watermark;
-//import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-//import org.junit.Test;
-//import org.junit.runner.RunWith;
-//import org.powermock.core.classloader.annotations.PowerMockIgnore;
-//import org.powermock.core.classloader.annotations.PrepareForTest;
-//import org.powermock.modules.junit4.PowerMockRunner;
-//
-//import java.io.IOException;
-//import java.lang.reflect.Field;
-//
-//import static org.junit.Assert.assertEquals;
-//import static org.junit.Assert.assertTrue;
-//
-///**
-// * Tests for asynchronous checkpoints.
-// */
-//@RunWith(PowerMockRunner.class)
-//@PrepareForTest(ResultPartitionWriter.class)
-//@PowerMockIgnore({"javax.management.*", "com.sun.jndi.*"})
-//@SuppressWarnings("serial")
-//public class StreamTaskAsyncCheckpointTest {
-//
-//	/**
-//	 * This ensures that asynchronous state handles are actually materialized asynchonously.
-//	 *
-//	 * <p>We use latches to block at various stages and see if the code still continues through
-//	 * the parts that are not asynchronous. If the checkpoint is not done asynchronously the
-//	 * test will simply lock forever.
-//	 * @throws Exception
-//	 */
-//	@Test
-//	public void testAsyncCheckpoints() throws Exception {
-//		final OneShotLatch delayCheckpointLatch = new OneShotLatch();
-//		final OneShotLatch ensureCheckpointLatch = new OneShotLatch();
-//
-//		final OneInputStreamTask<String, String> task = new OneInputStreamTask<>();
-//
-//		final OneInputStreamTaskTestHarness<String, String> testHarness = new OneInputStreamTaskTestHarness<>(task, BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO);
-//
-//		StreamConfig streamConfig = testHarness.getStreamConfig();
-//
-//		streamConfig.setStreamOperator(new AsyncCheckpointOperator());
-//
-//		StreamMockEnvironment mockEnv = new StreamMockEnvironment(
-//			testHarness.jobConfig,
-//			testHarness.taskConfig,
-//			testHarness.memorySize,
-//			new MockInputSplitProvider(),
-//			testHarness.bufferSize) {
-//
-//			@Override
-//			public ExecutionConfig getExecutionConfig() {
-//				return testHarness.executionConfig;
-//			}
-//
-//			@Override
-//			public void acknowledgeCheckpoint(long checkpointId) {
-//				super.acknowledgeCheckpoint(checkpointId);
-//			}
-//
-//			@Override
-//			public void acknowledgeCheckpoint(long checkpointId, StateHandle<?> state) {
-//				super.acknowledgeCheckpoint(checkpointId, state);
-//
-//				// block on the latch, to verify that triggerCheckpoint returns below,
-//				// even though the async checkpoint would not finish
-//				try {
-//					delayCheckpointLatch.await();
-//				} catch (InterruptedException e) {
-//					e.printStackTrace();
-//				}
-//
-//				assertTrue(state instanceof StreamTaskStateList);
-//				StreamTaskStateList stateList = (StreamTaskStateList) state;
-//
-//				// should be only one state
-//				StreamTaskState taskState = stateList.getState(this.getUserClassLoader())[0];
-//				StateHandle<?> operatorState = taskState.getOperatorState();
-//				assertTrue("It must be a TestStateHandle", operatorState instanceof TestStateHandle);
-//				TestStateHandle testState = (TestStateHandle) operatorState;
-//				assertEquals(42, testState.checkpointId);
-//				assertEquals(17, testState.timestamp);
-//
-//				// we now know that the checkpoint went through
-//				ensureCheckpointLatch.trigger();
-//			}
-//		};
-//
-//		testHarness.invoke(mockEnv);
-//
-//		// wait for the task to be running
-//		for (Field field: StreamTask.class.getDeclaredFields()) {
-//			if (field.getName().equals("isRunning")) {
-//				field.setAccessible(true);
-//				while (!field.getBoolean(task)) {
-//					Thread.sleep(10);
-//				}
-//
-//			}
-//		}
-//
-//		task.triggerCheckpoint(42, 17);
-//
-//		// now we allow the checkpoint
-//		delayCheckpointLatch.trigger();
-//
-//		// wait for the checkpoint to go through
-//		ensureCheckpointLatch.await();
-//
-//		testHarness.endInput();
-//		testHarness.waitForTaskCompletion();
-//	}
-//
-//
-//	// ------------------------------------------------------------------------
-//
-//	public static class AsyncCheckpointOperator
-//		extends AbstractStreamOperator<String>
-//		implements OneInputStreamOperator<String, String> {
-//		@Override
-//		public void processElement(StreamRecord<String> element) throws Exception {
-//			// we also don't care
-//		}
-//
-//		@Override
-//		public void processWatermark(Watermark mark) throws Exception {
-//			// not interested
-//		}
-//
-//
-//		@Override
-//		public StreamTaskState snapshotOperatorState(final long checkpointId, final long timestamp) throws Exception {
-//			StreamTaskState taskState = super.snapshotOperatorState(checkpointId, timestamp);
-//
-//			AsynchronousStateHandle<String> asyncState =
-//				new DataInputViewAsynchronousStateHandle(checkpointId, timestamp);
-//
-//			taskState.setOperatorState(asyncState);
-//
-//			return taskState;
-//		}
-//
-//		@Override
-//		public void restoreState(StreamTaskState taskState) throws Exception {
-//			super.restoreState(taskState);
-//		}
-//	}
-//
-//	private static class DataInputViewAsynchronousStateHandle extends AsynchronousStateHandle<String> {
-//
-//		private final long checkpointId;
-//		private final long timestamp;
-//
-//		public DataInputViewAsynchronousStateHandle(long checkpointId, long timestamp) {
-//			this.checkpointId = checkpointId;
-//			this.timestamp = timestamp;
-//		}
-//
-//		@Override
-//		public StateHandle<String> materialize() throws Exception {
-//			return new TestStateHandle(checkpointId, timestamp);
-//		}
-//
-//		@Override
-//		public long getStateSize() {
-//			return 0;
-//		}
-//
-//		@Override
-//		public void close() throws IOException {}
-//	}
-//
-//	private static class TestStateHandle implements StateHandle<String> {
-//
-//		public final long checkpointId;
-//		public final long timestamp;
-//
-//		public TestStateHandle(long checkpointId, long timestamp) {
-//			this.checkpointId = checkpointId;
-//			this.timestamp = timestamp;
-//		}
-//
-//		@Override
-//		public String getState(ClassLoader userCodeClassLoader) throws Exception {
-//			return null;
-//		}
-//
-//		@Override
-//		public void discardState() throws Exception {}
-//
-//		@Override
-//		public long getStateSize() {
-//			return 0;
-//		}
-//
-//		@Override
-//		public void close() throws IOException {}
-//	}
-//
-//	public static class DummyMapFunction<T> implements MapFunction<T, T> {
-//		@Override
-//		public T map(T value) { return value; }
-//	}
-//}


[33/50] [abbrv] flink git commit: [FLINK-4434] [rpc] Add a testing RPC service.

Posted by tr...@apache.org.
[FLINK-4434] [rpc] Add a testing RPC service.

This closes #2394.


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

Branch: refs/heads/flip-6
Commit: b540c7189a46ea4b661c525abd2b63dfd1efee98
Parents: 6c9f884
Author: Stephan Ewen <se...@apache.org>
Authored: Fri Aug 19 23:29:45 2016 +0200
Committer: Till Rohrmann <tr...@apache.org>
Committed: Thu Sep 8 17:26:58 2016 +0200

----------------------------------------------------------------------
 .../flink/runtime/rpc/RpcCompletenessTest.java  |   3 +
 .../flink/runtime/rpc/TestingGatewayBase.java   |  85 ++++++++++++++
 .../flink/runtime/rpc/TestingRpcService.java    | 115 +++++++++++++++++++
 3 files changed, 203 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/b540c718/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java
index 97cf0cb..b8aad62 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/RpcCompletenessTest.java
@@ -41,9 +41,11 @@ import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 public class RpcCompletenessTest extends TestLogger {
+
 	private static final Class<?> futureClass = Future.class;
 
 	@Test
+	@SuppressWarnings({"rawtypes", "unchecked"})
 	public void testRpcCompleteness() {
 		Reflections reflections = new Reflections("org.apache.flink");
 
@@ -64,6 +66,7 @@ public class RpcCompletenessTest extends TestLogger {
 		}
 	}
 
+	@SuppressWarnings("rawtypes")
 	private void checkCompleteness(Class<? extends RpcEndpoint> rpcEndpoint, Class<? extends RpcGateway> rpcGateway) {
 		Method[] gatewayMethods = rpcGateway.getDeclaredMethods();
 		Method[] serverMethods = rpcEndpoint.getDeclaredMethods();

http://git-wip-us.apache.org/repos/asf/flink/blob/b540c718/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingGatewayBase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingGatewayBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingGatewayBase.java
new file mode 100644
index 0000000..4256135
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingGatewayBase.java
@@ -0,0 +1,85 @@
+/*
+ * 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.rpc;
+
+import akka.dispatch.Futures;
+import scala.concurrent.Future;
+import scala.concurrent.Promise;
+
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+/**
+ * Utility base class for testing gateways
+ */
+public abstract class TestingGatewayBase implements RpcGateway {
+
+	private final ScheduledExecutorService executor;
+
+	protected TestingGatewayBase() {
+		this.executor = Executors.newSingleThreadScheduledExecutor();
+	}
+
+	// ------------------------------------------------------------------------
+	//  shutdown
+	// ------------------------------------------------------------------------
+
+	public void stop() {
+		executor.shutdownNow();
+	}
+
+	@Override
+	protected void finalize() throws Throwable {
+		super.finalize();
+		executor.shutdownNow();
+	}
+
+	// ------------------------------------------------------------------------
+	//  utilities
+	// ------------------------------------------------------------------------
+
+	public <T> Future<T> futureWithTimeout(long timeoutMillis) {
+		Promise<T> promise = Futures.<T>promise();
+		executor.schedule(new FutureTimeout(promise), timeoutMillis, TimeUnit.MILLISECONDS);
+		return promise.future();
+	}
+
+	// ------------------------------------------------------------------------
+	
+	private static final class FutureTimeout implements Runnable {
+
+		private final Promise<?> promise;
+
+		private FutureTimeout(Promise<?> promise) {
+			this.promise = promise;
+		}
+
+		@Override
+		public void run() {
+			try {
+				promise.failure(new TimeoutException());
+			} catch (Throwable t) {
+				System.err.println("CAUGHT AN ERROR IN THE TEST: " + t.getMessage());
+				t.printStackTrace();
+			}
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/b540c718/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingRpcService.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingRpcService.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingRpcService.java
new file mode 100644
index 0000000..7e92e8d
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingRpcService.java
@@ -0,0 +1,115 @@
+/*
+ * 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.rpc;
+
+import akka.dispatch.Futures;
+import akka.util.Timeout;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.rpc.akka.AkkaRpcService;
+
+import scala.concurrent.Future;
+import scala.concurrent.duration.FiniteDuration;
+
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * An RPC Service implementation for testing. This RPC service acts as a replacement for
+ * teh regular RPC service for cases where tests need to return prepared mock gateways instead of
+ * proper RPC gateways.
+ * 
+ * <p>The TestingRpcService can be used for example in the following fashion,
+ * using <i>Mockito</i> for mocks and verification:
+ * 
+ * <pre>{@code
+ * TestingRpcService rpc = new TestingRpcService();
+ *
+ * ResourceManagerGateway testGateway = mock(ResourceManagerGateway.class);
+ * rpc.registerGateway("myAddress", testGateway);
+ * 
+ * MyComponentToTest component = new MyComponentToTest();
+ * component.triggerSomethingThatCallsTheGateway();
+ * 
+ * verify(testGateway, timeout(1000)).theTestMethod(any(UUID.class), anyString());
+ * }</pre>
+ */
+public class TestingRpcService extends AkkaRpcService {
+
+	/** Map of pre-registered connections */
+	private final ConcurrentHashMap<String, RpcGateway> registeredConnections;
+
+	/**
+	 * Creates a new {@code TestingRpcService}. 
+	 */
+	public TestingRpcService() {
+		this(new Configuration());
+	}
+
+	/**
+	 * Creates a new {@code TestingRpcService}, using the given configuration. 
+	 */
+	public TestingRpcService(Configuration configuration) {
+		super(AkkaUtils.createLocalActorSystem(configuration), new Timeout(new FiniteDuration(10, TimeUnit.SECONDS)));
+
+		this.registeredConnections = new ConcurrentHashMap<>();
+	}
+
+	// ------------------------------------------------------------------------
+
+	@Override
+	public void stopService() {
+		super.stopService();
+		registeredConnections.clear();
+	}
+
+	// ------------------------------------------------------------------------
+	// connections
+	// ------------------------------------------------------------------------
+
+	public void registerGateway(String address, RpcGateway gateway) {
+		checkNotNull(address);
+		checkNotNull(gateway);
+
+		if (registeredConnections.putIfAbsent(address, gateway) != null) {
+			throw new IllegalStateException("a gateway is already registered under " + address);
+		}
+	}
+
+	@Override
+	public <C extends RpcGateway> Future<C> connect(String address, Class<C> clazz) {
+		RpcGateway gateway = registeredConnections.get(address);
+
+		if (gateway != null) {
+			if (clazz.isAssignableFrom(gateway.getClass())) {
+				@SuppressWarnings("unchecked")
+				C typedGateway = (C) gateway;
+				return Futures.successful(typedGateway);
+			} else {
+				return Futures.failed(
+						new Exception("Gateway registered under " + address + " is not of type " + clazz));
+			}
+		} else {
+			return Futures.failed(new Exception("No gateway registered under that name"));
+		}
+	}
+}
\ No newline at end of file


[06/50] [abbrv] flink git commit: [FLINK-4459] [distributed runtime] Introduce SlotProvider for Scheduler

Posted by tr...@apache.org.
[FLINK-4459] [distributed runtime] Introduce SlotProvider for Scheduler

This closes #2424


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

Branch: refs/heads/flip-6
Commit: 6e40f59015dcdb8529691318e8f5a33e831252b8
Parents: 502a79d
Author: Kurt Young <yk...@gmail.com>
Authored: Fri Aug 26 17:51:40 2016 +0800
Committer: Stephan Ewen <se...@apache.org>
Committed: Sun Sep 4 23:09:59 2016 +0200

----------------------------------------------------------------------
 .../flink/runtime/executiongraph/Execution.java |  15 +-
 .../runtime/executiongraph/ExecutionGraph.java  |  32 +-
 .../executiongraph/ExecutionJobVertex.java      |   7 +-
 .../runtime/executiongraph/ExecutionVertex.java |   8 +-
 .../flink/runtime/instance/SlotProvider.java    |  48 +++
 .../runtime/jobmanager/scheduler/Scheduler.java |  27 +-
 .../ExecutionGraphMetricsTest.java              |   8 +-
 .../ExecutionVertexSchedulingTest.java          |  28 +-
 .../TerminalStateDeadlockTest.java              |  27 +-
 .../ScheduleWithCoLocationHintTest.java         | 303 +++++++++++--------
 .../scheduler/SchedulerIsolatedTasksTest.java   |  45 ++-
 .../scheduler/SchedulerSlotSharingTest.java     | 230 +++++++-------
 12 files changed, 430 insertions(+), 348 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/6e40f590/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java
index 846df49..6826365 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java
@@ -34,13 +34,13 @@ import org.apache.flink.runtime.execution.ExecutionState;
 import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
 import org.apache.flink.runtime.instance.ActorGateway;
 import org.apache.flink.runtime.instance.SimpleSlot;
+import org.apache.flink.runtime.instance.SlotProvider;
 import org.apache.flink.runtime.io.network.ConnectionID;
 import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
 import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
 import org.apache.flink.runtime.jobmanager.scheduler.CoLocationConstraint;
 import org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException;
 import org.apache.flink.runtime.jobmanager.scheduler.ScheduledUnit;
-import org.apache.flink.runtime.jobmanager.scheduler.Scheduler;
 import org.apache.flink.runtime.jobmanager.scheduler.SlotAllocationFuture;
 import org.apache.flink.runtime.jobmanager.scheduler.SlotAllocationFutureAction;
 import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup;
@@ -271,15 +271,15 @@ public class Execution {
 	 *       to be scheduled immediately and no resource is available. If the task is accepted by the schedule, any
 	 *       error sets the vertex state to failed and triggers the recovery logic.
 	 * 
-	 * @param scheduler The scheduler to use to schedule this execution attempt.
+	 * @param slotProvider The slot provider to use to allocate slot for this execution attempt.
 	 * @param queued Flag to indicate whether the scheduler may queue this task if it cannot
 	 *               immediately deploy it.
 	 * 
 	 * @throws IllegalStateException Thrown, if the vertex is not in CREATED state, which is the only state that permits scheduling.
 	 * @throws NoResourceAvailableException Thrown is no queued scheduling is allowed and no resources are currently available.
 	 */
-	public boolean scheduleForExecution(Scheduler scheduler, boolean queued) throws NoResourceAvailableException {
-		if (scheduler == null) {
+	public boolean scheduleForExecution(SlotProvider slotProvider, boolean queued) throws NoResourceAvailableException {
+		if (slotProvider == null) {
 			throw new IllegalArgumentException("Cannot send null Scheduler when scheduling execution.");
 		}
 
@@ -299,9 +299,8 @@ public class Execution {
 
 			// IMPORTANT: To prevent leaks of cluster resources, we need to make sure that slots are returned
 			//     in all cases where the deployment failed. we use many try {} finally {} clauses to assure that
+			final SlotAllocationFuture future = slotProvider.allocateSlot(toSchedule, queued);
 			if (queued) {
-				SlotAllocationFuture future = scheduler.scheduleQueued(toSchedule);
-
 				future.setFutureAction(new SlotAllocationFutureAction() {
 					@Override
 					public void slotAllocated(SimpleSlot slot) {
@@ -319,7 +318,7 @@ public class Execution {
 				});
 			}
 			else {
-				SimpleSlot slot = scheduler.scheduleImmediately(toSchedule);
+				SimpleSlot slot = future.get();
 				try {
 					deployToSlot(slot);
 				}
@@ -560,7 +559,7 @@ public class Execution {
 					public Boolean call() throws Exception {
 						try {
 							consumerVertex.scheduleForExecution(
-									consumerVertex.getExecutionGraph().getScheduler(),
+									consumerVertex.getExecutionGraph().getSlotProvider(),
 									consumerVertex.getExecutionGraph().isQueuedSchedulingAllowed());
 						} catch (Throwable t) {
 							consumerVertex.fail(new IllegalStateException("Could not schedule consumer " +

http://git-wip-us.apache.org/repos/asf/flink/blob/6e40f590/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 92cab41..585e9f3 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
@@ -40,6 +40,7 @@ import org.apache.flink.runtime.checkpoint.stats.CheckpointStatsTracker;
 import org.apache.flink.runtime.execution.ExecutionState;
 import org.apache.flink.runtime.execution.SuppressRestartsException;
 import org.apache.flink.runtime.executiongraph.restart.RestartStrategy;
+import org.apache.flink.runtime.instance.SlotProvider;
 import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
 import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
 import org.apache.flink.runtime.jobgraph.JobStatus;
@@ -47,7 +48,6 @@ import org.apache.flink.runtime.jobgraph.JobVertex;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.jobgraph.ScheduleMode;
 import org.apache.flink.runtime.jobmanager.scheduler.CoLocationGroup;
-import org.apache.flink.runtime.jobmanager.scheduler.Scheduler;
 import org.apache.flink.runtime.query.KvStateLocationRegistry;
 import org.apache.flink.runtime.taskmanager.TaskExecutionState;
 import org.apache.flink.runtime.util.SerializableObject;
@@ -197,8 +197,8 @@ public class ExecutionGraph {
 
 	// ------ Fields that are relevant to the execution and need to be cleared before archiving  -------
 
-	/** The scheduler to use for scheduling new tasks as they are needed */
-	private Scheduler scheduler;
+	/** The slot provider to use for allocating slots for tasks as they are needed */
+	private SlotProvider slotProvider;
 
 	/** Strategy to use for restarts */
 	private RestartStrategy restartStrategy;
@@ -470,8 +470,8 @@ public class ExecutionGraph {
 		return jsonPlan;
 	}
 
-	public Scheduler getScheduler() {
-		return scheduler;
+	public SlotProvider getSlotProvider() {
+		return slotProvider;
 	}
 
 	public JobID getJobID() {
@@ -670,17 +670,17 @@ public class ExecutionGraph {
 		}
 	}
 
-	public void scheduleForExecution(Scheduler scheduler) throws JobException {
-		if (scheduler == null) {
+	public void scheduleForExecution(SlotProvider slotProvider) throws JobException {
+		if (slotProvider == null) {
 			throw new IllegalArgumentException("Scheduler must not be null.");
 		}
 
-		if (this.scheduler != null && this.scheduler != scheduler) {
-			throw new IllegalArgumentException("Cannot use different schedulers for the same job");
+		if (this.slotProvider != null && this.slotProvider != slotProvider) {
+			throw new IllegalArgumentException("Cannot use different slot providers for the same job");
 		}
 
 		if (transitionState(JobStatus.CREATED, JobStatus.RUNNING)) {
-			this.scheduler = scheduler;
+			this.slotProvider = slotProvider;
 
 			switch (scheduleMode) {
 
@@ -688,14 +688,14 @@ public class ExecutionGraph {
 					// simply take the vertices without inputs.
 					for (ExecutionJobVertex ejv : this.tasks.values()) {
 						if (ejv.getJobVertex().isInputVertex()) {
-							ejv.scheduleAll(scheduler, allowQueuedScheduling);
+							ejv.scheduleAll(slotProvider, allowQueuedScheduling);
 						}
 					}
 					break;
 
 				case EAGER:
 					for (ExecutionJobVertex ejv : getVerticesTopologically()) {
-						ejv.scheduleAll(scheduler, allowQueuedScheduling);
+						ejv.scheduleAll(slotProvider, allowQueuedScheduling);
 					}
 					break;
 
@@ -850,8 +850,8 @@ public class ExecutionGraph {
 					throw new IllegalStateException("Can only restart job from state restarting.");
 				}
 
-				if (scheduler == null) {
-					throw new IllegalStateException("The execution graph has not been scheduled before - scheduler is null.");
+				if (slotProvider == null) {
+					throw new IllegalStateException("The execution graph has not been scheduled before - slotProvider is null.");
 				}
 
 				this.currentExecutions.clear();
@@ -885,7 +885,7 @@ public class ExecutionGraph {
 				}
 			}
 
-			scheduleForExecution(scheduler);
+			scheduleForExecution(slotProvider);
 		}
 		catch (Throwable t) {
 			fail(t);
@@ -917,7 +917,7 @@ public class ExecutionGraph {
 
 		// clear the non-serializable fields
 		restartStrategy = null;
-		scheduler = null;
+		slotProvider = null;
 		checkpointCoordinator = null;
 		executionContext = null;
 		kvStateLocationRegistry = null;

http://git-wip-us.apache.org/repos/asf/flink/blob/6e40f590/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 d3dc8fe..1ac9522 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,6 +29,7 @@ import org.apache.flink.runtime.JobException;
 import org.apache.flink.runtime.accumulators.AccumulatorRegistry;
 import org.apache.flink.runtime.accumulators.StringifiedAccumulatorResult;
 import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.instance.SlotProvider;
 import org.apache.flink.runtime.jobgraph.JobVertex;
 import org.apache.flink.runtime.jobgraph.IntermediateDataSet;
 import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
@@ -37,7 +38,6 @@ 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;
-import org.apache.flink.runtime.jobmanager.scheduler.Scheduler;
 import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup;
 import org.apache.flink.runtime.util.SerializableObject;
 import org.apache.flink.util.Preconditions;
@@ -289,12 +289,13 @@ public class ExecutionJobVertex {
 	//  Actions
 	//---------------------------------------------------------------------------------------------
 	
-	public void scheduleAll(Scheduler scheduler, boolean queued) throws NoResourceAvailableException {
+	public void scheduleAll(SlotProvider slotProvider, boolean queued) throws NoResourceAvailableException {
+		
 		ExecutionVertex[] vertices = this.taskVertices;
 
 		// kick off the tasks
 		for (ExecutionVertex ev : vertices) {
-			ev.scheduleForExecution(scheduler, queued);
+			ev.scheduleForExecution(slotProvider, queued);
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/6e40f590/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 88e1b88..a8d5ee4 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
@@ -27,6 +27,7 @@ import org.apache.flink.runtime.deployment.PartialInputChannelDeploymentDescript
 import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor;
 import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor;
 import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.instance.SlotProvider;
 import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
 import org.apache.flink.runtime.instance.ActorGateway;
 import org.apache.flink.runtime.instance.SimpleSlot;
@@ -40,12 +41,11 @@ import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.jobmanager.scheduler.CoLocationConstraint;
 import org.apache.flink.runtime.jobmanager.scheduler.CoLocationGroup;
 import org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException;
-import org.apache.flink.runtime.jobmanager.scheduler.Scheduler;
 import org.apache.flink.util.SerializedValue;
-
 import org.apache.flink.runtime.state.ChainedStateHandle;
 import org.apache.flink.runtime.state.KeyGroupsStateHandle;
 import org.apache.flink.runtime.state.StreamStateHandle;
+
 import org.slf4j.Logger;
 
 import scala.concurrent.duration.FiniteDuration;
@@ -443,8 +443,8 @@ public class ExecutionVertex {
 		}
 	}
 
-	public boolean scheduleForExecution(Scheduler scheduler, boolean queued) throws NoResourceAvailableException {
-		return this.currentExecution.scheduleForExecution(scheduler, queued);
+	public boolean scheduleForExecution(SlotProvider slotProvider, boolean queued) throws NoResourceAvailableException {
+		return this.currentExecution.scheduleForExecution(slotProvider, queued);
 	}
 
 	public void deployToSlot(SimpleSlot slot) throws JobException {

http://git-wip-us.apache.org/repos/asf/flink/blob/6e40f590/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotProvider.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotProvider.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotProvider.java
new file mode 100644
index 0000000..b2c23a5
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/SlotProvider.java
@@ -0,0 +1,48 @@
+/*
+ * 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.instance;
+
+import org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException;
+import org.apache.flink.runtime.jobmanager.scheduler.ScheduledUnit;
+import org.apache.flink.runtime.jobmanager.scheduler.SlotAllocationFuture;
+
+/**
+ * The slot provider is responsible for preparing slots for ready-to-run tasks.
+ * 
+ * <p>It supports two allocating modes:
+ * <ul>
+ *     <li>Immediate allocating: A request for a task slot immediately gets satisfied, we can call
+ *         {@link SlotAllocationFuture#get()} to get the allocated slot.</li>
+ *     <li>Queued allocating: A request for a task slot is queued and returns a future that will be
+ *         fulfilled as soon as a slot becomes available.</li>
+ * </ul>
+ */
+public interface SlotProvider {
+
+	/**
+	 * Allocating slot with specific requirement.
+	 *
+	 * @param task         The task to allocate the slot for
+	 * @param allowQueued  Whether allow the task be queued if we do not have enough resource
+	 * @return The future of the allocation
+	 * 
+	 * @throws NoResourceAvailableException
+	 */
+	SlotAllocationFuture allocateSlot(ScheduledUnit task, boolean allowQueued) throws NoResourceAvailableException;
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/6e40f590/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java
index 734972d..c9cdd00 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java
@@ -39,6 +39,7 @@ import org.apache.commons.lang3.tuple.ImmutablePair;
 import org.apache.commons.lang3.tuple.Pair;
 
 import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.instance.SlotProvider;
 import org.apache.flink.runtime.instance.SlotSharingGroupAssignment;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.instance.SharedSlot;
@@ -65,7 +66,7 @@ import scala.concurrent.ExecutionContext;
  *         fulfilled as soon as a slot becomes available.</li>
  * </ul>
  */
-public class Scheduler implements InstanceListener, SlotAvailabilityListener {
+public class Scheduler implements InstanceListener, SlotAvailabilityListener, SlotProvider {
 
 	/** Scheduler-wide logger */
 	private static final Logger LOG = LoggerFactory.getLogger(Scheduler.class);
@@ -129,30 +130,24 @@ public class Scheduler implements InstanceListener, SlotAvailabilityListener {
 	// ------------------------------------------------------------------------
 	//  Scheduling
 	// ------------------------------------------------------------------------
-	
-	public SimpleSlot scheduleImmediately(ScheduledUnit task) throws NoResourceAvailableException {
-		Object ret = scheduleTask(task, false);
-		if (ret instanceof SimpleSlot) {
-			return (SimpleSlot) ret;
-		}
-		else {
-			throw new RuntimeException();
-		}
-	}
-	
-	public SlotAllocationFuture scheduleQueued(ScheduledUnit task) throws NoResourceAvailableException {
-		Object ret = scheduleTask(task, true);
+
+
+	@Override
+	public SlotAllocationFuture allocateSlot(ScheduledUnit task, boolean allowQueued) 
+			throws NoResourceAvailableException {
+
+		final Object ret = scheduleTask(task, allowQueued);
 		if (ret instanceof SimpleSlot) {
 			return new SlotAllocationFuture((SimpleSlot) ret);
 		}
-		if (ret instanceof SlotAllocationFuture) {
+		else if (ret instanceof SlotAllocationFuture) {
 			return (SlotAllocationFuture) ret;
 		}
 		else {
 			throw new RuntimeException();
 		}
 	}
-	
+
 	/**
 	 * Returns either a {@link org.apache.flink.runtime.instance.SimpleSlot}, or a {@link SlotAllocationFuture}.
 	 */

http://git-wip-us.apache.org/repos/asf/flink/blob/6e40f590/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphMetricsTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphMetricsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphMetricsTest.java
index d5520fd..aa5925f 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphMetricsTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphMetricsTest.java
@@ -33,6 +33,7 @@ import org.apache.flink.runtime.execution.ExecutionState;
 import org.apache.flink.runtime.executiongraph.restart.RestartStrategy;
 import org.apache.flink.runtime.instance.ActorGateway;
 import org.apache.flink.runtime.instance.Instance;
+import org.apache.flink.runtime.jobmanager.scheduler.SlotAllocationFuture;
 import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
 import org.apache.flink.runtime.instance.SimpleSlot;
 import org.apache.flink.runtime.instance.Slot;
@@ -70,8 +71,8 @@ import java.util.concurrent.TimeUnit;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
+
+import static org.mockito.Mockito.*;
 
 public class ExecutionGraphMetricsTest extends TestLogger {
 
@@ -135,7 +136,8 @@ public class ExecutionGraphMetricsTest extends TestLogger {
 			when(simpleSlot.setExecutedVertex(Matchers.any(Execution.class))).thenReturn(true);
 			when(simpleSlot.getRoot()).thenReturn(rootSlot);
 
-			when(scheduler.scheduleImmediately(Matchers.any(ScheduledUnit.class))).thenReturn(simpleSlot);
+			when(scheduler.allocateSlot(any(ScheduledUnit.class), anyBoolean()))
+					.thenReturn(new SlotAllocationFuture(simpleSlot));
 
 			
 

http://git-wip-us.apache.org/repos/asf/flink/blob/6e40f590/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 5e9ee33..c576ce5 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
@@ -18,25 +18,29 @@
 
 package org.apache.flink.runtime.executiongraph;
 
-import static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.getExecutionVertex;
-import static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.getInstance;
-import static org.junit.Assert.*;
-import static org.mockito.Mockito.*;
-
 import org.apache.flink.runtime.akka.AkkaUtils;
 import org.apache.flink.runtime.execution.ExecutionState;
 import org.apache.flink.runtime.instance.DummyActorGateway;
 import org.apache.flink.runtime.instance.Instance;
 import org.apache.flink.runtime.instance.SimpleSlot;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
-import org.apache.flink.runtime.jobmanager.scheduler.Scheduler;
 import org.apache.flink.runtime.jobmanager.scheduler.ScheduledUnit;
+import org.apache.flink.runtime.jobmanager.scheduler.Scheduler;
 import org.apache.flink.runtime.jobmanager.scheduler.SlotAllocationFuture;
 import org.apache.flink.runtime.testingUtils.TestingUtils;
-import org.junit.Test;
 
+import org.junit.Test;
 import org.mockito.Matchers;
 
+import static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.getExecutionVertex;
+import static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.getInstance;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Mockito.anyBoolean;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
 public class ExecutionVertexSchedulingTest {
 
 	@Test
@@ -54,7 +58,8 @@ public class ExecutionVertexSchedulingTest {
 			assertTrue(slot.isReleased());
 
 			Scheduler scheduler = mock(Scheduler.class);
-			when(scheduler.scheduleImmediately(Matchers.any(ScheduledUnit.class))).thenReturn(slot);
+			when(scheduler.allocateSlot(Matchers.any(ScheduledUnit.class), anyBoolean()))
+				.thenReturn(new SlotAllocationFuture(slot));
 
 			assertEquals(ExecutionState.CREATED, vertex.getExecutionState());
 			// try to deploy to the slot
@@ -86,7 +91,7 @@ public class ExecutionVertexSchedulingTest {
 			final SlotAllocationFuture future = new SlotAllocationFuture();
 
 			Scheduler scheduler = mock(Scheduler.class);
-			when(scheduler.scheduleQueued(Matchers.any(ScheduledUnit.class))).thenReturn(future);
+			when(scheduler.allocateSlot(Matchers.any(ScheduledUnit.class), anyBoolean())).thenReturn(future);
 
 			assertEquals(ExecutionState.CREATED, vertex.getExecutionState());
 			// try to deploy to the slot
@@ -117,7 +122,8 @@ public class ExecutionVertexSchedulingTest {
 			final SimpleSlot slot = instance.allocateSimpleSlot(ejv.getJobId());
 
 			Scheduler scheduler = mock(Scheduler.class);
-			when(scheduler.scheduleImmediately(Matchers.any(ScheduledUnit.class))).thenReturn(slot);
+			when(scheduler.allocateSlot(Matchers.any(ScheduledUnit.class), anyBoolean()))
+				.thenReturn(new SlotAllocationFuture(slot));
 
 			assertEquals(ExecutionState.CREATED, vertex.getExecutionState());
 
@@ -130,4 +136,4 @@ public class ExecutionVertexSchedulingTest {
 			fail(e.getMessage());
 		}
 	}
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/6e40f590/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/TerminalStateDeadlockTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/TerminalStateDeadlockTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/TerminalStateDeadlockTest.java
index 870ae05..4cae7c2 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/TerminalStateDeadlockTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/TerminalStateDeadlockTest.java
@@ -30,6 +30,7 @@ import org.apache.flink.runtime.instance.Instance;
 import org.apache.flink.runtime.taskmanager.TaskManagerLocation;
 import org.apache.flink.runtime.instance.InstanceID;
 import org.apache.flink.runtime.instance.SimpleSlot;
+import org.apache.flink.runtime.instance.SlotProvider;
 import org.apache.flink.runtime.jobgraph.JobVertex;
 import org.apache.flink.runtime.jobgraph.JobStatus;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
@@ -54,12 +55,12 @@ import java.util.concurrent.TimeUnit;
 import static org.junit.Assert.*;
 
 public class TerminalStateDeadlockTest {
-	
+
 	private final Field stateField;
 	private final Field resourceField;
 	private final Field execGraphStateField;
-	private final Field execGraphSchedulerField;
-	
+	private final Field execGraphSlotProviderField;
+
 	private final SimpleSlot resource;
 
 
@@ -75,8 +76,8 @@ public class TerminalStateDeadlockTest {
 			this.execGraphStateField = ExecutionGraph.class.getDeclaredField("state");
 			this.execGraphStateField.setAccessible(true);
 
-			this.execGraphSchedulerField = ExecutionGraph.class.getDeclaredField("scheduler");
-			this.execGraphSchedulerField.setAccessible(true);
+			this.execGraphSlotProviderField = ExecutionGraph.class.getDeclaredField("slotProvider");
+			this.execGraphSlotProviderField.setAccessible(true);
 			
 			// the dummy resource
 			ResourceID resourceId = ResourceID.generate();
@@ -96,11 +97,9 @@ public class TerminalStateDeadlockTest {
 			throw new RuntimeException();
 		}
 	}
-	
-	 
-	
+
 	// ------------------------------------------------------------------------
-	
+
 	@Test
 	public void testProvokeDeadlock() {
 		try {
@@ -135,7 +134,7 @@ public class TerminalStateDeadlockTest {
 				initializeExecution(e2);
 
 				execGraphStateField.set(eg, JobStatus.FAILING);
-				execGraphSchedulerField.set(eg, scheduler);
+				execGraphSlotProviderField.set(eg, scheduler);
 				
 				Runnable r1 = new Runnable() {
 					@Override
@@ -173,12 +172,10 @@ public class TerminalStateDeadlockTest {
 	
 	static class TestExecGraph extends ExecutionGraph {
 
-		private static final long serialVersionUID = -7606144898417942044L;
-		
 		private static final Configuration EMPTY_CONFIG = new Configuration();
 
 		private static final FiniteDuration TIMEOUT = new FiniteDuration(30, TimeUnit.SECONDS);
-		
+
 		private volatile boolean done;
 
 		TestExecGraph(JobID jobId) throws IOException {
@@ -193,14 +190,14 @@ public class TerminalStateDeadlockTest {
 		}
 
 		@Override
-		public void scheduleForExecution(Scheduler scheduler) {
+		public void scheduleForExecution(SlotProvider slotProvider) {
 			// notify that we are done with the "restarting"
 			synchronized (this) {
 				done = true;
 				this.notifyAll();
 			}
 		}
-		
+
 		public void waitTillDone() {
 			try {
 				synchronized (this) {

http://git-wip-us.apache.org/repos/asf/flink/blob/6e40f590/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleWithCoLocationHintTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleWithCoLocationHintTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleWithCoLocationHintTest.java
index eab4fea..b803702 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleWithCoLocationHintTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleWithCoLocationHintTest.java
@@ -63,18 +63,18 @@ public class ScheduleWithCoLocationHintTest {
 			CoLocationConstraint c6 = new CoLocationConstraint(ccg);
 
 			// schedule 4 tasks from the first vertex group
-			SimpleSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 0, 6), sharingGroup, c1));
-			SimpleSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 1, 6), sharingGroup, c2));
-			SimpleSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 2, 6), sharingGroup, c3));
-			SimpleSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 3, 6), sharingGroup, c4));
-			SimpleSlot s5 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 0, 6), sharingGroup, c1));
-			SimpleSlot s6 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 1, 6), sharingGroup, c2));
-			SimpleSlot s7 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 2, 6), sharingGroup, c3));
-			SimpleSlot s8 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 4, 6), sharingGroup, c5));
-			SimpleSlot s9 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 5, 6), sharingGroup, c6));
-			SimpleSlot s10 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 3, 6), sharingGroup, c4));
-			SimpleSlot s11 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 4, 6), sharingGroup, c5));
-			SimpleSlot s12 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 5, 6), sharingGroup, c6));
+			SimpleSlot s1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 0, 6), sharingGroup, c1), false).get();
+			SimpleSlot s2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 1, 6), sharingGroup, c2), false).get();
+			SimpleSlot s3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 2, 6), sharingGroup, c3), false).get();
+			SimpleSlot s4 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 3, 6), sharingGroup, c4), false).get();
+			SimpleSlot s5 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 0, 6), sharingGroup, c1), false).get();
+			SimpleSlot s6 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 1, 6), sharingGroup, c2), false).get();
+			SimpleSlot s7 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 2, 6), sharingGroup, c3), false).get();
+			SimpleSlot s8 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 4, 6), sharingGroup, c5), false).get();
+			SimpleSlot s9 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 5, 6), sharingGroup, c6), false).get();
+			SimpleSlot s10 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 3, 6), sharingGroup, c4), false).get();
+			SimpleSlot s11 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 4, 6), sharingGroup, c5), false).get();
+			SimpleSlot s12 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 5, 6), sharingGroup, c6), false).get();
 
 			assertNotNull(s1);
 			assertNotNull(s2);
@@ -109,22 +109,22 @@ public class ScheduleWithCoLocationHintTest {
 			assertEquals(s4.getTaskManagerID(), s10.getTaskManagerID());
 			assertEquals(s8.getTaskManagerID(), s11.getTaskManagerID());
 			assertEquals(s9.getTaskManagerID(), s12.getTaskManagerID());
-			
+
 			assertEquals(c1.getLocation(), s1.getTaskManagerLocation());
 			assertEquals(c2.getLocation(), s2.getTaskManagerLocation());
 			assertEquals(c3.getLocation(), s3.getTaskManagerLocation());
 			assertEquals(c4.getLocation(), s4.getTaskManagerLocation());
 			assertEquals(c5.getLocation(), s8.getTaskManagerLocation());
 			assertEquals(c6.getLocation(), s9.getTaskManagerLocation());
-			
+
 			// check the scheduler's bookkeeping
 			assertEquals(0, scheduler.getNumberOfAvailableSlots());
-			
+
 			// the first assignments are unconstrained, co.-scheduling is constrained
 			assertEquals(6, scheduler.getNumberOfLocalizedAssignments());
 			assertEquals(0, scheduler.getNumberOfNonLocalizedAssignments());
 			assertEquals(6, scheduler.getNumberOfUnconstrainedAssignments());
-			
+
 			// release some slots, be sure that new available ones come up
 			s1.releaseSlot();
 			s2.releaseSlot();
@@ -135,10 +135,11 @@ public class ScheduleWithCoLocationHintTest {
 			s11.releaseSlot();
 			s12.releaseSlot();
 			assertTrue(scheduler.getNumberOfAvailableSlots() >= 1);
-			
-			SimpleSlot single = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(new JobVertexID(), 0, 1)));
+
+			SimpleSlot single = scheduler.allocateSlot(
+					new ScheduledUnit(getTestVertex(new JobVertexID(), 0, 1)), false).get();
 			assertNotNull(single);
-			
+
 			s1.releaseSlot();
 			s2.releaseSlot();
 			s3.releaseSlot();
@@ -149,9 +150,9 @@ public class ScheduleWithCoLocationHintTest {
 			s9.releaseSlot();
 			s11.releaseSlot();
 			s12.releaseSlot();
-			
+
 			assertEquals(5, scheduler.getNumberOfAvailableSlots());
-			
+
 			assertEquals(6, scheduler.getNumberOfLocalizedAssignments());
 			assertEquals(0, scheduler.getNumberOfNonLocalizedAssignments());
 			assertEquals(7, scheduler.getNumberOfUnconstrainedAssignments());
@@ -161,7 +162,7 @@ public class ScheduleWithCoLocationHintTest {
 			fail(e.getMessage());
 		}
 	}
-	
+
 	@Test
 	public void scheduleWithIntermediateRelease() {
 		try {
@@ -169,34 +170,37 @@ public class ScheduleWithCoLocationHintTest {
 			JobVertexID jid2 = new JobVertexID();
 			JobVertexID jid3 = new JobVertexID();
 			JobVertexID jid4 = new JobVertexID();
-			
+
 			Scheduler scheduler = new Scheduler(TestingUtils.directExecutionContext());
-			
+
 			Instance i1 = getRandomInstance(1);
 			Instance i2 = getRandomInstance(1);
-			
+
 			scheduler.newInstanceAvailable(i1);
 			scheduler.newInstanceAvailable(i2);
-			
+
 			assertEquals(2, scheduler.getNumberOfAvailableSlots());
-			
+
 			SlotSharingGroup sharingGroup = new SlotSharingGroup();
 			CoLocationConstraint c1 = new CoLocationConstraint(new CoLocationGroup());
-			
-			SimpleSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 0, 1), sharingGroup, c1));
-			SimpleSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 0, 1), sharingGroup, c1));
-			
-			SimpleSlot sSolo = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid4, 0, 1)));
-			
+
+			SimpleSlot s1 = scheduler.allocateSlot(
+					new ScheduledUnit(getTestVertex(jid1, 0, 1), sharingGroup, c1), false).get();
+			SimpleSlot s2 = scheduler.allocateSlot(
+					new ScheduledUnit(getTestVertex(jid2, 0, 1), sharingGroup, c1), false).get();
+
+			SimpleSlot sSolo = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid4, 0, 1)), false).get();
+
 			ResourceID taskManager = s1.getTaskManagerID();
-			
+
 			s1.releaseSlot();
 			s2.releaseSlot();
 			sSolo.releaseSlot();
-			
-			SimpleSlot sNew = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid3, 0, 1), sharingGroup, c1));
+
+			SimpleSlot sNew = scheduler.allocateSlot(
+					new ScheduledUnit(getTestVertex(jid3, 0, 1), sharingGroup, c1), false).get();
 			assertEquals(taskManager, sNew.getTaskManagerID());
-			
+
 			assertEquals(2, scheduler.getNumberOfLocalizedAssignments());
 			assertEquals(0, scheduler.getNumberOfNonLocalizedAssignments());
 			assertEquals(2, scheduler.getNumberOfUnconstrainedAssignments());
@@ -206,41 +210,41 @@ public class ScheduleWithCoLocationHintTest {
 			fail(e.getMessage());
 		}
 	}
-	
+
 	@Test
 	public void scheduleWithReleaseNoResource() {
 		try {
 			JobVertexID jid1 = new JobVertexID();
 			JobVertexID jid2 = new JobVertexID();
 			JobVertexID jid3 = new JobVertexID();
-			
+
 			Scheduler scheduler = new Scheduler(TestingUtils.directExecutionContext());
-			
+
 			Instance i1 = getRandomInstance(1);
 			Instance i2 = getRandomInstance(1);
-			
+
 			scheduler.newInstanceAvailable(i1);
 			scheduler.newInstanceAvailable(i2);
-			
+
 			assertEquals(2, scheduler.getNumberOfAvailableSlots());
-			
+
 			SlotSharingGroup sharingGroup = new SlotSharingGroup();
 			CoLocationConstraint c1 = new CoLocationConstraint(new CoLocationGroup());
-			
-			SimpleSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 0, 1), sharingGroup, c1));
+
+			SimpleSlot s1 = scheduler.allocateSlot(
+					new ScheduledUnit(getTestVertex(jid1, 0, 1), sharingGroup, c1), false).get();
 			s1.releaseSlot();
-			
-			scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 0, 1)));
-			scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 1, 2)));
-			
-			
+
+			scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 0, 1)), false).get();
+			scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid2, 1, 2)), false).get();
+
 			try {
-				scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid3, 0, 1), sharingGroup, c1));
+				scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid3, 0, 1), sharingGroup, c1), false).get();
 				fail("Scheduled even though no resource was available.");
 			} catch (NoResourceAvailableException e) {
 				// expected
 			}
-			
+
 			assertEquals(0, scheduler.getNumberOfLocalizedAssignments());
 			assertEquals(0, scheduler.getNumberOfNonLocalizedAssignments());
 			assertEquals(3, scheduler.getNumberOfUnconstrainedAssignments());
@@ -250,7 +254,7 @@ public class ScheduleWithCoLocationHintTest {
 			fail(e.getMessage());
 		}
 	}
-	
+
 	@Test
 	public void scheduleMixedCoLocationSlotSharing() {
 		try {
@@ -276,27 +280,35 @@ public class ScheduleWithCoLocationHintTest {
 			SlotSharingGroup shareGroup = new SlotSharingGroup();
 
 			// first wave
-			scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 0, 4), shareGroup));
-			scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 2, 4), shareGroup));
-			scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 1, 4), shareGroup));
-			scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 3, 4), shareGroup));
+			scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 0, 4), shareGroup), false);
+			scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 2, 4), shareGroup), false);
+			scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 1, 4), shareGroup), false);
+			scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid1, 3, 4), shareGroup), false);
 			
 			// second wave
-			SimpleSlot s21 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 0, 4), shareGroup, clc1));
-			SimpleSlot s22 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 2, 4), shareGroup, clc2));
-			SimpleSlot s23 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 1, 4), shareGroup, clc3));
-			SimpleSlot s24 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 3, 4), shareGroup, clc4));
+			SimpleSlot s21 = scheduler.allocateSlot(
+					new ScheduledUnit(getTestVertex(jid2, 0, 4), shareGroup, clc1), false).get();
+			SimpleSlot s22 = scheduler.allocateSlot(
+					new ScheduledUnit(getTestVertex(jid2, 2, 4), shareGroup, clc2), false).get();
+			SimpleSlot s23 = scheduler.allocateSlot(
+					new ScheduledUnit(getTestVertex(jid2, 1, 4), shareGroup, clc3), false).get();
+			SimpleSlot s24 = scheduler.allocateSlot(
+					new ScheduledUnit(getTestVertex(jid2, 3, 4), shareGroup, clc4), false).get();
 			
 			// third wave
-			SimpleSlot s31 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid3, 1, 4), shareGroup, clc2));
-			SimpleSlot s32 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid3, 2, 4), shareGroup, clc3));
-			SimpleSlot s33 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid3, 3, 4), shareGroup, clc4));
-			SimpleSlot s34 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid3, 0, 4), shareGroup, clc1));
-			
-			scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid4, 0, 4), shareGroup));
-			scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid4, 1, 4), shareGroup));
-			scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid4, 2, 4), shareGroup));
-			scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid4, 3, 4), shareGroup));
+			SimpleSlot s31 = scheduler.allocateSlot(
+					new ScheduledUnit(getTestVertex(jid3, 1, 4), shareGroup, clc2), false).get();
+			SimpleSlot s32 = scheduler.allocateSlot(
+					new ScheduledUnit(getTestVertex(jid3, 2, 4), shareGroup, clc3), false).get();
+			SimpleSlot s33 = scheduler.allocateSlot(
+					new ScheduledUnit(getTestVertex(jid3, 3, 4), shareGroup, clc4), false).get();
+			SimpleSlot s34 = scheduler.allocateSlot(
+					new ScheduledUnit(getTestVertex(jid3, 0, 4), shareGroup, clc1), false).get();
+			
+			scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid4, 0, 4), shareGroup), false);
+			scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid4, 1, 4), shareGroup), false);
+			scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid4, 2, 4), shareGroup), false);
+			scheduler.allocateSlot(new ScheduledUnit(getTestVertex(jid4, 3, 4), shareGroup), false);
 			
 			assertEquals(s21.getTaskManagerID(), s34.getTaskManagerID());
 			assertEquals(s22.getTaskManagerID(), s31.getTaskManagerID());
@@ -341,20 +353,26 @@ public class ScheduleWithCoLocationHintTest {
 			CoLocationConstraint cc2 = new CoLocationConstraint(ccg);
 
 			// schedule something into the shared group so that both instances are in the sharing group
-			SimpleSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, loc1), sharingGroup));
-			SimpleSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, loc2), sharingGroup));
+			SimpleSlot s1 = scheduler.allocateSlot(
+					new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, loc1), sharingGroup), false).get();
+			SimpleSlot s2 = scheduler.allocateSlot(
+					new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, loc2), sharingGroup), false).get();
 			
 			// schedule one locally to instance 1
-			SimpleSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, loc1), sharingGroup, cc1));
+			SimpleSlot s3 = scheduler.allocateSlot(
+					new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, loc1), sharingGroup, cc1), false).get();
 
 			// schedule with co location constraint (yet unassigned) and a preference for
 			// instance 1, but it can only get instance 2
-			SimpleSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, loc1), sharingGroup, cc2));
+			SimpleSlot s4 = scheduler.allocateSlot(
+					new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, loc1), sharingGroup, cc2), false).get();
 			
 			// schedule something into the assigned co-location constraints and check that they override the
 			// other preferences
-			SimpleSlot s5 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid3, 0, 2, loc2), sharingGroup, cc1));
-			SimpleSlot s6 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid3, 1, 2, loc1), sharingGroup, cc2));
+			SimpleSlot s5 = scheduler.allocateSlot(
+					new ScheduledUnit(getTestVertexWithLocation(jid3, 0, 2, loc2), sharingGroup, cc1), false).get();
+			SimpleSlot s6 = scheduler.allocateSlot(
+					new ScheduledUnit(getTestVertexWithLocation(jid3, 1, 2, loc1), sharingGroup, cc2), false).get();
 			
 			// check that each slot got three
 			assertEquals(3, s1.getRoot().getNumberLeaves());
@@ -386,13 +404,13 @@ public class ScheduleWithCoLocationHintTest {
 			fail(e.getMessage());
 		}
 	}
-	
+
 	@Test
 	public void testSlotReleasedInBetween() {
 		try {
 			JobVertexID jid1 = new JobVertexID();
 			JobVertexID jid2 = new JobVertexID();
-			
+
 			Scheduler scheduler = new Scheduler(TestingUtils.directExecutionContext());
 
 			Instance i1 = getRandomInstance(1);
@@ -403,36 +421,40 @@ public class ScheduleWithCoLocationHintTest {
 
 			scheduler.newInstanceAvailable(i2);
 			scheduler.newInstanceAvailable(i1);
-			
+
 			assertEquals(2, scheduler.getNumberOfAvailableSlots());
-			
+
 			SlotSharingGroup sharingGroup = new SlotSharingGroup();
-			
+
 			CoLocationGroup ccg = new CoLocationGroup();
 			CoLocationConstraint cc1 = new CoLocationConstraint(ccg);
 			CoLocationConstraint cc2 = new CoLocationConstraint(ccg);
 
-			SimpleSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, loc1), sharingGroup, cc1));
-			SimpleSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, loc2), sharingGroup, cc2));
-			
+			SimpleSlot s1 = scheduler.allocateSlot(
+					new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, loc1), sharingGroup, cc1), false).get();
+			SimpleSlot s2 = scheduler.allocateSlot(
+					new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, loc2), sharingGroup, cc2), false).get();
+
 			s1.releaseSlot();
 			s2.releaseSlot();
-			
+
 			assertEquals(2, scheduler.getNumberOfAvailableSlots());
 			assertEquals(0, sharingGroup.getTaskAssignment().getNumberOfSlots());
 
-			SimpleSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, loc2), sharingGroup, cc1));
-			SimpleSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, loc1), sharingGroup, cc2));
-			
+			SimpleSlot s3 = scheduler.allocateSlot(
+					new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, loc2), sharingGroup, cc1), false).get();
+			SimpleSlot s4 = scheduler.allocateSlot(
+					new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, loc1), sharingGroup, cc2), false).get();
+
 			// still preserves the previous instance mapping)
 			assertEquals(i1.getTaskManagerID(), s3.getTaskManagerID());
 			assertEquals(i2.getTaskManagerID(), s4.getTaskManagerID());
-			
+
 			s3.releaseSlot();
 			s4.releaseSlot();
 
 			assertEquals(2, scheduler.getNumberOfAvailableSlots());
-			
+
 			assertEquals(4, scheduler.getNumberOfLocalizedAssignments());
 			assertEquals(0, scheduler.getNumberOfNonLocalizedAssignments());
 			assertEquals(0, scheduler.getNumberOfUnconstrainedAssignments());
@@ -442,14 +464,14 @@ public class ScheduleWithCoLocationHintTest {
 			fail(e.getMessage());
 		}
 	}
-	
+
 	@Test
 	public void testSlotReleasedInBetweenAndNoNewLocal() {
 		try {
 			JobVertexID jid1 = new JobVertexID();
 			JobVertexID jid2 = new JobVertexID();
 			JobVertexID jidx = new JobVertexID();
-			
+
 			Scheduler scheduler = new Scheduler(TestingUtils.directExecutionContext());
 
 			Instance i1 = getRandomInstance(1);
@@ -460,41 +482,46 @@ public class ScheduleWithCoLocationHintTest {
 
 			scheduler.newInstanceAvailable(i2);
 			scheduler.newInstanceAvailable(i1);
-			
+
 			assertEquals(2, scheduler.getNumberOfAvailableSlots());
-			
+
 			SlotSharingGroup sharingGroup = new SlotSharingGroup();
-			
+
 			CoLocationGroup ccg = new CoLocationGroup();
 			CoLocationConstraint cc1 = new CoLocationConstraint(ccg);
 			CoLocationConstraint cc2 = new CoLocationConstraint(ccg);
 
-			SimpleSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, loc1), sharingGroup, cc1));
-			SimpleSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, loc2), sharingGroup, cc2));
-			
+			SimpleSlot s1 = scheduler.allocateSlot(
+					new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, loc1), sharingGroup, cc1), false).get();
+			SimpleSlot s2 = scheduler.allocateSlot(
+					new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, loc2), sharingGroup, cc2), false).get();
+
 			s1.releaseSlot();
 			s2.releaseSlot();
-			
+
 			assertEquals(2, scheduler.getNumberOfAvailableSlots());
 			assertEquals(0, sharingGroup.getTaskAssignment().getNumberOfSlots());
 
-			SimpleSlot sa = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jidx, 0, 2)));
-			SimpleSlot sb = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jidx, 1, 2)));
-			
+			SimpleSlot sa = scheduler.allocateSlot(
+					new ScheduledUnit(getTestVertexWithLocation(jidx, 0, 2)), false).get();
+			SimpleSlot sb = scheduler.allocateSlot(
+					new ScheduledUnit(getTestVertexWithLocation(jidx, 1, 2)), false).get();
+
 			try {
-				scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, loc2), sharingGroup, cc1));
+				scheduler.allocateSlot(
+						new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, loc2), sharingGroup, cc1), false);
 				fail("should not be able to find a resource");
 			} catch (NoResourceAvailableException e) {
 				// good
 			} catch (Exception e) {
 				fail("wrong exception");
 			}
-			
+
 			sa.releaseSlot();
 			sb.releaseSlot();
 
 			assertEquals(2, scheduler.getNumberOfAvailableSlots());
-			
+
 			assertEquals(2, scheduler.getNumberOfLocalizedAssignments());
 			assertEquals(0, scheduler.getNumberOfNonLocalizedAssignments());
 			assertEquals(2, scheduler.getNumberOfUnconstrainedAssignments());
@@ -504,15 +531,15 @@ public class ScheduleWithCoLocationHintTest {
 			fail(e.getMessage());
 		}
 	}
-	
+
 	@Test
 	public void testScheduleOutOfOrder() {
 		try {
 			JobVertexID jid1 = new JobVertexID();
 			JobVertexID jid2 = new JobVertexID();
-			
+
 			Scheduler scheduler = new Scheduler(TestingUtils.directExecutionContext());
-			
+
 			Instance i1 = getRandomInstance(1);
 			Instance i2 = getRandomInstance(1);
 
@@ -520,11 +547,11 @@ public class ScheduleWithCoLocationHintTest {
 
 			scheduler.newInstanceAvailable(i2);
 			scheduler.newInstanceAvailable(i1);
-			
+
 			assertEquals(2, scheduler.getNumberOfAvailableSlots());
-			
+
 			SlotSharingGroup sharingGroup = new SlotSharingGroup();
-			
+
 			CoLocationGroup ccg = new CoLocationGroup();
 			CoLocationConstraint cc1 = new CoLocationConstraint(ccg);
 			CoLocationConstraint cc2 = new CoLocationConstraint(ccg);
@@ -532,33 +559,37 @@ public class ScheduleWithCoLocationHintTest {
 			// schedule something from the second job vertex id before the first is filled,
 			// and give locality preferences that hint at using the same shared slot for both
 			// co location constraints (which we seek to prevent)
-			SimpleSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, loc1), sharingGroup, cc1));
-			SimpleSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, loc1), sharingGroup, cc2));
+			SimpleSlot s1 = scheduler.allocateSlot(
+					new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, loc1), sharingGroup, cc1), false).get();
+			SimpleSlot s2 = scheduler.allocateSlot(
+					new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, loc1), sharingGroup, cc2), false).get();
+
+			SimpleSlot s3 = scheduler.allocateSlot(
+					new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, loc1), sharingGroup, cc1), false).get();
+			SimpleSlot s4 = scheduler.allocateSlot(
+					new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, loc1), sharingGroup, cc2), false).get();
 
-			SimpleSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, loc1), sharingGroup, cc1));
-			SimpleSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, loc1), sharingGroup, cc2));
-			
 			// check that each slot got three
 			assertEquals(2, s1.getRoot().getNumberLeaves());
 			assertEquals(2, s2.getRoot().getNumberLeaves());
-			
+
 			assertEquals(s1.getTaskManagerID(), s3.getTaskManagerID());
 			assertEquals(s2.getTaskManagerID(), s4.getTaskManagerID());
-			
+
 			// check the scheduler's bookkeeping
 			assertEquals(0, scheduler.getNumberOfAvailableSlots());
-			
+
 			assertEquals(3, scheduler.getNumberOfLocalizedAssignments());
 			assertEquals(1, scheduler.getNumberOfNonLocalizedAssignments());
 			assertEquals(0, scheduler.getNumberOfUnconstrainedAssignments());
-			
+
 			// release some slots, be sure that new available ones come up
 			s1.releaseSlot();
 			s2.releaseSlot();
 			s3.releaseSlot();
 			s4.releaseSlot();
 			assertEquals(2, scheduler.getNumberOfAvailableSlots());
-			
+
 			assertEquals(0, sharingGroup.getTaskAssignment().getNumberOfSlots());
 			assertEquals(0, sharingGroup.getTaskAssignment().getNumberOfAvailableSlotsForGroup(jid1));
 			assertEquals(0, sharingGroup.getTaskAssignment().getNumberOfAvailableSlotsForGroup(jid2));
@@ -568,15 +599,15 @@ public class ScheduleWithCoLocationHintTest {
 			fail(e.getMessage());
 		}
 	}
-	
+
 	@Test
 	public void nonColocationFollowsCoLocation() {
 		try {
 			JobVertexID jid1 = new JobVertexID();
 			JobVertexID jid2 = new JobVertexID();
-			
+
 			Scheduler scheduler = new Scheduler(TestingUtils.directExecutionContext());
-			
+
 			Instance i1 = getRandomInstance(1);
 			Instance i2 = getRandomInstance(1);
 
@@ -585,32 +616,36 @@ public class ScheduleWithCoLocationHintTest {
 
 			scheduler.newInstanceAvailable(i2);
 			scheduler.newInstanceAvailable(i1);
-			
+
 			assertEquals(2, scheduler.getNumberOfAvailableSlots());
 			
 			SlotSharingGroup sharingGroup = new SlotSharingGroup();
-			
+
 			CoLocationGroup ccg = new CoLocationGroup();
 			CoLocationConstraint cc1 = new CoLocationConstraint(ccg);
 			CoLocationConstraint cc2 = new CoLocationConstraint(ccg);
 
-			SimpleSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, loc1), sharingGroup, cc1));
-			SimpleSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, loc2), sharingGroup, cc2));
+			SimpleSlot s1 = scheduler.allocateSlot(
+					new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, loc1), sharingGroup, cc1), false).get();
+			SimpleSlot s2 = scheduler.allocateSlot(
+					new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, loc2), sharingGroup, cc2), false).get();
+
+			SimpleSlot s3 = scheduler.allocateSlot(
+					new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, loc1), sharingGroup), false).get();
+			SimpleSlot s4 = scheduler.allocateSlot(
+					new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, loc1), sharingGroup), false).get();
 
-			SimpleSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, loc1), sharingGroup));
-			SimpleSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, loc1), sharingGroup));
-			
 			// check that each slot got two
 			assertEquals(2, s1.getRoot().getNumberLeaves());
 			assertEquals(2, s2.getRoot().getNumberLeaves());
-			
+
 			s1.releaseSlot();
 			s2.releaseSlot();
 			s3.releaseSlot();
 			s4.releaseSlot();
-			
+
 			assertEquals(2, scheduler.getNumberOfAvailableSlots());
-			
+
 			assertEquals(0, sharingGroup.getTaskAssignment().getNumberOfSlots());
 			assertEquals(0, sharingGroup.getTaskAssignment().getNumberOfAvailableSlotsForGroup(jid1));
 			assertEquals(0, sharingGroup.getTaskAssignment().getNumberOfAvailableSlotsForGroup(jid2));

http://git-wip-us.apache.org/repos/asf/flink/blob/6e40f590/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerIsolatedTasksTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerIsolatedTasksTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerIsolatedTasksTest.java
index 25498c4..d78f551 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerIsolatedTasksTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerIsolatedTasksTest.java
@@ -35,7 +35,6 @@ import static org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils.a
 import static org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils.getDummyTask;
 import static org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils.getRandomInstance;
 import static org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils.getTestVertex;
-
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
@@ -122,17 +121,17 @@ public class SchedulerIsolatedTasksTest {
 			assertEquals(5, scheduler.getNumberOfAvailableSlots());
 			
 			// schedule something into all slots
-			SimpleSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getDummyTask()));
-			SimpleSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getDummyTask()));
-			SimpleSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getDummyTask()));
-			SimpleSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getDummyTask()));
-			SimpleSlot s5 = scheduler.scheduleImmediately(new ScheduledUnit(getDummyTask()));
+			SimpleSlot s1 = scheduler.allocateSlot(new ScheduledUnit(getDummyTask()), false).get();
+			SimpleSlot s2 = scheduler.allocateSlot(new ScheduledUnit(getDummyTask()), false).get();
+			SimpleSlot s3 = scheduler.allocateSlot(new ScheduledUnit(getDummyTask()), false).get();
+			SimpleSlot s4 = scheduler.allocateSlot(new ScheduledUnit(getDummyTask()), false).get();
+			SimpleSlot s5 = scheduler.allocateSlot(new ScheduledUnit(getDummyTask()), false).get();
 			
 			// the slots should all be different
 			assertTrue(areAllDistinct(s1, s2, s3, s4, s5));
 			
 			try {
-				scheduler.scheduleImmediately(new ScheduledUnit(getDummyTask()));
+				scheduler.allocateSlot(new ScheduledUnit(getDummyTask()), false);
 				fail("Scheduler accepted scheduling request without available resource.");
 			}
 			catch (NoResourceAvailableException e) {
@@ -145,8 +144,8 @@ public class SchedulerIsolatedTasksTest {
 			assertEquals(2, scheduler.getNumberOfAvailableSlots());
 			
 			// now we can schedule some more slots
-			SimpleSlot s6 = scheduler.scheduleImmediately(new ScheduledUnit(getDummyTask()));
-			SimpleSlot s7 = scheduler.scheduleImmediately(new ScheduledUnit(getDummyTask()));
+			SimpleSlot s6 = scheduler.allocateSlot(new ScheduledUnit(getDummyTask()), false).get();
+			SimpleSlot s7 = scheduler.allocateSlot(new ScheduledUnit(getDummyTask()), false).get();
 			
 			assertTrue(areAllDistinct(s1, s2, s3, s4, s5, s6, s7));
 			
@@ -245,7 +244,7 @@ public class SchedulerIsolatedTasksTest {
 			disposeThread.start();
 
 			for (int i = 0; i < NUM_TASKS_TO_SCHEDULE; i++) {
-				SlotAllocationFuture future = scheduler.scheduleQueued(new ScheduledUnit(getDummyTask()));
+				SlotAllocationFuture future = scheduler.allocateSlot(new ScheduledUnit(getDummyTask()), true);
 				future.setFutureAction(action);
 				allAllocatedSlots.add(future);
 			}
@@ -287,11 +286,11 @@ public class SchedulerIsolatedTasksTest {
 			scheduler.newInstanceAvailable(i3);
 			
 			List<SimpleSlot> slots = new ArrayList<SimpleSlot>();
-			slots.add(scheduler.scheduleImmediately(new ScheduledUnit(getDummyTask())));
-			slots.add(scheduler.scheduleImmediately(new ScheduledUnit(getDummyTask())));
-			slots.add(scheduler.scheduleImmediately(new ScheduledUnit(getDummyTask())));
-			slots.add(scheduler.scheduleImmediately(new ScheduledUnit(getDummyTask())));
-			slots.add(scheduler.scheduleImmediately(new ScheduledUnit(getDummyTask())));
+			slots.add(scheduler.allocateSlot(new ScheduledUnit(getDummyTask()), false).get());
+			slots.add(scheduler.allocateSlot(new ScheduledUnit(getDummyTask()), false).get());
+			slots.add(scheduler.allocateSlot(new ScheduledUnit(getDummyTask()), false).get());
+			slots.add(scheduler.allocateSlot(new ScheduledUnit(getDummyTask()), false).get());
+			slots.add(scheduler.allocateSlot(new ScheduledUnit(getDummyTask()), false).get());
 			
 			i2.markDead();
 			
@@ -312,7 +311,7 @@ public class SchedulerIsolatedTasksTest {
 			
 			// cannot get another slot, since all instances are dead
 			try {
-				scheduler.scheduleImmediately(new ScheduledUnit(getDummyTask()));
+				scheduler.allocateSlot(new ScheduledUnit(getDummyTask()), false).get();
 				fail("Scheduler served a slot from a dead instance");
 			}
 			catch (NoResourceAvailableException e) {
@@ -347,7 +346,7 @@ public class SchedulerIsolatedTasksTest {
 			scheduler.newInstanceAvailable(i3);
 			
 			// schedule something on an arbitrary instance
-			SimpleSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(new Instance[0])));
+			SimpleSlot s1 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(new Instance[0])), false).get();
 			
 			// figure out how we use the location hints
 			Instance first = (Instance) s1.getOwner();
@@ -355,28 +354,28 @@ public class SchedulerIsolatedTasksTest {
 			Instance third = first == i3 ? i2 : i3;
 			
 			// something that needs to go to the first instance again
-			SimpleSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(s1.getTaskManagerLocation())));
+			SimpleSlot s2 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(s1.getTaskManagerLocation())), false).get();
 			assertEquals(first, s2.getOwner());
 
 			// first or second --> second, because first is full
-			SimpleSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(first, second)));
+			SimpleSlot s3 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(first, second)), false).get();
 			assertEquals(second, s3.getOwner());
 			
 			// first or third --> third (because first is full)
-			SimpleSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(first, third)));
-			SimpleSlot s5 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(first, third)));
+			SimpleSlot s4 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(first, third)), false).get();
+			SimpleSlot s5 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(first, third)), false).get();
 			assertEquals(third, s4.getOwner());
 			assertEquals(third, s5.getOwner());
 			
 			// first or third --> second, because all others are full
-			SimpleSlot s6 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(first, third)));
+			SimpleSlot s6 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(first, third)), false).get();
 			assertEquals(second, s6.getOwner());
 			
 			// release something on the first and second instance
 			s2.releaseSlot();
 			s6.releaseSlot();
 			
-			SimpleSlot s7 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(first, third)));
+			SimpleSlot s7 = scheduler.allocateSlot(new ScheduledUnit(getTestVertex(first, third)), false).get();
 			assertEquals(first, s7.getOwner());
 			
 			assertEquals(1, scheduler.getNumberOfUnconstrainedAssignments());


[38/50] [abbrv] flink git commit: [FLINK-4363] Implement TaskManager basic startup of all components in java

Posted by tr...@apache.org.
[FLINK-4363] Implement TaskManager basic startup of all components in java

This closes #2400


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

Branch: refs/heads/flip-6
Commit: 73de8427b798d95d1dbcdaf16f186131d3043a30
Parents: 20fefaa
Author: \u6dd8\u6c5f <ta...@alibaba-inc.com>
Authored: Tue Aug 30 11:28:14 2016 +0800
Committer: Till Rohrmann <tr...@apache.org>
Committed: Thu Sep 8 17:27:00 2016 +0200

----------------------------------------------------------------------
 .../runtime/rpc/taskexecutor/TaskExecutor.java  | 686 ++++++++++++++++++-
 .../taskexecutor/TaskExecutorConfiguration.java | 151 ++++
 .../rpc/taskexecutor/TaskExecutorTest.java      |   8 +-
 3 files changed, 822 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/73de8427/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java
index f201e00..36d6310 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java
@@ -18,15 +18,60 @@
 
 package org.apache.flink.runtime.rpc.taskexecutor;
 
+import akka.actor.ActorSystem;
+import akka.dispatch.ExecutionContexts$;
+import akka.util.Timeout;
+import com.typesafe.config.Config;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.IllegalConfigurationException;
+import org.apache.flink.core.memory.HeapMemorySegment;
+import org.apache.flink.core.memory.HybridMemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
+import org.apache.flink.core.memory.MemoryType;
+import org.apache.flink.runtime.akka.AkkaUtils;
 import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.runtime.clusterframework.types.ResourceID;
 import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.instance.InstanceConnectionInfo;
+import org.apache.flink.runtime.io.disk.iomanager.IOManager;
+import org.apache.flink.runtime.io.disk.iomanager.IOManager.IOMode;
+import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
+import org.apache.flink.runtime.io.network.NetworkEnvironment;
+import org.apache.flink.runtime.io.network.netty.NettyConfig;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.rpc.RpcEndpoint;
 import org.apache.flink.runtime.rpc.RpcMethod;
 import org.apache.flink.runtime.rpc.RpcService;
-
+import org.apache.flink.runtime.rpc.akka.AkkaRpcService;
+import org.apache.flink.runtime.taskmanager.MemoryLogger;
+import org.apache.flink.runtime.util.EnvironmentInformation;
+import org.apache.flink.runtime.util.LeaderRetrievalUtils;
+import org.apache.flink.runtime.taskmanager.NetworkEnvironmentConfiguration;
+import org.apache.flink.util.MathUtils;
+import org.apache.flink.util.NetUtils;
+
+import scala.Tuple2;
+import scala.Option;
+import scala.Some;
+import scala.concurrent.ExecutionContext;
+import scala.concurrent.duration.Duration;
+import scala.concurrent.duration.FiniteDuration;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
 import java.util.UUID;
+import java.util.concurrent.ForkJoinPool;
+import java.util.concurrent.TimeUnit;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
@@ -36,12 +81,29 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
  */
 public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 
+	private static final Logger LOG = LoggerFactory.getLogger(TaskExecutor.class);
+
 	/** The unique resource ID of this TaskExecutor */
 	private final ResourceID resourceID;
 
 	/** The access to the leader election and metadata storage services */
 	private final HighAvailabilityServices haServices;
 
+	/** The task manager configuration */
+	private final TaskExecutorConfiguration taskExecutorConfig;
+
+	/** The I/O manager component in the task manager */
+	private final IOManager ioManager;
+
+	/** The memory manager component in the task manager */
+	private final MemoryManager memoryManager;
+
+	/** The network component in the task manager */
+	private final NetworkEnvironment networkEnvironment;
+
+	/** The number of slots in the task manager, should be 1 for YARN */
+	private final int numberOfSlots;
+
 	// --------- resource manager --------
 
 	private TaskExecutorToResourceManagerConnection resourceManagerConnection;
@@ -49,22 +111,24 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 	// ------------------------------------------------------------------------
 
 	public TaskExecutor(
+			TaskExecutorConfiguration taskExecutorConfig,
+			ResourceID resourceID,
+			MemoryManager memoryManager,
+			IOManager ioManager,
+			NetworkEnvironment networkEnvironment,
+			int numberOfSlots,
 			RpcService rpcService,
-			HighAvailabilityServices haServices,
-			ResourceID resourceID) {
+			HighAvailabilityServices haServices) {
 
 		super(rpcService);
 
-		this.haServices = checkNotNull(haServices);
+		this.taskExecutorConfig = checkNotNull(taskExecutorConfig);
 		this.resourceID = checkNotNull(resourceID);
-	}
-
-	// ------------------------------------------------------------------------
-	//  Properties
-	// ------------------------------------------------------------------------
-
-	public ResourceID getResourceID() {
-		return resourceID;
+		this.memoryManager = checkNotNull(memoryManager);
+		this.ioManager = checkNotNull(ioManager);
+		this.networkEnvironment = checkNotNull(networkEnvironment);
+		this.numberOfSlots = checkNotNull(numberOfSlots);
+		this.haServices = checkNotNull(haServices);
 	}
 
 	// ------------------------------------------------------------------------
@@ -83,7 +147,6 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 		}
 	}
 
-
 	// ------------------------------------------------------------------------
 	//  RPC methods - ResourceManager related
 	// ------------------------------------------------------------------------
@@ -94,12 +157,12 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 			if (newLeaderAddress != null) {
 				// the resource manager switched to a new leader
 				log.info("ResourceManager leader changed from {} to {}. Registering at new leader.",
-						resourceManagerConnection.getResourceManagerAddress(), newLeaderAddress);
+					resourceManagerConnection.getResourceManagerAddress(), newLeaderAddress);
 			}
 			else {
 				// address null means that the current leader is lost without a new leader being there, yet
 				log.info("Current ResourceManager {} lost leader status. Waiting for new ResourceManager leader.",
-						resourceManagerConnection.getResourceManagerAddress());
+					resourceManagerConnection.getResourceManagerAddress());
 			}
 
 			// drop the current connection or connection attempt
@@ -112,21 +175,604 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 		// establish a connection to the new leader
 		if (newLeaderAddress != null) {
 			log.info("Attempting to register at ResourceManager {}", newLeaderAddress);
-			resourceManagerConnection = 
-					new TaskExecutorToResourceManagerConnection(log, this, newLeaderAddress, newLeaderId);
+			resourceManagerConnection =
+				new TaskExecutorToResourceManagerConnection(log, this, newLeaderAddress, newLeaderId);
 			resourceManagerConnection.start();
 		}
 	}
 
+	/**
+	 * Starts and runs the TaskManager.
+	 * <p/>
+	 * This method first tries to select the network interface to use for the TaskManager
+	 * communication. The network interface is used both for the actor communication
+	 * (coordination) as well as for the data exchange between task managers. Unless
+	 * the hostname/interface is explicitly configured in the configuration, this
+	 * method will try out various interfaces and methods to connect to the JobManager
+	 * and select the one where the connection attempt is successful.
+	 * <p/>
+	 * After selecting the network interface, this method brings up an actor system
+	 * for the TaskManager and its actors, starts the TaskManager's services
+	 * (library cache, shuffle network stack, ...), and starts the TaskManager itself.
+	 *
+	 * @param configuration    The configuration for the TaskManager.
+	 * @param resourceID       The id of the resource which the task manager will run on.
+	 */
+	public static void selectNetworkInterfaceAndRunTaskManager(
+		Configuration configuration,
+		ResourceID resourceID) throws Exception {
+
+		final InetSocketAddress taskManagerAddress = selectNetworkInterfaceAndPort(configuration);
+
+		runTaskManager(taskManagerAddress.getHostName(), resourceID, taskManagerAddress.getPort(), configuration);
+	}
+
+	private static InetSocketAddress selectNetworkInterfaceAndPort(Configuration configuration)
+		throws Exception {
+		String taskManagerHostname = configuration.getString(ConfigConstants.TASK_MANAGER_HOSTNAME_KEY, null);
+		if (taskManagerHostname != null) {
+			LOG.info("Using configured hostname/address for TaskManager: " + taskManagerHostname);
+		} else {
+			LeaderRetrievalService leaderRetrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
+			FiniteDuration lookupTimeout = AkkaUtils.getLookupTimeout(configuration);
+
+			InetAddress taskManagerAddress = LeaderRetrievalUtils.findConnectingAddress(leaderRetrievalService, lookupTimeout);
+			taskManagerHostname = taskManagerAddress.getHostName();
+			LOG.info("TaskManager will use hostname/address '{}' ({}) for communication.",
+				taskManagerHostname, taskManagerAddress.getHostAddress());
+		}
+
+		// if no task manager port has been configured, use 0 (system will pick any free port)
+		final int actorSystemPort = configuration.getInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, 0);
+		if (actorSystemPort < 0 || actorSystemPort > 65535) {
+			throw new IllegalConfigurationException("Invalid value for '" +
+				ConfigConstants.TASK_MANAGER_IPC_PORT_KEY +
+				"' (port for the TaskManager actor system) : " + actorSystemPort +
+				" - Leave config parameter empty or use 0 to let the system choose a port automatically.");
+		}
+
+		return new InetSocketAddress(taskManagerHostname, actorSystemPort);
+	}
+
+	/**
+	 * Starts and runs the TaskManager. Brings up an actor system for the TaskManager and its
+	 * actors, starts the TaskManager's services (library cache, shuffle network stack, ...),
+	 * and starts the TaskManager itself.
+	 * <p/>
+	 * This method will also spawn a process reaper for the TaskManager (kill the process if
+	 * the actor fails) and optionally start the JVM memory logging thread.
+	 *
+	 * @param taskManagerHostname The hostname/address of the interface where the actor system
+	 *                            will communicate.
+	 * @param resourceID          The id of the resource which the task manager will run on.
+	 * @param actorSystemPort   The port at which the actor system will communicate.
+	 * @param configuration       The configuration for the TaskManager.
+	 */
+	private static void runTaskManager(
+		String taskManagerHostname,
+		ResourceID resourceID,
+		int actorSystemPort,
+		final Configuration configuration) throws Exception {
+
+		LOG.info("Starting TaskManager");
+
+		// Bring up the TaskManager actor system first, bind it to the given address.
+
+		LOG.info("Starting TaskManager actor system at " +
+			NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort));
+
+		final ActorSystem taskManagerSystem;
+		try {
+			Tuple2<String, Object> address = new Tuple2<String, Object>(taskManagerHostname, actorSystemPort);
+			Config akkaConfig = AkkaUtils.getAkkaConfig(configuration, new Some<>(address));
+			LOG.debug("Using akka configuration\n " + akkaConfig);
+			taskManagerSystem = AkkaUtils.createActorSystem(akkaConfig);
+		} catch (Throwable t) {
+			if (t instanceof org.jboss.netty.channel.ChannelException) {
+				Throwable cause = t.getCause();
+				if (cause != null && t.getCause() instanceof java.net.BindException) {
+					String address = NetUtils.hostAndPortToUrlString(taskManagerHostname, actorSystemPort);
+					throw new IOException("Unable to bind TaskManager actor system to address " +
+						address + " - " + cause.getMessage(), t);
+				}
+			}
+			throw new Exception("Could not create TaskManager actor system", t);
+		}
+
+		// start akka rpc service based on actor system
+		final Timeout timeout = new Timeout(AkkaUtils.getTimeout(configuration).toMillis(), TimeUnit.MILLISECONDS);
+		final AkkaRpcService akkaRpcService = new AkkaRpcService(taskManagerSystem, timeout);
+
+		// start high availability service to implement getResourceManagerLeaderRetriever method only
+		final HighAvailabilityServices haServices = new HighAvailabilityServices() {
+			@Override
+			public LeaderRetrievalService getResourceManagerLeaderRetriever() throws Exception {
+				return LeaderRetrievalUtils.createLeaderRetrievalService(configuration);
+			}
+
+			@Override
+			public LeaderElectionService getResourceManagerLeaderElectionService() throws Exception {
+				return null;
+			}
+
+			@Override
+			public LeaderElectionService getJobMasterLeaderElectionService(JobID jobID) throws Exception {
+				return null;
+			}
+		};
+
+		// start all the TaskManager services (network stack,  library cache, ...)
+		// and the TaskManager actor
+		try {
+			LOG.info("Starting TaskManager actor");
+			TaskExecutor taskExecutor = startTaskManagerComponentsAndActor(
+				configuration,
+				resourceID,
+				akkaRpcService,
+				taskManagerHostname,
+				haServices,
+				false);
+
+			taskExecutor.start();
+
+			// if desired, start the logging daemon that periodically logs the memory usage information
+			if (LOG.isInfoEnabled() && configuration.getBoolean(
+				ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD,
+				ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_START_LOG_THREAD)) {
+				LOG.info("Starting periodic memory usage logger");
+
+				long interval = configuration.getLong(
+					ConfigConstants.TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS,
+					ConfigConstants.DEFAULT_TASK_MANAGER_DEBUG_MEMORY_USAGE_LOG_INTERVAL_MS);
+
+				MemoryLogger logger = new MemoryLogger(LOG, interval, taskManagerSystem);
+				logger.start();
+			}
+
+			// block until everything is done
+			taskManagerSystem.awaitTermination();
+		} catch (Throwable t) {
+			LOG.error("Error while starting up taskManager", t);
+			try {
+				taskManagerSystem.shutdown();
+			} catch (Throwable tt) {
+				LOG.warn("Could not cleanly shut down actor system", tt);
+			}
+			throw t;
+		}
+	}
+
+	// --------------------------------------------------------------------------
+	//  Starting and running the TaskManager
+	// --------------------------------------------------------------------------
+
+	/**
+	 * @param configuration                 The configuration for the TaskManager.
+	 * @param resourceID                    The id of the resource which the task manager will run on.
+	 * @param rpcService                  The rpc service which is used to start and connect to the TaskManager RpcEndpoint .
+	 * @param taskManagerHostname       The hostname/address that describes the TaskManager's data location.
+	 * @param haServices        Optionally, a high availability service can be provided. If none is given,
+	 *                                      then a HighAvailabilityServices is constructed from the configuration.
+	 * @param localTaskManagerCommunication     If true, the TaskManager will not initiate the TCP network stack.
+	 * @return An ActorRef to the TaskManager actor.
+	 * @throws org.apache.flink.configuration.IllegalConfigurationException     Thrown, if the given config contains illegal values.
+	 * @throws java.io.IOException      Thrown, if any of the I/O components (such as buffer pools,
+	 *                                       I/O manager, ...) cannot be properly started.
+	 * @throws java.lang.Exception      Thrown is some other error occurs while parsing the configuration
+	 *                                      or starting the TaskManager components.
+	 */
+	public static TaskExecutor startTaskManagerComponentsAndActor(
+		Configuration configuration,
+		ResourceID resourceID,
+		RpcService rpcService,
+		String taskManagerHostname,
+		HighAvailabilityServices haServices,
+		boolean localTaskManagerCommunication) throws Exception {
+
+		final TaskExecutorConfiguration taskExecutorConfig = parseTaskManagerConfiguration(
+			configuration, taskManagerHostname, localTaskManagerCommunication);
+
+		MemoryType memType = taskExecutorConfig.getNetworkConfig().memoryType();
+
+		// pre-start checks
+		checkTempDirs(taskExecutorConfig.getTmpDirPaths());
+
+		ExecutionContext executionContext = ExecutionContexts$.MODULE$.fromExecutor(new ForkJoinPool());
+
+		// we start the network first, to make sure it can allocate its buffers first
+		final NetworkEnvironment network = new NetworkEnvironment(
+			executionContext,
+			taskExecutorConfig.getTimeout(),
+			taskExecutorConfig.getNetworkConfig(),
+			taskExecutorConfig.getConnectionInfo());
+
+		// computing the amount of memory to use depends on how much memory is available
+		// it strictly needs to happen AFTER the network stack has been initialized
+
+		// check if a value has been configured
+		long configuredMemory = configuration.getLong(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, -1L);
+		checkConfigParameter(configuredMemory == -1 || configuredMemory > 0, configuredMemory,
+			ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY,
+			"MemoryManager needs at least one MB of memory. " +
+				"If you leave this config parameter empty, the system automatically " +
+				"pick a fraction of the available memory.");
+
+		final long memorySize;
+		boolean preAllocateMemory = configuration.getBoolean(
+			ConfigConstants.TASK_MANAGER_MEMORY_PRE_ALLOCATE_KEY,
+			ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_PRE_ALLOCATE);
+		if (configuredMemory > 0) {
+			if (preAllocateMemory) {
+				LOG.info("Using {} MB for managed memory." , configuredMemory);
+			} else {
+				LOG.info("Limiting managed memory to {} MB, memory will be allocated lazily." , configuredMemory);
+			}
+			memorySize = configuredMemory << 20; // megabytes to bytes
+		} else {
+			float fraction = configuration.getFloat(
+				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
+				ConfigConstants.DEFAULT_MEMORY_MANAGER_MEMORY_FRACTION);
+			checkConfigParameter(fraction > 0.0f && fraction < 1.0f, fraction,
+				ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
+				"MemoryManager fraction of the free memory must be between 0.0 and 1.0");
+
+			if (memType == MemoryType.HEAP) {
+				long relativeMemSize = (long) (EnvironmentInformation.getSizeOfFreeHeapMemoryWithDefrag() * fraction);
+				if (preAllocateMemory) {
+					LOG.info("Using {} of the currently free heap space for managed heap memory ({} MB)." ,
+						fraction , relativeMemSize >> 20);
+				} else {
+					LOG.info("Limiting managed memory to {} of the currently free heap space ({} MB), " +
+						"memory will be allocated lazily." , fraction , relativeMemSize >> 20);
+				}
+				memorySize = relativeMemSize;
+			} else if (memType == MemoryType.OFF_HEAP) {
+				// The maximum heap memory has been adjusted according to the fraction
+				long maxMemory = EnvironmentInformation.getMaxJvmHeapMemory();
+				long directMemorySize = (long) (maxMemory / (1.0 - fraction) * fraction);
+				if (preAllocateMemory) {
+					LOG.info("Using {} of the maximum memory size for managed off-heap memory ({} MB)." ,
+						fraction, directMemorySize >> 20);
+				} else {
+					LOG.info("Limiting managed memory to {} of the maximum memory size ({} MB)," +
+						" memory will be allocated lazily.", fraction, directMemorySize >> 20);
+				}
+				memorySize = directMemorySize;
+			} else {
+				throw new RuntimeException("No supported memory type detected.");
+			}
+		}
+
+		// now start the memory manager
+		final MemoryManager memoryManager;
+		try {
+			memoryManager = new MemoryManager(
+				memorySize,
+				taskExecutorConfig.getNumberOfSlots(),
+				taskExecutorConfig.getNetworkConfig().networkBufferSize(),
+				memType,
+				preAllocateMemory);
+		} catch (OutOfMemoryError e) {
+			if (memType == MemoryType.HEAP) {
+				throw new Exception("OutOfMemory error (" + e.getMessage() +
+					") while allocating the TaskManager heap memory (" + memorySize + " bytes).", e);
+			} else if (memType == MemoryType.OFF_HEAP) {
+				throw new Exception("OutOfMemory error (" + e.getMessage() +
+					") while allocating the TaskManager off-heap memory (" + memorySize +
+					" bytes).Try increasing the maximum direct memory (-XX:MaxDirectMemorySize)", e);
+			} else {
+				throw e;
+			}
+		}
+
+		// start the I/O manager, it will create some temp directories.
+		final IOManager ioManager = new IOManagerAsync(taskExecutorConfig.getTmpDirPaths());
+
+		final TaskExecutor taskExecutor = new TaskExecutor(
+			taskExecutorConfig,
+			resourceID,
+			memoryManager,
+			ioManager,
+			network,
+			taskExecutorConfig.getNumberOfSlots(),
+			rpcService,
+			haServices);
+
+		return taskExecutor;
+	}
+
+	// --------------------------------------------------------------------------
+	//  Parsing and checking the TaskManager Configuration
+	// --------------------------------------------------------------------------
+
+	/**
+	 * Utility method to extract TaskManager config parameters from the configuration and to
+	 * sanity check them.
+	 *
+	 * @param configuration                 The configuration.
+	 * @param taskManagerHostname           The host name under which the TaskManager communicates.
+	 * @param localTaskManagerCommunication             True, to skip initializing the network stack.
+	 *                                      Use only in cases where only one task manager runs.
+	 * @return TaskExecutorConfiguration that wrappers InstanceConnectionInfo, NetworkEnvironmentConfiguration, etc.
+	 */
+	private static TaskExecutorConfiguration parseTaskManagerConfiguration(
+		Configuration configuration,
+		String taskManagerHostname,
+		boolean localTaskManagerCommunication) throws Exception {
+
+		// ------- read values from the config and check them ---------
+		//                      (a lot of them)
+
+		// ----> hosts / ports for communication and data exchange
+
+		int dataport = configuration.getInteger(ConfigConstants.TASK_MANAGER_DATA_PORT_KEY,
+			ConfigConstants.DEFAULT_TASK_MANAGER_DATA_PORT);
+		if (dataport == 0) {
+			dataport = NetUtils.getAvailablePort();
+		}
+		checkConfigParameter(dataport > 0, dataport, ConfigConstants.TASK_MANAGER_DATA_PORT_KEY,
+			"Leave config parameter empty or use 0 to let the system choose a port automatically.");
+
+		InetAddress taskManagerAddress = InetAddress.getByName(taskManagerHostname);
+		final InstanceConnectionInfo connectionInfo = new InstanceConnectionInfo(taskManagerAddress, dataport);
+
+		// ----> memory / network stack (shuffles/broadcasts), task slots, temp directories
+
+		// we need this because many configs have been written with a "-1" entry
+		int slots = configuration.getInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 1);
+		if (slots == -1) {
+			slots = 1;
+		}
+		checkConfigParameter(slots >= 1, slots, ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS,
+			"Number of task slots must be at least one.");
+
+		final int numNetworkBuffers = configuration.getInteger(
+			ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY,
+			ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_NUM_BUFFERS);
+		checkConfigParameter(numNetworkBuffers > 0, numNetworkBuffers,
+			ConfigConstants.TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY, "");
+
+		final int pageSize = configuration.getInteger(
+			ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
+			ConfigConstants.DEFAULT_TASK_MANAGER_MEMORY_SEGMENT_SIZE);
+		// check page size of for minimum size
+		checkConfigParameter(pageSize >= MemoryManager.MIN_PAGE_SIZE, pageSize,
+			ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
+			"Minimum memory segment size is " + MemoryManager.MIN_PAGE_SIZE);
+		// check page size for power of two
+		checkConfigParameter(MathUtils.isPowerOf2(pageSize), pageSize,
+			ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
+			"Memory segment size must be a power of 2.");
+
+		// check whether we use heap or off-heap memory
+		final MemoryType memType;
+		if (configuration.getBoolean(ConfigConstants.TASK_MANAGER_MEMORY_OFF_HEAP_KEY, false)) {
+			memType = MemoryType.OFF_HEAP;
+		} else {
+			memType = MemoryType.HEAP;
+		}
+
+		// initialize the memory segment factory accordingly
+		if (memType == MemoryType.HEAP) {
+			if (!MemorySegmentFactory.initializeIfNotInitialized(HeapMemorySegment.FACTORY)) {
+				throw new Exception("Memory type is set to heap memory, but memory segment " +
+					"factory has been initialized for off-heap memory segments");
+			}
+		} else {
+			if (!MemorySegmentFactory.initializeIfNotInitialized(HybridMemorySegment.FACTORY)) {
+				throw new Exception("Memory type is set to off-heap memory, but memory segment " +
+					"factory has been initialized for heap memory segments");
+			}
+		}
+
+		final String[] tmpDirs = configuration.getString(
+			ConfigConstants.TASK_MANAGER_TMP_DIR_KEY,
+			ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH).split(",|" + File.pathSeparator);
+
+		final NettyConfig nettyConfig;
+		if (!localTaskManagerCommunication) {
+			nettyConfig = new NettyConfig(connectionInfo.address(), connectionInfo.dataPort(), pageSize, slots, configuration);
+		} else {
+			nettyConfig = null;
+		}
+
+		// Default spill I/O mode for intermediate results
+		final String syncOrAsync = configuration.getString(
+			ConfigConstants.TASK_MANAGER_NETWORK_DEFAULT_IO_MODE,
+			ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_DEFAULT_IO_MODE);
+
+		final IOMode ioMode;
+		if (syncOrAsync.equals("async")) {
+			ioMode = IOManager.IOMode.ASYNC;
+		} else {
+			ioMode = IOManager.IOMode.SYNC;
+		}
+
+		final int queryServerPort =  configuration.getInteger(
+			ConfigConstants.QUERYABLE_STATE_SERVER_PORT,
+			ConfigConstants.DEFAULT_QUERYABLE_STATE_SERVER_PORT);
+
+		final int queryServerNetworkThreads =  configuration.getInteger(
+			ConfigConstants.QUERYABLE_STATE_SERVER_NETWORK_THREADS,
+			ConfigConstants.DEFAULT_QUERYABLE_STATE_SERVER_NETWORK_THREADS);
+
+		final int queryServerQueryThreads =  configuration.getInteger(
+			ConfigConstants.QUERYABLE_STATE_SERVER_QUERY_THREADS,
+			ConfigConstants.DEFAULT_QUERYABLE_STATE_SERVER_QUERY_THREADS);
+
+		final NetworkEnvironmentConfiguration networkConfig = new NetworkEnvironmentConfiguration(
+			numNetworkBuffers,
+			pageSize,
+			memType,
+			ioMode,
+			queryServerPort,
+			queryServerNetworkThreads,
+			queryServerQueryThreads,
+			localTaskManagerCommunication ? Option.<NettyConfig>empty() : new Some<>(nettyConfig),
+			new Tuple2<>(500, 3000));
+
+		// ----> timeouts, library caching, profiling
+
+		final FiniteDuration timeout;
+		try {
+			timeout = AkkaUtils.getTimeout(configuration);
+		} catch (Exception e) {
+			throw new IllegalArgumentException(
+				"Invalid format for '" + ConfigConstants.AKKA_ASK_TIMEOUT +
+					"'.Use formats like '50 s' or '1 min' to specify the timeout.");
+		}
+		LOG.info("Messages between TaskManager and JobManager have a max timeout of " + timeout);
+
+		final long cleanupInterval = configuration.getLong(
+			ConfigConstants.LIBRARY_CACHE_MANAGER_CLEANUP_INTERVAL,
+			ConfigConstants.DEFAULT_LIBRARY_CACHE_MANAGER_CLEANUP_INTERVAL) * 1000;
+
+		final FiniteDuration finiteRegistrationDuration;
+		try {
+			Duration maxRegistrationDuration = Duration.create(configuration.getString(
+				ConfigConstants.TASK_MANAGER_MAX_REGISTRATION_DURATION,
+				ConfigConstants.DEFAULT_TASK_MANAGER_MAX_REGISTRATION_DURATION));
+			if (maxRegistrationDuration.isFinite()) {
+				finiteRegistrationDuration = new FiniteDuration(maxRegistrationDuration.toSeconds(), TimeUnit.SECONDS);
+			} else {
+				finiteRegistrationDuration = null;
+			}
+		} catch (NumberFormatException e) {
+			throw new IllegalArgumentException("Invalid format for parameter " +
+				ConfigConstants.TASK_MANAGER_MAX_REGISTRATION_DURATION, e);
+		}
+
+		final FiniteDuration initialRegistrationPause;
+		try {
+			Duration pause = Duration.create(configuration.getString(
+				ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE,
+				ConfigConstants.DEFAULT_TASK_MANAGER_INITIAL_REGISTRATION_PAUSE));
+			if (pause.isFinite()) {
+				initialRegistrationPause = new FiniteDuration(pause.toSeconds(), TimeUnit.SECONDS);
+			} else {
+				throw new IllegalArgumentException("The initial registration pause must be finite: " + pause);
+			}
+		} catch (NumberFormatException e) {
+			throw new IllegalArgumentException("Invalid format for parameter " +
+				ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE, e);
+		}
+
+		final FiniteDuration maxRegistrationPause;
+		try {
+			Duration pause = Duration.create(configuration.getString(
+				ConfigConstants.TASK_MANAGER_MAX_REGISTARTION_PAUSE,
+				ConfigConstants.DEFAULT_TASK_MANAGER_MAX_REGISTRATION_PAUSE));
+			if (pause.isFinite()) {
+				maxRegistrationPause = new FiniteDuration(pause.toSeconds(), TimeUnit.SECONDS);
+			} else {
+				throw new IllegalArgumentException("The maximum registration pause must be finite: " + pause);
+			}
+		} catch (NumberFormatException e) {
+			throw new IllegalArgumentException("Invalid format for parameter " +
+				ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE, e);
+		}
+
+		final FiniteDuration refusedRegistrationPause;
+		try {
+			Duration pause = Duration.create(configuration.getString(
+				ConfigConstants.TASK_MANAGER_REFUSED_REGISTRATION_PAUSE,
+				ConfigConstants.DEFAULT_TASK_MANAGER_REFUSED_REGISTRATION_PAUSE));
+			if (pause.isFinite()) {
+				refusedRegistrationPause = new FiniteDuration(pause.toSeconds(), TimeUnit.SECONDS);
+			} else {
+				throw new IllegalArgumentException("The refused registration pause must be finite: " + pause);
+			}
+		} catch (NumberFormatException e) {
+			throw new IllegalArgumentException("Invalid format for parameter " +
+				ConfigConstants.TASK_MANAGER_INITIAL_REGISTRATION_PAUSE, e);
+		}
+
+		return new TaskExecutorConfiguration(
+			tmpDirs,
+			cleanupInterval,
+			connectionInfo,
+			networkConfig,
+			timeout,
+			finiteRegistrationDuration,
+			slots,
+			configuration,
+			initialRegistrationPause,
+			maxRegistrationPause,
+			refusedRegistrationPause);
+	}
+
+	/**
+	 * Validates a condition for a config parameter and displays a standard exception, if the
+	 * the condition does not hold.
+	 *
+	 * @param condition    The condition that must hold. If the condition is false, an exception is thrown.
+	 * @param parameter    The parameter value. Will be shown in the exception message.
+	 * @param name         The name of the config parameter. Will be shown in the exception message.
+	 * @param errorMessage The optional custom error message to append to the exception message.
+	 */
+	private static void checkConfigParameter(
+		boolean condition,
+		Object parameter,
+		String name,
+		String errorMessage) {
+		if (!condition) {
+			throw new IllegalConfigurationException("Invalid configuration value for " + name + " : " + parameter + " - " + errorMessage);
+		}
+	}
+
+	/**
+	 * Validates that all the directories denoted by the strings do actually exist, are proper
+	 * directories (not files), and are writable.
+	 *
+	 * @param tmpDirs The array of directory paths to check.
+	 * @throws Exception Thrown if any of the directories does not exist or is not writable
+	 *                   or is a file, rather than a directory.
+	 */
+	private static void checkTempDirs(String[] tmpDirs) throws IOException {
+		for (String dir : tmpDirs) {
+			if (dir != null && !dir.equals("")) {
+				File file = new File(dir);
+				if (!file.exists()) {
+					throw new IOException("Temporary file directory " + file.getAbsolutePath() + " does not exist.");
+				}
+				if (!file.isDirectory()) {
+					throw new IOException("Temporary file directory " + file.getAbsolutePath() + " is not a directory.");
+				}
+				if (!file.canWrite()) {
+					throw new IOException("Temporary file directory " + file.getAbsolutePath() + " is not writable.");
+				}
+
+				if (LOG.isInfoEnabled()) {
+					long totalSpaceGb = file.getTotalSpace() >> 30;
+					long usableSpaceGb = file.getUsableSpace() >> 30;
+					double usablePercentage = (double)usableSpaceGb / totalSpaceGb * 100;
+					String path = file.getAbsolutePath();
+					LOG.info(String.format("Temporary file directory '%s': total %d GB, " + "usable %d GB (%.2f%% usable)",
+						path, totalSpaceGb, usableSpaceGb, usablePercentage));
+				}
+			} else {
+				throw new IllegalArgumentException("Temporary file directory #$id is null.");
+			}
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  Properties
+	// ------------------------------------------------------------------------
+
+	public ResourceID getResourceID() {
+		return resourceID;
+	}
+
 	// ------------------------------------------------------------------------
-	//  Error handling
+	//  Error Handling
 	// ------------------------------------------------------------------------
 
 	/**
 	 * Notifies the TaskExecutor that a fatal error has occurred and it cannot proceed.
 	 * This method should be used when asynchronous threads want to notify the
 	 * TaskExecutor of a fatal error.
-	 * 
+	 *
 	 * @param t The exception describing the fatal error
 	 */
 	void onFatalErrorAsync(final Throwable t) {
@@ -141,7 +787,7 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 	/**
 	 * Notifies the TaskExecutor that a fatal error has occurred and it cannot proceed.
 	 * This method must only be called from within the TaskExecutor's main thread.
-	 * 
+	 *
 	 * @param t The exception describing the fatal error
 	 */
 	void onFatalError(Throwable t) {

http://git-wip-us.apache.org/repos/asf/flink/blob/73de8427/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorConfiguration.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorConfiguration.java
new file mode 100644
index 0000000..32484e1
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorConfiguration.java
@@ -0,0 +1,151 @@
+/*
+ * 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.rpc.taskexecutor;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.instance.InstanceConnectionInfo;
+import org.apache.flink.runtime.taskmanager.NetworkEnvironmentConfiguration;
+
+import scala.concurrent.duration.FiniteDuration;
+
+import java.io.Serializable;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * {@link TaskExecutor} Configuration
+ */
+public class TaskExecutorConfiguration implements Serializable {
+
+	private static final long serialVersionUID = 1L;
+
+	private final String[] tmpDirPaths;
+
+	private final long cleanupInterval;
+
+	private final int numberOfSlots;
+
+	private final Configuration configuration;
+
+	private final FiniteDuration timeout;
+	private final FiniteDuration maxRegistrationDuration;
+	private final FiniteDuration initialRegistrationPause;
+	private final FiniteDuration maxRegistrationPause;
+	private final FiniteDuration refusedRegistrationPause;
+
+	private final NetworkEnvironmentConfiguration networkConfig;
+
+	private final InstanceConnectionInfo connectionInfo;
+
+	public TaskExecutorConfiguration(
+			String[] tmpDirPaths,
+			long cleanupInterval,
+			InstanceConnectionInfo connectionInfo,
+			NetworkEnvironmentConfiguration networkConfig,
+			FiniteDuration timeout,
+			FiniteDuration maxRegistrationDuration,
+			int numberOfSlots,
+			Configuration configuration) {
+
+		this (tmpDirPaths,
+			cleanupInterval,
+			connectionInfo,
+			networkConfig,
+			timeout,
+			maxRegistrationDuration,
+			numberOfSlots,
+			configuration,
+			new FiniteDuration(500, TimeUnit.MILLISECONDS),
+			new FiniteDuration(30, TimeUnit.SECONDS),
+			new FiniteDuration(10, TimeUnit.SECONDS));
+	}
+
+	public TaskExecutorConfiguration(
+			String[] tmpDirPaths,
+			long cleanupInterval,
+			InstanceConnectionInfo connectionInfo,
+			NetworkEnvironmentConfiguration networkConfig,
+			FiniteDuration timeout,
+			FiniteDuration maxRegistrationDuration,
+			int numberOfSlots,
+			Configuration configuration,
+			FiniteDuration initialRegistrationPause,
+			FiniteDuration maxRegistrationPause,
+			FiniteDuration refusedRegistrationPause) {
+
+		this.tmpDirPaths = checkNotNull(tmpDirPaths);
+		this.cleanupInterval = checkNotNull(cleanupInterval);
+		this.connectionInfo = checkNotNull(connectionInfo);
+		this.networkConfig = checkNotNull(networkConfig);
+		this.timeout = checkNotNull(timeout);
+		this.maxRegistrationDuration = maxRegistrationDuration;
+		this.numberOfSlots = checkNotNull(numberOfSlots);
+		this.configuration = checkNotNull(configuration);
+		this.initialRegistrationPause = checkNotNull(initialRegistrationPause);
+		this.maxRegistrationPause = checkNotNull(maxRegistrationPause);
+		this.refusedRegistrationPause = checkNotNull(refusedRegistrationPause);
+	}
+
+	// --------------------------------------------------------------------------------------------
+	//  Properties
+	// --------------------------------------------------------------------------------------------
+
+	public String[] getTmpDirPaths() {
+		return tmpDirPaths;
+	}
+
+	public long getCleanupInterval() {
+		return cleanupInterval;
+	}
+
+	public InstanceConnectionInfo getConnectionInfo() { return connectionInfo; }
+
+	public NetworkEnvironmentConfiguration getNetworkConfig() { return networkConfig; }
+
+	public FiniteDuration getTimeout() {
+		return timeout;
+	}
+
+	public FiniteDuration getMaxRegistrationDuration() {
+		return maxRegistrationDuration;
+	}
+
+	public int getNumberOfSlots() {
+		return numberOfSlots;
+	}
+
+	public Configuration getConfiguration() {
+		return configuration;
+	}
+
+	public FiniteDuration getInitialRegistrationPause() {
+		return initialRegistrationPause;
+	}
+
+	public FiniteDuration getMaxRegistrationPause() {
+		return maxRegistrationPause;
+	}
+
+	public FiniteDuration getRefusedRegistrationPause() {
+		return refusedRegistrationPause;
+	}
+
+}
+

http://git-wip-us.apache.org/repos/asf/flink/blob/73de8427/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorTest.java
index b831ead..25a670c 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorTest.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.runtime.rpc.taskexecutor;
 
+import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.clusterframework.types.ResourceID;
 import org.apache.flink.runtime.highavailability.NonHaServices;
 import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices;
@@ -49,9 +50,9 @@ public class TaskExecutorTest extends TestLogger {
 			rpc.registerGateway(resourceManagerAddress, rmGateway);
 
 			NonHaServices haServices = new NonHaServices(resourceManagerAddress);
-			TaskExecutor taskManager = new TaskExecutor(rpc, haServices, resourceID);
+			TaskExecutor taskManager = TaskExecutor.startTaskManagerComponentsAndActor(
+				new Configuration(), resourceID, rpc, "localhost", haServices, true);
 			String taskManagerAddress = taskManager.getAddress();
-
 			taskManager.start();
 
 			verify(rmGateway, timeout(5000)).registerTaskExecutor(
@@ -84,7 +85,8 @@ public class TaskExecutorTest extends TestLogger {
 			TestingHighAvailabilityServices haServices = new TestingHighAvailabilityServices();
 			haServices.setResourceManagerLeaderRetriever(testLeaderService);
 
-			TaskExecutor taskManager = new TaskExecutor(rpc, haServices, resourceID);
+			TaskExecutor taskManager = TaskExecutor.startTaskManagerComponentsAndActor(
+				new Configuration(), resourceID, rpc, "localhost", haServices, true);
 			String taskManagerAddress = taskManager.getAddress();
 			taskManager.start();
 


[50/50] [abbrv] flink git commit: [FLINK-4528] [rpc] Marks main thread execution methods in RpcEndpoint as protected

Posted by tr...@apache.org.
[FLINK-4528] [rpc] Marks main thread execution methods in RpcEndpoint as protected

Give main thread execution context into the TaskExecutorToResourceManagerConnection


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

Branch: refs/heads/flip-6
Commit: 002f65060dfb2d9121c0d78fb1d33dc152f99a26
Parents: 72468d1
Author: Till Rohrmann <tr...@apache.org>
Authored: Mon Aug 29 15:49:59 2016 +0200
Committer: Till Rohrmann <tr...@apache.org>
Committed: Thu Sep 8 17:27:01 2016 +0200

----------------------------------------------------------------------
 .../apache/flink/runtime/rpc/RpcEndpoint.java   |   8 +-
 .../runtime/taskexecutor/TaskExecutor.java      |   7 +-
 ...TaskExecutorToResourceManagerConnection.java |  26 ++-
 .../flink/runtime/rpc/AsyncCallsTest.java       | 216 ++++++++++++++++++
 .../flink/runtime/rpc/akka/AsyncCallsTest.java  | 219 -------------------
 5 files changed, 242 insertions(+), 234 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/002f6506/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java
index 7b3f8a1..e9e2b2c 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java
@@ -161,7 +161,7 @@ public abstract class RpcEndpoint<C extends RpcGateway> {
 	 *
 	 * @return Main thread execution context
 	 */
-	public ExecutionContext getMainThreadExecutionContext() {
+	protected ExecutionContext getMainThreadExecutionContext() {
 		return mainThreadExecutionContext;
 	}
 
@@ -184,7 +184,7 @@ public abstract class RpcEndpoint<C extends RpcGateway> {
 	 *
 	 * @param runnable Runnable to be executed in the main thread of the underlying RPC endpoint
 	 */
-	public void runAsync(Runnable runnable) {
+	protected void runAsync(Runnable runnable) {
 		((MainThreadExecutor) self).runAsync(runnable);
 	}
 
@@ -195,7 +195,7 @@ public abstract class RpcEndpoint<C extends RpcGateway> {
 	 * @param runnable Runnable to be executed
 	 * @param delay    The delay after which the runnable will be executed
 	 */
-	public void scheduleRunAsync(Runnable runnable, long delay, TimeUnit unit) {
+	protected void scheduleRunAsync(Runnable runnable, long delay, TimeUnit unit) {
 		((MainThreadExecutor) self).scheduleRunAsync(runnable, unit.toMillis(delay));
 	}
 
@@ -209,7 +209,7 @@ public abstract class RpcEndpoint<C extends RpcGateway> {
 	 * @param <V> Return type of the callable
 	 * @return Future for the result of the callable.
 	 */
-	public <V> Future<V> callAsync(Callable<V> callable, Timeout timeout) {
+	protected <V> Future<V> callAsync(Callable<V> callable, Timeout timeout) {
 		return ((MainThreadExecutor) self).callAsync(callable, timeout);
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/002f6506/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
index 4871b96..735730b 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java
@@ -176,7 +176,12 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 		if (newLeaderAddress != null) {
 			log.info("Attempting to register at ResourceManager {}", newLeaderAddress);
 			resourceManagerConnection =
-				new TaskExecutorToResourceManagerConnection(log, this, newLeaderAddress, newLeaderId);
+				new TaskExecutorToResourceManagerConnection(
+					log,
+					this,
+					newLeaderAddress,
+					newLeaderId,
+					getMainThreadExecutionContext());
 			resourceManagerConnection.start();
 		}
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/002f6506/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorToResourceManagerConnection.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorToResourceManagerConnection.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorToResourceManagerConnection.java
index 25332a0..28062b6 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorToResourceManagerConnection.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutorToResourceManagerConnection.java
@@ -31,6 +31,7 @@ import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway;
 
 import org.slf4j.Logger;
 
+import scala.concurrent.ExecutionContext;
 import scala.concurrent.Future;
 import scala.concurrent.duration.FiniteDuration;
 
@@ -55,9 +56,12 @@ public class TaskExecutorToResourceManagerConnection {
 
 	private final String resourceManagerAddress;
 
+	/** Execution context to be used to execute the on complete action of the ResourceManagerRegistration */
+	private final ExecutionContext executionContext;
+
 	private TaskExecutorToResourceManagerConnection.ResourceManagerRegistration pendingRegistration;
 
-	private ResourceManagerGateway registeredResourceManager;
+	private volatile ResourceManagerGateway registeredResourceManager;
 
 	private InstanceID registrationId;
 
@@ -66,15 +70,17 @@ public class TaskExecutorToResourceManagerConnection {
 
 
 	public TaskExecutorToResourceManagerConnection(
-			Logger log,
-			TaskExecutor taskExecutor,
-			String resourceManagerAddress,
-			UUID resourceManagerLeaderId) {
+		Logger log,
+		TaskExecutor taskExecutor,
+		String resourceManagerAddress,
+		UUID resourceManagerLeaderId,
+		ExecutionContext executionContext) {
 
 		this.log = checkNotNull(log);
 		this.taskExecutor = checkNotNull(taskExecutor);
 		this.resourceManagerAddress = checkNotNull(resourceManagerAddress);
 		this.resourceManagerLeaderId = checkNotNull(resourceManagerLeaderId);
+		this.executionContext = checkNotNull(executionContext);
 	}
 
 	// ------------------------------------------------------------------------
@@ -93,22 +99,22 @@ public class TaskExecutorToResourceManagerConnection {
 		pendingRegistration.startRegistration();
 
 		Future<Tuple2<ResourceManagerGateway, TaskExecutorRegistrationSuccess>> future = pendingRegistration.getFuture();
-		
+
 		future.onSuccess(new OnSuccess<Tuple2<ResourceManagerGateway, TaskExecutorRegistrationSuccess>>() {
 			@Override
 			public void onSuccess(Tuple2<ResourceManagerGateway, TaskExecutorRegistrationSuccess> result) {
-				registeredResourceManager = result.f0;
 				registrationId = result.f1.getRegistrationId();
+				registeredResourceManager = result.f0;
 			}
-		}, taskExecutor.getMainThreadExecutionContext());
+		}, executionContext);
 		
 		// this future should only ever fail if there is a bug, not if the registration is declined
 		future.onFailure(new OnFailure() {
 			@Override
 			public void onFailure(Throwable failure) {
-				taskExecutor.onFatalError(failure);
+				taskExecutor.onFatalErrorAsync(failure);
 			}
-		}, taskExecutor.getMainThreadExecutionContext());
+		}, executionContext);
 	}
 
 	public void close() {

http://git-wip-us.apache.org/repos/asf/flink/blob/002f6506/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/AsyncCallsTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/AsyncCallsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/AsyncCallsTest.java
new file mode 100644
index 0000000..1791056
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/AsyncCallsTest.java
@@ -0,0 +1,216 @@
+/*
+ * 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.rpc;
+
+import akka.actor.ActorSystem;
+import akka.util.Timeout;
+
+import org.apache.flink.core.testutils.OneShotLatch;
+import org.apache.flink.runtime.akka.AkkaUtils;
+
+import org.apache.flink.runtime.rpc.akka.AkkaRpcService;
+import org.apache.flink.util.TestLogger;
+import org.junit.AfterClass;
+import org.junit.Test;
+
+import scala.concurrent.Await;
+import scala.concurrent.Future;
+import scala.concurrent.duration.FiniteDuration;
+
+import java.util.concurrent.Callable;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.ReentrantLock;
+
+import static org.junit.Assert.*;
+
+public class AsyncCallsTest extends TestLogger {
+
+	// ------------------------------------------------------------------------
+	//  shared test members
+	// ------------------------------------------------------------------------
+
+	private static ActorSystem actorSystem = AkkaUtils.createDefaultActorSystem();
+
+	private static AkkaRpcService akkaRpcService =
+			new AkkaRpcService(actorSystem, new Timeout(10000, TimeUnit.MILLISECONDS));
+
+	@AfterClass
+	public static void shutdown() {
+		akkaRpcService.stopService();
+		actorSystem.shutdown();
+	}
+
+
+	// ------------------------------------------------------------------------
+	//  tests
+	// ------------------------------------------------------------------------
+
+	@Test
+	public void testScheduleWithNoDelay() throws Exception {
+
+		// to collect all the thread references
+		final ReentrantLock lock = new ReentrantLock();
+		final AtomicBoolean concurrentAccess = new AtomicBoolean(false);
+
+		TestEndpoint testEndpoint = new TestEndpoint(akkaRpcService, lock);
+		testEndpoint.start();
+		TestGateway gateway = testEndpoint.getSelf();
+
+		// a bunch of gateway calls
+		gateway.someCall();
+		gateway.anotherCall();
+		gateway.someCall();
+
+		// run something asynchronously
+		for (int i = 0; i < 10000; i++) {
+			testEndpoint.runAsync(new Runnable() {
+				@Override
+				public void run() {
+					boolean holdsLock = lock.tryLock();
+					if (holdsLock) {
+						lock.unlock();
+					} else {
+						concurrentAccess.set(true);
+					}
+				}
+			});
+		}
+	
+		Future<String> result = testEndpoint.callAsync(new Callable<String>() {
+			@Override
+			public String call() throws Exception {
+				boolean holdsLock = lock.tryLock();
+				if (holdsLock) {
+					lock.unlock();
+				} else {
+					concurrentAccess.set(true);
+				}
+				return "test";
+			}
+		}, new Timeout(30, TimeUnit.SECONDS));
+		String str = Await.result(result, new FiniteDuration(30, TimeUnit.SECONDS));
+		assertEquals("test", str);
+
+		// validate that no concurrent access happened
+		assertFalse("Rpc Endpoint had concurrent access", testEndpoint.hasConcurrentAccess());
+		assertFalse("Rpc Endpoint had concurrent access", concurrentAccess.get());
+
+		akkaRpcService.stopServer(testEndpoint.getSelf());
+	}
+
+	@Test
+	public void testScheduleWithDelay() throws Exception {
+
+		// to collect all the thread references
+		final ReentrantLock lock = new ReentrantLock();
+		final AtomicBoolean concurrentAccess = new AtomicBoolean(false);
+		final OneShotLatch latch = new OneShotLatch();
+
+		final long delay = 200;
+
+		TestEndpoint testEndpoint = new TestEndpoint(akkaRpcService, lock);
+		testEndpoint.start();
+
+		// run something asynchronously
+		testEndpoint.runAsync(new Runnable() {
+			@Override
+			public void run() {
+				boolean holdsLock = lock.tryLock();
+				if (holdsLock) {
+					lock.unlock();
+				} else {
+					concurrentAccess.set(true);
+				}
+			}
+		});
+
+		final long start = System.nanoTime();
+
+		testEndpoint.scheduleRunAsync(new Runnable() {
+			@Override
+			public void run() {
+				boolean holdsLock = lock.tryLock();
+				if (holdsLock) {
+					lock.unlock();
+				} else {
+					concurrentAccess.set(true);
+				}
+				latch.trigger();
+			}
+		}, delay, TimeUnit.MILLISECONDS);
+
+		latch.await();
+		final long stop = System.nanoTime();
+
+		// validate that no concurrent access happened
+		assertFalse("Rpc Endpoint had concurrent access", testEndpoint.hasConcurrentAccess());
+		assertFalse("Rpc Endpoint had concurrent access", concurrentAccess.get());
+
+		assertTrue("call was not properly delayed", ((stop - start) / 1000000) >= delay);
+	}
+
+	// ------------------------------------------------------------------------
+	//  test RPC endpoint
+	// ------------------------------------------------------------------------
+	
+	public interface TestGateway extends RpcGateway {
+
+		void someCall();
+
+		void anotherCall();
+	}
+
+	@SuppressWarnings("unused")
+	public static class TestEndpoint extends RpcEndpoint<TestGateway> {
+
+		private final ReentrantLock lock;
+
+		private volatile boolean concurrentAccess;
+
+		public TestEndpoint(RpcService rpcService, ReentrantLock lock) {
+			super(rpcService);
+			this.lock = lock;
+		}
+
+		@RpcMethod
+		public void someCall() {
+			boolean holdsLock = lock.tryLock();
+			if (holdsLock) {
+				lock.unlock();
+			} else {
+				concurrentAccess = true;
+			}
+		}
+
+		@RpcMethod
+		public void anotherCall() {
+			boolean holdsLock = lock.tryLock();
+			if (holdsLock) {
+				lock.unlock();
+			} else {
+				concurrentAccess = true;
+			}
+		}
+
+		public boolean hasConcurrentAccess() {
+			return concurrentAccess;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/002f6506/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AsyncCallsTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AsyncCallsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AsyncCallsTest.java
deleted file mode 100644
index d33987c..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AsyncCallsTest.java
+++ /dev/null
@@ -1,219 +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.rpc.akka;
-
-import akka.actor.ActorSystem;
-import akka.util.Timeout;
-
-import org.apache.flink.core.testutils.OneShotLatch;
-import org.apache.flink.runtime.akka.AkkaUtils;
-import org.apache.flink.runtime.rpc.RpcEndpoint;
-import org.apache.flink.runtime.rpc.RpcGateway;
-import org.apache.flink.runtime.rpc.RpcMethod;
-import org.apache.flink.runtime.rpc.RpcService;
-
-import org.apache.flink.util.TestLogger;
-import org.junit.AfterClass;
-import org.junit.Test;
-
-import scala.concurrent.Await;
-import scala.concurrent.Future;
-import scala.concurrent.duration.FiniteDuration;
-
-import java.util.concurrent.Callable;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.locks.ReentrantLock;
-
-import static org.junit.Assert.*;
-
-public class AsyncCallsTest extends TestLogger {
-
-	// ------------------------------------------------------------------------
-	//  shared test members
-	// ------------------------------------------------------------------------
-
-	private static ActorSystem actorSystem = AkkaUtils.createDefaultActorSystem();
-
-	private static AkkaRpcService akkaRpcService = 
-			new AkkaRpcService(actorSystem, new Timeout(10000, TimeUnit.MILLISECONDS));
-
-	@AfterClass
-	public static void shutdown() {
-		akkaRpcService.stopService();
-		actorSystem.shutdown();
-	}
-
-
-	// ------------------------------------------------------------------------
-	//  tests
-	// ------------------------------------------------------------------------
-
-	@Test
-	public void testScheduleWithNoDelay() throws Exception {
-
-		// to collect all the thread references
-		final ReentrantLock lock = new ReentrantLock();
-		final AtomicBoolean concurrentAccess = new AtomicBoolean(false);
-
-		TestEndpoint testEndpoint = new TestEndpoint(akkaRpcService, lock);
-		testEndpoint.start();
-		TestGateway gateway = testEndpoint.getSelf();
-
-		// a bunch of gateway calls
-		gateway.someCall();
-		gateway.anotherCall();
-		gateway.someCall();
-
-		// run something asynchronously
-		for (int i = 0; i < 10000; i++) {
-			testEndpoint.runAsync(new Runnable() {
-				@Override
-				public void run() {
-					boolean holdsLock = lock.tryLock();
-					if (holdsLock) {
-						lock.unlock();
-					} else {
-						concurrentAccess.set(true);
-					}
-				}
-			});
-		}
-	
-		Future<String> result = testEndpoint.callAsync(new Callable<String>() {
-			@Override
-			public String call() throws Exception {
-				boolean holdsLock = lock.tryLock();
-				if (holdsLock) {
-					lock.unlock();
-				} else {
-					concurrentAccess.set(true);
-				}
-				return "test";
-			}
-		}, new Timeout(30, TimeUnit.SECONDS));
-		String str = Await.result(result, new FiniteDuration(30, TimeUnit.SECONDS));
-		assertEquals("test", str);
-
-		// validate that no concurrent access happened
-		assertFalse("Rpc Endpoint had concurrent access", testEndpoint.hasConcurrentAccess());
-		assertFalse("Rpc Endpoint had concurrent access", concurrentAccess.get());
-
-		akkaRpcService.stopServer(testEndpoint.getSelf());
-	}
-
-	@Test
-	public void testScheduleWithDelay() throws Exception {
-
-		// to collect all the thread references
-		final ReentrantLock lock = new ReentrantLock();
-		final AtomicBoolean concurrentAccess = new AtomicBoolean(false);
-		final OneShotLatch latch = new OneShotLatch();
-
-		final long delay = 200;
-
-		TestEndpoint testEndpoint = new TestEndpoint(akkaRpcService, lock);
-		testEndpoint.start();
-
-		// run something asynchronously
-		testEndpoint.runAsync(new Runnable() {
-			@Override
-			public void run() {
-				boolean holdsLock = lock.tryLock();
-				if (holdsLock) {
-					lock.unlock();
-				} else {
-					concurrentAccess.set(true);
-				}
-			}
-		});
-
-		final long start = System.nanoTime();
-
-		testEndpoint.scheduleRunAsync(new Runnable() {
-			@Override
-			public void run() {
-				boolean holdsLock = lock.tryLock();
-				if (holdsLock) {
-					lock.unlock();
-				} else {
-					concurrentAccess.set(true);
-				}
-				latch.trigger();
-			}
-		}, delay, TimeUnit.MILLISECONDS);
-
-		latch.await();
-		final long stop = System.nanoTime();
-
-		// validate that no concurrent access happened
-		assertFalse("Rpc Endpoint had concurrent access", testEndpoint.hasConcurrentAccess());
-		assertFalse("Rpc Endpoint had concurrent access", concurrentAccess.get());
-
-		assertTrue("call was not properly delayed", ((stop - start) / 1000000) >= delay);
-	}
-
-	// ------------------------------------------------------------------------
-	//  test RPC endpoint
-	// ------------------------------------------------------------------------
-	
-	interface TestGateway extends RpcGateway {
-
-		void someCall();
-
-		void anotherCall();
-	}
-
-	@SuppressWarnings("unused")
-	public static class TestEndpoint extends RpcEndpoint<TestGateway> {
-
-		private final ReentrantLock lock;
-
-		private volatile boolean concurrentAccess;
-
-		public TestEndpoint(RpcService rpcService, ReentrantLock lock) {
-			super(rpcService);
-			this.lock = lock;
-		}
-
-		@RpcMethod
-		public void someCall() {
-			boolean holdsLock = lock.tryLock();
-			if (holdsLock) {
-				lock.unlock();
-			} else {
-				concurrentAccess = true;
-			}
-		}
-
-		@RpcMethod
-		public void anotherCall() {
-			boolean holdsLock = lock.tryLock();
-			if (holdsLock) {
-				lock.unlock();
-			} else {
-				concurrentAccess = true;
-			}
-		}
-
-		public boolean hasConcurrentAccess() {
-			return concurrentAccess;
-		}
-	}
-}


[36/50] [abbrv] flink git commit: [FLINK-4400] [cluster mngmt] Implement leadership election among JobMasters

Posted by tr...@apache.org.
[FLINK-4400] [cluster mngmt] Implement leadership election among JobMasters

Adapt related components to the changes in HighAvailabilityServices

Add comments for getJobMasterElectionService in HighAvailabilityServices

This closes #2377.


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

Branch: refs/heads/flip-6
Commit: 81a35c182fff7dc666f8d88c8c58ba9a94038e05
Parents: 223eb6f
Author: xiaogang.sxg <xi...@alibaba-inc.com>
Authored: Wed Aug 17 13:46:00 2016 +0800
Committer: Till Rohrmann <tr...@apache.org>
Committed: Thu Sep 8 17:26:59 2016 +0200

----------------------------------------------------------------------
 .../HighAvailabilityServices.java               |   9 +
 .../runtime/highavailability/NonHaServices.java |   8 +
 .../flink/runtime/rpc/jobmaster/JobMaster.java  | 318 +++++++++----------
 .../runtime/rpc/akka/AkkaRpcServiceTest.java    |  53 +---
 4 files changed, 179 insertions(+), 209 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/81a35c18/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServices.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServices.java
index 094d36f..73e4f1f 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServices.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/HighAvailabilityServices.java
@@ -18,6 +18,8 @@
 
 package org.apache.flink.runtime.highavailability;
 
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
 
 /**
@@ -36,4 +38,11 @@ public interface HighAvailabilityServices {
 	 * Gets the leader retriever for the cluster's resource manager.
 	 */
 	LeaderRetrievalService getResourceManagerLeaderRetriever() throws Exception;
+
+	/**
+	 * Gets the leader election service for the given job.
+	 *
+	 * @param jobID The identifier of the job running the election.
+	 */
+	LeaderElectionService getJobMasterLeaderElectionService(JobID jobID) throws Exception;
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/81a35c18/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/NonHaServices.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/NonHaServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/NonHaServices.java
index b8c2ed8..3d2769b 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/NonHaServices.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/NonHaServices.java
@@ -18,6 +18,9 @@
 
 package org.apache.flink.runtime.highavailability;
 
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
+import org.apache.flink.runtime.leaderelection.StandaloneLeaderElectionService;
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
 import org.apache.flink.runtime.leaderretrieval.StandaloneLeaderRetrievalService;
 
@@ -56,4 +59,9 @@ public class NonHaServices implements HighAvailabilityServices {
 	public LeaderRetrievalService getResourceManagerLeaderRetriever() throws Exception {
 		return new StandaloneLeaderRetrievalService(resourceManagerAddress, new UUID(0, 0));
 	}
+
+	@Override
+	public LeaderElectionService getJobMasterLeaderElectionService(JobID jobID) throws Exception {
+		return new StandaloneLeaderElectionService();
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/81a35c18/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/jobmaster/JobMaster.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/jobmaster/JobMaster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/jobmaster/JobMaster.java
index e53cd68..49b200b 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/jobmaster/JobMaster.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/jobmaster/JobMaster.java
@@ -18,68 +18,77 @@
 
 package org.apache.flink.runtime.rpc.jobmaster;
 
-import akka.dispatch.Futures;
-import akka.dispatch.Mapper;
-import akka.dispatch.OnComplete;
-import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobmanager.RecoveryMode;
+import org.apache.flink.runtime.leaderelection.LeaderContender;
+import org.apache.flink.runtime.leaderelection.LeaderElectionService;
 import org.apache.flink.runtime.messages.Acknowledge;
 import org.apache.flink.runtime.rpc.RpcMethod;
-import org.apache.flink.runtime.rpc.resourcemanager.JobMasterRegistration;
-import org.apache.flink.runtime.rpc.resourcemanager.RegistrationResponse;
 import org.apache.flink.runtime.rpc.resourcemanager.ResourceManagerGateway;
 import org.apache.flink.runtime.rpc.RpcEndpoint;
 import org.apache.flink.runtime.rpc.RpcService;
 import org.apache.flink.runtime.taskmanager.TaskExecutionState;
 import org.apache.flink.util.Preconditions;
-import scala.Tuple2;
-import scala.concurrent.ExecutionContext;
-import scala.concurrent.ExecutionContext$;
-import scala.concurrent.Future;
-import scala.concurrent.duration.Deadline;
-import scala.concurrent.duration.FiniteDuration;
 
 import java.util.UUID;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.ScheduledThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
 
 /**
  * JobMaster implementation. The job master is responsible for the execution of a single
  * {@link org.apache.flink.runtime.jobgraph.JobGraph}.
- *
+ * <p>
  * It offers the following methods as part of its rpc interface to interact with the JobMaster
  * remotely:
  * <ul>
- *     <li>{@link #registerAtResourceManager(String)} triggers the registration at the resource manager</li>
  *     <li>{@link #updateTaskExecutionState(TaskExecutionState)} updates the task execution state for
  * given task</li>
  * </ul>
  */
 public class JobMaster extends RpcEndpoint<JobMasterGateway> {
-	/** Execution context for future callbacks */
-	private final ExecutionContext executionContext;
-
-	/** Execution context for scheduled runnables */
-	private final ScheduledExecutorService scheduledExecutorService;
-
-	private final FiniteDuration initialRegistrationTimeout = new FiniteDuration(500, TimeUnit.MILLISECONDS);
-	private final FiniteDuration maxRegistrationTimeout = new FiniteDuration(30, TimeUnit.SECONDS);
-	private final FiniteDuration registrationDuration = new FiniteDuration(365, TimeUnit.DAYS);
-	private final long failedRegistrationDelay = 10000;
 
 	/** Gateway to connected resource manager, null iff not connected */
 	private ResourceManagerGateway resourceManager = null;
 
-	/** UUID to filter out old registration runs */
-	private UUID currentRegistrationRun;
+	/** Logical representation of the job */
+	private final JobGraph jobGraph;
+	private final JobID jobID;
+
+	/** Configuration of the job */
+	private final Configuration configuration;
+	private final RecoveryMode recoveryMode;
+
+	/** Service to contend for and retrieve the leadership of JM and RM */
+	private final HighAvailabilityServices highAvailabilityServices;
+
+	/** Leader Management */
+	private LeaderElectionService leaderElectionService = null;
+	private UUID leaderSessionID;
+
+	/**
+	 * The JM's Constructor
+	 *
+	 * @param jobGraph The representation of the job's execution plan
+	 * @param configuration The job's configuration
+	 * @param rpcService The RPC service at which the JM serves
+	 * @param highAvailabilityService The cluster's HA service from the JM can elect and retrieve leaders.
+	 */
+	public JobMaster(
+		JobGraph jobGraph,
+		Configuration configuration,
+		RpcService rpcService,
+		HighAvailabilityServices highAvailabilityService) {
 
-	public JobMaster(RpcService rpcService, ExecutorService executorService) {
 		super(rpcService);
-		executionContext = ExecutionContext$.MODULE$.fromExecutor(
-			Preconditions.checkNotNull(executorService));
-		scheduledExecutorService = new ScheduledThreadPoolExecutor(1);
+
+		this.jobGraph = Preconditions.checkNotNull(jobGraph);
+		this.jobID = Preconditions.checkNotNull(jobGraph.getJobID());
+
+		this.configuration = Preconditions.checkNotNull(configuration);
+		this.recoveryMode = RecoveryMode.fromConfig(configuration);
+
+		this.highAvailabilityServices = Preconditions.checkNotNull(highAvailabilityService);
 	}
 
 	public ResourceManagerGateway getResourceManager() {
@@ -87,6 +96,91 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 	}
 
 	//----------------------------------------------------------------------------------------------
+	// Initialization methods
+	//----------------------------------------------------------------------------------------------
+	public void start() {
+		super.start();
+
+		// register at the election once the JM starts
+		registerAtElectionService();
+	}
+
+
+	//----------------------------------------------------------------------------------------------
+	// JobMaster Leadership methods
+	//----------------------------------------------------------------------------------------------
+
+	/**
+	 * Retrieves the election service and contend for the leadership.
+	 */
+	private void registerAtElectionService() {
+		try {
+			leaderElectionService = highAvailabilityServices.getJobMasterLeaderElectionService(jobID);
+			leaderElectionService.start(new JobMasterLeaderContender());
+		} catch (Exception e) {
+			throw new RuntimeException("Fail to register at the election of JobMaster", e);
+		}
+	}
+
+	/**
+	 * Start the execution when the leadership is granted.
+	 *
+	 * @param newLeaderSessionID The identifier of the new leadership session
+	 */
+	public void grantJobMasterLeadership(final UUID newLeaderSessionID) {
+		runAsync(new Runnable() {
+			@Override
+			public void run() {
+				log.info("JobManager {} grants leadership with session id {}.", getAddress(), newLeaderSessionID);
+
+				// The operation may be blocking, but since JM is idle before it grants the leadership, it's okay that
+				// JM waits here for the operation's completeness.
+				leaderSessionID = newLeaderSessionID;
+				leaderElectionService.confirmLeaderSessionID(newLeaderSessionID);
+
+				// TODO:: execute the job when the leadership is granted.
+			}
+		});
+	}
+
+	/**
+	 * Stop the execution when the leadership is revoked.
+	 */
+	public void revokeJobMasterLeadership() {
+		runAsync(new Runnable() {
+			@Override
+			public void run() {
+				log.info("JobManager {} was revoked leadership.", getAddress());
+
+				// TODO:: cancel the job's execution and notify all listeners
+				cancelAndClearEverything(new Exception("JobManager is no longer the leader."));
+
+				leaderSessionID = null;
+			}
+		});
+	}
+
+	/**
+	 * Handles error occurring in the leader election service
+	 *
+	 * @param exception Exception thrown in the leader election service
+	 */
+	public void onJobMasterElectionError(final Exception exception) {
+		runAsync(new Runnable() {
+			@Override
+			public void run() {
+				log.error("Received an error from the LeaderElectionService.", exception);
+
+				// TODO:: cancel the job's execution and shutdown the JM
+				cancelAndClearEverything(exception);
+
+				leaderSessionID = null;
+			}
+		});
+
+	}
+
+	//----------------------------------------------------------------------------------------------
 	// RPC methods
 	//----------------------------------------------------------------------------------------------
 
@@ -109,18 +203,7 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 	 */
 	@RpcMethod
 	public void registerAtResourceManager(final String address) {
-		currentRegistrationRun = UUID.randomUUID();
-
-		Future<ResourceManagerGateway> resourceManagerFuture = getRpcService().connect(address, ResourceManagerGateway.class);
-
-		handleResourceManagerRegistration(
-			new JobMasterRegistration(getAddress()),
-			1,
-			resourceManagerFuture,
-			currentRegistrationRun,
-			initialRegistrationTimeout,
-			maxRegistrationTimeout,
-			registrationDuration.fromNow());
+		//TODO:: register at the RM
 	}
 
 	//----------------------------------------------------------------------------------------------
@@ -128,124 +211,37 @@ public class JobMaster extends RpcEndpoint<JobMasterGateway> {
 	//----------------------------------------------------------------------------------------------
 
 	/**
-	 * Helper method to handle the resource manager registration process. If a registration attempt
-	 * times out, then a new attempt with the doubled time out is initiated. The whole registration
-	 * process has a deadline. Once this deadline is overdue without successful registration, the
-	 * job master shuts down.
+	 * Cancel the current job and notify all listeners the job's cancellation.
 	 *
-	 * @param jobMasterRegistration Job master registration info which is sent to the resource
-	 *                              manager
-	 * @param attemptNumber Registration attempt number
-	 * @param resourceManagerFuture Future of the resource manager gateway
-	 * @param registrationRun UUID describing the current registration run
-	 * @param timeout Timeout of the last registration attempt
-	 * @param maxTimeout Maximum timeout between registration attempts
-	 * @param deadline Deadline for the registration
+	 * @param cause Cause for the cancelling.
 	 */
-	void handleResourceManagerRegistration(
-		final JobMasterRegistration jobMasterRegistration,
-		final int attemptNumber,
-		final Future<ResourceManagerGateway> resourceManagerFuture,
-		final UUID registrationRun,
-		final FiniteDuration timeout,
-		final FiniteDuration maxTimeout,
-		final Deadline deadline) {
-
-		// filter out concurrent registration runs
-		if (registrationRun.equals(currentRegistrationRun)) {
-
-			log.info("Start registration attempt #{}.", attemptNumber);
-
-			if (deadline.isOverdue()) {
-				// we've exceeded our registration deadline. This means that we have to shutdown the JobMaster
-				log.error("Exceeded registration deadline without successfully registering at the ResourceManager.");
-				shutDown();
-			} else {
-				Future<Tuple2<RegistrationResponse, ResourceManagerGateway>> registrationResponseFuture = resourceManagerFuture.flatMap(new Mapper<ResourceManagerGateway, Future<Tuple2<RegistrationResponse, ResourceManagerGateway>>>() {
-					@Override
-					public Future<Tuple2<RegistrationResponse, ResourceManagerGateway>> apply(ResourceManagerGateway resourceManagerGateway) {
-						return resourceManagerGateway.registerJobMaster(jobMasterRegistration, timeout).zip(Futures.successful(resourceManagerGateway));
-					}
-				}, executionContext);
-
-				registrationResponseFuture.onComplete(new OnComplete<Tuple2<RegistrationResponse, ResourceManagerGateway>>() {
-					@Override
-					public void onComplete(Throwable failure, Tuple2<RegistrationResponse, ResourceManagerGateway> tuple) throws Throwable {
-						if (failure != null) {
-							if (failure instanceof TimeoutException) {
-								// we haven't received an answer in the given timeout interval,
-								// so increase it and try again.
-								final FiniteDuration newTimeout = timeout.$times(2L).min(maxTimeout);
-
-								handleResourceManagerRegistration(
-									jobMasterRegistration,
-									attemptNumber + 1,
-									resourceManagerFuture,
-									registrationRun,
-									newTimeout,
-									maxTimeout,
-									deadline);
-							} else {
-								log.error("Received unknown error while registering at the ResourceManager.", failure);
-								shutDown();
-							}
-						} else {
-							final RegistrationResponse response = tuple._1();
-							final ResourceManagerGateway gateway = tuple._2();
-
-							if (response.isSuccess()) {
-								finishResourceManagerRegistration(gateway, response.getInstanceID());
-							} else {
-								log.info("The registration was refused. Try again.");
-
-								scheduledExecutorService.schedule(new Runnable() {
-									@Override
-									public void run() {
-										// we have to execute scheduled runnable in the main thread
-										// because we need consistency wrt currentRegistrationRun
-										runAsync(new Runnable() {
-											@Override
-											public void run() {
-												// our registration attempt was refused. Start over.
-												handleResourceManagerRegistration(
-													jobMasterRegistration,
-													1,
-													resourceManagerFuture,
-													registrationRun,
-													initialRegistrationTimeout,
-													maxTimeout,
-													deadline);
-											}
-										});
-									}
-								}, failedRegistrationDelay, TimeUnit.MILLISECONDS);
-							}
-						}
-					}
-				}, getMainThreadExecutionContext()); // use the main thread execution context to execute the call back in the main thread
-			}
-		} else {
-			log.info("Discard out-dated registration run.");
-		}
+	private void cancelAndClearEverything(Throwable cause) {
+		// currently, nothing to do here
 	}
 
-	/**
-	 * Finish the resource manager registration by setting the new resource manager gateway.
-	 *
-	 * @param resourceManager New resource manager gateway
-	 * @param instanceID Instance id assigned by the resource manager
-	 */
-	void finishResourceManagerRegistration(ResourceManagerGateway resourceManager, InstanceID instanceID) {
-		log.info("Successfully registered at the ResourceManager under instance id {}.", instanceID);
-		this.resourceManager = resourceManager;
-	}
+	// ------------------------------------------------------------------------
+	//  Utility classes
+	// ------------------------------------------------------------------------
+	private class JobMasterLeaderContender implements LeaderContender {
 
-	/**
-	 * Return if the job master is connected to a resource manager.
-	 *
-	 * @return true if the job master is connected to the resource manager
-	 */
-	public boolean isConnected() {
-		return resourceManager != null;
+		@Override
+		public void grantLeadership(UUID leaderSessionID) {
+			JobMaster.this.grantJobMasterLeadership(leaderSessionID);
+		}
+
+		@Override
+		public void revokeLeadership() {
+			JobMaster.this.revokeJobMasterLeadership();
+		}
+
+		@Override
+		public String getAddress() {
+			return JobMaster.this.getAddress();
+		}
+
+		@Override
+		public void handleError(Exception exception) {
+			onJobMasterElectionError(exception);
+		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/81a35c18/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java
index 7b4ab89..2790cf8 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java
@@ -20,9 +20,12 @@ package org.apache.flink.runtime.rpc.akka;
 
 import akka.actor.ActorSystem;
 import akka.util.Timeout;
-
 import org.apache.flink.core.testutils.OneShotLatch;
+import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
+import org.apache.flink.runtime.highavailability.NonHaServices;
+import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.rpc.jobmaster.JobMaster;
 import org.apache.flink.runtime.rpc.resourcemanager.ResourceManagerGateway;
 import org.apache.flink.runtime.rpc.resourcemanager.ResourceManager;
@@ -31,6 +34,7 @@ import org.apache.flink.util.TestLogger;
 import org.junit.AfterClass;
 import org.junit.Test;
 
+import org.mockito.Mockito;
 import scala.concurrent.duration.Deadline;
 import scala.concurrent.duration.FiniteDuration;
 
@@ -80,51 +84,4 @@ public class AkkaRpcServiceTest extends TestLogger {
 
 		assertTrue("call was not properly delayed", ((stop - start) / 1000000) >= delay);
 	}
-
-	// ------------------------------------------------------------------------
-	//  specific component tests - should be moved to the test classes
-	//  for those components
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Tests that the {@link JobMaster} can connect to the {@link ResourceManager} using the
-	 * {@link AkkaRpcService}.
-	 */
-	@Test
-	public void testJobMasterResourceManagerRegistration() throws Exception {
-		Timeout akkaTimeout = new Timeout(10, TimeUnit.SECONDS);
-		ActorSystem actorSystem = AkkaUtils.createDefaultActorSystem();
-		ActorSystem actorSystem2 = AkkaUtils.createDefaultActorSystem();
-		AkkaRpcService akkaRpcService = new AkkaRpcService(actorSystem, akkaTimeout);
-		AkkaRpcService akkaRpcService2 = new AkkaRpcService(actorSystem2, akkaTimeout);
-		ExecutorService executorService = new ForkJoinPool();
-
-		ResourceManager resourceManager = new ResourceManager(akkaRpcService, executorService);
-		JobMaster jobMaster = new JobMaster(akkaRpcService2, executorService);
-
-		resourceManager.start();
-		jobMaster.start();
-
-		ResourceManagerGateway rm = resourceManager.getSelf();
-
-		assertTrue(rm instanceof AkkaGateway);
-
-		AkkaGateway akkaClient = (AkkaGateway) rm;
-
-		
-		jobMaster.registerAtResourceManager(AkkaUtils.getAkkaURL(actorSystem, akkaClient.getRpcEndpoint()));
-
-		// wait for successful registration
-		FiniteDuration timeout = new FiniteDuration(200, TimeUnit.SECONDS);
-		Deadline deadline = timeout.fromNow();
-
-		while (deadline.hasTimeLeft() && !jobMaster.isConnected()) {
-			Thread.sleep(100);
-		}
-
-		assertFalse(deadline.isOverdue());
-
-		jobMaster.shutDown();
-		resourceManager.shutDown();
-	}
 }


[35/50] [abbrv] flink git commit: [FLINK-4355] [cluster management] Add tests for the TaskManager -> ResourceManager registration.

Posted by tr...@apache.org.
[FLINK-4355] [cluster management] Add tests for the TaskManager -> ResourceManager registration.

This closes #2395.


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

Branch: refs/heads/flip-6
Commit: 223eb6f5e895295d425f0a3c4a64e5bdbc3cee5c
Parents: b540c71
Author: Stephan Ewen <se...@apache.org>
Authored: Fri Aug 19 23:45:54 2016 +0200
Committer: Till Rohrmann <tr...@apache.org>
Committed: Thu Sep 8 17:26:59 2016 +0200

----------------------------------------------------------------------
 .../rpc/registration/RetryingRegistration.java  |   4 +
 .../runtime/rpc/taskexecutor/SlotReport.java    |  38 ---
 .../runtime/rpc/taskexecutor/TaskExecutor.java  |  12 +
 ...TaskExecutorToResourceManagerConnection.java |   4 +
 .../TestingHighAvailabilityServices.java        |  53 +++
 .../flink/runtime/rpc/TestingGatewayBase.java   |  18 +-
 .../registration/RetryingRegistrationTest.java  | 336 +++++++++++++++++++
 .../registration/TestRegistrationGateway.java   |  85 +++++
 .../rpc/taskexecutor/TaskExecutorTest.java      |  92 ++++-
 9 files changed, 602 insertions(+), 40 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/223eb6f5/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/registration/RetryingRegistration.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/registration/RetryingRegistration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/registration/RetryingRegistration.java
index 4c93684..dcb5011 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/registration/RetryingRegistration.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/registration/RetryingRegistration.java
@@ -58,12 +58,16 @@ public abstract class RetryingRegistration<Gateway extends RpcGateway, Success e
 	//  default configuration values
 	// ------------------------------------------------------------------------
 
+	/** default value for the initial registration timeout (milliseconds) */
 	private static final long INITIAL_REGISTRATION_TIMEOUT_MILLIS = 100;
 
+	/** default value for the maximum registration timeout, after exponential back-off (milliseconds) */
 	private static final long MAX_REGISTRATION_TIMEOUT_MILLIS = 30000;
 
+	/** The pause (milliseconds) made after an registration attempt caused an exception (other than timeout) */
 	private static final long ERROR_REGISTRATION_DELAY_MILLIS = 10000;
 
+	/** The pause (milliseconds) made after the registration attempt was refused */
 	private static final long REFUSED_REGISTRATION_DELAY_MILLIS = 30000;
 
 	// ------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/flink/blob/223eb6f5/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/SlotReport.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/SlotReport.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/SlotReport.java
deleted file mode 100644
index e42fa4a..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/SlotReport.java
+++ /dev/null
@@ -1,38 +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.rpc.taskexecutor;
-
-import java.io.Serializable;
-
-/**
- * A report about the current status of all slots of the TaskExecutor, describing
- * which slots are available and allocated, and what jobs (JobManagers) the allocated slots
- * have been allocated to.
- */
-public class SlotReport implements Serializable{
-
-	private static final long serialVersionUID = 1L;
-
-	// ------------------------------------------------------------------------
-	
-	@Override
-	public String toString() {
-		return "SlotReport";
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/223eb6f5/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java
index 1a637bb..f201e00 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutor.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.runtime.rpc.taskexecutor;
 
+import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.runtime.clusterframework.types.ResourceID;
 import org.apache.flink.runtime.highavailability.HighAvailabilityServices;
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener;
@@ -72,6 +73,8 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 
 	@Override
 	public void start() {
+		super.start();
+
 		// start by connecting to the ResourceManager
 		try {
 			haServices.getResourceManagerLeaderRetriever().start(new ResourceManagerLeaderListener());
@@ -148,6 +151,15 @@ public class TaskExecutor extends RpcEndpoint<TaskExecutorGateway> {
 	}
 
 	// ------------------------------------------------------------------------
+	//  Access to fields for testing
+	// ------------------------------------------------------------------------
+
+	@VisibleForTesting
+	TaskExecutorToResourceManagerConnection getResourceManagerConnection() {
+		return resourceManagerConnection;
+	}
+
+	// ------------------------------------------------------------------------
 	//  Utility classes
 	// ------------------------------------------------------------------------
 

http://git-wip-us.apache.org/repos/asf/flink/blob/223eb6f5/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorToResourceManagerConnection.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorToResourceManagerConnection.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorToResourceManagerConnection.java
index ef75862..f398b7d 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorToResourceManagerConnection.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorToResourceManagerConnection.java
@@ -40,6 +40,9 @@ import java.util.concurrent.TimeUnit;
 import static org.apache.flink.util.Preconditions.checkNotNull;
 import static org.apache.flink.util.Preconditions.checkState;
 
+/**
+ * The connection between a TaskExecutor and the ResourceManager.
+ */
 public class TaskExecutorToResourceManagerConnection {
 
 	/** the logger for all log messages of this class */
@@ -87,6 +90,7 @@ public class TaskExecutorToResourceManagerConnection {
 				log, taskExecutor.getRpcService(),
 				resourceManagerAddress, resourceManagerLeaderId,
 				taskExecutor.getAddress(), taskExecutor.getResourceID());
+		registration.startRegistration();
 
 		Future<Tuple2<ResourceManagerGateway, TaskExecutorRegistrationSuccess>> future = registration.getFuture();
 		

http://git-wip-us.apache.org/repos/asf/flink/blob/223eb6f5/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingHighAvailabilityServices.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingHighAvailabilityServices.java b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingHighAvailabilityServices.java
new file mode 100644
index 0000000..3a9f943
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/TestingHighAvailabilityServices.java
@@ -0,0 +1,53 @@
+/*
+ * 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.highavailability;
+
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+
+/**
+ * A variant of the HighAvailabilityServices for testing. Each individual service can be set
+ * to an arbitrary implementation, such as a mock or default service.
+ */
+public class TestingHighAvailabilityServices implements HighAvailabilityServices {
+
+	private volatile LeaderRetrievalService resourceManagerLeaderRetriever;
+
+
+	// ------------------------------------------------------------------------
+	//  Setters for mock / testing implementations
+	// ------------------------------------------------------------------------
+
+	public void setResourceManagerLeaderRetriever(LeaderRetrievalService resourceManagerLeaderRetriever) {
+		this.resourceManagerLeaderRetriever = resourceManagerLeaderRetriever;
+	}
+	
+	// ------------------------------------------------------------------------
+	//  HA Services Methods
+	// ------------------------------------------------------------------------
+
+	@Override
+	public LeaderRetrievalService getResourceManagerLeaderRetriever() throws Exception {
+		LeaderRetrievalService service = this.resourceManagerLeaderRetriever;
+		if (service != null) {
+			return service;
+		} else {
+			throw new IllegalStateException("ResourceManagerLeaderRetriever has not been set");
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/223eb6f5/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingGatewayBase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingGatewayBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingGatewayBase.java
index 4256135..8133a87 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingGatewayBase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/TestingGatewayBase.java
@@ -34,8 +34,15 @@ public abstract class TestingGatewayBase implements RpcGateway {
 
 	private final ScheduledExecutorService executor;
 
-	protected TestingGatewayBase() {
+	private final String address;
+
+	protected TestingGatewayBase(final String address) {
 		this.executor = Executors.newSingleThreadScheduledExecutor();
+		this.address = address;
+	}
+
+	protected TestingGatewayBase() {
+		this("localhost");
 	}
 
 	// ------------------------------------------------------------------------
@@ -53,6 +60,15 @@ public abstract class TestingGatewayBase implements RpcGateway {
 	}
 
 	// ------------------------------------------------------------------------
+	//  Base class methods
+	// ------------------------------------------------------------------------
+
+	@Override
+	public String getAddress() {
+		return address;
+	}
+
+	// ------------------------------------------------------------------------
 	//  utilities
 	// ------------------------------------------------------------------------
 

http://git-wip-us.apache.org/repos/asf/flink/blob/223eb6f5/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/registration/RetryingRegistrationTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/registration/RetryingRegistrationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/registration/RetryingRegistrationTest.java
new file mode 100644
index 0000000..9508825
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/registration/RetryingRegistrationTest.java
@@ -0,0 +1,336 @@
+/*
+ * 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.rpc.registration;
+
+import akka.dispatch.Futures;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.runtime.rpc.RpcService;
+import org.apache.flink.runtime.rpc.TestingRpcService;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Test;
+
+import org.slf4j.LoggerFactory;
+
+import scala.concurrent.Await;
+import scala.concurrent.ExecutionContext$;
+import scala.concurrent.Future;
+import scala.concurrent.Promise;
+import scala.concurrent.duration.FiniteDuration;
+
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeoutException;
+
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
+
+/**
+ * Tests for the generic retrying registration class, validating the failure, retry, and back-off behavior.
+ */
+public class RetryingRegistrationTest extends TestLogger {
+
+	@Test
+	public void testSimpleSuccessfulRegistration() throws Exception {
+		final String testId = "laissez les bon temps roulez";
+		final String testEndpointAddress = "<test-address>";
+		final UUID leaderId = UUID.randomUUID();
+
+		// an endpoint that immediately returns success
+		TestRegistrationGateway testGateway = new TestRegistrationGateway(new TestRegistrationSuccess(testId));
+		TestingRpcService rpc = new TestingRpcService();
+
+		try {
+			rpc.registerGateway(testEndpointAddress, testGateway);
+
+			TestRetryingRegistration registration = new TestRetryingRegistration(rpc, testEndpointAddress, leaderId);
+			registration.startRegistration();
+
+			Future<Tuple2<TestRegistrationGateway, TestRegistrationSuccess>> future = registration.getFuture();
+			assertNotNull(future);
+
+			// multiple accesses return the same future
+			assertEquals(future, registration.getFuture());
+
+			Tuple2<TestRegistrationGateway, TestRegistrationSuccess> success = 
+					Await.result(future, new FiniteDuration(10, SECONDS));
+
+			// validate correct invocation and result
+			assertEquals(testId, success.f1.getCorrelationId());
+			assertEquals(leaderId, testGateway.getInvocations().take().leaderId());
+		}
+		finally {
+			testGateway.stop();
+			rpc.stopService();
+		}
+	}
+	
+	@Test
+	public void testPropagateFailures() throws Exception {
+		final String testExceptionMessage = "testExceptionMessage";
+
+		// RPC service that fails with exception upon the connection
+		RpcService rpc = mock(RpcService.class);
+		when(rpc.connect(anyString(), any(Class.class))).thenThrow(new RuntimeException(testExceptionMessage));
+
+		TestRetryingRegistration registration = new TestRetryingRegistration(rpc, "testaddress", UUID.randomUUID());
+		registration.startRegistration();
+
+		Future<?> future = registration.getFuture();
+		assertTrue(future.failed().isCompleted());
+
+		assertEquals(testExceptionMessage, future.failed().value().get().get().getMessage());
+	}
+
+	@Test
+	public void testRetryConnectOnFailure() throws Exception {
+		final String testId = "laissez les bon temps roulez";
+		final UUID leaderId = UUID.randomUUID();
+
+		ExecutorService executor = Executors.newCachedThreadPool();
+		TestRegistrationGateway testGateway = new TestRegistrationGateway(new TestRegistrationSuccess(testId));
+
+		try {
+			// RPC service that fails upon the first connection, but succeeds on the second
+			RpcService rpc = mock(RpcService.class);
+			when(rpc.connect(anyString(), any(Class.class))).thenReturn(
+					Futures.failed(new Exception("test connect failure")),  // first connection attempt fails
+					Futures.successful(testGateway)                         // second connection attempt succeeds
+			);
+			when(rpc.getExecutionContext()).thenReturn(ExecutionContext$.MODULE$.fromExecutor(executor));
+
+			TestRetryingRegistration registration = new TestRetryingRegistration(rpc, "foobar address", leaderId);
+			registration.startRegistration();
+
+			Tuple2<TestRegistrationGateway, TestRegistrationSuccess> success =
+					Await.result(registration.getFuture(), new FiniteDuration(10, SECONDS));
+
+			// validate correct invocation and result
+			assertEquals(testId, success.f1.getCorrelationId());
+			assertEquals(leaderId, testGateway.getInvocations().take().leaderId());
+		}
+		finally {
+			testGateway.stop();
+			executor.shutdown();
+		}
+	}
+
+	@Test
+	public void testRetriesOnTimeouts() throws Exception {
+		final String testId = "rien ne va plus";
+		final String testEndpointAddress = "<test-address>";
+		final UUID leaderId = UUID.randomUUID();
+
+		// an endpoint that immediately returns futures with timeouts before returning a successful future
+		TestRegistrationGateway testGateway = new TestRegistrationGateway(
+				null, // timeout
+				null, // timeout
+				new TestRegistrationSuccess(testId) // success
+		);
+
+		TestingRpcService rpc = new TestingRpcService();
+
+		try {
+			rpc.registerGateway(testEndpointAddress, testGateway);
+	
+			TestRetryingRegistration registration = new TestRetryingRegistration(rpc, testEndpointAddress, leaderId);
+	
+			long started = System.nanoTime();
+			registration.startRegistration();
+	
+			Future<Tuple2<TestRegistrationGateway, TestRegistrationSuccess>> future = registration.getFuture();
+			Tuple2<TestRegistrationGateway, TestRegistrationSuccess> success =
+					Await.result(future, new FiniteDuration(10, SECONDS));
+	
+			long finished = System.nanoTime();
+			long elapsedMillis = (finished - started) / 1000000;
+	
+			// validate correct invocation and result
+			assertEquals(testId, success.f1.getCorrelationId());
+			assertEquals(leaderId, testGateway.getInvocations().take().leaderId());
+	
+			// validate that some retry-delay / back-off behavior happened
+			assertTrue("retries did not properly back off", elapsedMillis >= 3 * TestRetryingRegistration.INITIAL_TIMEOUT);
+		}
+		finally {
+			rpc.stopService();
+			testGateway.stop();
+		}
+	}
+
+	@Test
+	public void testDecline() throws Exception {
+		final String testId = "qui a coupe le fromage";
+		final String testEndpointAddress = "<test-address>";
+		final UUID leaderId = UUID.randomUUID();
+
+		TestingRpcService rpc = new TestingRpcService();
+
+		TestRegistrationGateway testGateway = new TestRegistrationGateway(
+				null, // timeout
+				new RegistrationResponse.Decline("no reason "),
+				null, // timeout
+				new TestRegistrationSuccess(testId) // success
+		);
+
+		try {
+			rpc.registerGateway(testEndpointAddress, testGateway);
+
+			TestRetryingRegistration registration = new TestRetryingRegistration(rpc, testEndpointAddress, leaderId);
+
+			long started = System.nanoTime();
+			registration.startRegistration();
+	
+			Future<Tuple2<TestRegistrationGateway, TestRegistrationSuccess>> future = registration.getFuture();
+			Tuple2<TestRegistrationGateway, TestRegistrationSuccess> success =
+					Await.result(future, new FiniteDuration(10, SECONDS));
+
+			long finished = System.nanoTime();
+			long elapsedMillis = (finished - started) / 1000000;
+
+			// validate correct invocation and result
+			assertEquals(testId, success.f1.getCorrelationId());
+			assertEquals(leaderId, testGateway.getInvocations().take().leaderId());
+
+			// validate that some retry-delay / back-off behavior happened
+			assertTrue("retries did not properly back off", elapsedMillis >= 
+					2 * TestRetryingRegistration.INITIAL_TIMEOUT + TestRetryingRegistration.DELAY_ON_DECLINE);
+		}
+		finally {
+			testGateway.stop();
+			rpc.stopService();
+		}
+	}
+	
+	@Test
+	@SuppressWarnings("unchecked")
+	public void testRetryOnError() throws Exception {
+		final String testId = "Petit a petit, l'oiseau fait son nid";
+		final String testEndpointAddress = "<test-address>";
+		final UUID leaderId = UUID.randomUUID();
+
+		TestingRpcService rpc = new TestingRpcService();
+
+		try {
+			// gateway that upon calls first responds with a failure, then with a success
+			TestRegistrationGateway testGateway = mock(TestRegistrationGateway.class);
+
+			when(testGateway.registrationCall(any(UUID.class), anyLong())).thenReturn(
+					Futures.<RegistrationResponse>failed(new Exception("test exception")),
+					Futures.<RegistrationResponse>successful(new TestRegistrationSuccess(testId)));
+			
+			rpc.registerGateway(testEndpointAddress, testGateway);
+
+			TestRetryingRegistration registration = new TestRetryingRegistration(rpc, testEndpointAddress, leaderId);
+
+			long started = System.nanoTime();
+			registration.startRegistration();
+
+			Future<Tuple2<TestRegistrationGateway, TestRegistrationSuccess>> future = registration.getFuture();
+			Tuple2<TestRegistrationGateway, TestRegistrationSuccess> success =
+					Await.result(future, new FiniteDuration(10, SECONDS));
+
+			long finished = System.nanoTime();
+			long elapsedMillis = (finished - started) / 1000000;
+			
+			assertEquals(testId, success.f1.getCorrelationId());
+
+			// validate that some retry-delay / back-off behavior happened
+			assertTrue("retries did not properly back off",
+					elapsedMillis >= TestRetryingRegistration.DELAY_ON_ERROR);
+		}
+		finally {
+			rpc.stopService();
+		}
+	}
+
+	@Test
+	public void testCancellation() throws Exception {
+		final String testEndpointAddress = "my-test-address";
+		final UUID leaderId = UUID.randomUUID();
+
+		TestingRpcService rpc = new TestingRpcService();
+
+		try {
+			Promise<RegistrationResponse> result = Futures.promise();
+
+			TestRegistrationGateway testGateway = mock(TestRegistrationGateway.class);
+			when(testGateway.registrationCall(any(UUID.class), anyLong())).thenReturn(result.future());
+
+			rpc.registerGateway(testEndpointAddress, testGateway);
+
+			TestRetryingRegistration registration = new TestRetryingRegistration(rpc, testEndpointAddress, leaderId);
+			registration.startRegistration();
+
+			// cancel and fail the current registration attempt
+			registration.cancel();
+			result.failure(new TimeoutException());
+
+			// there should not be a second registration attempt
+			verify(testGateway, atMost(1)).registrationCall(any(UUID.class), anyLong());
+		}
+		finally {
+			rpc.stopService();
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  test registration
+	// ------------------------------------------------------------------------
+
+	private static class TestRegistrationSuccess extends RegistrationResponse.Success {
+		private static final long serialVersionUID = 5542698790917150604L;
+
+		private final String correlationId;
+
+		private TestRegistrationSuccess(String correlationId) {
+			this.correlationId = correlationId;
+		}
+
+		public String getCorrelationId() {
+			return correlationId;
+		}
+	}
+
+	private static class TestRetryingRegistration extends RetryingRegistration<TestRegistrationGateway, TestRegistrationSuccess> {
+
+		// we use shorter timeouts here to speed up the tests
+		static final long INITIAL_TIMEOUT = 20;
+		static final long MAX_TIMEOUT = 200;
+		static final long DELAY_ON_ERROR = 200;
+		static final long DELAY_ON_DECLINE = 200;
+
+		public TestRetryingRegistration(RpcService rpc, String targetAddress, UUID leaderId) {
+			super(LoggerFactory.getLogger(RetryingRegistrationTest.class),
+					rpc, "TestEndpoint",
+					TestRegistrationGateway.class,
+					targetAddress, leaderId,
+					INITIAL_TIMEOUT, MAX_TIMEOUT, DELAY_ON_ERROR, DELAY_ON_DECLINE);
+		}
+
+		@Override
+		protected Future<RegistrationResponse> invokeRegistration(
+				TestRegistrationGateway gateway, UUID leaderId, long timeoutMillis) {
+			return gateway.registrationCall(leaderId, timeoutMillis);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/223eb6f5/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/registration/TestRegistrationGateway.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/registration/TestRegistrationGateway.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/registration/TestRegistrationGateway.java
new file mode 100644
index 0000000..a049e48
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/registration/TestRegistrationGateway.java
@@ -0,0 +1,85 @@
+/*
+ * 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.rpc.registration;
+
+import akka.dispatch.Futures;
+
+import org.apache.flink.runtime.rpc.TestingGatewayBase;
+import org.apache.flink.util.Preconditions;
+
+import scala.concurrent.Future;
+
+import java.util.UUID;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+
+public class TestRegistrationGateway extends TestingGatewayBase {
+
+	private final BlockingQueue<RegistrationCall> invocations;
+
+	private final RegistrationResponse[] responses;
+
+	private int pos;
+
+	public TestRegistrationGateway(RegistrationResponse... responses) {
+		Preconditions.checkArgument(responses != null && responses.length > 0);
+
+		this.invocations = new LinkedBlockingQueue<>();
+		this.responses = responses;
+		
+	}
+
+	// ------------------------------------------------------------------------
+
+	public Future<RegistrationResponse> registrationCall(UUID leaderId, long timeout) {
+		invocations.add(new RegistrationCall(leaderId, timeout));
+
+		RegistrationResponse response = responses[pos];
+		if (pos < responses.length - 1) {
+			pos++;
+		}
+
+		// return a completed future (for a proper value), or one that never completes and will time out (for null)
+		return response != null ? Futures.successful(response) : this.<RegistrationResponse>futureWithTimeout(timeout);
+	}
+
+	public BlockingQueue<RegistrationCall> getInvocations() {
+		return invocations;
+	}
+
+	// ------------------------------------------------------------------------
+
+	public static class RegistrationCall {
+		private final UUID leaderId;
+		private final long timeout;
+
+		public RegistrationCall(UUID leaderId, long timeout) {
+			this.leaderId = leaderId;
+			this.timeout = timeout;
+		}
+
+		public UUID leaderId() {
+			return leaderId;
+		}
+
+		public long timeout() {
+			return timeout;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/223eb6f5/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorTest.java
index 9f9bab3..b831ead 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/taskexecutor/TaskExecutorTest.java
@@ -18,8 +18,98 @@
 
 package org.apache.flink.runtime.rpc.taskexecutor;
 
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.highavailability.NonHaServices;
+import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices;
+import org.apache.flink.runtime.leaderelection.TestingLeaderRetrievalService;
+import org.apache.flink.runtime.rpc.TestingRpcService;
+import org.apache.flink.runtime.rpc.resourcemanager.ResourceManagerGateway;
 import org.apache.flink.util.TestLogger;
 
+import org.junit.Test;
+
+import scala.concurrent.duration.FiniteDuration;
+
+import java.util.UUID;
+
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
+
 public class TaskExecutorTest extends TestLogger {
-	
+
+	@Test
+	public void testImmediatelyRegistersIfLeaderIsKnown() throws Exception {
+		final ResourceID resourceID = ResourceID.generate();
+		final String resourceManagerAddress = "/resource/manager/address/one";
+
+		final TestingRpcService rpc = new TestingRpcService();
+		try {
+			// register a mock resource manager gateway
+			ResourceManagerGateway rmGateway = mock(ResourceManagerGateway.class);
+			rpc.registerGateway(resourceManagerAddress, rmGateway);
+
+			NonHaServices haServices = new NonHaServices(resourceManagerAddress);
+			TaskExecutor taskManager = new TaskExecutor(rpc, haServices, resourceID);
+			String taskManagerAddress = taskManager.getAddress();
+
+			taskManager.start();
+
+			verify(rmGateway, timeout(5000)).registerTaskExecutor(
+					any(UUID.class), eq(taskManagerAddress), eq(resourceID), any(FiniteDuration.class));
+		}
+		finally {
+			rpc.stopService();
+		}
+	}
+
+	@Test
+	public void testTriggerRegistrationOnLeaderChange() throws Exception {
+		final ResourceID resourceID = ResourceID.generate();
+
+		final String address1 = "/resource/manager/address/one";
+		final String address2 = "/resource/manager/address/two";
+		final UUID leaderId1 = UUID.randomUUID();
+		final UUID leaderId2 = UUID.randomUUID();
+
+		final TestingRpcService rpc = new TestingRpcService();
+		try {
+			// register the mock resource manager gateways
+			ResourceManagerGateway rmGateway1 = mock(ResourceManagerGateway.class);
+			ResourceManagerGateway rmGateway2 = mock(ResourceManagerGateway.class);
+			rpc.registerGateway(address1, rmGateway1);
+			rpc.registerGateway(address2, rmGateway2);
+
+			TestingLeaderRetrievalService testLeaderService = new TestingLeaderRetrievalService();
+
+			TestingHighAvailabilityServices haServices = new TestingHighAvailabilityServices();
+			haServices.setResourceManagerLeaderRetriever(testLeaderService);
+
+			TaskExecutor taskManager = new TaskExecutor(rpc, haServices, resourceID);
+			String taskManagerAddress = taskManager.getAddress();
+			taskManager.start();
+
+			// no connection initially, since there is no leader
+			assertNull(taskManager.getResourceManagerConnection());
+
+			// define a leader and see that a registration happens
+			testLeaderService.notifyListener(address1, leaderId1);
+
+			verify(rmGateway1, timeout(5000)).registerTaskExecutor(
+					eq(leaderId1), eq(taskManagerAddress), eq(resourceID), any(FiniteDuration.class));
+			assertNotNull(taskManager.getResourceManagerConnection());
+
+			// cancel the leader 
+			testLeaderService.notifyListener(null, null);
+
+			// set a new leader, see that a registration happens 
+			testLeaderService.notifyListener(address2, leaderId2);
+
+			verify(rmGateway2, timeout(5000)).registerTaskExecutor(
+					eq(leaderId2), eq(taskManagerAddress), eq(resourceID), any(FiniteDuration.class));
+			assertNotNull(taskManager.getResourceManagerConnection());
+		}
+		finally {
+			rpc.stopService();
+		}
+	}
 }


[11/50] [abbrv] flink git commit: [FLINK-4265] [dataset api] Add a NoOpOperator

Posted by tr...@apache.org.
[FLINK-4265] [dataset api] Add a NoOpOperator

Adds a NoOpOperator which is unwound in OperatorTranslation.translate.
This will be first used by Gelly as a placeholder to support implicit
operator reuse.

This closes #2294


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

Branch: refs/heads/flip-6
Commit: 66d4b8724b9e9b09225d2bbd3132dc2efdcf843a
Parents: cab76f6
Author: Greg Hogan <co...@greghogan.com>
Authored: Mon Jul 25 14:05:56 2016 -0400
Committer: Greg Hogan <co...@greghogan.com>
Committed: Tue Sep 6 09:25:48 2016 -0400

----------------------------------------------------------------------
 .../flink/api/java/operators/NoOpOperator.java  | 51 ++++++++++++++++++++
 .../api/java/operators/OperatorTranslation.java | 20 ++++----
 2 files changed, 62 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/66d4b872/flink-java/src/main/java/org/apache/flink/api/java/operators/NoOpOperator.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/NoOpOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/NoOpOperator.java
new file mode 100644
index 0000000..369ab9e
--- /dev/null
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/NoOpOperator.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.api.java.operators;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.util.Preconditions;
+
+/**
+ * This operator will be ignored during translation.
+ *
+ * @param <IN> The type of the data set passed through the operator.
+ */
+@Internal
+public class NoOpOperator<IN> extends DataSet<IN> {
+
+	private DataSet<IN> input;
+
+	public NoOpOperator(DataSet<IN> input, TypeInformation<IN> resultType) {
+		super(input.getExecutionEnvironment(), resultType);
+
+		this.input = input;
+	}
+
+	public DataSet<IN> getInput() {
+		return input;
+	}
+
+	public void setInput(DataSet<IN> input) {
+		Preconditions.checkNotNull(input);
+
+		this.input = input;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/66d4b872/flink-java/src/main/java/org/apache/flink/api/java/operators/OperatorTranslation.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/OperatorTranslation.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/OperatorTranslation.java
index 74811a3..3f44d58 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/OperatorTranslation.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/OperatorTranslation.java
@@ -19,15 +19,8 @@
 package org.apache.flink.api.java.operators;
 
 import org.apache.flink.annotation.Internal;
-import org.apache.flink.api.common.Plan;
-import org.apache.flink.api.java.DataSet;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
 import org.apache.flink.api.common.InvalidProgramException;
+import org.apache.flink.api.common.Plan;
 import org.apache.flink.api.common.operators.AbstractUdfOperator;
 import org.apache.flink.api.common.operators.BinaryOperatorInformation;
 import org.apache.flink.api.common.operators.GenericDataSinkBase;
@@ -35,8 +28,14 @@ import org.apache.flink.api.common.operators.Operator;
 import org.apache.flink.api.common.operators.UnaryOperatorInformation;
 import org.apache.flink.api.common.operators.base.BulkIterationBase;
 import org.apache.flink.api.common.operators.base.DeltaIterationBase;
+import org.apache.flink.api.java.DataSet;
 import org.apache.flink.configuration.Configuration;
 
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
 @Internal
 public class OperatorTranslation {
 	
@@ -70,7 +69,10 @@ public class OperatorTranslation {
 	
 	
 	private <T> Operator<T> translate(DataSet<T> dataSet) {
-		
+		while (dataSet instanceof NoOpOperator) {
+			dataSet = ((NoOpOperator<T>) dataSet).getInput();
+		}
+
 		// check if we have already translated that data set (operation or source)
 		Operator<?> previous = (Operator<?>) this.translated.get(dataSet);
 		if (previous != null) {


[12/50] [abbrv] flink git commit: [FLINK-4436] Unclosed DataOutputBuffer in Utils#setTokensFor()

Posted by tr...@apache.org.
[FLINK-4436] Unclosed DataOutputBuffer in Utils#setTokensFor()

This closes #2402


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

Branch: refs/heads/flip-6
Commit: 7e07bde8c5b33e9985260416fdf75e15df102efa
Parents: 66d4b87
Author: f7753 <ma...@gmail.com>
Authored: Mon Aug 22 21:07:55 2016 +0800
Committer: zentol <ch...@apache.org>
Committed: Thu Sep 8 12:04:25 2016 +0200

----------------------------------------------------------------------
 .../src/main/java/org/apache/flink/yarn/Utils.java   | 15 ++++++++-------
 .../flink/yarn/YarnApplicationMasterRunner.java      |  3 +--
 2 files changed, 9 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/7e07bde8/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 d5bad2f..1496d61 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
@@ -155,15 +155,16 @@ public final class Utils {
 			LOG.info("Adding user token " + id + " with " + token);
 			credentials.addToken(id, token);
 		}
-		DataOutputBuffer dob = new DataOutputBuffer();
-		credentials.writeTokenStorageToStream(dob);
+		try (DataOutputBuffer dob = new DataOutputBuffer()) {
+			credentials.writeTokenStorageToStream(dob);
 
-		if(LOG.isDebugEnabled()) {
-			LOG.debug("Wrote tokens. Credentials buffer length: " + dob.getLength());
-		}
+			if(LOG.isDebugEnabled()) {
+				LOG.debug("Wrote tokens. Credentials buffer length: " + dob.getLength());
+			}
 
-		ByteBuffer securityTokens = ByteBuffer.wrap(dob.getData(), 0, dob.getLength());
-		amContainer.setTokens(securityTokens);
+			ByteBuffer securityTokens = ByteBuffer.wrap(dob.getData(), 0, dob.getLength());
+			amContainer.setTokens(securityTokens);
+		}
 	}
 
 	/**

http://git-wip-us.apache.org/repos/asf/flink/blob/7e07bde8/flink-yarn/src/main/java/org/apache/flink/yarn/YarnApplicationMasterRunner.java
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnApplicationMasterRunner.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnApplicationMasterRunner.java
index 4637b97..7453344 100644
--- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnApplicationMasterRunner.java
+++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnApplicationMasterRunner.java
@@ -599,10 +599,9 @@ public class YarnApplicationMasterRunner {
 
 		ctx.setEnvironment(containerEnv);
 
-		try {
+		try (DataOutputBuffer dob = new DataOutputBuffer()) {
 			UserGroupInformation user = UserGroupInformation.getCurrentUser();
 			Credentials credentials = user.getCredentials();
-			DataOutputBuffer dob = new DataOutputBuffer();
 			credentials.writeTokenStorageToStream(dob);
 			ByteBuffer securityTokens = ByteBuffer.wrap(dob.getData(), 0, dob.getLength());
 			ctx.setTokens(securityTokens);


[21/50] [abbrv] flink git commit: [FLINK-4346] [rpc] Add new RPC abstraction

Posted by tr...@apache.org.
[FLINK-4346] [rpc] Add new RPC abstraction


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

Branch: refs/heads/flip-6
Commit: 0f36fb7f10c1b40005b897d85fec04e858c4909b
Parents: 0735b5b
Author: Till Rohrmann <tr...@apache.org>
Authored: Wed Aug 3 19:31:34 2016 +0200
Committer: Till Rohrmann <tr...@apache.org>
Committed: Thu Sep 8 17:26:56 2016 +0200

----------------------------------------------------------------------
 flink-runtime/pom.xml                           |   5 +
 .../flink/runtime/rpc/MainThreadExecutor.java   |  54 +++
 .../apache/flink/runtime/rpc/RpcEndpoint.java   | 182 +++++++++++
 .../apache/flink/runtime/rpc/RpcGateway.java    |  25 ++
 .../org/apache/flink/runtime/rpc/RpcMethod.java |  35 ++
 .../apache/flink/runtime/rpc/RpcService.java    |  74 +++++
 .../apache/flink/runtime/rpc/RpcTimeout.java    |  34 ++
 .../flink/runtime/rpc/akka/AkkaGateway.java     |  29 ++
 .../flink/runtime/rpc/akka/AkkaRpcService.java  | 145 ++++++++
 .../flink/runtime/rpc/akka/BaseAkkaActor.java   |  50 +++
 .../flink/runtime/rpc/akka/BaseAkkaGateway.java |  41 +++
 .../rpc/akka/jobmaster/JobMasterAkkaActor.java  |  58 ++++
 .../akka/jobmaster/JobMasterAkkaGateway.java    |  57 ++++
 .../rpc/akka/messages/CallableMessage.java      |  33 ++
 .../runtime/rpc/akka/messages/CancelTask.java   |  36 ++
 .../runtime/rpc/akka/messages/ExecuteTask.java  |  36 ++
 .../messages/RegisterAtResourceManager.java     |  36 ++
 .../rpc/akka/messages/RegisterJobMaster.java    |  36 ++
 .../runtime/rpc/akka/messages/RequestSlot.java  |  37 +++
 .../rpc/akka/messages/RunnableMessage.java      |  31 ++
 .../akka/messages/UpdateTaskExecutionState.java |  37 +++
 .../ResourceManagerAkkaActor.java               |  65 ++++
 .../ResourceManagerAkkaGateway.java             |  67 ++++
 .../taskexecutor/TaskExecutorAkkaActor.java     |  77 +++++
 .../taskexecutor/TaskExecutorAkkaGateway.java   |  59 ++++
 .../flink/runtime/rpc/jobmaster/JobMaster.java  | 249 ++++++++++++++
 .../runtime/rpc/jobmaster/JobMasterGateway.java |  45 +++
 .../resourcemanager/JobMasterRegistration.java  |  35 ++
 .../resourcemanager/RegistrationResponse.java   |  43 +++
 .../rpc/resourcemanager/ResourceManager.java    |  94 ++++++
 .../resourcemanager/ResourceManagerGateway.java |  58 ++++
 .../rpc/resourcemanager/SlotAssignment.java     |  25 ++
 .../rpc/resourcemanager/SlotRequest.java        |  25 ++
 .../runtime/rpc/taskexecutor/TaskExecutor.java  |  82 +++++
 .../rpc/taskexecutor/TaskExecutorGateway.java   |  48 +++
 .../flink/runtime/rpc/RpcCompletenessTest.java  | 327 +++++++++++++++++++
 .../runtime/rpc/akka/AkkaRpcServiceTest.java    |  81 +++++
 .../rpc/taskexecutor/TaskExecutorTest.java      |  92 ++++++
 .../runtime/util/DirectExecutorService.java     | 234 +++++++++++++
 flink-tests/pom.xml                             |   1 -
 pom.xml                                         |   7 +
 41 files changed, 2784 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/0f36fb7f/flink-runtime/pom.xml
----------------------------------------------------------------------
diff --git a/flink-runtime/pom.xml b/flink-runtime/pom.xml
index 5fea8fb..09c6fd0 100644
--- a/flink-runtime/pom.xml
+++ b/flink-runtime/pom.xml
@@ -189,6 +189,11 @@ under the License.
 			<artifactId>akka-testkit_${scala.binary.version}</artifactId>
 		</dependency>
 
+		<dependency>
+			<groupId>org.reflections</groupId>
+			<artifactId>reflections</artifactId>
+		</dependency>
+
 	</dependencies>
 
 	<build>

http://git-wip-us.apache.org/repos/asf/flink/blob/0f36fb7f/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/MainThreadExecutor.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/MainThreadExecutor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/MainThreadExecutor.java
new file mode 100644
index 0000000..e06711e
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/MainThreadExecutor.java
@@ -0,0 +1,54 @@
+/*
+ * 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.rpc;
+
+import akka.util.Timeout;
+import scala.concurrent.Future;
+
+import java.util.concurrent.Callable;
+import java.util.concurrent.TimeoutException;
+
+/**
+ * Interface to execute {@link Runnable} and {@link Callable} in the main thread of the underlying
+ * rpc server.
+ *
+ * This interface is intended to be implemented by the self gateway in a {@link RpcEndpoint}
+ * implementation which allows to dispatch local procedures to the main thread of the underlying
+ * rpc server.
+ */
+public interface MainThreadExecutor {
+	/**
+	 * Execute the runnable in the main thread of the underlying rpc server.
+	 *
+	 * @param runnable Runnable to be executed
+	 */
+	void runAsync(Runnable runnable);
+
+	/**
+	 * Execute the callable in the main thread of the underlying rpc server and return a future for
+	 * the callable result. If the future is not completed within the given timeout, the returned
+	 * future will throw a {@link TimeoutException}.
+	 *
+	 * @param callable Callable to be executed
+	 * @param timeout Timeout for the future to complete
+	 * @param <V> Return value of the callable
+	 * @return Future of the callable result
+	 */
+	<V> Future<V> callAsync(Callable<V> callable, Timeout timeout);
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0f36fb7f/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java
new file mode 100644
index 0000000..3d8757f
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java
@@ -0,0 +1,182 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.rpc;
+
+import akka.util.Timeout;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.concurrent.ExecutionContext;
+import scala.concurrent.Future;
+
+import java.util.concurrent.Callable;
+
+/**
+ * Base class for rpc endpoints. Distributed components which offer remote procedure calls have to
+ * extend the rpc endpoint base class.
+ *
+ * The main idea is that a rpc endpoint is backed by a rpc server which has a single thread
+ * processing the rpc calls. Thus, by executing all state changing operations within the main
+ * thread, we don't have to reason about concurrent accesses. The rpc provides provides
+ * {@link #runAsync(Runnable)}, {@link #callAsync(Callable, Timeout)} and the
+ * {@link #getMainThreadExecutionContext()} to execute code in the rpc server's main thread.
+ *
+ * @param <C> Rpc gateway counterpart for the implementing rpc endpoint
+ */
+public abstract class RpcEndpoint<C extends RpcGateway> {
+
+	protected final Logger log = LoggerFactory.getLogger(getClass());
+
+	/** Rpc service to be used to start the rpc server and to obtain rpc gateways */
+	private final RpcService rpcService;
+
+	/** Self gateway which can be used to schedule asynchronous calls on yourself */
+	private C self;
+
+	/**
+	 * The main thread execution context to be used to execute future callbacks in the main thread
+	 * of the executing rpc server.
+	 *
+	 * IMPORTANT: The main thread context is only available after the rpc server has been started.
+	 */
+	private MainThreadExecutionContext mainThreadExecutionContext;
+
+	public RpcEndpoint(RpcService rpcService) {
+		this.rpcService = rpcService;
+	}
+
+	/**
+	 * Get self-gateway which should be used to run asynchronous rpc calls on this endpoint.
+	 *
+	 * IMPORTANT: Always issue local method calls via the self-gateway if the current thread
+	 * is not the main thread of the underlying rpc server, e.g. from within a future callback.
+	 *
+	 * @return Self gateway
+	 */
+	public C getSelf() {
+		return self;
+	}
+
+	/**
+	 * Execute the runnable in the main thread of the underlying rpc server.
+	 *
+	 * @param runnable Runnable to be executed in the main thread of the underlying rpc server
+	 */
+	public void runAsync(Runnable runnable) {
+		((MainThreadExecutor) self).runAsync(runnable);
+	}
+
+	/**
+	 * Execute the callable in the main thread of the underlying rpc server returning a future for
+	 * the result of the callable. If the callable is not completed within the given timeout, then
+	 * the future will be failed with a {@link java.util.concurrent.TimeoutException}.
+	 *
+	 * @param callable Callable to be executed in the main thread of the underlying rpc server
+	 * @param timeout Timeout for the callable to be completed
+	 * @param <V> Return type of the callable
+	 * @return Future for the result of the callable.
+	 */
+	public <V> Future<V> callAsync(Callable<V> callable, Timeout timeout) {
+		return ((MainThreadExecutor) self).callAsync(callable, timeout);
+	}
+
+	/**
+	 * Gets the main thread execution context. The main thread execution context can be used to
+	 * execute tasks in the main thread of the underlying rpc server.
+	 *
+	 * @return Main thread execution context
+	 */
+	public ExecutionContext getMainThreadExecutionContext() {
+		return mainThreadExecutionContext;
+	}
+
+	/**
+	 * Gets the used rpc service.
+	 *
+	 * @return Rpc service
+	 */
+	public RpcService getRpcService() {
+		return rpcService;
+	}
+
+	/**
+	 * Starts the underlying rpc server via the rpc service and creates the main thread execution
+	 * context. This makes the rpc endpoint effectively reachable from the outside.
+	 *
+	 * Can be overriden to add rpc endpoint specific start up code. Should always call the parent
+	 * start method.
+	 */
+	public void start() {
+		self = rpcService.startServer(this);
+		mainThreadExecutionContext = new MainThreadExecutionContext((MainThreadExecutor) self);
+	}
+
+
+	/**
+	 * Shuts down the underlying rpc server via the rpc service.
+	 *
+	 * Can be overriden to add rpc endpoint specific shut down code. Should always call the parent
+	 * shut down method.
+	 */
+	public void shutDown() {
+		rpcService.stopServer(self);
+	}
+
+	/**
+	 * Gets the address of the underlying rpc server. The address should be fully qualified so that
+	 * a remote system can connect to this rpc server via this address.
+	 *
+	 * @return Fully qualified address of the underlying rpc server
+	 */
+	public String getAddress() {
+		return rpcService.getAddress(self);
+	}
+
+	/**
+	 * Execution context which executes runnables in the main thread context. A reported failure
+	 * will cause the underlying rpc server to shut down.
+	 */
+	private class MainThreadExecutionContext implements ExecutionContext {
+		private final MainThreadExecutor gateway;
+
+		MainThreadExecutionContext(MainThreadExecutor gateway) {
+			this.gateway = gateway;
+		}
+
+		@Override
+		public void execute(Runnable runnable) {
+			gateway.runAsync(runnable);
+		}
+
+		@Override
+		public void reportFailure(final Throwable t) {
+			gateway.runAsync(new Runnable() {
+				@Override
+				public void run() {
+					log.error("Encountered failure in the main thread execution context.", t);
+					shutDown();
+				}
+			});
+		}
+
+		@Override
+		public ExecutionContext prepare() {
+			return this;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0f36fb7f/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcGateway.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcGateway.java
new file mode 100644
index 0000000..e3a16b4
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcGateway.java
@@ -0,0 +1,25 @@
+/*
+ * 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.rpc;
+
+/**
+ * Rpc gateway interface which has to be implemented by Rpc gateways.
+ */
+public interface RpcGateway {
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0f36fb7f/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcMethod.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcMethod.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcMethod.java
new file mode 100644
index 0000000..875e557
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcMethod.java
@@ -0,0 +1,35 @@
+/*
+ * 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.rpc;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+/**
+ * Annotation for rpc method in a {@link RpcEndpoint} implementation. Every rpc method must have a
+ * respective counterpart in the {@link RpcGateway} implementation for this rpc server. The
+ * RpcCompletenessTest makes sure that the set of rpc methods in a rpc server and the set of
+ * gateway methods in the corresponding gateway implementation are identical.
+ */
+@Target(ElementType.METHOD)
+@Retention(RetentionPolicy.RUNTIME)
+public @interface RpcMethod {
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0f36fb7f/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java
new file mode 100644
index 0000000..90ff7b6
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcService.java
@@ -0,0 +1,74 @@
+/*
+ * 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.rpc;
+
+import scala.concurrent.Future;
+
+/**
+ * Interface for rpc services. An rpc service is used to start and connect to a {@link RpcEndpoint}.
+ * Connecting to a rpc server will return a {@link RpcGateway} which can be used to call remote
+ * procedures.
+ */
+public interface RpcService {
+
+	/**
+	 * Connect to a remote rpc server under the provided address. Returns a rpc gateway which can
+	 * be used to communicate with the rpc server.
+	 *
+	 * @param address Address of the remote rpc server
+	 * @param clazz Class of the rpc gateway to return
+	 * @param <C> Type of the rpc gateway to return
+	 * @return Future containing the rpc gateway
+	 */
+	<C extends RpcGateway> Future<C> connect(String address, Class<C> clazz);
+
+	/**
+	 * Start a rpc server which forwards the remote procedure calls to the provided rpc endpoint.
+	 *
+	 * @param rpcEndpoint Rpc protocl to dispath the rpcs to
+	 * @param <S> Type of the rpc endpoint
+	 * @param <C> Type of the self rpc gateway associated with the rpc server
+	 * @return Self gateway to dispatch remote procedure calls to oneself
+	 */
+	<S extends RpcEndpoint, C extends RpcGateway> C startServer(S rpcEndpoint);
+
+	/**
+	 * Stop the underlying rpc server of the provided self gateway.
+	 *
+	 * @param selfGateway Self gateway describing the underlying rpc server
+	 * @param <C> Type of the rpc gateway
+	 */
+	<C extends RpcGateway> void stopServer(C selfGateway);
+
+	/**
+	 * Stop the rpc service shutting down all started rpc servers.
+	 */
+	void stopService();
+
+	/**
+	 * Get the fully qualified address of the underlying rpc server represented by the self gateway.
+	 * It must be possible to connect from a remote host to the rpc server via the returned fully
+	 * qualified address.
+	 *
+	 * @param selfGateway Self gateway associated with the underlying rpc server
+	 * @param <C> Type of the rpc gateway
+	 * @return Fully qualified address
+	 */
+	<C extends RpcGateway> String getAddress(C selfGateway);
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0f36fb7f/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcTimeout.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcTimeout.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcTimeout.java
new file mode 100644
index 0000000..3d36d47
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcTimeout.java
@@ -0,0 +1,34 @@
+/*
+ * 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.rpc;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+/**
+ * Annotation for {@link RpcGateway} methods to specify an additional timeout parameter for the
+ * returned future to be completed. The rest of the provided parameters is passed to the remote rpc
+ * server for the rpc.
+ */
+@Target(ElementType.PARAMETER)
+@Retention(RetentionPolicy.RUNTIME)
+public @interface RpcTimeout {
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0f36fb7f/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaGateway.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaGateway.java
new file mode 100644
index 0000000..a96a600
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaGateway.java
@@ -0,0 +1,29 @@
+/*
+ * 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.rpc.akka;
+
+import akka.actor.ActorRef;
+
+/**
+ * Interface for Akka based rpc gateways
+ */
+public interface AkkaGateway {
+
+	ActorRef getActorRef();
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0f36fb7f/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java
new file mode 100644
index 0000000..d55bd13
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcService.java
@@ -0,0 +1,145 @@
+/*
+ * 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.rpc.akka;
+
+import akka.actor.ActorIdentity;
+import akka.actor.ActorRef;
+import akka.actor.ActorSelection;
+import akka.actor.ActorSystem;
+import akka.actor.Identify;
+import akka.actor.PoisonPill;
+import akka.actor.Props;
+import akka.dispatch.Mapper;
+import akka.pattern.AskableActorSelection;
+import akka.util.Timeout;
+import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.rpc.jobmaster.JobMaster;
+import org.apache.flink.runtime.rpc.jobmaster.JobMasterGateway;
+import org.apache.flink.runtime.rpc.resourcemanager.ResourceManager;
+import org.apache.flink.runtime.rpc.resourcemanager.ResourceManagerGateway;
+import org.apache.flink.runtime.rpc.RpcGateway;
+import org.apache.flink.runtime.rpc.RpcEndpoint;
+import org.apache.flink.runtime.rpc.RpcService;
+import org.apache.flink.runtime.rpc.akka.jobmaster.JobMasterAkkaActor;
+import org.apache.flink.runtime.rpc.akka.jobmaster.JobMasterAkkaGateway;
+import org.apache.flink.runtime.rpc.akka.resourcemanager.ResourceManagerAkkaActor;
+import org.apache.flink.runtime.rpc.akka.resourcemanager.ResourceManagerAkkaGateway;
+import org.apache.flink.runtime.rpc.akka.taskexecutor.TaskExecutorAkkaActor;
+import org.apache.flink.runtime.rpc.akka.taskexecutor.TaskExecutorAkkaGateway;
+import org.apache.flink.runtime.rpc.taskexecutor.TaskExecutorGateway;
+import org.apache.flink.runtime.rpc.taskexecutor.TaskExecutor;
+import scala.concurrent.Future;
+
+import java.util.HashSet;
+import java.util.Set;
+
+public class AkkaRpcService implements RpcService {
+	private final ActorSystem actorSystem;
+	private final Timeout timeout;
+	private final Set<ActorRef> actors = new HashSet<>();
+
+	public AkkaRpcService(ActorSystem actorSystem, Timeout timeout) {
+		this.actorSystem = actorSystem;
+		this.timeout = timeout;
+	}
+
+	@Override
+	public <C extends RpcGateway> Future<C> connect(String address, final Class<C> clazz) {
+		ActorSelection actorSel = actorSystem.actorSelection(address);
+
+		AskableActorSelection asker = new AskableActorSelection(actorSel);
+
+		Future<Object> identify = asker.ask(new Identify(42), timeout);
+
+		return identify.map(new Mapper<Object, C>(){
+			public C apply(Object obj) {
+				ActorRef actorRef = ((ActorIdentity) obj).getRef();
+
+				if (clazz == TaskExecutorGateway.class) {
+					return (C) new TaskExecutorAkkaGateway(actorRef, timeout);
+				} else if (clazz == ResourceManagerGateway.class) {
+					return (C) new ResourceManagerAkkaGateway(actorRef, timeout);
+				} else if (clazz == JobMasterGateway.class) {
+					return (C) new JobMasterAkkaGateway(actorRef, timeout);
+				} else {
+					throw new RuntimeException("Could not find remote endpoint " + clazz);
+				}
+			}
+		}, actorSystem.dispatcher());
+	}
+
+	@Override
+	public <S extends RpcEndpoint, C extends RpcGateway> C startServer(S rpcEndpoint) {
+		ActorRef ref;
+		C self;
+		if (rpcEndpoint instanceof TaskExecutor) {
+			ref = actorSystem.actorOf(
+				Props.create(TaskExecutorAkkaActor.class, rpcEndpoint)
+			);
+
+			self = (C) new TaskExecutorAkkaGateway(ref, timeout);
+		} else if (rpcEndpoint instanceof ResourceManager) {
+			ref = actorSystem.actorOf(
+				Props.create(ResourceManagerAkkaActor.class, rpcEndpoint)
+			);
+
+			self = (C) new ResourceManagerAkkaGateway(ref, timeout);
+		} else if (rpcEndpoint instanceof JobMaster) {
+			ref = actorSystem.actorOf(
+				Props.create(JobMasterAkkaActor.class, rpcEndpoint)
+			);
+
+			self = (C) new JobMasterAkkaGateway(ref, timeout);
+		} else {
+			throw new RuntimeException("Could not start RPC server for class " + rpcEndpoint.getClass());
+		}
+
+		actors.add(ref);
+
+		return self;
+	}
+
+	@Override
+	public <C extends RpcGateway> void stopServer(C selfGateway) {
+		if (selfGateway instanceof AkkaGateway) {
+			AkkaGateway akkaClient = (AkkaGateway) selfGateway;
+
+			if (actors.contains(akkaClient.getActorRef())) {
+				akkaClient.getActorRef().tell(PoisonPill.getInstance(), ActorRef.noSender());
+			} else {
+				// don't stop this actor since it was not started by this RPC service
+			}
+		}
+	}
+
+	@Override
+	public void stopService() {
+		actorSystem.shutdown();
+		actorSystem.awaitTermination();
+	}
+
+	@Override
+	public <C extends RpcGateway> String getAddress(C selfGateway) {
+		if (selfGateway instanceof AkkaGateway) {
+			return AkkaUtils.getAkkaURL(actorSystem, ((AkkaGateway) selfGateway).getActorRef());
+		} else {
+			throw new RuntimeException("Cannot get address for non " + AkkaGateway.class.getName() + ".");
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0f36fb7f/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/BaseAkkaActor.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/BaseAkkaActor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/BaseAkkaActor.java
new file mode 100644
index 0000000..3cb499c
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/BaseAkkaActor.java
@@ -0,0 +1,50 @@
+/*
+ * 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.rpc.akka;
+
+import akka.actor.Status;
+import akka.actor.UntypedActor;
+import org.apache.flink.runtime.rpc.akka.messages.CallableMessage;
+import org.apache.flink.runtime.rpc.akka.messages.RunnableMessage;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class BaseAkkaActor extends UntypedActor {
+	private static final Logger LOG = LoggerFactory.getLogger(BaseAkkaActor.class);
+
+	@Override
+	public void onReceive(Object message) throws Exception {
+		if (message instanceof RunnableMessage) {
+			try {
+				((RunnableMessage) message).getRunnable().run();
+			} catch (Exception e) {
+				LOG.error("Encountered error while executing runnable.", e);
+			}
+		} else if (message instanceof CallableMessage<?>) {
+			try {
+				Object result = ((CallableMessage<?>) message).getCallable().call();
+				sender().tell(new Status.Success(result), getSelf());
+			} catch (Exception e) {
+				sender().tell(new Status.Failure(e), getSelf());
+			}
+		} else {
+			throw new RuntimeException("Unknown message " + message);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0f36fb7f/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/BaseAkkaGateway.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/BaseAkkaGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/BaseAkkaGateway.java
new file mode 100644
index 0000000..512790d
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/BaseAkkaGateway.java
@@ -0,0 +1,41 @@
+/*
+ * 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.rpc.akka;
+
+import akka.actor.ActorRef;
+import akka.pattern.Patterns;
+import akka.util.Timeout;
+import org.apache.flink.runtime.rpc.MainThreadExecutor;
+import org.apache.flink.runtime.rpc.akka.messages.CallableMessage;
+import org.apache.flink.runtime.rpc.akka.messages.RunnableMessage;
+import scala.concurrent.Future;
+
+import java.util.concurrent.Callable;
+
+public abstract class BaseAkkaGateway implements MainThreadExecutor, AkkaGateway {
+	@Override
+	public void runAsync(Runnable runnable) {
+		getActorRef().tell(new RunnableMessage(runnable), ActorRef.noSender());
+	}
+
+	@Override
+	public <V> Future<V> callAsync(Callable<V> callable, Timeout timeout) {
+		return (Future<V>) Patterns.ask(getActorRef(), new CallableMessage(callable), timeout);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0f36fb7f/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/jobmaster/JobMasterAkkaActor.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/jobmaster/JobMasterAkkaActor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/jobmaster/JobMasterAkkaActor.java
new file mode 100644
index 0000000..9e04ea9
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/jobmaster/JobMasterAkkaActor.java
@@ -0,0 +1,58 @@
+/*
+ * 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.rpc.akka.jobmaster;
+
+import akka.actor.ActorRef;
+import akka.actor.Status;
+import org.apache.flink.runtime.rpc.akka.BaseAkkaActor;
+import org.apache.flink.runtime.rpc.akka.messages.RegisterAtResourceManager;
+import org.apache.flink.runtime.rpc.jobmaster.JobMaster;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.rpc.akka.messages.UpdateTaskExecutionState;
+
+public class JobMasterAkkaActor extends BaseAkkaActor {
+	private final JobMaster jobMaster;
+
+	public JobMasterAkkaActor(JobMaster jobMaster) {
+		this.jobMaster = jobMaster;
+	}
+
+	@Override
+	public void onReceive(Object message) throws Exception {
+		if (message instanceof UpdateTaskExecutionState) {
+
+			final ActorRef sender = getSender();
+
+			UpdateTaskExecutionState updateTaskExecutionState = (UpdateTaskExecutionState) message;
+
+			try {
+				Acknowledge result = jobMaster.updateTaskExecutionState(updateTaskExecutionState.getTaskExecutionState());
+				sender.tell(new Status.Success(result), getSelf());
+			} catch (Exception e) {
+				sender.tell(new Status.Failure(e), getSelf());
+			}
+		} else if (message instanceof RegisterAtResourceManager) {
+			RegisterAtResourceManager registerAtResourceManager = (RegisterAtResourceManager) message;
+
+			jobMaster.registerAtResourceManager(registerAtResourceManager.getAddress());
+		} else {
+			super.onReceive(message);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0f36fb7f/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/jobmaster/JobMasterAkkaGateway.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/jobmaster/JobMasterAkkaGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/jobmaster/JobMasterAkkaGateway.java
new file mode 100644
index 0000000..e6bf061
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/jobmaster/JobMasterAkkaGateway.java
@@ -0,0 +1,57 @@
+/*
+ * 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.rpc.akka.jobmaster;
+
+import akka.actor.ActorRef;
+import akka.pattern.AskableActorRef;
+import akka.util.Timeout;
+import org.apache.flink.runtime.rpc.akka.BaseAkkaGateway;
+import org.apache.flink.runtime.rpc.akka.messages.RegisterAtResourceManager;
+import org.apache.flink.runtime.rpc.jobmaster.JobMasterGateway;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.rpc.akka.messages.UpdateTaskExecutionState;
+import org.apache.flink.runtime.taskmanager.TaskExecutionState;
+import scala.concurrent.Future;
+import scala.reflect.ClassTag$;
+
+public class JobMasterAkkaGateway extends BaseAkkaGateway implements JobMasterGateway {
+	private final AskableActorRef actorRef;
+	private final Timeout timeout;
+
+	public JobMasterAkkaGateway(ActorRef actorRef, Timeout timeout) {
+		this.actorRef = new AskableActorRef(actorRef);
+		this.timeout = timeout;
+	}
+
+	@Override
+	public Future<Acknowledge> updateTaskExecutionState(TaskExecutionState taskExecutionState) {
+		return actorRef.ask(new UpdateTaskExecutionState(taskExecutionState), timeout)
+			.mapTo(ClassTag$.MODULE$.<Acknowledge>apply(Acknowledge.class));
+	}
+
+	@Override
+	public void registerAtResourceManager(String address) {
+		actorRef.actorRef().tell(new RegisterAtResourceManager(address), actorRef.actorRef());
+	}
+
+	@Override
+	public ActorRef getActorRef() {
+		return actorRef.actorRef();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0f36fb7f/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/CallableMessage.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/CallableMessage.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/CallableMessage.java
new file mode 100644
index 0000000..f0e555f
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/CallableMessage.java
@@ -0,0 +1,33 @@
+/*
+ * 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.rpc.akka.messages;
+
+import java.util.concurrent.Callable;
+
+public class CallableMessage<V> {
+	private final Callable<V> callable;
+
+	public CallableMessage(Callable<V> callable) {
+		this.callable = callable;
+	}
+
+	public Callable<V> getCallable() {
+		return callable;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0f36fb7f/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/CancelTask.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/CancelTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/CancelTask.java
new file mode 100644
index 0000000..0b9e9dc
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/CancelTask.java
@@ -0,0 +1,36 @@
+/*
+ * 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.rpc.akka.messages;
+
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+
+import java.io.Serializable;
+
+public class CancelTask implements Serializable {
+	private static final long serialVersionUID = -2998176874447950595L;
+	private final ExecutionAttemptID executionAttemptID;
+
+	public CancelTask(ExecutionAttemptID executionAttemptID) {
+		this.executionAttemptID = executionAttemptID;
+	}
+
+	public ExecutionAttemptID getExecutionAttemptID() {
+		return executionAttemptID;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0f36fb7f/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/ExecuteTask.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/ExecuteTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/ExecuteTask.java
new file mode 100644
index 0000000..a83d539
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/ExecuteTask.java
@@ -0,0 +1,36 @@
+/*
+ * 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.rpc.akka.messages;
+
+import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor;
+
+import java.io.Serializable;
+
+public class ExecuteTask implements Serializable {
+	private static final long serialVersionUID = -6769958430967048348L;
+	private final TaskDeploymentDescriptor taskDeploymentDescriptor;
+
+	public ExecuteTask(TaskDeploymentDescriptor taskDeploymentDescriptor) {
+		this.taskDeploymentDescriptor = taskDeploymentDescriptor;
+	}
+
+	public TaskDeploymentDescriptor getTaskDeploymentDescriptor() {
+		return taskDeploymentDescriptor;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0f36fb7f/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RegisterAtResourceManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RegisterAtResourceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RegisterAtResourceManager.java
new file mode 100644
index 0000000..3ade082
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RegisterAtResourceManager.java
@@ -0,0 +1,36 @@
+/*
+ * 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.rpc.akka.messages;
+
+import java.io.Serializable;
+
+public class RegisterAtResourceManager implements Serializable {
+
+	private static final long serialVersionUID = -4175905742620903602L;
+
+	private final String address;
+
+	public RegisterAtResourceManager(String address) {
+		this.address = address;
+	}
+
+	public String getAddress() {
+		return address;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0f36fb7f/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RegisterJobMaster.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RegisterJobMaster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RegisterJobMaster.java
new file mode 100644
index 0000000..b35ea38
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RegisterJobMaster.java
@@ -0,0 +1,36 @@
+/*
+ * 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.rpc.akka.messages;
+
+import org.apache.flink.runtime.rpc.resourcemanager.JobMasterRegistration;
+
+import java.io.Serializable;
+
+public class RegisterJobMaster implements Serializable{
+	private static final long serialVersionUID = -4616879574192641507L;
+	private final JobMasterRegistration jobMasterRegistration;
+
+	public RegisterJobMaster(JobMasterRegistration jobMasterRegistration) {
+		this.jobMasterRegistration = jobMasterRegistration;
+	}
+
+	public JobMasterRegistration getJobMasterRegistration() {
+		return jobMasterRegistration;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0f36fb7f/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RequestSlot.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RequestSlot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RequestSlot.java
new file mode 100644
index 0000000..85ceeec
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RequestSlot.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.rpc.akka.messages;
+
+import org.apache.flink.runtime.rpc.resourcemanager.SlotRequest;
+
+import java.io.Serializable;
+
+public class RequestSlot implements Serializable {
+	private static final long serialVersionUID = 7207463889348525866L;
+
+	private final SlotRequest slotRequest;
+
+	public RequestSlot(SlotRequest slotRequest) {
+		this.slotRequest = slotRequest;
+	}
+
+	public SlotRequest getSlotRequest() {
+		return slotRequest;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0f36fb7f/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RunnableMessage.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RunnableMessage.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RunnableMessage.java
new file mode 100644
index 0000000..3556738
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RunnableMessage.java
@@ -0,0 +1,31 @@
+/*
+ * 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.rpc.akka.messages;
+
+public class RunnableMessage {
+	private final Runnable runnable;
+
+	public RunnableMessage(Runnable runnable) {
+		this.runnable = runnable;
+	}
+
+	public Runnable getRunnable() {
+		return runnable;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0f36fb7f/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/UpdateTaskExecutionState.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/UpdateTaskExecutionState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/UpdateTaskExecutionState.java
new file mode 100644
index 0000000..f89cd2f
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/UpdateTaskExecutionState.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.rpc.akka.messages;
+
+import org.apache.flink.runtime.taskmanager.TaskExecutionState;
+
+import java.io.Serializable;
+
+public class UpdateTaskExecutionState implements Serializable{
+	private static final long serialVersionUID = -6662229114427331436L;
+
+	private final TaskExecutionState taskExecutionState;
+
+	public UpdateTaskExecutionState(TaskExecutionState taskExecutionState) {
+		this.taskExecutionState = taskExecutionState;
+	}
+
+	public TaskExecutionState getTaskExecutionState() {
+		return taskExecutionState;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0f36fb7f/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/resourcemanager/ResourceManagerAkkaActor.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/resourcemanager/ResourceManagerAkkaActor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/resourcemanager/ResourceManagerAkkaActor.java
new file mode 100644
index 0000000..13101f9
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/resourcemanager/ResourceManagerAkkaActor.java
@@ -0,0 +1,65 @@
+/*
+ * 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.rpc.akka.resourcemanager;
+
+import akka.actor.ActorRef;
+import akka.actor.Status;
+import akka.pattern.Patterns;
+import org.apache.flink.runtime.rpc.akka.BaseAkkaActor;
+import org.apache.flink.runtime.rpc.resourcemanager.RegistrationResponse;
+import org.apache.flink.runtime.rpc.resourcemanager.ResourceManager;
+import org.apache.flink.runtime.rpc.resourcemanager.SlotAssignment;
+import org.apache.flink.runtime.rpc.akka.messages.RegisterJobMaster;
+import org.apache.flink.runtime.rpc.akka.messages.RequestSlot;
+import scala.concurrent.Future;
+
+public class ResourceManagerAkkaActor extends BaseAkkaActor {
+	private final ResourceManager resourceManager;
+
+	public ResourceManagerAkkaActor(ResourceManager resourceManager) {
+		this.resourceManager = resourceManager;
+	}
+
+	@Override
+	public void onReceive(Object message) throws Exception {
+		final ActorRef sender = getSender();
+
+		if (message instanceof RegisterJobMaster) {
+			RegisterJobMaster registerJobMaster = (RegisterJobMaster) message;
+
+			try {
+				Future<RegistrationResponse> response = resourceManager.registerJobMaster(registerJobMaster.getJobMasterRegistration());
+				Patterns.pipe(response, getContext().dispatcher()).to(sender());
+			} catch (Exception e) {
+				sender.tell(new Status.Failure(e), getSelf());
+			}
+		} else if (message instanceof RequestSlot) {
+			RequestSlot requestSlot = (RequestSlot) message;
+
+			try {
+				SlotAssignment response = resourceManager.requestSlot(requestSlot.getSlotRequest());
+				sender.tell(new Status.Success(response), getSelf());
+			} catch (Exception e) {
+				sender.tell(new Status.Failure(e), getSelf());
+			}
+		} else {
+			super.onReceive(message);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0f36fb7f/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/resourcemanager/ResourceManagerAkkaGateway.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/resourcemanager/ResourceManagerAkkaGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/resourcemanager/ResourceManagerAkkaGateway.java
new file mode 100644
index 0000000..1304707
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/resourcemanager/ResourceManagerAkkaGateway.java
@@ -0,0 +1,67 @@
+/*
+ * 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.rpc.akka.resourcemanager;
+
+import akka.actor.ActorRef;
+import akka.pattern.AskableActorRef;
+import akka.util.Timeout;
+import org.apache.flink.runtime.rpc.akka.BaseAkkaGateway;
+import org.apache.flink.runtime.rpc.resourcemanager.JobMasterRegistration;
+import org.apache.flink.runtime.rpc.resourcemanager.RegistrationResponse;
+import org.apache.flink.runtime.rpc.resourcemanager.ResourceManagerGateway;
+import org.apache.flink.runtime.rpc.resourcemanager.SlotAssignment;
+import org.apache.flink.runtime.rpc.resourcemanager.SlotRequest;
+import org.apache.flink.runtime.rpc.akka.messages.RegisterJobMaster;
+import org.apache.flink.runtime.rpc.akka.messages.RequestSlot;
+import scala.concurrent.Future;
+import scala.concurrent.duration.FiniteDuration;
+import scala.reflect.ClassTag$;
+
+public class ResourceManagerAkkaGateway extends BaseAkkaGateway implements ResourceManagerGateway {
+	private final AskableActorRef actorRef;
+	private final Timeout timeout;
+
+	public ResourceManagerAkkaGateway(ActorRef actorRef, Timeout timeout) {
+		this.actorRef = new AskableActorRef(actorRef);
+		this.timeout = timeout;
+	}
+
+	@Override
+	public Future<RegistrationResponse> registerJobMaster(JobMasterRegistration jobMasterRegistration, FiniteDuration timeout) {
+		return actorRef.ask(new RegisterJobMaster(jobMasterRegistration), new Timeout(timeout))
+			.mapTo(ClassTag$.MODULE$.<RegistrationResponse>apply(RegistrationResponse.class));
+	}
+
+	@Override
+	public Future<RegistrationResponse> registerJobMaster(JobMasterRegistration jobMasterRegistration) {
+		return actorRef.ask(new RegisterJobMaster(jobMasterRegistration), timeout)
+			.mapTo(ClassTag$.MODULE$.<RegistrationResponse>apply(RegistrationResponse.class));
+	}
+
+	@Override
+	public Future<SlotAssignment> requestSlot(SlotRequest slotRequest) {
+		return actorRef.ask(new RequestSlot(slotRequest), timeout)
+			.mapTo(ClassTag$.MODULE$.<SlotAssignment>apply(SlotAssignment.class));
+	}
+
+	@Override
+	public ActorRef getActorRef() {
+		return actorRef.actorRef();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0f36fb7f/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/taskexecutor/TaskExecutorAkkaActor.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/taskexecutor/TaskExecutorAkkaActor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/taskexecutor/TaskExecutorAkkaActor.java
new file mode 100644
index 0000000..ed522cc
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/taskexecutor/TaskExecutorAkkaActor.java
@@ -0,0 +1,77 @@
+/*
+ * 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.rpc.akka.taskexecutor;
+
+import akka.actor.ActorRef;
+import akka.actor.Status;
+import akka.dispatch.OnComplete;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.rpc.akka.BaseAkkaActor;
+import org.apache.flink.runtime.rpc.akka.messages.CancelTask;
+import org.apache.flink.runtime.rpc.akka.messages.ExecuteTask;
+import org.apache.flink.runtime.rpc.taskexecutor.TaskExecutorGateway;
+
+public class TaskExecutorAkkaActor extends BaseAkkaActor {
+	private final TaskExecutorGateway taskExecutor;
+
+	public TaskExecutorAkkaActor(TaskExecutorGateway taskExecutor) {
+		this.taskExecutor = taskExecutor;
+	}
+
+	@Override
+	public void onReceive(Object message) throws Exception {
+		final ActorRef sender = getSender();
+
+		if (message instanceof ExecuteTask) {
+			ExecuteTask executeTask = (ExecuteTask) message;
+
+			taskExecutor.executeTask(executeTask.getTaskDeploymentDescriptor()).onComplete(
+				new OnComplete<Acknowledge>() {
+					@Override
+					public void onComplete(Throwable failure, Acknowledge success) throws Throwable {
+						if (failure != null) {
+							sender.tell(new Status.Failure(failure), getSelf());
+						} else {
+							sender.tell(new Status.Success(Acknowledge.get()), getSelf());
+						}
+					}
+				},
+				getContext().dispatcher()
+			);
+		} else if (message instanceof CancelTask) {
+			CancelTask cancelTask = (CancelTask) message;
+
+			taskExecutor.cancelTask(cancelTask.getExecutionAttemptID()).onComplete(
+				new OnComplete<Acknowledge>() {
+					@Override
+					public void onComplete(Throwable failure, Acknowledge success) throws Throwable {
+						if (failure != null) {
+							sender.tell(new Status.Failure(failure), getSelf());
+						} else {
+							sender.tell(new Status.Success(Acknowledge.get()), getSelf());
+						}
+					}
+				},
+				getContext().dispatcher()
+			);
+		} else {
+			super.onReceive(message);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0f36fb7f/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/taskexecutor/TaskExecutorAkkaGateway.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/taskexecutor/TaskExecutorAkkaGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/taskexecutor/TaskExecutorAkkaGateway.java
new file mode 100644
index 0000000..7f0a522
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/taskexecutor/TaskExecutorAkkaGateway.java
@@ -0,0 +1,59 @@
+/*
+ * 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.rpc.akka.taskexecutor;
+
+import akka.actor.ActorRef;
+import akka.pattern.AskableActorRef;
+import akka.util.Timeout;
+import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.rpc.akka.BaseAkkaGateway;
+import org.apache.flink.runtime.rpc.akka.messages.CancelTask;
+import org.apache.flink.runtime.rpc.akka.messages.ExecuteTask;
+import org.apache.flink.runtime.rpc.taskexecutor.TaskExecutorGateway;
+import scala.concurrent.Future;
+import scala.reflect.ClassTag$;
+
+public class TaskExecutorAkkaGateway extends BaseAkkaGateway implements TaskExecutorGateway {
+	private final AskableActorRef actorRef;
+	private final Timeout timeout;
+
+	public TaskExecutorAkkaGateway(ActorRef actorRef, Timeout timeout) {
+		this.actorRef = new AskableActorRef(actorRef);
+		this.timeout = timeout;
+	}
+
+	@Override
+	public Future<Acknowledge> executeTask(TaskDeploymentDescriptor taskDeploymentDescriptor) {
+		return actorRef.ask(new ExecuteTask(taskDeploymentDescriptor), timeout)
+			.mapTo(ClassTag$.MODULE$.<Acknowledge>apply(Acknowledge.class));
+	}
+
+	@Override
+	public Future<Acknowledge> cancelTask(ExecutionAttemptID executionAttemptId) {
+		return actorRef.ask(new CancelTask(executionAttemptId), timeout)
+			.mapTo(ClassTag$.MODULE$.<Acknowledge>apply(Acknowledge.class));
+	}
+
+	@Override
+	public ActorRef getActorRef() {
+		return actorRef.actorRef();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0f36fb7f/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/jobmaster/JobMaster.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/jobmaster/JobMaster.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/jobmaster/JobMaster.java
new file mode 100644
index 0000000..b81b19c
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/jobmaster/JobMaster.java
@@ -0,0 +1,249 @@
+/*
+ * 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.rpc.jobmaster;
+
+import akka.dispatch.Futures;
+import akka.dispatch.Mapper;
+import akka.dispatch.OnComplete;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.rpc.RpcMethod;
+import org.apache.flink.runtime.rpc.resourcemanager.JobMasterRegistration;
+import org.apache.flink.runtime.rpc.resourcemanager.RegistrationResponse;
+import org.apache.flink.runtime.rpc.resourcemanager.ResourceManagerGateway;
+import org.apache.flink.runtime.rpc.RpcEndpoint;
+import org.apache.flink.runtime.rpc.RpcService;
+import org.apache.flink.runtime.taskmanager.TaskExecutionState;
+import scala.Tuple2;
+import scala.concurrent.ExecutionContext;
+import scala.concurrent.ExecutionContext$;
+import scala.concurrent.Future;
+import scala.concurrent.duration.Deadline;
+import scala.concurrent.duration.FiniteDuration;
+
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+/**
+ * JobMaster implementation. The job master is responsible for the execution of a single
+ * {@link org.apache.flink.runtime.jobgraph.JobGraph}.
+ *
+ * It offers the following methods as part of its rpc interface to interact with the JobMaster
+ * remotely:
+ * <ul>
+ *     <li>{@link #registerAtResourceManager(String)} triggers the registration at the resource manager</li>
+ *     <li>{@link #updateTaskExecutionState(TaskExecutionState)} updates the task execution state for
+ * given task</li>
+ * </ul>
+ */
+public class JobMaster extends RpcEndpoint<JobMasterGateway> {
+	/** Execution context for future callbacks */
+	private final ExecutionContext executionContext;
+
+	/** Execution context for scheduled runnables */
+	private final ScheduledExecutorService scheduledExecutorService;
+
+	private final FiniteDuration initialRegistrationTimeout = new FiniteDuration(500, TimeUnit.MILLISECONDS);
+	private final FiniteDuration maxRegistrationTimeout = new FiniteDuration(30, TimeUnit.SECONDS);
+	private final FiniteDuration registrationDuration = new FiniteDuration(365, TimeUnit.DAYS);
+	private final long failedRegistrationDelay = 10000;
+
+	/** Gateway to connected resource manager, null iff not connected */
+	private ResourceManagerGateway resourceManager = null;
+
+	/** UUID to filter out old registration runs */
+	private UUID currentRegistrationRun;
+
+	public JobMaster(RpcService rpcService, ExecutorService executorService) {
+		super(rpcService);
+		executionContext = ExecutionContext$.MODULE$.fromExecutor(executorService);
+		scheduledExecutorService = new ScheduledThreadPoolExecutor(1);
+	}
+
+	public ResourceManagerGateway getResourceManager() {
+		return resourceManager;
+	}
+
+	//----------------------------------------------------------------------------------------------
+	// RPC methods
+	//----------------------------------------------------------------------------------------------
+
+	/**
+	 * Updates the task execution state for a given task.
+	 *
+	 * @param taskExecutionState New task execution state for a given task
+	 * @return Acknowledge the task execution state update
+	 */
+	@RpcMethod
+	public Acknowledge updateTaskExecutionState(TaskExecutionState taskExecutionState) {
+		System.out.println("TaskExecutionState: " + taskExecutionState);
+		return Acknowledge.get();
+	}
+
+	/**
+	 * Triggers the registration of the job master at the resource manager.
+	 *
+	 * @param address Address of the resource manager
+	 */
+	@RpcMethod
+	public void registerAtResourceManager(final String address) {
+		currentRegistrationRun = UUID.randomUUID();
+
+		Future<ResourceManagerGateway> resourceManagerFuture = getRpcService().connect(address, ResourceManagerGateway.class);
+
+		handleResourceManagerRegistration(
+			new JobMasterRegistration(getAddress()),
+			1,
+			resourceManagerFuture,
+			currentRegistrationRun,
+			initialRegistrationTimeout,
+			maxRegistrationTimeout,
+			registrationDuration.fromNow());
+	}
+
+	//----------------------------------------------------------------------------------------------
+	// Helper methods
+	//----------------------------------------------------------------------------------------------
+
+	/**
+	 * Helper method to handle the resource manager registration process. If a registration attempt
+	 * times out, then a new attempt with the doubled time out is initiated. The whole registration
+	 * process has a deadline. Once this deadline is overdue without successful registration, the
+	 * job master shuts down.
+	 *
+	 * @param jobMasterRegistration Job master registration info which is sent to the resource
+	 *                              manager
+	 * @param attemptNumber Registration attempt number
+	 * @param resourceManagerFuture Future of the resource manager gateway
+	 * @param registrationRun UUID describing the current registration run
+	 * @param timeout Timeout of the last registration attempt
+	 * @param maxTimeout Maximum timeout between registration attempts
+	 * @param deadline Deadline for the registration
+	 */
+	void handleResourceManagerRegistration(
+		final JobMasterRegistration jobMasterRegistration,
+		final int attemptNumber,
+		final Future<ResourceManagerGateway> resourceManagerFuture,
+		final UUID registrationRun,
+		final FiniteDuration timeout,
+		final FiniteDuration maxTimeout,
+		final Deadline deadline) {
+
+		// filter out concurrent registration runs
+		if (registrationRun.equals(currentRegistrationRun)) {
+
+			log.info("Start registration attempt #{}.", attemptNumber);
+
+			if (deadline.isOverdue()) {
+				// we've exceeded our registration deadline. This means that we have to shutdown the JobMaster
+				log.error("Exceeded registration deadline without successfully registering at the ResourceManager.");
+				shutDown();
+			} else {
+				Future<Tuple2<RegistrationResponse, ResourceManagerGateway>> registrationResponseFuture = resourceManagerFuture.flatMap(new Mapper<ResourceManagerGateway, Future<Tuple2<RegistrationResponse, ResourceManagerGateway>>>() {
+					@Override
+					public Future<Tuple2<RegistrationResponse, ResourceManagerGateway>> apply(ResourceManagerGateway resourceManagerGateway) {
+						return resourceManagerGateway.registerJobMaster(jobMasterRegistration, timeout).zip(Futures.successful(resourceManagerGateway));
+					}
+				}, executionContext);
+
+				registrationResponseFuture.onComplete(new OnComplete<Tuple2<RegistrationResponse, ResourceManagerGateway>>() {
+					@Override
+					public void onComplete(Throwable failure, Tuple2<RegistrationResponse, ResourceManagerGateway> tuple) throws Throwable {
+						if (failure != null) {
+							if (failure instanceof TimeoutException) {
+								// we haven't received an answer in the given timeout interval,
+								// so increase it and try again.
+								final FiniteDuration newTimeout = timeout.$times(2L).min(maxTimeout);
+
+								handleResourceManagerRegistration(
+									jobMasterRegistration,
+									attemptNumber + 1,
+									resourceManagerFuture,
+									registrationRun,
+									newTimeout,
+									maxTimeout,
+									deadline);
+							} else {
+								log.error("Received unknown error while registering at the ResourceManager.", failure);
+								shutDown();
+							}
+						} else {
+							final RegistrationResponse response = tuple._1();
+							final ResourceManagerGateway gateway = tuple._2();
+
+							if (response.isSuccess()) {
+								finishResourceManagerRegistration(gateway, response.getInstanceID());
+							} else {
+								log.info("The registration was refused. Try again.");
+
+								scheduledExecutorService.schedule(new Runnable() {
+									@Override
+									public void run() {
+										// we have to execute scheduled runnable in the main thread
+										// because we need consistency wrt currentRegistrationRun
+										runAsync(new Runnable() {
+											@Override
+											public void run() {
+												// our registration attempt was refused. Start over.
+												handleResourceManagerRegistration(
+													jobMasterRegistration,
+													1,
+													resourceManagerFuture,
+													registrationRun,
+													initialRegistrationTimeout,
+													maxTimeout,
+													deadline);
+											}
+										});
+									}
+								}, failedRegistrationDelay, TimeUnit.MILLISECONDS);
+							}
+						}
+					}
+				}, getMainThreadExecutionContext()); // use the main thread execution context to execute the call back in the main thread
+			}
+		} else {
+			log.info("Discard out-dated registration run.");
+		}
+	}
+
+	/**
+	 * Finish the resource manager registration by setting the new resource manager gateway.
+	 *
+	 * @param resourceManager New resource manager gateway
+	 * @param instanceID Instance id assigned by the resource manager
+	 */
+	void finishResourceManagerRegistration(ResourceManagerGateway resourceManager, InstanceID instanceID) {
+		log.info("Successfully registered at the ResourceManager under instance id {}.", instanceID);
+		this.resourceManager = resourceManager;
+	}
+
+	/**
+	 * Return if the job master is connected to a resource manager.
+	 *
+	 * @return true if the job master is connected to the resource manager
+	 */
+	public boolean isConnected() {
+		return resourceManager != null;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0f36fb7f/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/jobmaster/JobMasterGateway.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/jobmaster/JobMasterGateway.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/jobmaster/JobMasterGateway.java
new file mode 100644
index 0000000..17a4c3a
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/jobmaster/JobMasterGateway.java
@@ -0,0 +1,45 @@
+/*
+ * 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.rpc.jobmaster;
+
+import org.apache.flink.runtime.messages.Acknowledge;
+import org.apache.flink.runtime.rpc.RpcGateway;
+import org.apache.flink.runtime.taskmanager.TaskExecutionState;
+import scala.concurrent.Future;
+
+/**
+ * {@link JobMaster} rpc gateway interface
+ */
+public interface JobMasterGateway extends RpcGateway {
+
+	/**
+	 * Updates the task execution state for a given task.
+	 *
+	 * @param taskExecutionState New task execution state for a given task
+	 * @return Future acknowledge of the task execution state update
+	 */
+	Future<Acknowledge> updateTaskExecutionState(TaskExecutionState taskExecutionState);
+
+	/**
+	 * Triggers the registration of the job master at the resource manager.
+	 *
+	 * @param address Address of the resource manager
+	 */
+	void registerAtResourceManager(final String address);
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0f36fb7f/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/JobMasterRegistration.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/JobMasterRegistration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/JobMasterRegistration.java
new file mode 100644
index 0000000..7a2deae
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/JobMasterRegistration.java
@@ -0,0 +1,35 @@
+/*
+ * 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.rpc.resourcemanager;
+
+import java.io.Serializable;
+
+public class JobMasterRegistration implements Serializable {
+	private static final long serialVersionUID = 8411214999193765202L;
+
+	private final String address;
+
+	public JobMasterRegistration(String address) {
+		this.address = address;
+	}
+
+	public String getAddress() {
+		return address;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0f36fb7f/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/RegistrationResponse.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/RegistrationResponse.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/RegistrationResponse.java
new file mode 100644
index 0000000..8ac9e49
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/RegistrationResponse.java
@@ -0,0 +1,43 @@
+/*
+ * 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.rpc.resourcemanager;
+
+import org.apache.flink.runtime.instance.InstanceID;
+
+import java.io.Serializable;
+
+public class RegistrationResponse implements Serializable {
+	private static final long serialVersionUID = -2379003255993119993L;
+
+	private final boolean isSuccess;
+	private final InstanceID instanceID;
+
+	public RegistrationResponse(boolean isSuccess, InstanceID instanceID) {
+		this.isSuccess = isSuccess;
+		this.instanceID = instanceID;
+	}
+
+	public boolean isSuccess() {
+		return isSuccess;
+	}
+
+	public InstanceID getInstanceID() {
+		return instanceID;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/0f36fb7f/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManager.java
new file mode 100644
index 0000000..c7e8def
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/resourcemanager/ResourceManager.java
@@ -0,0 +1,94 @@
+/*
+ * 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.rpc.resourcemanager;
+
+import akka.dispatch.Mapper;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.rpc.RpcMethod;
+import org.apache.flink.runtime.rpc.RpcEndpoint;
+import org.apache.flink.runtime.rpc.RpcService;
+import org.apache.flink.runtime.rpc.jobmaster.JobMaster;
+import org.apache.flink.runtime.rpc.jobmaster.JobMasterGateway;
+import scala.concurrent.ExecutionContext;
+import scala.concurrent.ExecutionContext$;
+import scala.concurrent.Future;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ExecutorService;
+
+/**
+ * ResourceManager implementation. The resource manager is responsible for resource de-/allocation
+ * and bookkeeping.
+ *
+ * It offers the following methods as part of its rpc interface to interact with the him remotely:
+ * <ul>
+ *     <li>{@link #registerJobMaster(JobMasterRegistration)} registers a {@link JobMaster} at the resource manager</li>
+ *     <li>{@link #requestSlot(SlotRequest)} requests a slot from the resource manager</li>
+ * </ul>
+ */
+public class ResourceManager extends RpcEndpoint<ResourceManagerGateway> {
+	private final ExecutionContext executionContext;
+	private final Map<JobMasterGateway, InstanceID> jobMasterGateways;
+
+	public ResourceManager(RpcService rpcService, ExecutorService executorService) {
+		super(rpcService);
+		this.executionContext = ExecutionContext$.MODULE$.fromExecutor(executorService);
+		this.jobMasterGateways = new HashMap<>();
+	}
+
+	/**
+	 * Register a {@link JobMaster} at the resource manager.
+	 *
+	 * @param jobMasterRegistration Job master registration information
+	 * @return Future registration response
+	 */
+	@RpcMethod
+	public Future<RegistrationResponse> registerJobMaster(JobMasterRegistration jobMasterRegistration) {
+		Future<JobMasterGateway> jobMasterFuture = getRpcService().connect(jobMasterRegistration.getAddress(), JobMasterGateway.class);
+
+		return jobMasterFuture.map(new Mapper<JobMasterGateway, RegistrationResponse>() {
+			@Override
+			public RegistrationResponse apply(final JobMasterGateway jobMasterGateway) {
+				InstanceID instanceID;
+
+				if (jobMasterGateways.containsKey(jobMasterGateway)) {
+					instanceID = jobMasterGateways.get(jobMasterGateway);
+				} else {
+					instanceID = new InstanceID();
+					jobMasterGateways.put(jobMasterGateway, instanceID);
+				}
+
+				return new RegistrationResponse(true, instanceID);
+			}
+		}, getMainThreadExecutionContext());
+	}
+
+	/**
+	 * Requests a slot from the resource manager.
+	 *
+	 * @param slotRequest Slot request
+	 * @return Slot assignment
+	 */
+	@RpcMethod
+	public SlotAssignment requestSlot(SlotRequest slotRequest) {
+		System.out.println("SlotRequest: " + slotRequest);
+		return new SlotAssignment();
+	}
+}


[19/50] [abbrv] flink git commit: [FLINK-4368] [distributed runtime] Eagerly initialize the RPC endpoint members

Posted by tr...@apache.org.
[FLINK-4368] [distributed runtime] Eagerly initialize the RPC endpoint members

This closes #2351


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

Branch: refs/heads/flip-6
Commit: 506aac073a97494c8685492b18adc65fb2786829
Parents: 0f36fb7
Author: Stephan Ewen <se...@apache.org>
Authored: Wed Aug 10 18:27:21 2016 +0200
Committer: Till Rohrmann <tr...@apache.org>
Committed: Thu Sep 8 17:26:56 2016 +0200

----------------------------------------------------------------------
 .../flink/runtime/rpc/MainThreadExecutor.java   |   9 +-
 .../apache/flink/runtime/rpc/RpcEndpoint.java   | 156 +++++++++++--------
 .../runtime/rpc/akka/AkkaRpcServiceTest.java    |   4 +-
 3 files changed, 99 insertions(+), 70 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/506aac07/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/MainThreadExecutor.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/MainThreadExecutor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/MainThreadExecutor.java
index e06711e..14b2997 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/MainThreadExecutor.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/MainThreadExecutor.java
@@ -26,22 +26,23 @@ import java.util.concurrent.TimeoutException;
 
 /**
  * Interface to execute {@link Runnable} and {@link Callable} in the main thread of the underlying
- * rpc server.
+ * RPC endpoint.
  *
- * This interface is intended to be implemented by the self gateway in a {@link RpcEndpoint}
+ * <p>This interface is intended to be implemented by the self gateway in a {@link RpcEndpoint}
  * implementation which allows to dispatch local procedures to the main thread of the underlying
  * rpc server.
  */
 public interface MainThreadExecutor {
+
 	/**
-	 * Execute the runnable in the main thread of the underlying rpc server.
+	 * Execute the runnable in the main thread of the underlying RPC endpoint.
 	 *
 	 * @param runnable Runnable to be executed
 	 */
 	void runAsync(Runnable runnable);
 
 	/**
-	 * Execute the callable in the main thread of the underlying rpc server and return a future for
+	 * Execute the callable in the main thread of the underlying RPC endpoint and return a future for
 	 * the callable result. If the future is not completed within the given timeout, the returned
 	 * future will throw a {@link TimeoutException}.
 	 *

http://git-wip-us.apache.org/repos/asf/flink/blob/506aac07/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java
index 3d8757f..0d928a8 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java
@@ -19,85 +19,116 @@
 package org.apache.flink.runtime.rpc;
 
 import akka.util.Timeout;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import scala.concurrent.ExecutionContext;
 import scala.concurrent.Future;
 
 import java.util.concurrent.Callable;
 
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
 /**
- * Base class for rpc endpoints. Distributed components which offer remote procedure calls have to
- * extend the rpc endpoint base class.
+ * Base class for RPC endpoints. Distributed components which offer remote procedure calls have to
+ * extend the RPC endpoint base class. An RPC endpoint is backed by an {@link RpcService}. 
+ * 
+ * <h1>Endpoint and Gateway</h1>
+ * 
+ * To be done...
+ * 
+ * <h1>Single Threaded Endpoint Execution </h1>
+ * 
+ * <p>All RPC calls on the same endpoint are called by the same thread
+ * (referred to as the endpoint's <i>main thread</i>).
+ * Thus, by executing all state changing operations within the main 
+ * thread, we don't have to reason about concurrent accesses, in the same way in the Actor Model
+ * of Erlang or Akka.
  *
- * The main idea is that a rpc endpoint is backed by a rpc server which has a single thread
- * processing the rpc calls. Thus, by executing all state changing operations within the main
- * thread, we don't have to reason about concurrent accesses. The rpc provides provides
- * {@link #runAsync(Runnable)}, {@link #callAsync(Callable, Timeout)} and the
- * {@link #getMainThreadExecutionContext()} to execute code in the rpc server's main thread.
+ * <p>The RPC endpoint provides provides {@link #runAsync(Runnable)}, {@link #callAsync(Callable, Timeout)}
+  * and the {@link #getMainThreadExecutionContext()} to execute code in the RPC endoint's main thread.
  *
- * @param <C> Rpc gateway counterpart for the implementing rpc endpoint
+ * @param <C> The RPC gateway counterpart for the implementing RPC endpoint
  */
 public abstract class RpcEndpoint<C extends RpcGateway> {
 
 	protected final Logger log = LoggerFactory.getLogger(getClass());
 
-	/** Rpc service to be used to start the rpc server and to obtain rpc gateways */
+	// ------------------------------------------------------------------------
+
+	/** RPC service to be used to start the RPC server and to obtain rpc gateways */
 	private final RpcService rpcService;
 
 	/** Self gateway which can be used to schedule asynchronous calls on yourself */
-	private C self;
+	private final C self;
+
+	/** the fully qualified address of the this RPC endpoint */
+	private final String selfAddress;
+
+	/** The main thread execution context to be used to execute future callbacks in the main thread
+	 * of the executing rpc server. */
+	private final MainThreadExecutionContext mainThreadExecutionContext;
+
 
 	/**
-	 * The main thread execution context to be used to execute future callbacks in the main thread
-	 * of the executing rpc server.
-	 *
-	 * IMPORTANT: The main thread context is only available after the rpc server has been started.
+	 * Initializes the RPC endpoint.
+	 * 
+	 * @param rpcService The RPC server that dispatches calls to this RPC endpoint. 
 	 */
-	private MainThreadExecutionContext mainThreadExecutionContext;
-
 	public RpcEndpoint(RpcService rpcService) {
-		this.rpcService = rpcService;
+		this.rpcService = checkNotNull(rpcService, "rpcService");
+		this.self = rpcService.startServer(this);
+		this.selfAddress = rpcService.getAddress(self);
+		this.mainThreadExecutionContext = new MainThreadExecutionContext((MainThreadExecutor) self);
 	}
 
+	// ------------------------------------------------------------------------
+	//  Shutdown
+	// ------------------------------------------------------------------------
+
 	/**
-	 * Get self-gateway which should be used to run asynchronous rpc calls on this endpoint.
-	 *
-	 * IMPORTANT: Always issue local method calls via the self-gateway if the current thread
-	 * is not the main thread of the underlying rpc server, e.g. from within a future callback.
-	 *
-	 * @return Self gateway
+	 * Shuts down the underlying RPC endpoint via the RPC service.
+	 * After this method was called, the RPC endpoint will no longer be reachable, neither remotely,
+	 * not via its {@link #getSelf() self gateway}. It will also not accepts executions in main thread
+	 * any more (via {@link #callAsync(Callable, Timeout)} and {@link #runAsync(Runnable)}).
+	 * 
+	 * <p>This method can be overridden to add RPC endpoint specific shut down code.
+	 * The overridden method should always call the parent shut down method.
 	 */
-	public C getSelf() {
-		return self;
+	public void shutDown() {
+		rpcService.stopServer(self);
 	}
 
+	// ------------------------------------------------------------------------
+	//  Basic RPC endpoint properties
+	// ------------------------------------------------------------------------
+
 	/**
-	 * Execute the runnable in the main thread of the underlying rpc server.
+	 * Get self-gateway which should be used to run asynchronous RPC calls on this endpoint.
+	 *
+	 * <p><b>IMPORTANT</b>: Always issue local method calls via the self-gateway if the current thread
+	 * is not the main thread of the underlying rpc server, e.g. from within a future callback.
 	 *
-	 * @param runnable Runnable to be executed in the main thread of the underlying rpc server
+	 * @return The self gateway
 	 */
-	public void runAsync(Runnable runnable) {
-		((MainThreadExecutor) self).runAsync(runnable);
+	public C getSelf() {
+		return self;
 	}
 
 	/**
-	 * Execute the callable in the main thread of the underlying rpc server returning a future for
-	 * the result of the callable. If the callable is not completed within the given timeout, then
-	 * the future will be failed with a {@link java.util.concurrent.TimeoutException}.
+	 * Gets the address of the underlying RPC endpoint. The address should be fully qualified so that
+	 * a remote system can connect to this RPC endpoint via this address.
 	 *
-	 * @param callable Callable to be executed in the main thread of the underlying rpc server
-	 * @param timeout Timeout for the callable to be completed
-	 * @param <V> Return type of the callable
-	 * @return Future for the result of the callable.
+	 * @return Fully qualified address of the underlying RPC endpoint
 	 */
-	public <V> Future<V> callAsync(Callable<V> callable, Timeout timeout) {
-		return ((MainThreadExecutor) self).callAsync(callable, timeout);
+	public String getAddress() {
+		return selfAddress;
 	}
 
 	/**
 	 * Gets the main thread execution context. The main thread execution context can be used to
-	 * execute tasks in the main thread of the underlying rpc server.
+	 * execute tasks in the main thread of the underlying RPC endpoint.
 	 *
 	 * @return Main thread execution context
 	 */
@@ -106,52 +137,51 @@ public abstract class RpcEndpoint<C extends RpcGateway> {
 	}
 
 	/**
-	 * Gets the used rpc service.
+	 * Gets the endpoint's RPC service.
 	 *
-	 * @return Rpc service
+	 * @return The endpoint's RPC service
 	 */
 	public RpcService getRpcService() {
 		return rpcService;
 	}
 
-	/**
-	 * Starts the underlying rpc server via the rpc service and creates the main thread execution
-	 * context. This makes the rpc endpoint effectively reachable from the outside.
-	 *
-	 * Can be overriden to add rpc endpoint specific start up code. Should always call the parent
-	 * start method.
-	 */
-	public void start() {
-		self = rpcService.startServer(this);
-		mainThreadExecutionContext = new MainThreadExecutionContext((MainThreadExecutor) self);
-	}
-
+	// ------------------------------------------------------------------------
+	//  Asynchronous executions
+	// ------------------------------------------------------------------------
 
 	/**
-	 * Shuts down the underlying rpc server via the rpc service.
+	 * Execute the runnable in the main thread of the underlying RPC endpoint.
 	 *
-	 * Can be overriden to add rpc endpoint specific shut down code. Should always call the parent
-	 * shut down method.
+	 * @param runnable Runnable to be executed in the main thread of the underlying RPC endpoint
 	 */
-	public void shutDown() {
-		rpcService.stopServer(self);
+	public void runAsync(Runnable runnable) {
+		((MainThreadExecutor) self).runAsync(runnable);
 	}
 
 	/**
-	 * Gets the address of the underlying rpc server. The address should be fully qualified so that
-	 * a remote system can connect to this rpc server via this address.
+	 * Execute the callable in the main thread of the underlying RPC service, returning a future for
+	 * the result of the callable. If the callable is not completed within the given timeout, then
+	 * the future will be failed with a {@link java.util.concurrent.TimeoutException}.
 	 *
-	 * @return Fully qualified address of the underlying rpc server
+	 * @param callable Callable to be executed in the main thread of the underlying rpc server
+	 * @param timeout Timeout for the callable to be completed
+	 * @param <V> Return type of the callable
+	 * @return Future for the result of the callable.
 	 */
-	public String getAddress() {
-		return rpcService.getAddress(self);
+	public <V> Future<V> callAsync(Callable<V> callable, Timeout timeout) {
+		return ((MainThreadExecutor) self).callAsync(callable, timeout);
 	}
 
+	// ------------------------------------------------------------------------
+	//  Utilities
+	// ------------------------------------------------------------------------
+	
 	/**
 	 * Execution context which executes runnables in the main thread context. A reported failure
 	 * will cause the underlying rpc server to shut down.
 	 */
 	private class MainThreadExecutionContext implements ExecutionContext {
+
 		private final MainThreadExecutor gateway;
 
 		MainThreadExecutionContext(MainThreadExecutor gateway) {

http://git-wip-us.apache.org/repos/asf/flink/blob/506aac07/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java
index c5bac94..642a380 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java
@@ -54,15 +54,13 @@ public class AkkaRpcServiceTest extends TestLogger {
 		ResourceManager resourceManager = new ResourceManager(akkaRpcService, executorService);
 		JobMaster jobMaster = new JobMaster(akkaRpcService2, executorService);
 
-		resourceManager.start();
-
 		ResourceManagerGateway rm = resourceManager.getSelf();
 
 		assertTrue(rm instanceof AkkaGateway);
 
 		AkkaGateway akkaClient = (AkkaGateway) rm;
 
-		jobMaster.start();
+		
 		jobMaster.registerAtResourceManager(AkkaUtils.getAkkaURL(actorSystem, akkaClient.getActorRef()));
 
 		// wait for successful registration


[10/50] [abbrv] flink git commit: [FLINK-4559][kinesis-connector] Fix AWSUtil.getCredentialsProvider() to not return null

Posted by tr...@apache.org.
[FLINK-4559][kinesis-connector] Fix AWSUtil.getCredentialsProvider() to not return null

Return a DefaultAWSCredentialsProviderChain instead of null when
AWS_CREDENTIALS_PROVIDER config is set to "AUTO"

This closes #2470


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

Branch: refs/heads/flip-6
Commit: cab76f6e23f6b491ecde5856dd32e4de95a0af90
Parents: 87aea72
Author: Tzu-Li (Gordon) Tai <tz...@apache.org>
Authored: Sun Sep 4 01:37:39 2016 +0800
Committer: Tzu-Li (Gordon) Tai <tz...@apache.org>
Committed: Tue Sep 6 20:12:34 2016 +0800

----------------------------------------------------------------------
 .../streaming/connectors/kinesis/util/AWSUtil.java      | 12 ++++--------
 1 file changed, 4 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/cab76f6e/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/util/AWSUtil.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/util/AWSUtil.java b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/util/AWSUtil.java
index 06f967b..81c0b6b 100644
--- a/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/util/AWSUtil.java
+++ b/flink-streaming-connectors/flink-connector-kinesis/src/main/java/org/apache/flink/streaming/connectors/kinesis/util/AWSUtil.java
@@ -22,6 +22,7 @@ import com.amazonaws.ClientConfigurationFactory;
 import com.amazonaws.auth.AWSCredentials;
 import com.amazonaws.auth.AWSCredentialsProvider;
 import com.amazonaws.auth.BasicAWSCredentials;
+import com.amazonaws.auth.DefaultAWSCredentialsProviderChain;
 import com.amazonaws.auth.EnvironmentVariableCredentialsProvider;
 import com.amazonaws.auth.SystemPropertiesCredentialsProvider;
 import com.amazonaws.auth.profile.ProfileCredentialsProvider;
@@ -50,13 +51,8 @@ public class AWSUtil {
 		awsClientConfig.setUserAgent("Apache Flink " + EnvironmentInformation.getVersion() +
 			" (" + EnvironmentInformation.getRevisionInformation().commitId + ") Kinesis Connector");
 
-		AmazonKinesisClient client;
-		if (AWSUtil.getCredentialsProvider(configProps) != null) {
-			client = new AmazonKinesisClient(
-				AWSUtil.getCredentialsProvider(configProps).getCredentials(), awsClientConfig);
-		} else {
-			client = new AmazonKinesisClient(awsClientConfig);
-		}
+		AmazonKinesisClient client = new AmazonKinesisClient(
+			AWSUtil.getCredentialsProvider(configProps).getCredentials(), awsClientConfig);
 
 		client.setRegion(Region.getRegion(Regions.fromName(configProps.getProperty(AWSConfigConstants.AWS_REGION))));
 		if (configProps.containsKey(AWSConfigConstants.AWS_ENDPOINT)) {
@@ -94,7 +90,7 @@ public class AWSUtil {
 					: new ProfileCredentialsProvider(profileConfigPath, profileName);
 				break;
 			case AUTO:
-				credentialsProvider = null;
+				credentialsProvider = new DefaultAWSCredentialsProviderChain();
 				break;
 			default:
 			case BASIC:


[22/50] [abbrv] flink git commit: [FLINK-4384] [rpc] Add "scheduleRunAsync()" to the RpcEndpoint

Posted by tr...@apache.org.
[FLINK-4384] [rpc] Add "scheduleRunAsync()" to the RpcEndpoint

This closes #2360


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

Branch: refs/heads/flip-6
Commit: 2bf672708b7b39e5a29b207547abf8c7dff46fe2
Parents: 67e6f57
Author: Stephan Ewen <se...@apache.org>
Authored: Thu Aug 11 19:10:48 2016 +0200
Committer: Till Rohrmann <tr...@apache.org>
Committed: Thu Sep 8 17:26:56 2016 +0200

----------------------------------------------------------------------
 .../flink/runtime/rpc/MainThreadExecutor.java   |   9 +
 .../apache/flink/runtime/rpc/RpcEndpoint.java   |  12 ++
 .../runtime/rpc/akka/AkkaInvocationHandler.java |  13 +-
 .../flink/runtime/rpc/akka/AkkaRpcActor.java    |  15 +-
 .../runtime/rpc/akka/messages/RunAsync.java     |  24 ++-
 .../runtime/rpc/akka/AkkaRpcServiceTest.java    |   3 +
 .../flink/runtime/rpc/akka/AsyncCallsTest.java  | 216 +++++++++++++++++++
 7 files changed, 286 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/2bf67270/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/MainThreadExecutor.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/MainThreadExecutor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/MainThreadExecutor.java
index 882c1b7..4efb382 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/MainThreadExecutor.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/MainThreadExecutor.java
@@ -52,4 +52,13 @@ public interface MainThreadExecutor {
 	 * @return Future of the callable result
 	 */
 	<V> Future<V> callAsync(Callable<V> callable, Timeout callTimeout);
+
+	/**
+	 * Execute the runnable in the main thread of the underlying RPC endpoint, with
+	 * a delay of the given number of milliseconds.
+	 *
+	 * @param runnable Runnable to be executed
+	 * @param delay    The delay, in milliseconds, after which the runnable will be executed
+	 */
+	void scheduleRunAsync(Runnable runnable, long delay);
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/2bf67270/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java
index aef0803..44933d5 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/RpcEndpoint.java
@@ -28,6 +28,7 @@ import scala.concurrent.ExecutionContext;
 import scala.concurrent.Future;
 
 import java.util.concurrent.Callable;
+import java.util.concurrent.TimeUnit;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
@@ -168,6 +169,17 @@ public abstract class RpcEndpoint<C extends RpcGateway> {
 	}
 
 	/**
+	 * Execute the runnable in the main thread of the underlying RPC endpoint, with
+	 * a delay of the given number of milliseconds.
+	 *
+	 * @param runnable Runnable to be executed
+	 * @param delay    The delay after which the runnable will be executed
+	 */
+	public void scheduleRunAsync(Runnable runnable, long delay, TimeUnit unit) {
+		((MainThreadExecutor) self).scheduleRunAsync(runnable, unit.toMillis(delay));
+	}
+
+	/**
 	 * Execute the callable in the main thread of the underlying RPC service, returning a future for
 	 * the result of the callable. If the callable is not completed within the given timeout, then
 	 * the future will be failed with a {@link java.util.concurrent.TimeoutException}.

http://git-wip-us.apache.org/repos/asf/flink/blob/2bf67270/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaInvocationHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaInvocationHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaInvocationHandler.java
index e8e383a..580b161 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaInvocationHandler.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaInvocationHandler.java
@@ -38,6 +38,9 @@ import java.lang.reflect.Method;
 import java.util.BitSet;
 import java.util.concurrent.Callable;
 
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkArgument;
+
 /**
  * Invocation handler to be used with a {@link AkkaRpcActor}. The invocation handler wraps the
  * rpc in a {@link RpcInvocation} message and then sends it to the {@link AkkaRpcActor} where it is
@@ -106,9 +109,17 @@ class AkkaInvocationHandler implements InvocationHandler, AkkaGateway, MainThrea
 
 	@Override
 	public void runAsync(Runnable runnable) {
+		scheduleRunAsync(runnable, 0);
+	}
+
+	@Override
+	public void scheduleRunAsync(Runnable runnable, long delay) {
+		checkNotNull(runnable, "runnable");
+		checkArgument(delay >= 0, "delay must be zero or greater");
+		
 		// Unfortunately I couldn't find a way to allow only local communication. Therefore, the
 		// runnable field is transient transient
-		rpcServer.tell(new RunAsync(runnable), ActorRef.noSender());
+		rpcServer.tell(new RunAsync(runnable, delay), ActorRef.noSender());
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/2bf67270/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java
index 57da38a..18ccf1b 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/AkkaRpcActor.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.runtime.rpc.akka;
 
+import akka.actor.ActorRef;
 import akka.actor.Status;
 import akka.actor.UntypedActor;
 import akka.pattern.Patterns;
@@ -30,9 +31,11 @@ import org.apache.flink.util.Preconditions;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import scala.concurrent.Future;
+import scala.concurrent.duration.FiniteDuration;
 
 import java.lang.reflect.Method;
 import java.util.concurrent.Callable;
+import java.util.concurrent.TimeUnit;
 
 /**
  * Akka rpc actor which receives {@link RpcInvocation}, {@link RunAsync} and {@link CallAsync}
@@ -152,13 +155,23 @@ class AkkaRpcActor<C extends RpcGateway, T extends RpcEndpoint<C>> extends Untyp
 				"{} is only supported with local communication.",
 				runAsync.getClass().getName(),
 				runAsync.getClass().getName());
-		} else {
+		}
+		else if (runAsync.getDelay() == 0) {
+			// run immediately
 			try {
 				runAsync.getRunnable().run();
 			} catch (final Throwable e) {
 				LOG.error("Caught exception while executing runnable in main thread.", e);
 			}
 		}
+		else {
+			// schedule for later. send a new message after the delay, which will then be immediately executed 
+			FiniteDuration delay = new FiniteDuration(runAsync.getDelay(), TimeUnit.MILLISECONDS);
+			RunAsync message = new RunAsync(runAsync.getRunnable(), 0);
+
+			getContext().system().scheduler().scheduleOnce(delay, getSelf(), message,
+					getContext().dispatcher(), ActorRef.noSender());
+		}
 	}
 
 	/**

http://git-wip-us.apache.org/repos/asf/flink/blob/2bf67270/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RunAsync.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RunAsync.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RunAsync.java
index fb95852..c18906c 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RunAsync.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rpc/akka/messages/RunAsync.java
@@ -18,23 +18,39 @@
 
 package org.apache.flink.runtime.rpc.akka.messages;
 
-import org.apache.flink.util.Preconditions;
-
 import java.io.Serializable;
 
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkArgument;
+
 /**
  * Message for asynchronous runnable invocations
  */
 public final class RunAsync implements Serializable {
 	private static final long serialVersionUID = -3080595100695371036L;
 
+	/** The runnable to be executed. Transient, so it gets lost upon serialization */ 
 	private final transient Runnable runnable;
 
-	public RunAsync(Runnable runnable) {
-		this.runnable = Preconditions.checkNotNull(runnable);
+	/** The delay after which the runnable should be called */
+	private final long delay;
+
+	/**
+	 * 
+	 * @param runnable  The Runnable to run.
+	 * @param delay     The delay in milliseconds. Zero indicates immediate execution.
+	 */
+	public RunAsync(Runnable runnable, long delay) {
+		checkArgument(delay >= 0);
+		this.runnable = checkNotNull(runnable);
+		this.delay = delay;
 	}
 
 	public Runnable getRunnable() {
 		return runnable;
 	}
+
+	public long getDelay() {
+		return delay;
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/2bf67270/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java
index a4e1d7f..5e37e10 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AkkaRpcServiceTest.java
@@ -21,6 +21,9 @@ package org.apache.flink.runtime.rpc.akka;
 import akka.actor.ActorSystem;
 import akka.util.Timeout;
 import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.rpc.RpcEndpoint;
+import org.apache.flink.runtime.rpc.RpcGateway;
+import org.apache.flink.runtime.rpc.RpcService;
 import org.apache.flink.runtime.rpc.jobmaster.JobMaster;
 import org.apache.flink.runtime.rpc.resourcemanager.ResourceManagerGateway;
 import org.apache.flink.runtime.rpc.resourcemanager.ResourceManager;

http://git-wip-us.apache.org/repos/asf/flink/blob/2bf67270/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AsyncCallsTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AsyncCallsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AsyncCallsTest.java
new file mode 100644
index 0000000..f2ce52d
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rpc/akka/AsyncCallsTest.java
@@ -0,0 +1,216 @@
+/*
+ * 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.rpc.akka;
+
+import akka.actor.ActorSystem;
+import akka.util.Timeout;
+
+import org.apache.flink.core.testutils.OneShotLatch;
+import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.rpc.RpcEndpoint;
+import org.apache.flink.runtime.rpc.RpcGateway;
+import org.apache.flink.runtime.rpc.RpcMethod;
+import org.apache.flink.runtime.rpc.RpcService;
+
+import org.junit.AfterClass;
+import org.junit.Test;
+
+import scala.concurrent.Await;
+import scala.concurrent.Future;
+import scala.concurrent.duration.FiniteDuration;
+
+import java.util.concurrent.Callable;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.ReentrantLock;
+
+import static org.junit.Assert.*;
+
+public class AsyncCallsTest {
+
+	// ------------------------------------------------------------------------
+	//  shared test members
+	// ------------------------------------------------------------------------
+
+	private static ActorSystem actorSystem = AkkaUtils.createDefaultActorSystem();
+
+	private static AkkaRpcService akkaRpcService = 
+			new AkkaRpcService(actorSystem, new Timeout(10000, TimeUnit.MILLISECONDS));
+
+	@AfterClass
+	public static void shutdown() {
+		akkaRpcService.stopService();
+		actorSystem.shutdown();
+	}
+
+
+	// ------------------------------------------------------------------------
+	//  tests
+	// ------------------------------------------------------------------------
+
+	@Test
+	public void testScheduleWithNoDelay() throws Exception {
+
+		// to collect all the thread references
+		final ReentrantLock lock = new ReentrantLock();
+		final AtomicBoolean concurrentAccess = new AtomicBoolean(false);
+
+		TestEndpoint testEndpoint = new TestEndpoint(akkaRpcService, lock);
+		TestGateway gateway = testEndpoint.getSelf();
+
+		// a bunch of gateway calls
+		gateway.someCall();
+		gateway.anotherCall();
+		gateway.someCall();
+
+		// run something asynchronously
+		for (int i = 0; i < 10000; i++) {
+			testEndpoint.runAsync(new Runnable() {
+				@Override
+				public void run() {
+					boolean holdsLock = lock.tryLock();
+					if (holdsLock) {
+						lock.unlock();
+					} else {
+						concurrentAccess.set(true);
+					}
+				}
+			});
+		}
+	
+		Future<String> result = testEndpoint.callAsync(new Callable<String>() {
+			@Override
+			public String call() throws Exception {
+				boolean holdsLock = lock.tryLock();
+				if (holdsLock) {
+					lock.unlock();
+				} else {
+					concurrentAccess.set(true);
+				}
+				return "test";
+			}
+		}, new Timeout(30, TimeUnit.SECONDS));
+		String str = Await.result(result, new FiniteDuration(30, TimeUnit.SECONDS));
+		assertEquals("test", str);
+
+		// validate that no concurrent access happened
+		assertFalse("Rpc Endpoint had concurrent access", testEndpoint.hasConcurrentAccess());
+		assertFalse("Rpc Endpoint had concurrent access", concurrentAccess.get());
+
+		akkaRpcService.stopServer(testEndpoint.getSelf());
+	}
+
+	@Test
+	public void testScheduleWithDelay() throws Exception {
+
+		// to collect all the thread references
+		final ReentrantLock lock = new ReentrantLock();
+		final AtomicBoolean concurrentAccess = new AtomicBoolean(false);
+		final OneShotLatch latch = new OneShotLatch();
+
+		final long delay = 200;
+
+		TestEndpoint testEndpoint = new TestEndpoint(akkaRpcService, lock);
+
+		// run something asynchronously
+		testEndpoint.runAsync(new Runnable() {
+			@Override
+			public void run() {
+				boolean holdsLock = lock.tryLock();
+				if (holdsLock) {
+					lock.unlock();
+				} else {
+					concurrentAccess.set(true);
+				}
+			}
+		});
+
+		final long start = System.nanoTime();
+
+		testEndpoint.scheduleRunAsync(new Runnable() {
+			@Override
+			public void run() {
+				boolean holdsLock = lock.tryLock();
+				if (holdsLock) {
+					lock.unlock();
+				} else {
+					concurrentAccess.set(true);
+				}
+				latch.trigger();
+			}
+		}, delay, TimeUnit.MILLISECONDS);
+
+		latch.await();
+		final long stop = System.nanoTime();
+
+		// validate that no concurrent access happened
+		assertFalse("Rpc Endpoint had concurrent access", testEndpoint.hasConcurrentAccess());
+		assertFalse("Rpc Endpoint had concurrent access", concurrentAccess.get());
+
+		assertTrue("call was not properly delayed", ((stop - start) / 1000000) >= delay);
+	}
+
+	// ------------------------------------------------------------------------
+	//  test RPC endpoint
+	// ------------------------------------------------------------------------
+	
+	interface TestGateway extends RpcGateway {
+
+		void someCall();
+
+		void anotherCall();
+	}
+
+	@SuppressWarnings("unused")
+	public static class TestEndpoint extends RpcEndpoint<TestGateway> {
+
+		private final ReentrantLock lock;
+
+		private volatile boolean concurrentAccess;
+
+		public TestEndpoint(RpcService rpcService, ReentrantLock lock) {
+			super(rpcService);
+			this.lock = lock;
+		}
+
+		@RpcMethod
+		public void someCall() {
+			boolean holdsLock = lock.tryLock();
+			if (holdsLock) {
+				lock.unlock();
+			} else {
+				concurrentAccess = true;
+			}
+		}
+
+		@RpcMethod
+		public void anotherCall() {
+			boolean holdsLock = lock.tryLock();
+			if (holdsLock) {
+				lock.unlock();
+			} else {
+				concurrentAccess = true;
+			}
+		}
+
+		public boolean hasConcurrentAccess() {
+			return concurrentAccess;
+		}
+	}
+}