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

[1/9] flink git commit: [FLINK-1984] Mesos ResourceManager - T1 milestone

Repository: flink
Updated Branches:
  refs/heads/master 578e80e3c -> 842e3e7d1


http://git-wip-us.apache.org/repos/asf/flink/blob/d9b2be05/flink-mesos/src/test/scala/org/apache/flink/runtime/akka/FSMSpec.scala
----------------------------------------------------------------------
diff --git a/flink-mesos/src/test/scala/org/apache/flink/runtime/akka/FSMSpec.scala b/flink-mesos/src/test/scala/org/apache/flink/runtime/akka/FSMSpec.scala
new file mode 100644
index 0000000..b33e9aa
--- /dev/null
+++ b/flink-mesos/src/test/scala/org/apache/flink/runtime/akka/FSMSpec.scala
@@ -0,0 +1,40 @@
+package org.apache.flink.runtime.akka
+
+import java.util
+
+import org.mockito.ArgumentMatcher
+import org.scalatest.WordSpecLike
+
+import scala.collection.JavaConverters._
+
+/**
+  * Extends wordspec with FSM functionality.
+  */
+abstract class FSMSpec extends FSMSpecLike {
+}
+
+/**
+  * Implementation trait for class <code>FSMSpec</code>, which extends wordspec with FSM functionality.
+  *
+  * For example: "MyFSM" when inState {
+  *   "Connected" should handle {
+  *     "Disconnect" which {
+  *       "transitions to Disconnected" in (pending)
+  *     }
+  *   }
+  * }
+  *
+  */
+abstract trait FSMSpecLike extends WordSpecLike {
+  /**
+    * After word to describe the states that an FSM may be in.
+    */
+  def inState = afterWord("in state")
+
+  /**
+    * After word to describe the events that an FSM may handle in a given state.
+    * @return
+    */
+  def handle = afterWord("handle")
+}
+

http://git-wip-us.apache.org/repos/asf/flink/blob/d9b2be05/flink-runtime/src/main/java/org/apache/flink/runtime/util/ZooKeeperUtils.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/ZooKeeperUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/ZooKeeperUtils.java
index bb48c81..b585fe6 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/ZooKeeperUtils.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/ZooKeeperUtils.java
@@ -313,7 +313,7 @@ public class ZooKeeperUtils {
 	 * @return {@link FileSystemStateStorageHelper} instance
 	 * @throws IOException
 	 */
-	private static <T extends Serializable> FileSystemStateStorageHelper<T> createFileSystemStateStorage(
+	public static <T extends Serializable> FileSystemStateStorageHelper<T> createFileSystemStateStorage(
 			Configuration configuration,
 			String prefix) throws IOException {
 

http://git-wip-us.apache.org/repos/asf/flink/blob/d9b2be05/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 0e28d98..b706a1a 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
@@ -449,6 +449,7 @@ class JobManager(
 
     case msg: ResourceRemoved =>
       // we're being informed by the resource manager that a resource has become unavailable
+      // note: a Terminated event may already have removed the instance.
       val resourceID = msg.resourceId()
       log.debug(s"Resource has been removed: $resourceID")
 
@@ -1054,7 +1055,7 @@ class JobManager(
 
   /**
     * Handler to be executed when a task manager terminates.
-    * (Akka Deathwatch or notifiction from ResourceManager)
+    * (Akka Deathwatch or notification from ResourceManager)
     *
     * @param taskManager The ActorRef of the taskManager
     */

http://git-wip-us.apache.org/repos/asf/flink/blob/d9b2be05/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 0e8c3b7..c76b15c 100644
--- a/pom.xml
+++ b/pom.xml
@@ -74,6 +74,7 @@ under the License.
 		<module>flink-quickstart</module>
 		<module>flink-contrib</module>
 		<module>flink-dist</module>
+		<module>flink-mesos</module>
 		<module>flink-metrics</module>
 	</modules>
 


[6/9] flink git commit: [FLINK-1984] Mesos ResourceManager - T1 milstone (3)

Posted by mx...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/59eeea4c/flink-mesos/src/main/scala/org/apache/flink/runtime/clusterframework/ContaineredJobManager.scala
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/scala/org/apache/flink/runtime/clusterframework/ContaineredJobManager.scala b/flink-mesos/src/main/scala/org/apache/flink/runtime/clusterframework/ContaineredJobManager.scala
deleted file mode 100644
index 49c86b5..0000000
--- a/flink-mesos/src/main/scala/org/apache/flink/runtime/clusterframework/ContaineredJobManager.scala
+++ /dev/null
@@ -1,174 +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 java.util.concurrent.{TimeUnit, ExecutorService}
-
-import akka.actor.ActorRef
-
-import org.apache.flink.api.common.JobID
-import org.apache.flink.configuration.{Configuration => FlinkConfiguration, ConfigConstants}
-import org.apache.flink.runtime.checkpoint.savepoint.SavepointStore
-import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory
-import org.apache.flink.runtime.clusterframework.ApplicationStatus
-import org.apache.flink.runtime.executiongraph.restart.RestartStrategyFactory
-import org.apache.flink.runtime.clusterframework.messages._
-import org.apache.flink.runtime.jobgraph.JobStatus
-import org.apache.flink.runtime.jobmanager.{SubmittedJobGraphStore, JobManager}
-import org.apache.flink.runtime.leaderelection.LeaderElectionService
-import org.apache.flink.runtime.messages.JobManagerMessages.{RequestJobStatus, CurrentJobStatus, JobNotFound}
-import org.apache.flink.runtime.messages.Messages.Acknowledge
-import org.apache.flink.runtime.metrics.{MetricRegistry => FlinkMetricRegistry}
-import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager
-import org.apache.flink.runtime.instance.InstanceManager
-import org.apache.flink.runtime.jobmanager.scheduler.{Scheduler => FlinkScheduler}
-
-import scala.concurrent.duration._
-import scala.language.postfixOps
-
-
-/** JobManager actor for execution on Yarn or Mesos. It enriches the [[JobManager]] with additional messages
-  * to start/administer/stop the session.
-  *
-  * @param flinkConfiguration Configuration object for the actor
-  * @param executorService Execution context which is used to execute concurrent tasks in the
-  *                         [[org.apache.flink.runtime.executiongraph.ExecutionGraph]]
-  * @param instanceManager Instance manager to manage the registered
-  *                        [[org.apache.flink.runtime.taskmanager.TaskManager]]
-  * @param scheduler Scheduler to schedule Flink jobs
-  * @param libraryCacheManager Manager to manage uploaded jar files
-  * @param archive Archive for finished Flink jobs
-  * @param restartStrategyFactory Restart strategy to be used in case of a job recovery
-  * @param timeout Timeout for futures
-  * @param leaderElectionService LeaderElectionService to participate in the leader election
-  */
-abstract class ContaineredJobManager(
-                      flinkConfiguration: FlinkConfiguration,
-                      executorService: ExecutorService,
-                      instanceManager: InstanceManager,
-                      scheduler: FlinkScheduler,
-                      libraryCacheManager: BlobLibraryCacheManager,
-                      archive: ActorRef,
-                      restartStrategyFactory: RestartStrategyFactory,
-                      timeout: FiniteDuration,
-                      leaderElectionService: LeaderElectionService,
-                      submittedJobGraphs : SubmittedJobGraphStore,
-                      checkpointRecoveryFactory : CheckpointRecoveryFactory,
-                      savepointStore: SavepointStore,
-                      jobRecoveryTimeout: FiniteDuration,
-                      metricsRegistry: Option[FlinkMetricRegistry])
-  extends JobManager(
-    flinkConfiguration,
-    executorService,
-    instanceManager,
-    scheduler,
-    libraryCacheManager,
-    archive,
-    restartStrategyFactory,
-    timeout,
-    leaderElectionService,
-    submittedJobGraphs,
-    checkpointRecoveryFactory,
-    savepointStore,
-    jobRecoveryTimeout,
-    metricsRegistry) {
-
-  val jobPollingInterval: FiniteDuration
-
-  // indicates if this JM has been started in a dedicated (per-job) mode.
-  var stopWhenJobFinished: JobID = null
-
-  override def handleMessage: Receive = {
-    handleContainerMessage orElse super.handleMessage
-  }
-
-  def handleContainerMessage: Receive = {
-
-    case msg @ (_: RegisterInfoMessageListener | _: UnRegisterInfoMessageListener) =>
-      // forward to ResourceManager
-      currentResourceManager match {
-        case Some(rm) =>
-          // we forward the message
-          rm.forward(decorateMessage(msg))
-        case None =>
-        // client has to try again
-      }
-
-    case msg: ShutdownClusterAfterJob =>
-      val jobId = msg.jobId()
-      log.info(s"ApplicationMaster will shut down session when job $jobId has finished.")
-      stopWhenJobFinished = jobId
-      // trigger regular job status messages (if this is a dedicated/per-job cluster)
-      if (stopWhenJobFinished != null) {
-        context.system.scheduler.schedule(0 seconds,
-          jobPollingInterval,
-          new Runnable {
-            override def run(): Unit = {
-              self ! decorateMessage(RequestJobStatus(stopWhenJobFinished))
-            }
-          }
-        )(context.dispatcher)
-      }
-
-      sender() ! decorateMessage(Acknowledge)
-
-    case msg: GetClusterStatus =>
-      sender() ! decorateMessage(
-        new GetClusterStatusResponse(
-          instanceManager.getNumberOfRegisteredTaskManagers,
-          instanceManager.getTotalNumberOfSlots)
-      )
-
-    case jnf: JobNotFound =>
-      log.debug(s"Job with ID ${jnf.jobID} not found in JobManager")
-      if (stopWhenJobFinished == null) {
-        log.warn("The ApplicationMaster didn't expect to receive this message")
-      }
-
-    case jobStatus: CurrentJobStatus =>
-      if (stopWhenJobFinished == null) {
-        log.warn(s"Received job status $jobStatus which wasn't requested.")
-      } else {
-        if (stopWhenJobFinished != jobStatus.jobID) {
-          log.warn(s"Received job status for job ${jobStatus.jobID} but expected status for " +
-            s"job $stopWhenJobFinished")
-        } else {
-          if (jobStatus.status.isTerminalState) {
-            log.info(s"Job with ID ${jobStatus.jobID} is in terminal state ${jobStatus.status}. " +
-              s"Shutting down session")
-            if (jobStatus.status == JobStatus.FINISHED) {
-              self ! decorateMessage(
-                new StopCluster(
-                  ApplicationStatus.SUCCEEDED,
-                  s"The monitored job with ID ${jobStatus.jobID} has finished.")
-              )
-            } else {
-              self ! decorateMessage(
-                new StopCluster(
-                  ApplicationStatus.FAILED,
-                  s"The monitored job with ID ${jobStatus.jobID} has failed to complete.")
-              )
-            }
-          } else {
-            log.debug(s"Monitored job with ID ${jobStatus.jobID} is in state ${jobStatus.status}")
-          }
-        }
-      }
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/59eeea4c/flink-mesos/src/test/java/org/apache/flink/mesos/runtime/clusterframework/MesosFlinkResourceManagerTest.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/test/java/org/apache/flink/mesos/runtime/clusterframework/MesosFlinkResourceManagerTest.java b/flink-mesos/src/test/java/org/apache/flink/mesos/runtime/clusterframework/MesosFlinkResourceManagerTest.java
index 05fb033..f287e13 100644
--- a/flink-mesos/src/test/java/org/apache/flink/mesos/runtime/clusterframework/MesosFlinkResourceManagerTest.java
+++ b/flink-mesos/src/test/java/org/apache/flink/mesos/runtime/clusterframework/MesosFlinkResourceManagerTest.java
@@ -235,9 +235,9 @@ public class MesosFlinkResourceManagerTest {
 				protected void run() {
 					try {
 						// set the initial persistent state then initialize the RM
-						MesosWorkerStore.Worker worker1 = MesosWorkerStore.Worker.newTask(task1);
-						MesosWorkerStore.Worker worker2 = MesosWorkerStore.Worker.newTask(task2).launchTask(slave1, slave1host);
-						MesosWorkerStore.Worker worker3 = MesosWorkerStore.Worker.newTask(task3).launchTask(slave1, slave1host).releaseTask();
+						MesosWorkerStore.Worker worker1 = MesosWorkerStore.Worker.newWorker(task1);
+						MesosWorkerStore.Worker worker2 = MesosWorkerStore.Worker.newWorker(task2).launchWorker(slave1, slave1host);
+						MesosWorkerStore.Worker worker3 = MesosWorkerStore.Worker.newWorker(task3).launchWorker(slave1, slave1host).releaseWorker();
 						when(workerStore.getFrameworkID()).thenReturn(Option.apply(framework1));
 						when(workerStore.recoverWorkers()).thenReturn(Arrays.asList(worker1, worker2, worker3));
 						initialize();
@@ -276,7 +276,7 @@ public class MesosFlinkResourceManagerTest {
 				protected void run() {
 					try {
 						// set the initial persistent state then initialize the RM
-						MesosWorkerStore.Worker worker1launched = MesosWorkerStore.Worker.newTask(task1).launchTask(slave1, slave1host);
+						MesosWorkerStore.Worker worker1launched = MesosWorkerStore.Worker.newWorker(task1).launchWorker(slave1, slave1host);
 						when(workerStore.getFrameworkID()).thenReturn(Option.apply(framework1));
 						when(workerStore.recoverWorkers()).thenReturn(singletonList(worker1launched));
 						initialize();
@@ -306,7 +306,7 @@ public class MesosFlinkResourceManagerTest {
 				protected void run() {
 					try {
 						// set the initial state with a (recovered) launched worker
-						MesosWorkerStore.Worker worker1launched = MesosWorkerStore.Worker.newTask(task1).launchTask(slave1, slave1host);
+						MesosWorkerStore.Worker worker1launched = MesosWorkerStore.Worker.newWorker(task1).launchWorker(slave1, slave1host);
 						when(workerStore.getFrameworkID()).thenReturn(Option.apply(framework1));
 						when(workerStore.recoverWorkers()).thenReturn(singletonList(worker1launched));
 						initialize();
@@ -339,7 +339,7 @@ public class MesosFlinkResourceManagerTest {
 				protected void run() {
 					try {
 						// set the initial persistent state, initialize the RM, then register with task1 as a registered worker
-						MesosWorkerStore.Worker worker1 = MesosWorkerStore.Worker.newTask(task1).launchTask(slave1, slave1host);
+						MesosWorkerStore.Worker worker1 = MesosWorkerStore.Worker.newWorker(task1).launchWorker(slave1, slave1host);
 						when(workerStore.getFrameworkID()).thenReturn(Option.apply(framework1));
 						when(workerStore.recoverWorkers()).thenReturn(singletonList(worker1));
 						initialize();
@@ -351,12 +351,13 @@ public class MesosFlinkResourceManagerTest {
 
 						// verify that the worker was persisted, the internal state was updated, the task router was notified,
 						// and the launch coordinator was notified about the host assignment change
-						MesosWorkerStore.Worker worker2Released = worker1.releaseTask();
+						MesosWorkerStore.Worker worker2Released = worker1.releaseWorker();
 						verify(workerStore).putWorker(worker2Released);
 						assertThat(resourceManagerInstance.workersBeingReturned, hasEntry(extractResourceID(task1), worker2Released));
 						resourceManagerInstance.launchCoordinator.expectMsg(new LaunchCoordinator.Unassign(task1, slave1host));
 
 						// send the subsequent terminated message
+						when(workerStore.removeWorker(task1)).thenReturn(true);
 						resourceManager.tell(new TaskMonitor.TaskTerminated(task1, Protos.TaskStatus.newBuilder()
 							.setTaskId(task1).setSlaveId(slave1).setState(Protos.TaskState.TASK_FINISHED).build()));
 
@@ -391,7 +392,7 @@ public class MesosFlinkResourceManagerTest {
 
 						// verify that a new worker was persisted, the internal state was updated, the task router was notified,
 						// and the launch coordinator was asked to launch a task
-						MesosWorkerStore.Worker expected = MesosWorkerStore.Worker.newTask(task1);
+						MesosWorkerStore.Worker expected = MesosWorkerStore.Worker.newWorker(task1);
 						verify(workerStore).putWorker(expected);
 						assertThat(resourceManagerInstance.workersInNew, hasEntry(extractResourceID(task1), expected));
 						resourceManagerInstance.taskRouter.expectMsgClass(TaskMonitor.TaskGoalStateUpdated.class);
@@ -438,7 +439,7 @@ public class MesosFlinkResourceManagerTest {
 				protected void run() {
 					try {
 						// set the initial persistent state with a new task then initialize the RM
-						MesosWorkerStore.Worker worker1 = MesosWorkerStore.Worker.newTask(task1);
+						MesosWorkerStore.Worker worker1 = MesosWorkerStore.Worker.newWorker(task1);
 						when(workerStore.getFrameworkID()).thenReturn(Option.apply(framework1));
 						when(workerStore.recoverWorkers()).thenReturn(singletonList(worker1));
 						initialize();
@@ -455,7 +456,7 @@ public class MesosFlinkResourceManagerTest {
 
 						// verify that the worker was persisted, the internal state was updated,
 						// Mesos was asked to launch task1, and the task router was notified
-						MesosWorkerStore.Worker worker1launched = worker1.launchTask(slave1, slave1host);
+						MesosWorkerStore.Worker worker1launched = worker1.launchWorker(slave1, slave1host);
 						verify(workerStore).putWorker(worker1launched);
 						assertThat(resourceManagerInstance.workersInNew.entrySet(), empty());
 						assertThat(resourceManagerInstance.workersInLaunch, hasEntry(extractResourceID(task1), worker1launched));
@@ -505,7 +506,7 @@ public class MesosFlinkResourceManagerTest {
 				protected void run() {
 					try {
 						// set the initial persistent state with a launched worker that hasn't yet registered
-						MesosWorkerStore.Worker worker1launched = MesosWorkerStore.Worker.newTask(task1).launchTask(slave1, slave1host);
+						MesosWorkerStore.Worker worker1launched = MesosWorkerStore.Worker.newWorker(task1).launchWorker(slave1, slave1host);
 						when(workerStore.getFrameworkID()).thenReturn(Option.apply(framework1));
 						when(workerStore.recoverWorkers()).thenReturn(singletonList(worker1launched));
 						initialize();
@@ -513,6 +514,7 @@ public class MesosFlinkResourceManagerTest {
 
 						// tell the RM that a task failed (and prepare a replacement task)
 						when(workerStore.newTaskID()).thenReturn(task2);
+						when(workerStore.removeWorker(task1)).thenReturn(true);
 						resourceManager.tell(new SetWorkerPoolSize(1), jobManager);
 						resourceManager.tell(new TaskMonitor.TaskTerminated(task1, Protos.TaskStatus.newBuilder()
 							.setTaskId(task1).setSlaveId(slave1).setState(Protos.TaskState.TASK_FAILED).build()));
@@ -540,7 +542,7 @@ public class MesosFlinkResourceManagerTest {
 				protected void run() {
 					try {
 						// set the initial persistent state with a launched & registered worker
-						MesosWorkerStore.Worker worker1launched = MesosWorkerStore.Worker.newTask(task1).launchTask(slave1, slave1host);
+						MesosWorkerStore.Worker worker1launched = MesosWorkerStore.Worker.newWorker(task1).launchWorker(slave1, slave1host);
 						when(workerStore.getFrameworkID()).thenReturn(Option.apply(framework1));
 						when(workerStore.recoverWorkers()).thenReturn(singletonList(worker1launched));
 						initialize();
@@ -548,6 +550,7 @@ public class MesosFlinkResourceManagerTest {
 
 						// tell the RM that a task failed (and prepare a replacement task)
 						when(workerStore.newTaskID()).thenReturn(task2);
+						when(workerStore.removeWorker(task1)).thenReturn(true);
 						resourceManager.tell(new SetWorkerPoolSize(1), jobManager);
 						resourceManager.tell(new TaskMonitor.TaskTerminated(task1, Protos.TaskStatus.newBuilder()
 							.setTaskId(task1).setSlaveId(slave1).setState(Protos.TaskState.TASK_FAILED).build()));
@@ -582,7 +585,7 @@ public class MesosFlinkResourceManagerTest {
 
 						// verify that the Mesos framework is shutdown
 						verify(schedulerDriver).stop(false);
-						verify(workerStore).cleanup();
+						verify(workerStore).stop(true);
 						expectTerminated(resourceManager.actor());
 					}
 					catch(Exception ex) {

http://git-wip-us.apache.org/repos/asf/flink/blob/59eeea4c/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 3ab72cd..80186cf 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
@@ -31,7 +31,8 @@ object Matchers {
   def contentsMatch[T](plan: Seq[T]): java.util.Collection[T] = {
     org.mockito.Matchers.argThat(new ArgumentMatcher[java.util.Collection[T]] {
       override def matches(o: scala.Any): Boolean = o match {
-        case actual: java.util.Collection[T] => actual.size() == plan.size && actual.containsAll(plan.asJava)
+        case actual: java.util.Collection[T] =>
+          actual.size() == plan.size && actual.containsAll(plan.asJava)
         case _ => false
       }
     })
@@ -46,7 +47,8 @@ object TestFSMUtils {
     "$" + akka.util.Helpers.base64(l)
   }
 
-  def testFSMRef[S, D, T <: Actor: ClassTag](factory: \u21d2 T, supervisor: ActorRef)(implicit ev: T <:< FSM[S, D], system: ActorSystem): TestFSMRef[S, D, T] = {
+  def testFSMRef[S, D, T <: Actor: ClassTag](factory: \u21d2 T, supervisor: ActorRef)
+      (implicit ev: T <:< FSM[S, D], system: ActorSystem): TestFSMRef[S, D, T] = {
     new TestFSMRef(system, Props(factory), supervisor, TestFSMUtils.randomName)
   }
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/59eeea4c/flink-mesos/src/test/scala/org/apache/flink/mesos/scheduler/LaunchCoordinatorTest.scala
----------------------------------------------------------------------
diff --git a/flink-mesos/src/test/scala/org/apache/flink/mesos/scheduler/LaunchCoordinatorTest.scala b/flink-mesos/src/test/scala/org/apache/flink/mesos/scheduler/LaunchCoordinatorTest.scala
index be7d788..34c1f66 100644
--- a/flink-mesos/src/test/scala/org/apache/flink/mesos/scheduler/LaunchCoordinatorTest.scala
+++ b/flink-mesos/src/test/scala/org/apache/flink/mesos/scheduler/LaunchCoordinatorTest.scala
@@ -93,7 +93,9 @@ class LaunchCoordinatorTest
 
     val task: LaunchableTask = new LaunchableTask() {
       override def taskRequest: TaskRequest = generateTaskRequest
-      override def launch(slaveId: SlaveID, taskAssignment: TaskAssignmentResult): Protos.TaskInfo = {
+      override def launch(
+          slaveId: SlaveID,
+          taskAssignment: TaskAssignmentResult): Protos.TaskInfo = {
         Protos.TaskInfo.newBuilder
           .setTaskId(taskID).setName(taskID.getValue)
           .setCommand(Protos.CommandInfo.newBuilder.setValue("whoami"))
@@ -135,11 +137,12 @@ class LaunchCoordinatorTest
     */
   def taskAssignmentResult(lease: VirtualMachineLease, task: TaskRequest): TaskAssignmentResult = {
     val ports = lease.portRanges().get(0)
+    val assignedPorts = ports.getBeg to ports.getBeg + task.getPorts
     val r = mock(classOf[TaskAssignmentResult])
     when(r.getTaskId).thenReturn(task.getId)
     when(r.getHostname).thenReturn(lease.hostname())
     when(r.getAssignedPorts).thenReturn(
-      (ports.getBeg to ports.getBeg + task.getPorts).toList.asJava.asInstanceOf[java.util.List[Integer]])
+      assignedPorts.toList.asJava.asInstanceOf[java.util.List[Integer]])
     when(r.getRequest).thenReturn(task)
     when(r.isSuccessful).thenReturn(true)
     when(r.getFitness).thenReturn(1.0)
@@ -196,7 +199,8 @@ class LaunchCoordinatorTest
     */
   def taskSchedulerBuilder(optimizer: TaskScheduler) = new TaskSchedulerBuilder {
     var leaseRejectAction: Action1[VirtualMachineLease] = null
-    override def withLeaseRejectAction(action: Action1[VirtualMachineLease]): TaskSchedulerBuilder = {
+    override def withLeaseRejectAction(
+        action: Action1[VirtualMachineLease]): TaskSchedulerBuilder = {
       leaseRejectAction = action
       this
     }
@@ -225,7 +229,8 @@ class LaunchCoordinatorTest
     val optimizerBuilder = taskSchedulerBuilder(optimizer)
     val schedulerDriver = mock(classOf[SchedulerDriver])
     val trace = Mockito.inOrder(schedulerDriver)
-    val fsm = TestFSMRef(new LaunchCoordinator(testActor, config, schedulerDriver, optimizerBuilder))
+    val fsm =
+      TestFSMRef(new LaunchCoordinator(testActor, config, schedulerDriver, optimizerBuilder))
 
     val framework = randomFramework
     val task1 = randomTask
@@ -234,12 +239,14 @@ class LaunchCoordinatorTest
 
     val slave1 = {
       val slave = randomSlave
-      (slave._1, slave._2, randomOffer(framework, slave), randomOffer(framework, slave), randomOffer(framework, slave))
+      (slave._1, slave._2,
+        randomOffer(framework, slave), randomOffer(framework, slave), randomOffer(framework, slave))
     }
 
     val slave2 = {
       val slave = randomSlave
-      (slave._1, slave._2, randomOffer(framework, slave), randomOffer(framework, slave), randomOffer(framework, slave))
+      (slave._1, slave._2,
+        randomOffer(framework, slave), randomOffer(framework, slave), randomOffer(framework, slave))
     }
   }
 
@@ -337,10 +344,10 @@ class LaunchCoordinatorTest
           verify(schedulerDriver).suppressOffers()
         }
         "declines any outstanding offers" in new Context {
-          //fsm.setState(GatheringOffers, GatherData(newOffers = Seq(new VMLeaseObject(slave1._3))))
           fsm.setState(GatheringOffers, GatherData())
           fsm ! new Disconnected()
           verify(optimizer).expireAllLeases()
+          verify(optimizer).scheduleOnce(MM.any(), MM.any())
         }
       }
       "Disconnected" which {
@@ -351,7 +358,8 @@ class LaunchCoordinatorTest
           fsm.stateData.tasks should contain only (task1._2)
         }
         "transitions to Suspended with offer queue emptied" in new Context {
-          fsm.setState(GatheringOffers, GatherData(tasks = Seq(task1._2), newLeases = Seq(lease(slave1._3))))
+          fsm.setState(GatheringOffers,
+            GatherData(tasks = Seq(task1._2), newLeases = Seq(lease(slave1._3))))
           fsm ! new Disconnected()
           fsm.stateName should be (Suspended)
           fsm.stateData.newLeases should be (empty)
@@ -359,7 +367,8 @@ class LaunchCoordinatorTest
       }
       "Launch" which {
         "stays in GatheringOffers with updated task queue" in new Context {
-          fsm.setState(GatheringOffers, GatherData(tasks = Seq(task1._2), newLeases = Seq(lease(slave1._3))))
+          fsm.setState(GatheringOffers,
+            GatherData(tasks = Seq(task1._2), newLeases = Seq(lease(slave1._3))))
           fsm ! Launch(Seq(task2._2).asJava)
           fsm.stateName should be (GatheringOffers)
           fsm.stateData.tasks should contain only (task1._2, task2._2)
@@ -368,16 +377,19 @@ class LaunchCoordinatorTest
       }
       "ResourceOffers" which {
         "stays in GatheringOffers with offer queue updated" in new Context {
-          fsm.setState(GatheringOffers, GatherData(tasks = Seq(task1._2), newLeases = Seq(lease(slave1._3))))
+          fsm.setState(GatheringOffers,
+            GatherData(tasks = Seq(task1._2), newLeases = Seq(lease(slave1._3))))
           fsm ! new ResourceOffers(Seq(slave1._4, slave2._3).asJava)
           fsm.stateName should be (GatheringOffers)
           fsm.stateData.tasks should contain only (task1._2)
-          fsm.stateData.newLeases.map(_.getOffer) should contain only (slave1._3, slave1._4, slave2._3)
+          fsm.stateData.newLeases.map(_.getOffer) should contain only
+            (slave1._3, slave1._4, slave2._3)
         }
       }
       "OfferRescinded" which {
         "stays in GatheringOffers with offer queue updated" in new Context {
-          fsm.setState(GatheringOffers, GatherData(tasks = Seq(task1._2), newLeases = Seq(lease(slave1._3))))
+          fsm.setState(GatheringOffers,
+            GatherData(tasks = Seq(task1._2), newLeases = Seq(lease(slave1._3))))
           fsm ! new OfferRescinded(slave1._3.getId)
           verify(optimizer).expireLease(slave1._3.getId.getValue)
           fsm.stateName should be (GatheringOffers)
@@ -387,45 +399,60 @@ class LaunchCoordinatorTest
       }
       "StateTimeout" which {
         "sends AcceptOffers message for matched tasks" in new Context {
-          when(optimizer.scheduleOnce(MM.any(), MM.any())) thenAnswer scheduleOnce { (requests, newLeases) =>
-            val (lease, task) = (newLeases.head, requests.head)
-            schedulingResult(
-              successes = Seq(vmAssignmentResult(lease.hostname(), Seq(lease), Set(taskAssignmentResult(lease, task)))))
-          }
-          fsm.setState(GatheringOffers, GatherData(tasks = Seq(task1._2), newLeases = Seq(lease(slave1._3))))
+          when(optimizer.scheduleOnce(MM.any(), MM.any())) thenAnswer {
+            scheduleOnce { (requests, newLeases) =>
+              val (l, task) = (newLeases.head, requests.head)
+              val vm = vmAssignmentResult(l.hostname(), Seq(l), Set(taskAssignmentResult(l, task)))
+              schedulingResult(successes = Seq(vm))
+            }
+          } thenReturn(schedulingResult(successes = Nil))
+
+          fsm.setState(GatheringOffers,
+            GatherData(tasks = Seq(task1._2), newLeases = Seq(lease(slave1._3))))
           fsm ! StateTimeout
           val offers = expectMsgType[AcceptOffers]
           offers.hostname() should be (slave1._2)
           offers.offerIds() should contain only (slave1._3.getId)
         }
         "transitions to Idle when task queue is empty" in new Context {
-          when(optimizer.scheduleOnce(MM.any(), MM.any())) thenAnswer scheduleOnce { (requests, newLeases) =>
-            val (lease, task) = (newLeases.head, requests.head)
-            schedulingResult(
-              successes = Seq(vmAssignmentResult(lease.hostname(), Seq(lease), Set(taskAssignmentResult(lease, task)))))
-          }
-          fsm.setState(GatheringOffers, GatherData(tasks = Seq(task1._2), newLeases = Seq(lease(slave1._3))))
+          when(optimizer.scheduleOnce(MM.any(), MM.any())) thenAnswer {
+            scheduleOnce { (requests, newLeases) =>
+              val (l, task) = (newLeases.head, requests.head)
+              val vm = vmAssignmentResult(l.hostname(), Seq(l), Set(taskAssignmentResult(l, task)))
+              schedulingResult(successes = Seq(vm))
+            }
+          } thenReturn(schedulingResult(successes = Nil))
+
+          fsm.setState(GatheringOffers,
+            GatherData(tasks = Seq(task1._2), newLeases = Seq(lease(slave1._3))))
           fsm ! StateTimeout
           fsm.stateName should be (Idle)
           fsm.stateData.tasks should be (empty)
           fsm.stateData.newLeases should be (empty)
         }
         "stays in GatheringOffers when task queue is non-empty" in new Context {
-          when(optimizer.scheduleOnce(MM.any(), MM.any())) thenAnswer scheduleOnce { (requests, newLeases) =>
-            schedulingResult(successes = Nil)
+          when(optimizer.scheduleOnce(MM.any(), MM.any())) thenAnswer {
+            scheduleOnce { (requests, newLeases) =>
+              schedulingResult(successes = Nil)
+            }
           }
-          fsm.setState(GatheringOffers, GatherData(tasks = Seq(task1._2), newLeases = Seq(lease(slave1._3))))
+          fsm.setState(GatheringOffers,
+            GatherData(tasks = Seq(task1._2), newLeases = Seq(lease(slave1._3))))
           fsm ! StateTimeout
           fsm.stateName should be (GatheringOffers)
           fsm.stateData.tasks should contain only (task1._2)
           fsm.stateData.newLeases should be (empty)
         }
         "declines old offers" in new Context {
-          when(optimizer.scheduleOnce(MM.any(), MM.any())) thenAnswer scheduleOnce { (requests, newLeases) =>
-            optimizerBuilder.leaseRejectAction.call(newLeases.head)
-            schedulingResult(successes = Nil)
-          }
-          fsm.setState(GatheringOffers, GatherData(tasks = Seq(task1._2), newLeases = Seq(lease(slave1._3))))
+          when(optimizer.scheduleOnce(MM.any(), MM.any())) thenAnswer {
+            scheduleOnce { (requests, newLeases) =>
+              optimizerBuilder.leaseRejectAction.call(newLeases.head)
+              schedulingResult(successes = Nil)
+            }
+          } thenReturn(schedulingResult(successes = Nil))
+
+          fsm.setState(GatheringOffers,
+            GatherData(tasks = Seq(task1._2), newLeases = Seq(lease(slave1._3))))
           fsm ! StateTimeout
           verify(schedulerDriver).declineOffer(slave1._3.getId)
         }

http://git-wip-us.apache.org/repos/asf/flink/blob/59eeea4c/flink-mesos/src/test/scala/org/apache/flink/mesos/scheduler/ReconciliationCoordinatorTest.scala
----------------------------------------------------------------------
diff --git a/flink-mesos/src/test/scala/org/apache/flink/mesos/scheduler/ReconciliationCoordinatorTest.scala b/flink-mesos/src/test/scala/org/apache/flink/mesos/scheduler/ReconciliationCoordinatorTest.scala
index fb64115..c223852 100644
--- a/flink-mesos/src/test/scala/org/apache/flink/mesos/scheduler/ReconciliationCoordinatorTest.scala
+++ b/flink-mesos/src/test/scala/org/apache/flink/mesos/scheduler/ReconciliationCoordinatorTest.scala
@@ -54,7 +54,8 @@ class ReconciliationCoordinatorTest
   def randomTask = {
     val slaveID = Protos.SlaveID.newBuilder.setValue(UUID.randomUUID.toString).build()
     val taskID = Protos.TaskID.newBuilder.setValue(UUID.randomUUID.toString).build
-    val taskStatus = Protos.TaskStatus.newBuilder().setTaskId(taskID).setSlaveId(slaveID).setState(TASK_STAGING).build()
+    val taskStatus = Protos.TaskStatus.newBuilder()
+      .setTaskId(taskID).setSlaveId(slaveID).setState(TASK_STAGING).build()
     (taskID, taskStatus)
   }
 

http://git-wip-us.apache.org/repos/asf/flink/blob/59eeea4c/flink-mesos/src/test/scala/org/apache/flink/mesos/scheduler/TaskMonitorTest.scala
----------------------------------------------------------------------
diff --git a/flink-mesos/src/test/scala/org/apache/flink/mesos/scheduler/TaskMonitorTest.scala b/flink-mesos/src/test/scala/org/apache/flink/mesos/scheduler/TaskMonitorTest.scala
index ff32116..b4ef938 100644
--- a/flink-mesos/src/test/scala/org/apache/flink/mesos/scheduler/TaskMonitorTest.scala
+++ b/flink-mesos/src/test/scala/org/apache/flink/mesos/scheduler/TaskMonitorTest.scala
@@ -60,7 +60,8 @@ class TaskMonitorTest
 
   def randomTask(slaveID: Protos.SlaveID) = {
     val taskID = Protos.TaskID.newBuilder.setValue(UUID.randomUUID.toString).build
-    val taskStatus = Protos.TaskStatus.newBuilder().setTaskId(taskID).setSlaveId(slaveID).setState(TASK_STAGING).build()
+    val taskStatus = Protos.TaskStatus.newBuilder()
+      .setTaskId(taskID).setSlaveId(slaveID).setState(TASK_STAGING).build()
     (taskID, taskStatus)
   }
 
@@ -70,7 +71,8 @@ class TaskMonitorTest
     val slave = randomSlave
     val task = randomTask(slave._1)
     val parent = TestProbe()
-    val fsm = TestFSMUtils.testFSMRef(new TaskMonitor(config, schedulerDriver, New(task._1)), parent.ref)
+    val fsm =
+      TestFSMUtils.testFSMRef(new TaskMonitor(config, schedulerDriver, New(task._1)), parent.ref)
     parent.watch(fsm)
   }
 
@@ -79,13 +81,13 @@ class TaskMonitorTest
 
   def handlesStatusUpdate(state: TaskMonitorState) = {
     "StatusUpdate" which {
-      "transitions to Staging when goal state is Launched and status is TASK_STAGING|TASK_STARTING" in new Context {
+      "transitions to Staging when goal state is Launched and status is staging" in new Context {
         fsm.setState(state, StateData(Launched(task._1, slave._1)))
         fsm ! new StatusUpdate(task._2.toBuilder.setState(TASK_STAGING).build())
         fsm.stateName should be (Staging)
         fsm.stateData should be (StateData(Launched(task._1, slave._1)))
       }
-      "transitions to Running when goal state is Launched and status is TASK_RUNNING" in new Context {
+      "transitions to Running when goal state is Launched and status is running" in new Context {
         fsm.setState(state, StateData(Launched(task._1, slave._1)))
         fsm ! new StatusUpdate(task._2.toBuilder.setState(TASK_RUNNING).build())
         fsm.stateName should be (Running)
@@ -100,13 +102,13 @@ class TaskMonitorTest
         }
         parent.expectTerminated(fsm)
       }
-      "transitions to Killing when goal state is Released and status is TASK_STAGING|TASK_STARTING|TASK_RUNNING" in new Context {
+      "transitions to Killing when goal state is Released and status is running" in new Context {
         fsm.setState(state, StateData(Released(task._1, slave._1)))
         fsm ! new StatusUpdate(task._2.toBuilder.setState(TASK_RUNNING).build())
         fsm.stateName should be (Killing)
         fsm.stateData should be (StateData(Released(task._1, slave._1)))
       }
-      "stops when goal state is Released and status is TASK_KILLED" in new Context {
+      "stops when goal state is Released and status is killed" in new Context {
         fsm.setState(state, StateData(Released(task._1, slave._1)))
         fsm ! new StatusUpdate(task._2.toBuilder.setState(TASK_KILLED).build())
         parent.fishForMessage() {

http://git-wip-us.apache.org/repos/asf/flink/blob/59eeea4c/flink-mesos/src/test/scala/org/apache/flink/mesos/scheduler/TasksTest.scala
----------------------------------------------------------------------
diff --git a/flink-mesos/src/test/scala/org/apache/flink/mesos/scheduler/TasksTest.scala b/flink-mesos/src/test/scala/org/apache/flink/mesos/scheduler/TasksTest.scala
new file mode 100644
index 0000000..fcf2977
--- /dev/null
+++ b/flink-mesos/src/test/scala/org/apache/flink/mesos/scheduler/TasksTest.scala
@@ -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.mesos.scheduler
+
+import java.util.UUID
+
+import akka.actor._
+import akka.testkit._
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.mesos.TestFSMUtils
+import org.apache.flink.mesos.scheduler.ReconciliationCoordinator.Reconcile
+import org.apache.flink.mesos.scheduler.TaskMonitor._
+import org.apache.flink.mesos.scheduler.messages.{Connected, Disconnected, StatusUpdate}
+import org.apache.flink.runtime.akka.AkkaUtils
+import org.apache.mesos.Protos.TaskState._
+import org.apache.mesos.{Protos, SchedulerDriver}
+import org.junit.runner.RunWith
+import org.mockito.Mockito._
+import org.scalatest.junit.JUnitRunner
+import org.scalatest.{BeforeAndAfterAll, Matchers, WordSpecLike}
+
+import scala.collection.mutable.{Map => MutableMap}
+
+@RunWith(classOf[JUnitRunner])
+class TasksTest
+    extends WordSpecLike
+    with Matchers
+    with BeforeAndAfterAll {
+
+  lazy val config = new Configuration()
+  implicit lazy val system = AkkaUtils.createLocalActorSystem(config)
+
+  override def afterAll(): Unit = {
+    TestKit.shutdownActorSystem(system)
+  }
+
+  def randomSlave = {
+    val slaveID = Protos.SlaveID.newBuilder.setValue(UUID.randomUUID.toString).build
+    val hostname = s"host-${slaveID.getValue}"
+    (slaveID, hostname)
+  }
+
+  def randomTask(slaveID: Protos.SlaveID) = {
+    val taskID = Protos.TaskID.newBuilder.setValue(UUID.randomUUID.toString).build
+    val taskStatus = Protos.TaskStatus.newBuilder()
+      .setTaskId(taskID).setSlaveId(slaveID).setState(TASK_STAGING)
+    (taskID, taskStatus)
+  }
+
+  def childProbe(parent: ActorRefFactory): (TestProbe, ActorRef) = {
+    val probe = TestProbe()
+    val childRef = parent.actorOf(Props(
+      new Actor {
+        override def receive: Receive = {
+          case msg @ _ => probe.ref.forward(msg)
+        }
+      }
+    ))
+    (probe,childRef)
+  }
+
+  class Context(implicit val system: ActorSystem) extends TestKitBase with ImplicitSender {
+
+    case class MockTaskMonitor(probe: TestProbe, actorRef: ActorRef, task: TaskGoalState)
+
+    val schedulerDriver = mock(classOf[SchedulerDriver])
+
+    val slave = randomSlave
+    val task = randomTask(slave._1)
+
+    val taskActors = MutableMap[Protos.TaskID,MockTaskMonitor]()
+
+    val actor = {
+      val taskActorCreator = (factory: ActorRefFactory, task: TaskGoalState) => {
+        val (probe, taskActorRef) = childProbe(factory)
+        taskActors.put(task.taskID, MockTaskMonitor(probe, taskActorRef, task))
+        taskActorRef
+      }
+      TestActorRef[Tasks](
+        Props(classOf[Tasks], config, schedulerDriver, taskActorCreator),
+        testActor,
+        TestFSMUtils.randomName)
+    }
+  }
+
+  def handle = afterWord("handle")
+
+  "Tasks" should handle {
+
+    "(supervision)" which {
+      "escalates" in new Context {
+        actor ! TaskGoalStateUpdated(Launched(task._1, slave._1))
+        watch(actor)
+        taskActors(task._1).actorRef ! Kill
+        expectTerminated(actor)
+      }
+    }
+
+    "Connect" which {
+      "stores the connected message for later use" in new Context {
+        val msg = new Connected() {}
+        actor ! msg
+        actor.underlyingActor.registered should be (Some(msg))
+      }
+
+      "forwards the message to child tasks" in new Context {
+        val msg = new Connected() {}
+        actor ! TaskGoalStateUpdated(Launched(task._1, slave._1))
+        actor ! msg
+        taskActors(task._1).probe.expectMsg(msg)
+      }
+    }
+
+    "Disconnect" which {
+      "releases any connected message that was previously stored" in new Context {
+        actor.underlyingActor.registered = Some(new Connected() {})
+        actor ! new Disconnected()
+        actor.underlyingActor.registered should be (None)
+      }
+
+      "forwards the message to child tasks" in new Context {
+        val msg = new Disconnected() {}
+        actor ! TaskGoalStateUpdated(Launched(task._1, slave._1))
+        actor ! msg
+        taskActors(task._1).probe.expectMsg(msg)
+      }
+    }
+
+    "TaskGoalStateUpdated" which {
+      "creates a task monitor on-demand for a given task" in new Context {
+        val goal = Launched(task._1, slave._1)
+        actor ! TaskGoalStateUpdated(goal)
+        actor.underlyingActor.taskMap.contains(task._1) should be (true)
+        taskActors(task._1).task should be (goal)
+      }
+
+      "forwards the stored connected message to new monitor actors" in new Context {
+        val msg = new Connected() {}
+        val goal = Launched(task._1, slave._1)
+        actor ! msg
+        actor ! TaskGoalStateUpdated(goal)
+        taskActors(task._1).probe.expectMsg(msg)
+      }
+
+      "forwards the goal state to the task monitor" in new Context {
+        actor ! TaskGoalStateUpdated(Launched(task._1, slave._1))
+        val updateMsg = TaskGoalStateUpdated(Released(task._1, slave._1))
+        actor ! updateMsg
+        taskActors(task._1).probe.expectMsg(updateMsg)
+      }
+    }
+
+    "StatusUpdate" which {
+      "forwards the update to a task monitor" in new Context {
+        actor ! TaskGoalStateUpdated(Launched(task._1, slave._1))
+        val msg = new StatusUpdate(task._2.setState(TASK_RUNNING).build())
+        actor ! msg
+        taskActors(task._1).probe.expectMsg(msg)
+      }
+
+      "resumes monitoring of resurrected tasks" in new Context {
+        // in this scenario, no goal state is sent prior to the status update
+        actor ! new StatusUpdate(task._2.setState(TASK_RUNNING).build())
+        taskActors.contains(task._1) should be (true)
+        taskActors(task._1).task should be (Released(task._1, slave._1))
+      }
+    }
+
+    "Reconcile" which {
+      "forwards the message to the parent" in new Context {
+        val msg = new Reconcile(Seq(task._2.build()))
+        actor ! msg
+        expectMsg(msg)
+      }
+    }
+
+    "TaskTerminated" which {
+      "removes the task monitor ref" in new Context {
+        actor ! TaskGoalStateUpdated(Launched(task._1, slave._1))
+        actor.underlyingActor.taskMap.contains(task._1) should be (true)
+        actor ! TaskTerminated(task._1, task._2.setState(TASK_FAILED).build())
+        actor.underlyingActor.taskMap.contains(task._1) should be (false)
+      }
+
+      "forwards to the parent" in new Context {
+        actor ! TaskGoalStateUpdated(Launched(task._1, slave._1))
+        val msg = TaskTerminated(task._1, task._2.setState(TASK_FAILED).build())
+        actor ! msg
+        expectMsg(msg)
+      }
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/59eeea4c/flink-mesos/src/test/scala/org/apache/flink/runtime/akka/FSMSpec.scala
----------------------------------------------------------------------
diff --git a/flink-mesos/src/test/scala/org/apache/flink/runtime/akka/FSMSpec.scala b/flink-mesos/src/test/scala/org/apache/flink/runtime/akka/FSMSpec.scala
index 3252f94..44a7fdc 100644
--- a/flink-mesos/src/test/scala/org/apache/flink/runtime/akka/FSMSpec.scala
+++ b/flink-mesos/src/test/scala/org/apache/flink/runtime/akka/FSMSpec.scala
@@ -32,7 +32,8 @@ abstract class FSMSpec extends FSMSpecLike {
 }
 
 /**
-  * Implementation trait for class <code>FSMSpec</code>, which extends wordspec with FSM functionality.
+  * Implementation trait for class <code>FSMSpec</code>, which extends wordspec
+  * with FSM functionality.
   *
   * For example: "MyFSM" when inState {
   *   "Connected" should handle {

http://git-wip-us.apache.org/repos/asf/flink/blob/59eeea4c/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/ContaineredJobManager.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/ContaineredJobManager.scala b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/ContaineredJobManager.scala
new file mode 100644
index 0000000..45b404a
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/ContaineredJobManager.scala
@@ -0,0 +1,172 @@
+/*
+ * 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 java.util.concurrent.ExecutorService
+
+import akka.actor.ActorRef
+import org.apache.flink.api.common.JobID
+import org.apache.flink.configuration.{Configuration => FlinkConfiguration}
+import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory
+import org.apache.flink.runtime.checkpoint.savepoint.SavepointStore
+import org.apache.flink.runtime.clusterframework.messages._
+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.jobgraph.JobStatus
+import org.apache.flink.runtime.jobmanager.scheduler.{Scheduler => FlinkScheduler}
+import org.apache.flink.runtime.jobmanager.{JobManager, SubmittedJobGraphStore}
+import org.apache.flink.runtime.leaderelection.LeaderElectionService
+import org.apache.flink.runtime.messages.JobManagerMessages.{CurrentJobStatus, JobNotFound, RequestJobStatus}
+import org.apache.flink.runtime.messages.Messages.Acknowledge
+import org.apache.flink.runtime.metrics.{MetricRegistry => FlinkMetricRegistry}
+
+import scala.concurrent.duration._
+import scala.language.postfixOps
+
+
+/** JobManager actor for execution on Yarn or Mesos. It enriches the [[JobManager]] with additional messages
+  * to start/administer/stop the session.
+  *
+  * @param flinkConfiguration Configuration object for the actor
+  * @param executorService Execution context which is used to execute concurrent tasks in the
+  *                         [[org.apache.flink.runtime.executiongraph.ExecutionGraph]]
+  * @param instanceManager Instance manager to manage the registered
+  *                        [[org.apache.flink.runtime.taskmanager.TaskManager]]
+  * @param scheduler Scheduler to schedule Flink jobs
+  * @param libraryCacheManager Manager to manage uploaded jar files
+  * @param archive Archive for finished Flink jobs
+  * @param restartStrategyFactory Restart strategy to be used in case of a job recovery
+  * @param timeout Timeout for futures
+  * @param leaderElectionService LeaderElectionService to participate in the leader election
+  */
+abstract class ContaineredJobManager(
+    flinkConfiguration: FlinkConfiguration,
+    executorService: ExecutorService,
+    instanceManager: InstanceManager,
+    scheduler: FlinkScheduler,
+    libraryCacheManager: BlobLibraryCacheManager,
+    archive: ActorRef,
+    restartStrategyFactory: RestartStrategyFactory,
+    timeout: FiniteDuration,
+    leaderElectionService: LeaderElectionService,
+    submittedJobGraphs : SubmittedJobGraphStore,
+    checkpointRecoveryFactory : CheckpointRecoveryFactory,
+    savepointStore: SavepointStore,
+    jobRecoveryTimeout: FiniteDuration,
+    metricsRegistry: Option[FlinkMetricRegistry])
+  extends JobManager(
+    flinkConfiguration,
+    executorService,
+    instanceManager,
+    scheduler,
+    libraryCacheManager,
+    archive,
+    restartStrategyFactory,
+    timeout,
+    leaderElectionService,
+    submittedJobGraphs,
+    checkpointRecoveryFactory,
+    savepointStore,
+    jobRecoveryTimeout,
+    metricsRegistry) {
+
+  val jobPollingInterval: FiniteDuration
+
+  // indicates if this JM has been started in a dedicated (per-job) mode.
+  var stopWhenJobFinished: JobID = null
+
+  override def handleMessage: Receive = {
+    handleContainerMessage orElse super.handleMessage
+  }
+
+  def handleContainerMessage: Receive = {
+
+    case msg @ (_: RegisterInfoMessageListener | _: UnRegisterInfoMessageListener) =>
+      // forward to ResourceManager
+      currentResourceManager match {
+        case Some(rm) =>
+          // we forward the message
+          rm.forward(decorateMessage(msg))
+        case None =>
+          // client has to try again
+      }
+
+    case msg: ShutdownClusterAfterJob =>
+      val jobId = msg.jobId()
+      log.info(s"ApplicationMaster will shut down session when job $jobId has finished.")
+      stopWhenJobFinished = jobId
+      // trigger regular job status messages (if this is a dedicated/per-job cluster)
+      if (stopWhenJobFinished != null) {
+        context.system.scheduler.schedule(0 seconds,
+          jobPollingInterval,
+          new Runnable {
+            override def run(): Unit = {
+              self ! decorateMessage(RequestJobStatus(stopWhenJobFinished))
+            }
+          }
+        )(context.dispatcher)
+      }
+
+      sender() ! decorateMessage(Acknowledge)
+
+    case msg: GetClusterStatus =>
+      sender() ! decorateMessage(
+        new GetClusterStatusResponse(
+          instanceManager.getNumberOfRegisteredTaskManagers,
+          instanceManager.getTotalNumberOfSlots)
+      )
+
+    case jnf: JobNotFound =>
+      log.debug(s"Job with ID ${jnf.jobID} not found in JobManager")
+      if (stopWhenJobFinished == null) {
+        log.warn("The ApplicationMaster didn't expect to receive this message")
+      }
+
+    case jobStatus: CurrentJobStatus =>
+      if (stopWhenJobFinished == null) {
+        log.warn(s"Received job status $jobStatus which wasn't requested.")
+      } else {
+        if (stopWhenJobFinished != jobStatus.jobID) {
+          log.warn(s"Received job status for job ${jobStatus.jobID} but expected status for " +
+            s"job $stopWhenJobFinished")
+        } else {
+          if (jobStatus.status.isGloballyTerminalState) {
+            log.info(s"Job with ID ${jobStatus.jobID} is in terminal state ${jobStatus.status}. " +
+              s"Shutting down session")
+            if (jobStatus.status == JobStatus.FINISHED) {
+              self ! decorateMessage(
+                new StopCluster(
+                  ApplicationStatus.SUCCEEDED,
+                  s"The monitored job with ID ${jobStatus.jobID} has finished.")
+              )
+            } else {
+              self ! decorateMessage(
+                new StopCluster(
+                  ApplicationStatus.FAILED,
+                  s"The monitored job with ID ${jobStatus.jobID} has failed to complete.")
+              )
+            }
+          } else {
+            log.debug(s"Monitored job with ID ${jobStatus.jobID} is in state ${jobStatus.status}")
+          }
+        }
+      }
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/59eeea4c/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 87a2c98..4637b97 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
@@ -331,14 +331,6 @@ public class YarnApplicationMasterRunner {
 			// make sure that everything whatever ends up in the log
 			LOG.error("YARN Application Master initialization failed", t);
 
-			if (actorSystem != null) {
-				try {
-					actorSystem.shutdown();
-				} catch (Throwable tt) {
-					LOG.error("Error shutting down actor system", tt);
-				}
-			}
-
 			if (webMonitor != null) {
 				try {
 					webMonitor.stop();
@@ -347,6 +339,14 @@ public class YarnApplicationMasterRunner {
 				}
 			}
 
+			if (actorSystem != null) {
+				try {
+					actorSystem.shutdown();
+				} catch (Throwable tt) {
+					LOG.error("Error shutting down actor system", tt);
+				}
+			}
+
 			return INIT_ERROR_EXIT_CODE;
 		}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/59eeea4c/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnJobManager.scala
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnJobManager.scala b/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnJobManager.scala
index 94ad9f2..b9d52ae 100644
--- a/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnJobManager.scala
+++ b/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnJobManager.scala
@@ -18,26 +18,20 @@
 
 package org.apache.flink.yarn
 
-import java.util.concurrent.{TimeUnit, ExecutorService}
+import java.util.concurrent.{ExecutorService, TimeUnit}
 
 import akka.actor.ActorRef
-
-import org.apache.flink.api.common.JobID
-import org.apache.flink.configuration.{Configuration => FlinkConfiguration, ConfigConstants}
+import org.apache.flink.configuration.{ConfigConstants, Configuration => FlinkConfiguration}
 import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory
 import org.apache.flink.runtime.checkpoint.savepoint.SavepointStore
-import org.apache.flink.runtime.clusterframework.ApplicationStatus
-import org.apache.flink.runtime.executiongraph.restart.RestartStrategyFactory
-import org.apache.flink.runtime.clusterframework.messages._
-import org.apache.flink.runtime.jobgraph.JobStatus
-import org.apache.flink.runtime.jobmanager.{SubmittedJobGraphStore, JobManager}
-import org.apache.flink.runtime.leaderelection.LeaderElectionService
-import org.apache.flink.runtime.messages.JobManagerMessages.{RequestJobStatus, CurrentJobStatus, JobNotFound}
-import org.apache.flink.runtime.messages.Messages.Acknowledge
-import org.apache.flink.runtime.metrics.MetricRegistry
+import org.apache.flink.runtime.clusterframework.ContaineredJobManager
 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 => FlinkScheduler}
+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
@@ -73,7 +67,7 @@ class YarnJobManager(
     savepointStore: SavepointStore,
     jobRecoveryTimeout: FiniteDuration,
     metricsRegistry: Option[MetricRegistry])
-  extends JobManager(
+  extends ContaineredJobManager(
     flinkConfiguration,
     executorService,
     instanceManager,
@@ -95,85 +89,5 @@ class YarnJobManager(
       flinkConfiguration.getInteger(ConfigConstants.YARN_HEARTBEAT_DELAY_SECONDS, 5),
       TimeUnit.SECONDS)
 
-  // indicates if this AM has been started in a detached mode.
-  var stopWhenJobFinished: JobID = null
-
-  override def handleMessage: Receive = {
-    handleYarnMessage orElse super.handleMessage
-  }
-
-  def handleYarnMessage: Receive = {
-
-    case msg @ (_: RegisterInfoMessageListener | _: UnRegisterInfoMessageListener) =>
-      // forward to ResourceManager
-      currentResourceManager match {
-        case Some(rm) =>
-          // we forward the message
-          rm.forward(decorateMessage(msg))
-        case None =>
-          // client has to try again
-      }
-
-    case msg: ShutdownClusterAfterJob =>
-      val jobId = msg.jobId()
-      log.info(s"ApplicatonMaster will shut down YARN session when job $jobId has finished.")
-      stopWhenJobFinished = jobId
-      // trigger regular job status messages (if this is a per-job yarn cluster)
-      if (stopWhenJobFinished != null) {
-        context.system.scheduler.schedule(0 seconds,
-          YARN_HEARTBEAT_DELAY,
-          new Runnable {
-            override def run(): Unit = {
-              self ! decorateMessage(RequestJobStatus(stopWhenJobFinished))
-            }
-          }
-        )(context.dispatcher)
-      }
-
-      sender() ! decorateMessage(Acknowledge)
-
-    case msg: GetClusterStatus =>
-      sender() ! decorateMessage(
-        new GetClusterStatusResponse(
-          instanceManager.getNumberOfRegisteredTaskManagers,
-          instanceManager.getTotalNumberOfSlots)
-      )
-
-    case jnf: JobNotFound =>
-      log.debug(s"Job with ID ${jnf.jobID} not found in JobManager")
-      if (stopWhenJobFinished == null) {
-        log.warn("The ApplicationMaster didn't expect to receive this message")
-      }
-
-    case jobStatus: CurrentJobStatus =>
-      if (stopWhenJobFinished == null) {
-        log.warn(s"Received job status $jobStatus which wasn't requested.")
-      } else {
-        if (stopWhenJobFinished != jobStatus.jobID) {
-          log.warn(s"Received job status for job ${jobStatus.jobID} but expected status for " +
-            s"job $stopWhenJobFinished")
-        } else {
-          if (jobStatus.status.isGloballyTerminalState) {
-            log.info(s"Job with ID ${jobStatus.jobID} is in terminal state ${jobStatus.status}. " +
-              s"Shutting down YARN session")
-            if (jobStatus.status == JobStatus.FINISHED) {
-              self ! decorateMessage(
-                new StopCluster(
-                  ApplicationStatus.SUCCEEDED,
-                  s"The monitored job with ID ${jobStatus.jobID} has finished.")
-              )
-            } else {
-              self ! decorateMessage(
-                new StopCluster(
-                  ApplicationStatus.FAILED,
-                  s"The monitored job with ID ${jobStatus.jobID} has failed to complete.")
-              )
-            }
-          } else {
-            log.debug(s"Monitored job with ID ${jobStatus.jobID} is in state ${jobStatus.status}")
-          }
-        }
-      }
-  }
-
+  override val jobPollingInterval = YARN_HEARTBEAT_DELAY
 }


[8/9] flink git commit: [FLINK-1984] Mesos ResourceManager - T1 milestone (4)

Posted by mx...@apache.org.
[FLINK-1984] Mesos ResourceManager - T1 milestone (4)

- adjust pom.xml per feedback


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

Branch: refs/heads/master
Commit: 38a9534c64e3fa2c5e045e50d98768d68a1352ac
Parents: 59eeea4
Author: wrighe3 <er...@emc.com>
Authored: Wed Aug 24 15:02:05 2016 -0700
Committer: Maximilian Michels <mx...@apache.org>
Committed: Mon Aug 29 17:32:51 2016 +0200

----------------------------------------------------------------------
 flink-mesos/pom.xml | 65 ++++++++++++++++--------------------------------
 1 file changed, 22 insertions(+), 43 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/38a9534c/flink-mesos/pom.xml
----------------------------------------------------------------------
diff --git a/flink-mesos/pom.xml b/flink-mesos/pom.xml
index c344ab2..43b3195 100644
--- a/flink-mesos/pom.xml
+++ b/flink-mesos/pom.xml
@@ -40,12 +40,6 @@ under the License.
 			<groupId>org.apache.flink</groupId>
 			<artifactId>flink-runtime_2.10</artifactId>
 			<version>${project.version}</version>
-			<exclusions>
-				<exclusion>
-					<artifactId>hadoop-core</artifactId>
-					<groupId>org.apache.hadoop</groupId>
-				</exclusion>
-			</exclusions>
 		</dependency>
 		
 		<dependency>
@@ -62,13 +56,10 @@ under the License.
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-test-utils_2.10</artifactId>
+			<artifactId>flink-shaded-curator-recipes</artifactId>
 			<version>${project.version}</version>
-			<type>test-jar</type>
-			<scope>test</scope>
 		</dependency>
 
-
 		<dependency>
 			<groupId>com.typesafe.akka</groupId>
 			<artifactId>akka-actor_${scala.binary.version}</artifactId>
@@ -85,39 +76,6 @@ under the License.
 		</dependency>
 
 		<dependency>
-			<groupId>com.typesafe.akka</groupId>
-			<artifactId>akka-testkit_${scala.binary.version}</artifactId>
-		</dependency>
-
-		<!--<dependency>
-			<groupId>com.typesafe.akka</groupId>
-			<artifactId>akka-camel_${scala.binary.version}</artifactId>
-		</dependency>-->
-
-		<dependency>
-			<groupId>com.google.guava</groupId>
-			<artifactId>guava</artifactId>
-			<version>${guava.version}</version>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.curator</groupId>
-			<artifactId>curator-test</artifactId>
-			<version>${curator.version}</version>
-			<scope>test</scope>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-shaded-curator-recipes</artifactId>
-			<version>${project.version}</version>
-		</dependency>
-
-		<!-- ===================================================
-        Dependencies for Mesos
-		=================================================== -->
-
-		<dependency>
 			<groupId>org.apache.mesos</groupId>
 			<artifactId>mesos</artifactId>
 			<version>${mesos.version}</version>
@@ -144,6 +102,19 @@ under the License.
 		</dependency>
 
 		<dependency>
+			<groupId>org.apache.curator</groupId>
+			<artifactId>curator-test</artifactId>
+			<version>${curator.version}</version>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>com.typesafe.akka</groupId>
+			<artifactId>akka-testkit_${scala.binary.version}</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
 			<groupId>org.apache.flink</groupId>
 			<artifactId>flink-runtime_2.10</artifactId>
 			<version>${project.version}</version>
@@ -151,6 +122,14 @@ under the License.
 			<type>test-jar</type>
 		</dependency>
 
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-test-utils_2.10</artifactId>
+			<version>${project.version}</version>
+			<type>test-jar</type>
+			<scope>test</scope>
+		</dependency>
+
 	</dependencies>
 
 	<build>


[4/9] flink git commit: [FLINK-1984] Mesos ResourceManager - T1 milestone

Posted by mx...@apache.org.
[FLINK-1984] Mesos ResourceManager - T1 milestone

Implemented Mesos AppMaster including:
- runners for AppMaster and TaskManager
- MesosFlinkResourceManager as a Mesos framework
- ZK persistent storage for Mesos tasks
- reusable scheduler actors for:
  - offer handling using Netflix Fenzo (LaunchCoordinator)
  - reconciliation (ReconciliationCoordinator)
  - task monitoring (TaskMonitor)
  - connection monitoring (ConnectionMonitor)
- lightweight HTTP server to serve artifacts to the Mesos fetcher (ArtifactServer)
- scenario-based logging for:
  - connectivity issues
  - offer handling (receive, process, decline, rescind, accept)
- incorporated FLINK-4152, FLINK-3904, FLINK-4141, FLINK-3675, FLINK-4166


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

Branch: refs/heads/master
Commit: d9b2be054f7dadf902d74622352e3ec8dfdcd584
Parents: 578e80e
Author: wrighe3 <er...@emc.com>
Authored: Thu Jul 14 00:12:49 2016 -0700
Committer: Maximilian Michels <mx...@apache.org>
Committed: Mon Aug 29 17:27:10 2016 +0200

----------------------------------------------------------------------
 .../flink/configuration/ConfigConstants.java    |  76 ++
 flink-dist/pom.xml                              |   7 +-
 flink-mesos/pom.xml                             | 294 ++++++++
 .../main/java/org/apache/flink/mesos/Utils.java |  49 ++
 .../flink/mesos/cli/FlinkMesosSessionCli.java   |  41 ++
 .../clusterframework/LaunchableMesosWorker.java | 187 +++++
 .../MesosApplicationMasterRunner.java           | 600 +++++++++++++++
 .../clusterframework/MesosConfigKeys.java       |  26 +
 .../MesosFlinkResourceManager.java              | 737 +++++++++++++++++++
 .../MesosTaskManagerParameters.java             |  51 ++
 .../MesosTaskManagerRunner.java                 |  99 +++
 .../RegisteredMesosWorkerNode.scala             |  15 +
 .../store/MesosWorkerStore.java                 | 134 ++++
 .../store/StandaloneMesosWorkerStore.java       |  69 ++
 .../store/ZooKeeperMesosWorkerStore.java        | 272 +++++++
 .../flink/mesos/scheduler/LaunchableTask.java   |  24 +
 .../flink/mesos/scheduler/SchedulerProxy.java   |  87 +++
 .../mesos/scheduler/TaskSchedulerBuilder.java   |  16 +
 .../mesos/scheduler/messages/AcceptOffers.java  |  56 ++
 .../mesos/scheduler/messages/Connected.java     |   8 +
 .../mesos/scheduler/messages/Disconnected.java  |  12 +
 .../flink/mesos/scheduler/messages/Error.java   |  24 +
 .../scheduler/messages/OfferRescinded.java      |  26 +
 .../mesos/scheduler/messages/ReRegistered.java  |  30 +
 .../mesos/scheduler/messages/Registered.java    |  39 +
 .../scheduler/messages/ResourceOffers.java      |  30 +
 .../mesos/scheduler/messages/SlaveLost.java     |  26 +
 .../mesos/scheduler/messages/StatusUpdate.java  |  27 +
 .../flink/mesos/util/MesosArtifactServer.java   | 286 +++++++
 .../flink/mesos/util/MesosConfiguration.java    | 108 +++
 .../apache/flink/mesos/util/ZooKeeperUtils.java |  22 +
 flink-mesos/src/main/resources/log4j.properties |  27 +
 .../clusterframework/MesosJobManager.scala      |  66 ++
 .../clusterframework/MesosTaskManager.scala     |  47 ++
 .../mesos/scheduler/ConnectionMonitor.scala     | 108 +++
 .../mesos/scheduler/LaunchCoordinator.scala     | 331 +++++++++
 .../scheduler/ReconciliationCoordinator.scala   | 164 +++++
 .../flink/mesos/scheduler/TaskMonitor.scala     | 240 ++++++
 .../apache/flink/mesos/scheduler/Tasks.scala    |  96 +++
 .../ContaineredJobManager.scala                 | 174 +++++
 .../MesosFlinkResourceManagerTest.java          | 697 ++++++++++++++++++
 .../src/test/resources/log4j-test.properties    |  32 +
 flink-mesos/src/test/resources/logback-test.xml |  37 +
 .../scala/org/apache/flink/mesos/Utils.scala    |  34 +
 .../mesos/scheduler/LaunchCoordinatorTest.scala | 421 +++++++++++
 .../ReconciliationCoordinatorTest.scala         | 214 ++++++
 .../flink/mesos/scheduler/TaskMonitorTest.scala | 237 ++++++
 .../org/apache/flink/runtime/akka/FSMSpec.scala |  40 +
 .../flink/runtime/util/ZooKeeperUtils.java      |   2 +-
 .../flink/runtime/jobmanager/JobManager.scala   |   3 +-
 pom.xml                                         |   1 +
 51 files changed, 6446 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/d9b2be05/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
index 514c730..2fe27e0 100644
--- a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
+++ b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
@@ -426,6 +426,60 @@ public final class ConfigConstants {
 	public static final String YARN_APPLICATION_MASTER_PORT = "yarn.application-master.port";
 
 
+	// ------------------------ Mesos Configuration ------------------------
+
+	/**
+	 * The maximum number of failed Mesos tasks before entirely stopping
+	 * the Mesos session / job on Mesos.
+	 *
+	 * By default, we take the number of of initially requested tasks.
+	 */
+	public static final String MESOS_MAX_FAILED_TASKS = "mesos.maximum-failed-tasks";
+
+	/**
+	 * The Mesos master URL.
+	 *
+	 * The value should be in one of the following forms:
+	 * <pre>
+	 * {@code
+	 *     host:port
+	 *     zk://host1:port1,host2:port2,.../path
+	 *     zk://username:password@host1:port1,host2:port2,.../path
+	 *     file:///path/to/file (where file contains one of the above)
+	 * }
+	 * </pre>
+	 *
+	 */
+	public static final String MESOS_MASTER_URL = "mesos.master";
+
+	/**
+	 * The failover timeout for the Mesos scheduler, after which running tasks are automatically shut down.
+	 *
+	 * The default value is 600 (seconds).
+	 */
+	public static final String MESOS_FAILOVER_TIMEOUT_SECONDS = "mesos.failover-timeout";
+
+	/**
+	 * The config parameter defining the Mesos artifact server port to use.
+	 * Setting the port to 0 will let the OS choose an available port.
+	 */
+	public static final String MESOS_ARTIFACT_SERVER_PORT_KEY = "mesos.resourcemanager.artifactserver.port";
+
+	public static final String MESOS_RESOURCEMANAGER_FRAMEWORK_NAME = "mesos.resourcemanager.framework.name";
+
+	public static final String MESOS_RESOURCEMANAGER_FRAMEWORK_ROLE = "mesos.resourcemanager.framework.role";
+
+	public static final String MESOS_RESOURCEMANAGER_FRAMEWORK_PRINCIPAL = "mesos.resourcemanager.framework.principal";
+
+	public static final String MESOS_RESOURCEMANAGER_FRAMEWORK_SECRET = "mesos.resourcemanager.framework.secret";
+
+	/**
+	 * The cpus to acquire from Mesos.
+	 *
+	 * By default, we use the number of requested task slots.
+	 */
+	public static final String MESOS_RESOURCEMANAGER_TASKS_CPUS = "mesos.resourcemanager.tasks.cpus";
+
 	// ------------------------ Hadoop Configuration ------------------------
 
 	/**
@@ -736,6 +790,9 @@ public final class ConfigConstants {
 	@Deprecated
 	public static final String ZOOKEEPER_CHECKPOINT_COUNTER_PATH = "recovery.zookeeper.path.checkpoint-counter";
 
+	/** ZooKeeper root path (ZNode) for Mesos workers. */
+	public static final String ZOOKEEPER_MESOS_WORKERS_PATH = "recovery.zookeeper.path.mesos-workers";
+
 	/** Deprecated in favour of {@link #HA_ZOOKEEPER_SESSION_TIMEOUT}. */
 	@Deprecated
 	public static final String ZOOKEEPER_SESSION_TIMEOUT = "recovery.zookeeper.client.session-timeout";
@@ -983,6 +1040,23 @@ public final class ConfigConstants {
 	 */
 	public static final String DEFAULT_YARN_JOB_MANAGER_PORT = "0";
 
+	// ------ Mesos-Specific Configuration ------
+
+	/** The default failover timeout provided to Mesos (10 mins) */
+	public static final int DEFAULT_MESOS_FAILOVER_TIMEOUT_SECS = 10 * 60;
+
+	/**
+	 * The default network port to listen on for the Mesos artifact server.
+	 */
+	public static final int DEFAULT_MESOS_ARTIFACT_SERVER_PORT = 0;
+
+	/**
+	 * The default Mesos framework name for the ResourceManager to use.
+	 */
+	public static final String DEFAULT_MESOS_RESOURCEMANAGER_FRAMEWORK_NAME = "Flink";
+
+	public static final String DEFAULT_MESOS_RESOURCEMANAGER_FRAMEWORK_ROLE = "*";
+
 	// ------------------------ File System Behavior ------------------------
 
 	/**
@@ -1131,6 +1205,8 @@ public final class ConfigConstants {
 
 	public static final String DEFAULT_ZOOKEEPER_CHECKPOINT_COUNTER_PATH = "/checkpoint-counter";
 
+	public static final String DEFAULT_ZOOKEEPER_MESOS_WORKERS_PATH = "/mesos-workers";
+
 	public static final int DEFAULT_ZOOKEEPER_SESSION_TIMEOUT = 60000;
 
 	public static final int DEFAULT_ZOOKEEPER_CONNECTION_TIMEOUT = 15000;

http://git-wip-us.apache.org/repos/asf/flink/blob/d9b2be05/flink-dist/pom.xml
----------------------------------------------------------------------
diff --git a/flink-dist/pom.xml b/flink-dist/pom.xml
index 539ca8e..ec84adc 100644
--- a/flink-dist/pom.xml
+++ b/flink-dist/pom.xml
@@ -119,8 +119,13 @@ under the License.
 			<artifactId>flink-metrics-jmx</artifactId>
 			<version>${project.version}</version>
 		</dependency>
+        
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-mesos_2.10</artifactId>
+			<version>${project.version}</version>
+		</dependency>
 		
-
 	</dependencies>
 
 	<!-- See main pom.xml for explanation of profiles -->

http://git-wip-us.apache.org/repos/asf/flink/blob/d9b2be05/flink-mesos/pom.xml
----------------------------------------------------------------------
diff --git a/flink-mesos/pom.xml b/flink-mesos/pom.xml
new file mode 100644
index 0000000..c344ab2
--- /dev/null
+++ b/flink-mesos/pom.xml
@@ -0,0 +1,294 @@
+<!--
+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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+	<modelVersion>4.0.0</modelVersion>
+
+	<parent>
+		<groupId>org.apache.flink</groupId>
+		<artifactId>flink-parent</artifactId>
+		<version>1.1-SNAPSHOT</version>
+		<relativePath>..</relativePath>
+	</parent>
+	
+	<artifactId>flink-mesos_2.10</artifactId>
+	<name>flink-mesos</name>
+	<packaging>jar</packaging>
+
+    <properties>
+        <mesos.version>0.27.1</mesos.version>
+    </properties>
+	
+    <dependencies>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-runtime_2.10</artifactId>
+			<version>${project.version}</version>
+			<exclusions>
+				<exclusion>
+					<artifactId>hadoop-core</artifactId>
+					<groupId>org.apache.hadoop</groupId>
+				</exclusion>
+			</exclusions>
+		</dependency>
+		
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-clients_2.10</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>${shading-artifact.name}</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-test-utils_2.10</artifactId>
+			<version>${project.version}</version>
+			<type>test-jar</type>
+			<scope>test</scope>
+		</dependency>
+
+
+		<dependency>
+			<groupId>com.typesafe.akka</groupId>
+			<artifactId>akka-actor_${scala.binary.version}</artifactId>
+		</dependency>
+
+		<dependency>
+			<groupId>com.typesafe.akka</groupId>
+			<artifactId>akka-remote_${scala.binary.version}</artifactId>
+		</dependency>
+
+		<dependency>
+			<groupId>com.typesafe.akka</groupId>
+			<artifactId>akka-slf4j_${scala.binary.version}</artifactId>
+		</dependency>
+
+		<dependency>
+			<groupId>com.typesafe.akka</groupId>
+			<artifactId>akka-testkit_${scala.binary.version}</artifactId>
+		</dependency>
+
+		<!--<dependency>
+			<groupId>com.typesafe.akka</groupId>
+			<artifactId>akka-camel_${scala.binary.version}</artifactId>
+		</dependency>-->
+
+		<dependency>
+			<groupId>com.google.guava</groupId>
+			<artifactId>guava</artifactId>
+			<version>${guava.version}</version>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.curator</groupId>
+			<artifactId>curator-test</artifactId>
+			<version>${curator.version}</version>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-shaded-curator-recipes</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+
+		<!-- ===================================================
+        Dependencies for Mesos
+		=================================================== -->
+
+		<dependency>
+			<groupId>org.apache.mesos</groupId>
+			<artifactId>mesos</artifactId>
+			<version>${mesos.version}</version>
+		</dependency>
+
+		<dependency>
+			<groupId>com.netflix.fenzo</groupId>
+			<artifactId>fenzo-core</artifactId>
+			<version>0.9.3</version>
+		</dependency>
+
+		<dependency>
+			<groupId>tv.cntt</groupId>
+			<artifactId>netty-router</artifactId>
+			<version>1.10</version>
+		</dependency>
+
+		<!-- test dependencies -->
+
+		<dependency>
+			<groupId>org.scalatest</groupId>
+			<artifactId>scalatest_${scala.binary.version}</artifactId>
+			<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>
+
+	</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>
+
+			<!-- Relocate curator -->
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-shade-plugin</artifactId>
+				<executions>
+					<execution>
+						<id>shade-flink</id>
+						<phase>package</phase>
+						<goals>
+							<goal>shade</goal>
+						</goals>
+						<configuration>
+							<artifactSet>
+								<includes combine.children="append">
+									<include>org.apache.flink:flink-shaded-curator-recipes</include>
+								</includes>
+							</artifactSet>
+							<relocations combine.children="append">
+								<relocation>
+									<pattern>org.apache.curator</pattern>
+									<shadedPattern>org.apache.flink.shaded.org.apache.curator</shadedPattern>
+								</relocation>
+							</relocations>
+						</configuration>
+					</execution>
+				</executions>
+			</plugin>
+		</plugins>
+	</build>
+</project>

http://git-wip-us.apache.org/repos/asf/flink/blob/d9b2be05/flink-mesos/src/main/java/org/apache/flink/mesos/Utils.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/Utils.java b/flink-mesos/src/main/java/org/apache/flink/mesos/Utils.java
new file mode 100644
index 0000000..2509465
--- /dev/null
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/Utils.java
@@ -0,0 +1,49 @@
+package org.apache.flink.mesos;
+
+import org.apache.mesos.Protos;
+
+import java.net.URL;
+import java.util.Arrays;
+
+public class Utils {
+	/**
+	 * Construct a Mesos environment variable.
+     */
+	public static Protos.Environment.Variable variable(String name, String value) {
+		return Protos.Environment.Variable.newBuilder()
+			.setName(name)
+			.setValue(value)
+			.build();
+	}
+
+	/**
+	 * Construct a Mesos URI.
+     */
+	public static Protos.CommandInfo.URI uri(URL url, boolean cacheable) {
+		return Protos.CommandInfo.URI.newBuilder()
+			.setValue(url.toExternalForm())
+			.setExtract(false)
+			.setCache(cacheable)
+			.build();
+	}
+
+	public static Protos.Resource scalar(String name, double value) {
+		return Protos.Resource.newBuilder()
+			.setName(name)
+			.setType(Protos.Value.Type.SCALAR)
+			.setScalar(Protos.Value.Scalar.newBuilder().setValue(value))
+			.build();
+	}
+
+	public static Protos.Value.Range range(long begin, long end) {
+		return Protos.Value.Range.newBuilder().setBegin(begin).setEnd(end).build();
+	}
+
+	public static Protos.Resource ranges(String name, Protos.Value.Range... ranges) {
+		return Protos.Resource.newBuilder()
+			.setName(name)
+			.setType(Protos.Value.Type.RANGES)
+			.setRanges(Protos.Value.Ranges.newBuilder().addAllRange(Arrays.asList(ranges)).build())
+			.build();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/d9b2be05/flink-mesos/src/main/java/org/apache/flink/mesos/cli/FlinkMesosSessionCli.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/cli/FlinkMesosSessionCli.java b/flink-mesos/src/main/java/org/apache/flink/mesos/cli/FlinkMesosSessionCli.java
new file mode 100644
index 0000000..b767344
--- /dev/null
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/cli/FlinkMesosSessionCli.java
@@ -0,0 +1,41 @@
+package org.apache.flink.mesos.cli;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.flink.configuration.Configuration;
+
+import java.io.IOException;
+import java.util.Map;
+
+public class FlinkMesosSessionCli {
+
+	private static final ObjectMapper mapper = new ObjectMapper();
+
+	public static Configuration decodeDynamicProperties(String dynamicPropertiesEncoded) {
+		try {
+			Configuration configuration = new Configuration();
+			if(dynamicPropertiesEncoded != null) {
+				TypeReference<Map<String, String>> typeRef = new TypeReference<Map<String, String>>() {};
+				Map<String,String> props = mapper.readValue(dynamicPropertiesEncoded, typeRef);
+				for (Map.Entry<String, String> property : props.entrySet()) {
+					configuration.setString(property.getKey(), property.getValue());
+				}
+			}
+			return configuration;
+		}
+		catch(IOException ex) {
+			throw new IllegalArgumentException("unreadable encoded properties", ex);
+		}
+	}
+
+	public static String encodeDynamicProperties(Configuration configuration) {
+		try {
+			String dynamicPropertiesEncoded = mapper.writeValueAsString(configuration.toMap());
+			return dynamicPropertiesEncoded;
+		}
+		catch (JsonProcessingException ex) {
+			throw new IllegalArgumentException("unwritable properties", ex);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/d9b2be05/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorker.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorker.java b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorker.java
new file mode 100644
index 0000000..8abd79a
--- /dev/null
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorker.java
@@ -0,0 +1,187 @@
+package org.apache.flink.mesos.runtime.clusterframework;
+
+import com.netflix.fenzo.ConstraintEvaluator;
+import com.netflix.fenzo.TaskAssignmentResult;
+import com.netflix.fenzo.TaskRequest;
+import com.netflix.fenzo.VMTaskFitnessCalculator;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.mesos.cli.FlinkMesosSessionCli;
+import org.apache.flink.mesos.scheduler.LaunchableTask;
+import org.apache.mesos.Protos;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.flink.mesos.Utils.variable;
+import static org.apache.flink.mesos.Utils.range;
+import static org.apache.flink.mesos.Utils.ranges;
+import static org.apache.flink.mesos.Utils.scalar;
+
+/**
+ * Specifies how to launch a Mesos worker.
+ */
+public class LaunchableMesosWorker implements LaunchableTask {
+
+	/**
+	 * The set of configuration keys to be dynamically configured with a port allocated from Mesos.
+	 */
+	private static String[] TM_PORT_KEYS = {
+		"taskmanager.rpc.port",
+		"taskmanager.data.port" };
+
+	private final MesosTaskManagerParameters params;
+	private final Protos.TaskInfo.Builder template;
+	private final Protos.TaskID taskID;
+	private final Request taskRequest;
+
+	/**
+	 * Construct a launchable Mesos worker.
+	 * @param params the TM parameters such as memory, cpu to acquire.
+	 * @param template a template for the TaskInfo to be constructed at launch time.
+	 * @param taskID the taskID for this worker.
+     */
+	public LaunchableMesosWorker(MesosTaskManagerParameters params, Protos.TaskInfo.Builder template, Protos.TaskID taskID) {
+		this.params = params;
+		this.template = template;
+		this.taskID = taskID;
+		this.taskRequest = new Request();
+	}
+
+	public Protos.TaskID taskID() {
+		return taskID;
+	}
+
+	@Override
+	public TaskRequest taskRequest() {
+		return taskRequest;
+	}
+
+	class Request implements TaskRequest {
+		private final AtomicReference<TaskRequest.AssignedResources> assignedResources = new AtomicReference<>();
+
+		@Override
+		public String getId() {
+			return taskID.getValue();
+		}
+
+		@Override
+		public String taskGroupName() {
+			return "";
+		}
+
+		@Override
+		public double getCPUs() {
+			return params.cpus();
+		}
+
+		@Override
+		public double getMemory() {
+			return params.containeredParameters().taskManagerTotalMemoryMB();
+		}
+
+		@Override
+		public double getNetworkMbps() {
+			return 0.0;
+		}
+
+		@Override
+		public double getDisk() {
+			return 0.0;
+		}
+
+		@Override
+		public int getPorts() {
+			return TM_PORT_KEYS.length;
+		}
+
+		@Override
+		public Map<String, NamedResourceSetRequest> getCustomNamedResources() {
+			return Collections.emptyMap();
+		}
+
+		@Override
+		public List<? extends ConstraintEvaluator> getHardConstraints() {
+			return null;
+		}
+
+		@Override
+		public List<? extends VMTaskFitnessCalculator> getSoftConstraints() {
+			return null;
+		}
+
+		@Override
+		public void setAssignedResources(AssignedResources assignedResources) {
+			this.assignedResources.set(assignedResources);
+		}
+
+		@Override
+		public AssignedResources getAssignedResources() {
+			return assignedResources.get();
+		}
+
+		@Override
+		public String toString() {
+			return "Request{" +
+				"cpus=" + getCPUs() +
+				"memory=" + getMemory() +
+				'}';
+		}
+	}
+
+	/**
+	 * Construct the TaskInfo needed to launch the worker.
+	 * @param slaveId the assigned slave.
+	 * @param assignment the assignment details.
+     * @return a fully-baked TaskInfo.
+     */
+	@Override
+	public Protos.TaskInfo launch(Protos.SlaveID slaveId, TaskAssignmentResult assignment) {
+
+		final Configuration dynamicProperties = new Configuration();
+
+		// specialize the TaskInfo template with assigned resources, environment variables, etc
+		final Protos.TaskInfo.Builder taskInfo = template
+			.clone()
+			.setSlaveId(slaveId)
+			.setTaskId(taskID)
+			.setName(taskID.getValue())
+			.addResources(scalar("cpus", assignment.getRequest().getCPUs()))
+			.addResources(scalar("mem", assignment.getRequest().getMemory()));
+		//.addResources(scalar("disk", assignment.getRequest.getDisk).setRole("Flink"))
+
+		// use the assigned ports for the TM
+		if (assignment.getAssignedPorts().size() != TM_PORT_KEYS.length) {
+			throw new IllegalArgumentException("unsufficient # of ports assigned");
+		}
+		for (int i = 0; i < TM_PORT_KEYS.length; i++) {
+			int port = assignment.getAssignedPorts().get(i);
+			String key = TM_PORT_KEYS[i];
+			taskInfo.addResources(ranges("ports", range(port, port)));
+			dynamicProperties.setInteger(key, port);
+		}
+
+		// finalize environment variables
+		final Protos.Environment.Builder environmentBuilder = taskInfo.getCommandBuilder().getEnvironmentBuilder();
+
+		// propagate the Mesos task ID to the TM
+		environmentBuilder
+			.addVariables(variable(MesosConfigKeys.ENV_FLINK_CONTAINER_ID, taskInfo.getTaskId().getValue()));
+
+		// propagate the dynamic configuration properties to the TM
+		String dynamicPropertiesEncoded = FlinkMesosSessionCli.encodeDynamicProperties(dynamicProperties);
+		environmentBuilder
+			.addVariables(variable(MesosConfigKeys.ENV_DYNAMIC_PROPERTIES, dynamicPropertiesEncoded));
+
+		return taskInfo.build();
+	}
+
+	@Override
+	public String toString() {
+		return "LaunchableMesosWorker{" +
+			"taskID=" + taskID +
+			"taskRequest=" + taskRequest +
+			'}';
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/d9b2be05/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosApplicationMasterRunner.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosApplicationMasterRunner.java b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosApplicationMasterRunner.java
new file mode 100644
index 0000000..30f2258
--- /dev/null
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosApplicationMasterRunner.java
@@ -0,0 +1,600 @@
+package org.apache.flink.mesos.runtime.clusterframework;
+
+import akka.actor.ActorRef;
+import akka.actor.ActorSystem;
+import akka.actor.Props;
+
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.GlobalConfiguration;
+import org.apache.flink.configuration.IllegalConfigurationException;
+import org.apache.flink.mesos.cli.FlinkMesosSessionCli;
+import org.apache.flink.mesos.runtime.clusterframework.store.MesosWorkerStore;
+import org.apache.flink.mesos.runtime.clusterframework.store.StandaloneMesosWorkerStore;
+import org.apache.flink.mesos.runtime.clusterframework.store.ZooKeeperMesosWorkerStore;
+import org.apache.flink.mesos.util.MesosArtifactServer;
+import org.apache.flink.mesos.util.MesosConfiguration;
+import org.apache.flink.mesos.util.ZooKeeperUtils;
+import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.clusterframework.BootstrapTools;
+import org.apache.flink.runtime.clusterframework.ContaineredTaskManagerParameters;
+import org.apache.flink.runtime.jobmanager.JobManager;
+import org.apache.flink.runtime.jobmanager.MemoryArchivist;
+import org.apache.flink.runtime.jobmanager.RecoveryMode;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.runtime.process.ProcessReaper;
+import org.apache.flink.runtime.taskmanager.TaskManager;
+import org.apache.flink.runtime.util.EnvironmentInformation;
+import org.apache.flink.runtime.util.LeaderRetrievalUtils;
+import org.apache.flink.runtime.util.SignalHandler;
+import org.apache.flink.runtime.webmonitor.WebMonitor;
+
+import org.apache.hadoop.security.UserGroupInformation;
+
+import org.apache.mesos.Protos;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import scala.Option;
+import scala.concurrent.duration.Duration;
+import scala.concurrent.duration.FiniteDuration;
+
+import java.io.File;
+import java.net.InetAddress;
+import java.net.URL;
+import java.security.PrivilegedAction;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.flink.mesos.Utils.uri;
+import static org.apache.flink.mesos.Utils.variable;
+
+/**
+ * This class is the executable entry point for the Mesos Application Master.
+ * It starts actor system and the actors for {@link org.apache.flink.runtime.jobmanager.JobManager}
+ * and {@link MesosFlinkResourceManager}.
+ *
+ * The JobManager handles Flink job execution, while the MesosFlinkResourceManager handles container
+ * allocation and failure detection.
+ */
+public class MesosApplicationMasterRunner {
+	/** Logger */
+	protected static final Logger LOG = LoggerFactory.getLogger(MesosApplicationMasterRunner.class);
+
+	/** The maximum time that TaskManagers may be waiting to register at the JobManager,
+	 * before they quit */
+	private static final FiniteDuration TASKMANAGER_REGISTRATION_TIMEOUT = new FiniteDuration(5, TimeUnit.MINUTES);
+
+	/** The process environment variables */
+	private static final Map<String, String> ENV = System.getenv();
+
+	/** The exit code returned if the initialization of the application master failed */
+	private static final int INIT_ERROR_EXIT_CODE = 31;
+
+	/** The exit code returned if the process exits because a critical actor died */
+	private static final int ACTOR_DIED_EXIT_CODE = 32;
+
+	// ------------------------------------------------------------------------
+	//  Program entry point
+	// ------------------------------------------------------------------------
+
+	/**
+	 * The entry point for the Mesos AppMaster.
+	 *
+	 * @param args The command line arguments.
+	 */
+	public static void main(String[] args) {
+		EnvironmentInformation.logEnvironmentInfo(LOG, "Mesos AppMaster", args);
+		SignalHandler.register(LOG);
+
+		// run and exit with the proper return code
+		int returnCode = new MesosApplicationMasterRunner().run(args);
+		System.exit(returnCode);
+	}
+
+	/**
+	 * The instance entry point for the Mesos AppMaster. Obtains user group
+	 * information and calls the main work method {@link #runPrivileged()} as a
+	 * privileged action.
+	 *
+	 * @param args The command line arguments.
+	 * @return The process exit code.
+	 */
+	protected int run(String[] args) {
+		try {
+			LOG.debug("All environment variables: {}", ENV);
+
+			final UserGroupInformation currentUser;
+			try {
+				currentUser = UserGroupInformation.getCurrentUser();
+			} catch (Throwable t) {
+				throw new Exception("Cannot access UserGroupInformation information for current user", t);
+			}
+
+			LOG.info("Running Flink as user {}", currentUser.getShortUserName());
+
+			// run the actual work in a secured privileged action
+			return currentUser.doAs(new PrivilegedAction<Integer>() {
+				@Override
+				public Integer run() {
+					return runPrivileged();
+				}
+			});
+		}
+		catch (Throwable t) {
+			// make sure that everything whatever ends up in the log
+			LOG.error("Mesos AppMaster initialization failed", t);
+			return INIT_ERROR_EXIT_CODE;
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  Core work method
+	// ------------------------------------------------------------------------
+
+	/**
+	 * The main work method, must run as a privileged action.
+	 *
+	 * @return The return code for the Java process.
+	 */
+	protected int runPrivileged() {
+
+		ActorSystem actorSystem = null;
+		WebMonitor webMonitor = null;
+		MesosArtifactServer artifactServer = null;
+
+		try {
+			// ------- (1) load and parse / validate all configurations -------
+
+			// loading all config values here has the advantage that the program fails fast, if any
+			// configuration problem occurs
+
+			final String workingDir = ENV.get(MesosConfigKeys.ENV_MESOS_SANDBOX);
+			require(workingDir != null, "Sandbox directory variable (%s) not set", MesosConfigKeys.ENV_MESOS_SANDBOX);
+
+			final String sessionID = ENV.get(MesosConfigKeys.ENV_SESSION_ID);
+			require(sessionID != null, "Session ID (%s) not set", MesosConfigKeys.ENV_SESSION_ID);
+
+			// Note that we use the "appMasterHostname" given by the system, to make sure
+			// we use the hostnames consistently throughout akka.
+			// for akka "localhost" and "localhost.localdomain" are different actors.
+			final String appMasterHostname = InetAddress.getLocalHost().getHostName();
+
+			// Flink configuration
+			final Configuration dynamicProperties =
+				FlinkMesosSessionCli.decodeDynamicProperties(ENV.get(MesosConfigKeys.ENV_DYNAMIC_PROPERTIES));
+			LOG.debug("Mesos dynamic properties: {}", dynamicProperties);
+
+			final Configuration config = createConfiguration(workingDir, dynamicProperties);
+
+			// Mesos configuration
+			final MesosConfiguration mesosConfig = createMesosConfig(config, appMasterHostname);
+
+			// environment values related to TM
+			final int taskManagerContainerMemory;
+			final int numInitialTaskManagers;
+			final int slotsPerTaskManager;
+
+			try {
+				taskManagerContainerMemory = Integer.parseInt(ENV.get(MesosConfigKeys.ENV_TM_MEMORY));
+			} catch (NumberFormatException e) {
+				throw new RuntimeException("Invalid value for " + MesosConfigKeys.ENV_TM_MEMORY + " : "
+					+ e.getMessage());
+			}
+			try {
+				numInitialTaskManagers = Integer.parseInt(ENV.get(MesosConfigKeys.ENV_TM_COUNT));
+			} catch (NumberFormatException e) {
+				throw new RuntimeException("Invalid value for " + MesosConfigKeys.ENV_TM_COUNT + " : "
+					+ e.getMessage());
+			}
+			try {
+				slotsPerTaskManager = Integer.parseInt(ENV.get(MesosConfigKeys.ENV_SLOTS));
+			} catch (NumberFormatException e) {
+				throw new RuntimeException("Invalid value for " + MesosConfigKeys.ENV_SLOTS + " : "
+					+ e.getMessage());
+			}
+
+			final ContaineredTaskManagerParameters containeredParameters =
+				ContaineredTaskManagerParameters.create(config, taskManagerContainerMemory, slotsPerTaskManager);
+
+			final MesosTaskManagerParameters taskManagerParameters =
+				MesosTaskManagerParameters.create(config, containeredParameters);
+
+			LOG.info("TaskManagers will be created with {} task slots",
+				taskManagerParameters.containeredParameters().numSlots());
+			LOG.info("TaskManagers will be started with container size {} MB, JVM heap size {} MB, " +
+					"JVM direct memory limit {} MB, {} cpus",
+				taskManagerParameters.containeredParameters().taskManagerTotalMemoryMB(),
+				taskManagerParameters.containeredParameters().taskManagerHeapSizeMB(),
+				taskManagerParameters.containeredParameters().taskManagerDirectMemoryLimitMB(),
+				taskManagerParameters.cpus());
+
+			// JM endpoint, which should be explicitly configured by the dispatcher (based on acquired net resources)
+			final int listeningPort = config.getInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY,
+				ConfigConstants.DEFAULT_JOB_MANAGER_IPC_PORT);
+			require(listeningPort >= 0 && listeningPort <= 65536, "Config parameter \"" +
+				ConfigConstants.JOB_MANAGER_IPC_PORT_KEY + "\" is invalid, it must be between 0 and 65536");
+
+			// ----------------- (2) start the actor system -------------------
+
+			// try to start the actor system, JobManager and JobManager actor system
+			// using the configured address and ports
+			actorSystem = BootstrapTools.startActorSystem(config, appMasterHostname, listeningPort, LOG);
+
+			final String akkaHostname = AkkaUtils.getAddress(actorSystem).host().get();
+			final int akkaPort = (Integer) AkkaUtils.getAddress(actorSystem).port().get();
+
+			LOG.info("Actor system bound to hostname {}.", akkaHostname);
+
+			// try to start the artifact server
+			LOG.debug("Starting Artifact Server");
+			final int artifactServerPort = config.getInteger(ConfigConstants.MESOS_ARTIFACT_SERVER_PORT_KEY,
+				ConfigConstants.DEFAULT_MESOS_ARTIFACT_SERVER_PORT);
+			artifactServer = new MesosArtifactServer(sessionID, akkaHostname, artifactServerPort);
+
+			// ----------------- (3) Generate the configuration for the TaskManagers -------------------
+
+			final Configuration taskManagerConfig = BootstrapTools.generateTaskManagerConfiguration(
+				config, akkaHostname, akkaPort, slotsPerTaskManager, TASKMANAGER_REGISTRATION_TIMEOUT);
+			LOG.debug("TaskManager configuration: {}", taskManagerConfig);
+
+			final Protos.TaskInfo.Builder taskManagerContext = createTaskManagerContext(
+				config, mesosConfig, ENV,
+				taskManagerParameters, taskManagerConfig,
+				workingDir, getTaskManagerClass(), artifactServer, LOG);
+
+			// ----------------- (4) start the actors -------------------
+
+			// 1) JobManager & Archive (in non-HA case, the leader service takes this)
+			// 2) Web Monitor (we need its port to register)
+			// 3) Resource Master for Mesos
+			// 4) Process reapers for the JobManager and Resource Master
+
+			// 1: the JobManager
+			LOG.debug("Starting JobManager actor");
+
+			// we start the JobManager with its standard name
+			ActorRef jobManager = JobManager.startJobManagerActors(
+				config, actorSystem,
+				new scala.Some<>(JobManager.JOB_MANAGER_NAME()),
+				scala.Option.<String>empty(),
+				getJobManagerClass(),
+				getArchivistClass())._1();
+
+
+			// 2: the web monitor
+			LOG.debug("Starting Web Frontend");
+
+			webMonitor = BootstrapTools.startWebMonitorIfConfigured(config, actorSystem, jobManager, LOG);
+			if(webMonitor != null) {
+				final URL webMonitorURL = new URL("http", appMasterHostname, webMonitor.getServerPort(), "/");
+				mesosConfig.frameworkInfo().setWebuiUrl(webMonitorURL.toExternalForm());
+			}
+
+			// 3: Flink's Mesos ResourceManager
+			LOG.debug("Starting Mesos Flink Resource Manager");
+
+			// create the worker store to persist task information across restarts
+			MesosWorkerStore workerStore = createWorkerStore(config);
+
+			// we need the leader retrieval service here to be informed of new
+			// leader session IDs, even though there can be only one leader ever
+			LeaderRetrievalService leaderRetriever =
+				LeaderRetrievalUtils.createLeaderRetrievalService(config, jobManager);
+
+			Props resourceMasterProps = MesosFlinkResourceManager.createActorProps(
+				getResourceManagerClass(),
+				config,
+				mesosConfig,
+				workerStore,
+				leaderRetriever,
+				taskManagerParameters,
+				taskManagerContext,
+				numInitialTaskManagers,
+				LOG);
+
+			ActorRef resourceMaster = actorSystem.actorOf(resourceMasterProps, "Mesos_Resource_Master");
+
+
+			// 4: Process reapers
+			// The process reapers ensure that upon unexpected actor death, the process exits
+			// and does not stay lingering around unresponsive
+
+			LOG.debug("Starting process reapers for JobManager");
+
+			actorSystem.actorOf(
+				Props.create(ProcessReaper.class, resourceMaster, LOG, ACTOR_DIED_EXIT_CODE),
+				"Mesos_Resource_Master_Process_Reaper");
+
+			actorSystem.actorOf(
+				Props.create(ProcessReaper.class, jobManager, LOG, ACTOR_DIED_EXIT_CODE),
+				"JobManager_Process_Reaper");
+		}
+		catch (Throwable t) {
+			// make sure that everything whatever ends up in the log
+			LOG.error("Mesos JobManager initialization failed", t);
+
+			if (actorSystem != null) {
+				try {
+					actorSystem.shutdown();
+				} catch (Throwable tt) {
+					LOG.error("Error shutting down actor system", tt);
+				}
+			}
+
+			if (webMonitor != null) {
+				try {
+					webMonitor.stop();
+				} catch (Throwable ignored) {
+					LOG.warn("Failed to stop the web frontend", ignored);
+				}
+			}
+
+			if(artifactServer != null) {
+				try {
+					artifactServer.stop();
+				} catch (Throwable ignored) {
+					LOG.error("Failed to stop the artifact server", ignored);
+				}
+			}
+
+			return INIT_ERROR_EXIT_CODE;
+		}
+
+		// everything started, we can wait until all is done or the process is killed
+		LOG.info("Mesos JobManager started");
+
+		// wait until everything is done
+		actorSystem.awaitTermination();
+
+		// if we get here, everything work out jolly all right, and we even exited smoothly
+		if (webMonitor != null) {
+			try {
+				webMonitor.stop();
+			} catch (Throwable t) {
+				LOG.error("Failed to stop the web frontend", t);
+			}
+		}
+
+		try {
+			artifactServer.stop();
+		} catch (Throwable t) {
+			LOG.error("Failed to stop the artifact server", t);
+		}
+
+		return 0;
+	}
+
+	// ------------------------------------------------------------------------
+	//  For testing, this allows to override the actor classes used for
+	//  JobManager and the archive of completed jobs
+	// ------------------------------------------------------------------------
+
+	protected Class<? extends MesosFlinkResourceManager> getResourceManagerClass() {
+		return MesosFlinkResourceManager.class;
+	}
+
+	protected Class<? extends JobManager> getJobManagerClass() {
+		return MesosJobManager.class;
+	}
+
+	protected Class<? extends MemoryArchivist> getArchivistClass() {
+		return MemoryArchivist.class;
+	}
+
+	protected Class<? extends TaskManager> getTaskManagerClass() {
+		return MesosTaskManager.class;
+	}
+
+	/**
+	 * Validates a condition, throwing a RuntimeException if the condition is violated.
+	 *
+	 * @param condition The condition.
+	 * @param message The message for the runtime exception, with format variables as defined by
+	 *                {@link String#format(String, Object...)}.
+	 * @param values The format arguments.
+	 */
+	private static void require(boolean condition, String message, Object... values) {
+		if (!condition) {
+			throw new RuntimeException(String.format(message, values));
+		}
+	}
+
+	/**
+	 *
+	 * @param baseDirectory
+	 * @param additional
+	 *
+	 * @return The configuration to be used by the TaskManagers.
+	 */
+	@SuppressWarnings("deprecation")
+	private static Configuration createConfiguration(String baseDirectory, Configuration additional) {
+		LOG.info("Loading config from directory " + baseDirectory);
+
+		Configuration configuration = GlobalConfiguration.loadConfiguration(baseDirectory);
+
+		configuration.setString(ConfigConstants.FLINK_BASE_DIR_PATH_KEY, baseDirectory);
+
+		// add dynamic properties to JobManager configuration.
+		configuration.addAll(additional);
+
+		return configuration;
+	}
+
+	/**
+	 * Loads and validates the ResourceManager Mesos configuration from the given Flink configuration.
+	 */
+	public static MesosConfiguration createMesosConfig(Configuration flinkConfig, String hostname) {
+
+		Protos.FrameworkInfo.Builder frameworkInfo = Protos.FrameworkInfo.newBuilder()
+			.setUser("")
+			.setHostname(hostname);
+		Protos.Credential.Builder credential = null;
+
+		if(!flinkConfig.containsKey(ConfigConstants.MESOS_MASTER_URL)) {
+			throw new IllegalConfigurationException(ConfigConstants.MESOS_MASTER_URL + " must be configured.");
+		}
+		String masterUrl = flinkConfig.getString(ConfigConstants.MESOS_MASTER_URL, null);
+
+		Duration failoverTimeout = FiniteDuration.apply(
+			flinkConfig.getInteger(
+				ConfigConstants.MESOS_FAILOVER_TIMEOUT_SECONDS,
+				ConfigConstants.DEFAULT_MESOS_FAILOVER_TIMEOUT_SECS),
+			TimeUnit.SECONDS);
+		frameworkInfo.setFailoverTimeout(failoverTimeout.toSeconds());
+
+		frameworkInfo.setName(flinkConfig.getString(
+			ConfigConstants.MESOS_RESOURCEMANAGER_FRAMEWORK_NAME,
+			ConfigConstants.DEFAULT_MESOS_RESOURCEMANAGER_FRAMEWORK_NAME));
+
+		frameworkInfo.setRole(flinkConfig.getString(
+			ConfigConstants.MESOS_RESOURCEMANAGER_FRAMEWORK_ROLE,
+			ConfigConstants.DEFAULT_MESOS_RESOURCEMANAGER_FRAMEWORK_ROLE));
+
+		if(flinkConfig.containsKey(ConfigConstants.MESOS_RESOURCEMANAGER_FRAMEWORK_PRINCIPAL)) {
+			frameworkInfo.setPrincipal(flinkConfig.getString(
+				ConfigConstants.MESOS_RESOURCEMANAGER_FRAMEWORK_PRINCIPAL, null));
+
+			credential = Protos.Credential.newBuilder();
+			credential.setPrincipal(frameworkInfo.getPrincipal());
+
+			if(!flinkConfig.containsKey(ConfigConstants.MESOS_RESOURCEMANAGER_FRAMEWORK_SECRET)) {
+				throw new IllegalConfigurationException(ConfigConstants.MESOS_RESOURCEMANAGER_FRAMEWORK_SECRET + " must be configured.");
+			}
+			credential.setSecret(flinkConfig.getString(
+				ConfigConstants.MESOS_RESOURCEMANAGER_FRAMEWORK_SECRET, null));
+		}
+
+		MesosConfiguration mesos =
+			new MesosConfiguration(masterUrl, frameworkInfo, Option.apply(credential));
+
+		return mesos;
+	}
+
+	private MesosWorkerStore createWorkerStore(Configuration flinkConfig) throws Exception {
+		MesosWorkerStore workerStore;
+		RecoveryMode recoveryMode = RecoveryMode.fromConfig(flinkConfig);
+		if (recoveryMode == RecoveryMode.STANDALONE) {
+			workerStore = new StandaloneMesosWorkerStore();
+		}
+		else if (recoveryMode == RecoveryMode.ZOOKEEPER) {
+			// note: the store is responsible for closing the client.
+			CuratorFramework client = ZooKeeperUtils.startCuratorFramework(flinkConfig);
+			workerStore = ZooKeeperMesosWorkerStore.createMesosWorkerStore(client, flinkConfig);
+		}
+		else {
+			throw new IllegalConfigurationException("Unexpected recovery mode '" + recoveryMode + ".");
+		}
+
+		return workerStore;
+	}
+
+	/**
+	 * Creates a Mesos task info template, which describes how to bring up a TaskManager process as
+	 * a Mesos task.
+	 *
+	 * <p>This code is extremely Mesos-specific and registers all the artifacts that the TaskManager
+	 * needs (such as JAR file, config file, ...) and all environment variables in a task info record.
+	 * The Mesos fetcher then ensures that those artifacts will be copied into the task's sandbox directory.
+	 * A lightweight HTTP server serves the artifacts to the fetcher.
+	 *
+	 * <p>We do this work before we start the ResourceManager actor in order to fail early if
+	 * any of the operations here fail.
+	 *
+	 * @param flinkConfig
+	 *         The Flink configuration object.
+	 * @param mesosConfig
+	 *         The Mesos configuration object.
+	 * @param env
+	 *         The environment variables.
+	 * @param tmParams
+	 *         The TaskManager container memory parameters.
+	 * @param taskManagerConfig
+	 *         The configuration for the TaskManagers.
+	 * @param workingDirectory
+	 *         The current application master container's working directory.
+	 * @param taskManagerMainClass
+	 *         The class with the main method.
+	 * @param artifactServer
+	 *         The artifact server.
+	 * @param log
+	 *         The logger.
+	 *
+	 * @return The task info template for the TaskManager processes.
+	 *
+	 * @throws Exception Thrown if the task info could not be created, for example if
+	 *                   the resources could not be copied.
+	 */
+	public static Protos.TaskInfo.Builder createTaskManagerContext(
+		Configuration flinkConfig,
+		MesosConfiguration mesosConfig,
+		Map<String, String> env,
+		MesosTaskManagerParameters tmParams,
+		Configuration taskManagerConfig,
+		String workingDirectory,
+		Class<?> taskManagerMainClass,
+		MesosArtifactServer artifactServer,
+		Logger log) throws Exception {
+
+
+		Protos.TaskInfo.Builder info = Protos.TaskInfo.newBuilder();
+		Protos.CommandInfo.Builder cmd = Protos.CommandInfo.newBuilder();
+
+		log.info("Setting up artifacts for TaskManagers");
+
+		String shipListString = env.get(MesosConfigKeys.ENV_CLIENT_SHIP_FILES);
+		require(shipListString != null, "Environment variable %s not set", MesosConfigKeys.ENV_CLIENT_SHIP_FILES);
+
+		String clientUsername = env.get(MesosConfigKeys.ENV_CLIENT_USERNAME);
+		require(clientUsername != null, "Environment variable %s not set", MesosConfigKeys.ENV_CLIENT_USERNAME);
+
+		String classPathString = env.get(MesosConfigKeys.ENV_FLINK_CLASSPATH);
+		require(classPathString != null, "Environment variable %s not set", MesosConfigKeys.ENV_FLINK_CLASSPATH);
+
+		// register the Flink jar
+		final File flinkJarFile = new File(workingDirectory, "flink.jar");
+		cmd.addUris(uri(artifactServer.addFile(flinkJarFile, "flink.jar"), true));
+
+		// register the TaskManager configuration
+		final File taskManagerConfigFile =
+			new File(workingDirectory, UUID.randomUUID() + "-taskmanager-conf.yaml");
+		LOG.debug("Writing TaskManager configuration to {}", taskManagerConfigFile.getAbsolutePath());
+		BootstrapTools.writeConfiguration(taskManagerConfig, taskManagerConfigFile);
+		cmd.addUris(uri(artifactServer.addFile(taskManagerConfigFile, GlobalConfiguration.FLINK_CONF_FILENAME), true));
+
+		// prepare additional files to be shipped
+		for (String pathStr : shipListString.split(",")) {
+			if (!pathStr.isEmpty()) {
+				File shipFile = new File(workingDirectory, pathStr);
+				cmd.addUris(uri(artifactServer.addFile(shipFile, shipFile.getName()), true));
+			}
+		}
+
+		log.info("Creating task info for TaskManagers");
+
+		// build the launch command
+		boolean hasLogback = new File(workingDirectory, "logback.xml").exists();
+		boolean hasLog4j = new File(workingDirectory, "log4j.properties").exists();
+
+		String launchCommand = BootstrapTools.getTaskManagerShellCommand(
+			flinkConfig, tmParams.containeredParameters(), ".", ".",
+			hasLogback, hasLog4j, taskManagerMainClass);
+		cmd.setValue(launchCommand);
+
+		// build the environment variables
+		Protos.Environment.Builder envBuilder = Protos.Environment.newBuilder();
+		for (Map.Entry<String, String> entry : tmParams.containeredParameters().taskManagerEnv().entrySet()) {
+			envBuilder.addVariables(variable(entry.getKey(), entry.getValue()));
+		}
+		envBuilder.addVariables(variable(MesosConfigKeys.ENV_CLASSPATH, classPathString));
+		envBuilder.addVariables(variable(MesosConfigKeys.ENV_CLIENT_USERNAME, clientUsername));
+
+		cmd.setEnvironment(envBuilder);
+
+		info.setCommand(cmd);
+
+		return info;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/d9b2be05/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosConfigKeys.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosConfigKeys.java b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosConfigKeys.java
new file mode 100644
index 0000000..3173286
--- /dev/null
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosConfigKeys.java
@@ -0,0 +1,26 @@
+package org.apache.flink.mesos.runtime.clusterframework;
+
+/**
+ * The Mesos environment variables used for settings of the containers.
+ */
+public class MesosConfigKeys {
+	// ------------------------------------------------------------------------
+	//  Environment variable names
+	// ------------------------------------------------------------------------
+
+	public static final String ENV_TM_MEMORY = "_CLIENT_TM_MEMORY";
+	public static final String ENV_TM_COUNT = "_CLIENT_TM_COUNT";
+	public static final String ENV_SLOTS = "_SLOTS";
+	public static final String ENV_CLIENT_SHIP_FILES = "_CLIENT_SHIP_FILES";
+	public static final String ENV_CLIENT_USERNAME = "_CLIENT_USERNAME";
+	public static final String ENV_DYNAMIC_PROPERTIES = "_DYNAMIC_PROPERTIES";
+	public static final String ENV_FLINK_CONTAINER_ID = "_FLINK_CONTAINER_ID";
+	public static final String ENV_FLINK_TMP_DIR = "_FLINK_TMP_DIR";
+	public static final String ENV_FLINK_CLASSPATH = "_FLINK_CLASSPATH";
+	public static final String ENV_CLASSPATH = "CLASSPATH";
+	public static final String ENV_MESOS_SANDBOX = "MESOS_SANDBOX";
+	public static final String ENV_SESSION_ID = "_CLIENT_SESSION_ID";
+
+	/** Private constructor to prevent instantiation */
+	private MesosConfigKeys() {}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/d9b2be05/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosFlinkResourceManager.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosFlinkResourceManager.java b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosFlinkResourceManager.java
new file mode 100644
index 0000000..483c7b7
--- /dev/null
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosFlinkResourceManager.java
@@ -0,0 +1,737 @@
+package org.apache.flink.mesos.runtime.clusterframework;
+
+import akka.actor.ActorRef;
+import akka.actor.Props;
+import com.netflix.fenzo.TaskRequest;
+import com.netflix.fenzo.TaskScheduler;
+import com.netflix.fenzo.VirtualMachineLease;
+import com.netflix.fenzo.functions.Action1;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.mesos.runtime.clusterframework.store.MesosWorkerStore;
+import org.apache.flink.mesos.scheduler.ConnectionMonitor;
+import org.apache.flink.mesos.scheduler.LaunchableTask;
+import org.apache.flink.mesos.scheduler.LaunchCoordinator;
+import org.apache.flink.mesos.scheduler.ReconciliationCoordinator;
+import org.apache.flink.mesos.scheduler.SchedulerProxy;
+import org.apache.flink.mesos.scheduler.TaskMonitor;
+import org.apache.flink.mesos.scheduler.TaskSchedulerBuilder;
+import org.apache.flink.mesos.scheduler.Tasks;
+import org.apache.flink.mesos.scheduler.messages.AcceptOffers;
+import org.apache.flink.mesos.scheduler.messages.Disconnected;
+import org.apache.flink.mesos.scheduler.messages.Error;
+import org.apache.flink.mesos.scheduler.messages.OfferRescinded;
+import org.apache.flink.mesos.scheduler.messages.ReRegistered;
+import org.apache.flink.mesos.scheduler.messages.Registered;
+import org.apache.flink.mesos.scheduler.messages.ResourceOffers;
+import org.apache.flink.mesos.scheduler.messages.StatusUpdate;
+import org.apache.flink.mesos.util.MesosConfiguration;
+import org.apache.flink.runtime.clusterframework.ApplicationStatus;
+import org.apache.flink.runtime.clusterframework.FlinkResourceManager;
+import org.apache.flink.runtime.clusterframework.messages.FatalErrorOccurred;
+import org.apache.flink.runtime.clusterframework.messages.StopCluster;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.mesos.Protos;
+import org.apache.mesos.Protos.FrameworkInfo;
+import org.apache.mesos.SchedulerDriver;
+import org.slf4j.Logger;
+import scala.Option;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static java.util.Objects.requireNonNull;
+
+/**
+ * Flink Resource Manager for Apache Mesos.
+ */
+public class MesosFlinkResourceManager extends FlinkResourceManager<RegisteredMesosWorkerNode> {
+
+	/** The Mesos configuration (master and framework info) */
+	private final MesosConfiguration mesosConfig;
+
+	/** The TaskManager container parameters (like container memory size) */
+	private final MesosTaskManagerParameters taskManagerParameters;
+
+	/** Context information used to start a TaskManager Java process */
+	private final Protos.TaskInfo.Builder taskManagerLaunchContext;
+
+	/** Number of failed Mesos tasks before stopping the application. -1 means infinite. */
+	private final int maxFailedTasks;
+
+	/** Callback handler for the asynchronous Mesos scheduler */
+	private SchedulerProxy schedulerCallbackHandler;
+
+	/** Mesos scheduler driver */
+	private SchedulerDriver schedulerDriver;
+
+	private ActorRef connectionMonitor;
+
+	private ActorRef taskRouter;
+
+	private ActorRef launchCoordinator;
+
+	private ActorRef reconciliationCoordinator;
+
+	private MesosWorkerStore workerStore;
+
+	final Map<ResourceID, MesosWorkerStore.Worker> workersInNew;
+	final Map<ResourceID, MesosWorkerStore.Worker> workersInLaunch;
+	final Map<ResourceID, MesosWorkerStore.Worker> workersBeingReturned;
+
+	/** The number of failed tasks since the master became active */
+	private int failedTasksSoFar;
+
+	public MesosFlinkResourceManager(
+		Configuration flinkConfig,
+		MesosConfiguration mesosConfig,
+		MesosWorkerStore workerStore,
+		LeaderRetrievalService leaderRetrievalService,
+		MesosTaskManagerParameters taskManagerParameters,
+		Protos.TaskInfo.Builder taskManagerLaunchContext,
+		int maxFailedTasks,
+		int numInitialTaskManagers) {
+
+		super(numInitialTaskManagers, flinkConfig, leaderRetrievalService);
+
+		this.mesosConfig = requireNonNull(mesosConfig);
+
+		this.workerStore = requireNonNull(workerStore);
+
+		this.taskManagerParameters = requireNonNull(taskManagerParameters);
+		this.taskManagerLaunchContext = requireNonNull(taskManagerLaunchContext);
+		this.maxFailedTasks = maxFailedTasks;
+
+		this.workersInNew = new HashMap<>();
+		this.workersInLaunch = new HashMap<>();
+		this.workersBeingReturned = new HashMap<>();
+	}
+
+	// ------------------------------------------------------------------------
+	//  Mesos-specific behavior
+	// ------------------------------------------------------------------------
+
+	@Override
+	protected void initialize() throws Exception {
+		LOG.info("Initializing Mesos resource master");
+
+		workerStore.start();
+
+		// create the scheduler driver to communicate with Mesos
+		schedulerCallbackHandler = new SchedulerProxy(self());
+
+		// register with Mesos
+		FrameworkInfo.Builder frameworkInfo = mesosConfig.frameworkInfo()
+			.clone()
+			.setCheckpoint(true);
+
+		Option<Protos.FrameworkID> frameworkID = workerStore.getFrameworkID();
+		if(frameworkID.isEmpty()) {
+			LOG.info("Registering as new framework.");
+		}
+		else {
+			LOG.info("Recovery scenario: re-registering using framework ID {}.", frameworkID.get().getValue());
+			frameworkInfo.setId(frameworkID.get());
+		}
+
+		MesosConfiguration initializedMesosConfig = mesosConfig.withFrameworkInfo(frameworkInfo);
+		MesosConfiguration.logMesosConfig(LOG, initializedMesosConfig);
+		schedulerDriver = initializedMesosConfig.createDriver(schedulerCallbackHandler, false);
+
+		// create supporting actors
+		connectionMonitor = createConnectionMonitor();
+		launchCoordinator = createLaunchCoordinator();
+		reconciliationCoordinator = createReconciliationCoordinator();
+		taskRouter = createTaskRouter();
+
+		recoverWorkers();
+
+		connectionMonitor.tell(new ConnectionMonitor.Start(), self());
+		schedulerDriver.start();
+	}
+
+	protected ActorRef createConnectionMonitor() {
+		return context().actorOf(
+			ConnectionMonitor.createActorProps(ConnectionMonitor.class, config),
+			"connectionMonitor");
+	}
+
+	protected ActorRef createTaskRouter() {
+		return context().actorOf(
+			Tasks.createActorProps(Tasks.class, config, schedulerDriver, TaskMonitor.class),
+			"tasks");
+	}
+
+	protected ActorRef createLaunchCoordinator() {
+		return context().actorOf(
+			LaunchCoordinator.createActorProps(LaunchCoordinator.class, self(), config, schedulerDriver, createOptimizer()),
+			"launchCoordinator");
+	}
+
+	protected ActorRef createReconciliationCoordinator() {
+		return context().actorOf(
+			ReconciliationCoordinator.createActorProps(ReconciliationCoordinator.class, config, schedulerDriver),
+			"reconciliationCoordinator");
+	}
+
+	@Override
+	public void postStop() {
+		LOG.info("Stopping Mesos resource master");
+		super.postStop();
+	}
+
+	// ------------------------------------------------------------------------
+	//  Actor messages
+	// ------------------------------------------------------------------------
+
+	@Override
+	protected void handleMessage(Object message) {
+
+		// check for Mesos-specific actor messages first
+
+		// --- messages about Mesos connection
+		if (message instanceof Registered) {
+			registered((Registered) message);
+		} else if (message instanceof ReRegistered) {
+			reregistered((ReRegistered) message);
+		} else if (message instanceof Disconnected) {
+			disconnected((Disconnected) message);
+		} else if (message instanceof Error) {
+			error(((Error) message).message());
+
+		// --- messages about offers
+		} else if (message instanceof ResourceOffers || message instanceof OfferRescinded) {
+			launchCoordinator.tell(message, self());
+		} else if (message instanceof AcceptOffers) {
+			acceptOffers((AcceptOffers) message);
+
+		// --- messages about tasks
+		} else if (message instanceof StatusUpdate) {
+			taskStatusUpdated((StatusUpdate) message);
+		} else if (message instanceof ReconciliationCoordinator.Reconcile) {
+			// a reconciliation request from a task
+			reconciliationCoordinator.tell(message, self());
+		} else if (message instanceof TaskMonitor.TaskTerminated) {
+			// a termination message from a task
+			TaskMonitor.TaskTerminated msg = (TaskMonitor.TaskTerminated) message;
+			taskTerminated(msg.taskID(), msg.status());
+
+		} else  {
+			// message handled by the generic resource master code
+			super.handleMessage(message);
+		}
+	}
+
+	/**
+	 * Called to shut down the cluster (not a failover situation).
+	 *
+	 * @param finalStatus The application status to report.
+	 * @param optionalDiagnostics An optional diagnostics message.
+     */
+	@Override
+	protected void shutdownApplication(ApplicationStatus finalStatus, String optionalDiagnostics) {
+
+		LOG.info("Shutting down and unregistering as a Mesos framework.");
+		try {
+			// unregister the framework, which implicitly removes all tasks.
+			schedulerDriver.stop(false);
+		}
+		catch(Exception ex) {
+			LOG.warn("unable to unregister the framework", ex);
+		}
+
+		try {
+			workerStore.cleanup();
+		}
+		catch(Exception ex) {
+			LOG.warn("unable to cleanup the ZooKeeper state", ex);
+		}
+
+		context().stop(self());
+	}
+
+	@Override
+	protected void fatalError(String message, Throwable error) {
+		// we do not unregister, but cause a hard fail of this process, to have it
+		// restarted by the dispatcher
+		LOG.error("FATAL ERROR IN MESOS APPLICATION MASTER: " + message, error);
+		LOG.error("Shutting down process");
+
+		// kill this process, this will make an external supervisor (the dispatcher) restart the process
+		System.exit(EXIT_CODE_FATAL_ERROR);
+	}
+
+	// ------------------------------------------------------------------------
+	//  Worker Management
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Recover framework/worker information persisted by a prior incarnation of the RM.
+	 */
+	private void recoverWorkers() throws Exception {
+		// if this application master starts as part of an ApplicationMaster/JobManager recovery,
+		// then some worker tasks are most likely still alive and we can re-obtain them
+		final List<MesosWorkerStore.Worker> tasksFromPreviousAttempts = workerStore.recoverWorkers();
+
+		if (!tasksFromPreviousAttempts.isEmpty()) {
+			LOG.info("Retrieved {} TaskManagers from previous attempt", tasksFromPreviousAttempts.size());
+
+			List<Tuple2<TaskRequest,String>> toAssign = new ArrayList<>(tasksFromPreviousAttempts.size());
+			List<LaunchableTask> toLaunch = new ArrayList<>(tasksFromPreviousAttempts.size());
+
+			for (final MesosWorkerStore.Worker worker : tasksFromPreviousAttempts) {
+				LaunchableMesosWorker launchable = createLaunchableMesosWorker(worker.taskID());
+
+				switch(worker.state()) {
+					case New:
+						workersInNew.put(extractResourceID(worker.taskID()), worker);
+						toLaunch.add(launchable);
+						break;
+					case Launched:
+						workersInLaunch.put(extractResourceID(worker.taskID()), worker);
+						toAssign.add(new Tuple2<>(launchable.taskRequest(), worker.hostname().get()));
+						break;
+					case Released:
+						workersBeingReturned.put(extractResourceID(worker.taskID()), worker);
+						break;
+				}
+				taskRouter.tell(new TaskMonitor.TaskGoalStateUpdated(extractGoalState(worker)), self());
+			}
+
+			// tell the launch coordinator about prior assignments
+			if(toAssign.size() >= 1) {
+				launchCoordinator.tell(new LaunchCoordinator.Assign(toAssign), self());
+			}
+			// tell the launch coordinator to launch any new tasks
+			if(toLaunch.size() >= 1) {
+				launchCoordinator.tell(new LaunchCoordinator.Launch(toLaunch), self());
+			}
+		}
+	}
+
+	/**
+	 * Plan for some additional workers to be launched.
+	 *
+	 * @param numWorkers The number of workers to allocate.
+     */
+	@Override
+	protected void requestNewWorkers(int numWorkers) {
+
+		try {
+			List<TaskMonitor.TaskGoalStateUpdated> toMonitor = new ArrayList<>(numWorkers);
+			List<LaunchableTask> toLaunch = new ArrayList<>(numWorkers);
+
+			// generate new workers into persistent state and launch associated actors
+			for (int i = 0; i < numWorkers; i++) {
+				MesosWorkerStore.Worker worker = MesosWorkerStore.Worker.newTask(workerStore.newTaskID());
+				workerStore.putWorker(worker);
+				workersInNew.put(extractResourceID(worker.taskID()), worker);
+
+				LaunchableMesosWorker launchable = createLaunchableMesosWorker(worker.taskID());
+
+				LOG.info("Scheduling Mesos task {} with ({} MB, {} cpus).",
+					launchable.taskID().getValue(), launchable.taskRequest().getMemory(), launchable.taskRequest().getCPUs());
+
+				toMonitor.add(new TaskMonitor.TaskGoalStateUpdated(extractGoalState(worker)));
+				toLaunch.add(launchable);
+			}
+
+			// tell the task router about the new plans
+			for (TaskMonitor.TaskGoalStateUpdated update : toMonitor) {
+				taskRouter.tell(update, self());
+			}
+
+			// tell the launch coordinator to launch the new tasks
+			if(toLaunch.size() >= 1) {
+				launchCoordinator.tell(new LaunchCoordinator.Launch(toLaunch), self());
+			}
+		}
+		catch(Exception ex) {
+			fatalError("unable to request new workers", ex);
+		}
+	}
+
+	/**
+	 * Accept offers as advised by the launch coordinator.
+	 *
+	 * Acceptance is routed through the RM to update the persistent state before
+	 * forwarding the message to Mesos.
+     */
+	private void acceptOffers(AcceptOffers msg) {
+
+		try {
+			List<TaskMonitor.TaskGoalStateUpdated> toMonitor = new ArrayList<>(msg.operations().size());
+
+			// transition the persistent state of some tasks to Launched
+			for (Protos.Offer.Operation op : msg.operations()) {
+				if (op.getType() != Protos.Offer.Operation.Type.LAUNCH) {
+					continue;
+				}
+				for (Protos.TaskInfo info : op.getLaunch().getTaskInfosList()) {
+					MesosWorkerStore.Worker worker = workersInNew.remove(extractResourceID(info.getTaskId()));
+					assert (worker != null);
+
+					worker = worker.launchTask(info.getSlaveId(), msg.hostname());
+					workerStore.putWorker(worker);
+					workersInLaunch.put(extractResourceID(worker.taskID()), worker);
+
+					LOG.info("Launching Mesos task {} on host {}.",
+						worker.taskID().getValue(), worker.hostname().get());
+
+					toMonitor.add(new TaskMonitor.TaskGoalStateUpdated(extractGoalState(worker)));
+				}
+			}
+
+			// tell the task router about the new plans
+			for (TaskMonitor.TaskGoalStateUpdated update : toMonitor) {
+				taskRouter.tell(update, self());
+			}
+
+			// send the acceptance message to Mesos
+			schedulerDriver.acceptOffers(msg.offerIds(), msg.operations(), msg.filters());
+		}
+		catch(Exception ex) {
+			fatalError("unable to accept offers", ex);
+		}
+	}
+
+	/**
+	 * Handle a task status change.
+     */
+	private void taskStatusUpdated(StatusUpdate message) {
+		taskRouter.tell(message, self());
+		reconciliationCoordinator.tell(message, self());
+		schedulerDriver.acknowledgeStatusUpdate(message.status());
+	}
+
+	/**
+	 * Accept the given started worker into the internal state.
+	 *
+	 * @param resourceID The worker resource id
+	 * @return A registered worker node record.
+	 */
+	@Override
+	protected RegisteredMesosWorkerNode workerStarted(ResourceID resourceID) {
+		MesosWorkerStore.Worker inLaunch = workersInLaunch.remove(resourceID);
+		if (inLaunch == null) {
+			// Worker was not in state "being launched", this can indicate that the TaskManager
+			// in this worker was already registered or that the container was not started
+			// by this resource manager. Simply ignore this resourceID.
+			return null;
+		}
+		return new RegisteredMesosWorkerNode(inLaunch);
+	}
+
+	/**
+	 * Accept the given registered workers into the internal state.
+	 *
+	 * @param toConsolidate The worker IDs known previously to the JobManager.
+	 * @return A collection of registered worker node records.
+     */
+	@Override
+	protected Collection<RegisteredMesosWorkerNode> reacceptRegisteredWorkers(Collection<ResourceID> toConsolidate) {
+
+		// we check for each task manager if we recognize its Mesos task ID
+		List<RegisteredMesosWorkerNode> accepted = new ArrayList<>(toConsolidate.size());
+		for (ResourceID resourceID : toConsolidate) {
+			MesosWorkerStore.Worker worker = workersInLaunch.remove(resourceID);
+			if (worker != null) {
+				LOG.info("Mesos worker consolidation recognizes TaskManager {}.", resourceID);
+				accepted.add(new RegisteredMesosWorkerNode(worker));
+			}
+			else {
+				if(isStarted(resourceID)) {
+					LOG.info("TaskManager {} has already been registered at the resource manager.", resourceID);
+				}
+				else {
+					LOG.info("Mesos worker consolidation does not recognize TaskManager {}.", resourceID);
+				}
+			}
+		}
+		return accepted;
+	}
+
+	/**
+	 * Release the given pending worker.
+	 */
+	@Override
+	protected void releasePendingWorker(ResourceID id) {
+		MesosWorkerStore.Worker worker = workersInLaunch.remove(id);
+		if (worker != null) {
+			releaseWorker(worker);
+		} else {
+			LOG.error("Cannot find worker {} to release. Ignoring request.", id);
+		}
+	}
+
+	/**
+	 * Release the given started worker.
+	 */
+	@Override
+	protected void releaseStartedWorker(RegisteredMesosWorkerNode worker) {
+		releaseWorker(worker.task());
+	}
+
+	/**
+	 * Plan for the removal of the given worker.
+     */
+	private void releaseWorker(MesosWorkerStore.Worker worker) {
+		try {
+			LOG.info("Releasing worker {}", worker.taskID());
+
+			// update persistent state of worker to Released
+			worker = worker.releaseTask();
+			workerStore.putWorker(worker);
+			workersBeingReturned.put(extractResourceID(worker.taskID()), worker);
+			taskRouter.tell(new TaskMonitor.TaskGoalStateUpdated(extractGoalState(worker)), self());
+
+			if (worker.hostname().isDefined()) {
+				// tell the launch coordinator that the task is being unassigned from the host, for planning purposes
+				launchCoordinator.tell(new LaunchCoordinator.Unassign(worker.taskID(), worker.hostname().get()), self());
+			}
+		}
+		catch (Exception ex) {
+			fatalError("unable to release worker", ex);
+		}
+	}
+
+	@Override
+	protected int getNumWorkerRequestsPending() {
+		return workersInNew.size();
+	}
+
+	@Override
+	protected int getNumWorkersPendingRegistration() {
+		return workersInLaunch.size();
+	}
+
+	// ------------------------------------------------------------------------
+	//  Callbacks from the Mesos Master
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Called when connected to Mesos as a new framework.
+	 */
+	private void registered(Registered message) {
+		connectionMonitor.tell(message, self());
+
+		try {
+			workerStore.setFrameworkID(Option.apply(message.frameworkId()));
+		}
+		catch(Exception ex) {
+			fatalError("unable to store the assigned framework ID", ex);
+			return;
+		}
+
+		launchCoordinator.tell(message, self());
+		reconciliationCoordinator.tell(message, self());
+		taskRouter.tell(message, self());
+	}
+
+	/**
+	 * Called when reconnected to Mesos following a failover event.
+	 */
+	private void reregistered(ReRegistered message) {
+		connectionMonitor.tell(message, self());
+		launchCoordinator.tell(message, self());
+		reconciliationCoordinator.tell(message, self());
+		taskRouter.tell(message, self());
+	}
+
+	/**
+	 * Called when disconnected from Mesos.
+	 */
+	private void disconnected(Disconnected message) {
+		connectionMonitor.tell(message, self());
+		launchCoordinator.tell(message, self());
+		reconciliationCoordinator.tell(message, self());
+		taskRouter.tell(message, self());
+	}
+
+	/**
+	 * Called when an error is reported by the scheduler callback.
+	 */
+	private void error(String message) {
+		self().tell(new FatalErrorOccurred("Connection to Mesos failed", new Exception(message)), self());
+	}
+
+	/**
+	 * Invoked when a Mesos task reaches a terminal status.
+     */
+	private void taskTerminated(Protos.TaskID taskID, Protos.TaskStatus status) {
+		// this callback occurs for failed containers and for released containers alike
+
+		final ResourceID id = extractResourceID(taskID);
+
+		try {
+			workerStore.removeWorker(taskID);
+		}
+		catch(Exception ex) {
+			fatalError("unable to remove worker", ex);
+			return;
+		}
+
+		// check if this is a failed task or a released task
+		if (workersBeingReturned.remove(id) != null) {
+			// regular finished worker that we released
+			LOG.info("Worker {} finished successfully with diagnostics: {}",
+				id, status.getMessage());
+		} else {
+			// failed worker, either at startup, or running
+			final MesosWorkerStore.Worker launched = workersInLaunch.remove(id);
+			if (launched != null) {
+				LOG.info("Mesos task {} failed, with a TaskManager in launch or registration. " +
+					"State: {} Reason: {} ({})", id, status.getState(), status.getReason(), status.getMessage());
+				// we will trigger re-acquiring new workers at the end
+			} else {
+				// failed registered worker
+				LOG.info("Mesos task {} failed, with a registered TaskManager. " +
+					"State: {} Reason: {} ({})", id, status.getState(), status.getReason(), status.getMessage());
+
+				// notify the generic logic, which notifies the JobManager, etc.
+				notifyWorkerFailed(id, "Mesos task " + id + " failed.  State: " + status.getState());
+			}
+
+			// general failure logging
+			failedTasksSoFar++;
+
+			String diagMessage = String.format("Diagnostics for task %s in state %s : " +
+					"reason=%s message=%s",
+				id, status.getState(), status.getReason(), status.getMessage());
+			sendInfoMessage(diagMessage);
+
+			LOG.info(diagMessage);
+			LOG.info("Total number of failed tasks so far: " + failedTasksSoFar);
+
+			// maxFailedTasks == -1 is infinite number of retries.
+			if (maxFailedTasks >= 0 && failedTasksSoFar > maxFailedTasks) {
+				String msg = "Stopping Mesos session because the number of failed tasks ("
+					+ failedTasksSoFar + ") exceeded the maximum failed tasks ("
+					+ maxFailedTasks + "). This number is controlled by the '"
+					+ ConfigConstants.MESOS_MAX_FAILED_TASKS + "' configuration setting. "
+					+ "By default its the number of requested tasks.";
+
+				LOG.error(msg);
+				self().tell(decorateMessage(new StopCluster(ApplicationStatus.FAILED, msg)),
+					ActorRef.noSender());
+
+				// no need to do anything else
+				return;
+			}
+		}
+
+		// in case failed containers were among the finished containers, make
+		// sure we re-examine and request new ones
+		triggerCheckWorkers();
+	}
+
+	// ------------------------------------------------------------------------
+	//  Utilities
+	// ------------------------------------------------------------------------
+
+	private LaunchableMesosWorker createLaunchableMesosWorker(Protos.TaskID taskID) {
+		LaunchableMesosWorker launchable =
+			new LaunchableMesosWorker(taskManagerParameters, taskManagerLaunchContext, taskID);
+		return launchable;
+	}
+
+	/**
+	 * Extracts a unique ResourceID from the Mesos task.
+	 *
+	 * @param taskId the Mesos TaskID
+	 * @return The ResourceID for the container
+	 */
+	static ResourceID extractResourceID(Protos.TaskID taskId) {
+		return new ResourceID(taskId.getValue());
+	}
+
+	/**
+	 * Extracts the Mesos task goal state from the worker information.
+	 * @param worker the persistent worker information.
+	 * @return goal state information for the {@Link TaskMonitor}.
+     */
+	static TaskMonitor.TaskGoalState extractGoalState(MesosWorkerStore.Worker worker) {
+		switch(worker.state()) {
+			case New: return new TaskMonitor.New(worker.taskID());
+			case Launched: return new TaskMonitor.Launched(worker.taskID(), worker.slaveID().get());
+			case Released: return new TaskMonitor.Released(worker.taskID(), worker.slaveID().get());
+			default: throw new IllegalArgumentException();
+		}
+	}
+
+	/**
+	 * Creates the Fenzo optimizer (builder).
+	 * The builder is an indirection to faciliate unit testing of the Launch Coordinator.
+     */
+	private static TaskSchedulerBuilder createOptimizer() {
+		return new TaskSchedulerBuilder() {
+			TaskScheduler.Builder builder = new TaskScheduler.Builder();
+
+			@Override
+			public TaskSchedulerBuilder withLeaseRejectAction(Action1<VirtualMachineLease> action) {
+				builder.withLeaseRejectAction(action);
+				return this;
+			}
+
+			@Override
+			public TaskScheduler build() {
+				return builder.build();
+			}
+		};
+	}
+
+	/**
+	 * Creates the props needed to instantiate this actor.
+	 *
+	 * Rather than extracting and validating parameters in the constructor, this factory method takes
+	 * care of that. That way, errors occur synchronously, and are not swallowed simply in a
+	 * failed asynchronous attempt to start the actor.
+
+	 * @param actorClass
+	 *             The actor class, to allow overriding this actor with subclasses for testing.
+	 * @param flinkConfig
+	 *             The Flink configuration object.
+	 * @param taskManagerParameters
+	 *             The parameters for launching TaskManager containers.
+	 * @param taskManagerLaunchContext
+	 *             The parameters for launching the TaskManager processes in the TaskManager containers.
+	 * @param numInitialTaskManagers
+	 *             The initial number of TaskManagers to allocate.
+	 * @param log
+	 *             The logger to log to.
+	 *
+	 * @return The Props object to instantiate the MesosFlinkResourceManager actor.
+	 */
+	public static Props createActorProps(Class<? extends MesosFlinkResourceManager> actorClass,
+			Configuration flinkConfig,
+			MesosConfiguration mesosConfig,
+			MesosWorkerStore workerStore,
+			LeaderRetrievalService leaderRetrievalService,
+			MesosTaskManagerParameters taskManagerParameters,
+			Protos.TaskInfo.Builder taskManagerLaunchContext,
+			int numInitialTaskManagers,
+			Logger log)
+	{
+		final int maxFailedTasks = flinkConfig.getInteger(
+			ConfigConstants.MESOS_MAX_FAILED_TASKS, numInitialTaskManagers);
+		if (maxFailedTasks >= 0) {
+			log.info("Mesos framework tolerates {} failed tasks before giving up",
+				maxFailedTasks);
+		}
+
+		return Props.create(actorClass,
+			flinkConfig,
+			mesosConfig,
+			workerStore,
+			leaderRetrievalService,
+			taskManagerParameters,
+			taskManagerLaunchContext,
+			maxFailedTasks,
+			numInitialTaskManagers);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/d9b2be05/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManagerParameters.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManagerParameters.java b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManagerParameters.java
new file mode 100644
index 0000000..b3956aa
--- /dev/null
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManagerParameters.java
@@ -0,0 +1,51 @@
+package org.apache.flink.mesos.runtime.clusterframework;
+
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.clusterframework.ContaineredTaskManagerParameters;
+
+import static java.util.Objects.requireNonNull;
+
+public class MesosTaskManagerParameters {
+
+	private double cpus;
+
+	private ContaineredTaskManagerParameters containeredParameters;
+
+	public MesosTaskManagerParameters(double cpus, ContaineredTaskManagerParameters containeredParameters) {
+		requireNonNull(containeredParameters);
+		this.cpus = cpus;
+		this.containeredParameters = containeredParameters;
+	}
+
+	public double cpus() {
+		return cpus;
+	}
+
+	public ContaineredTaskManagerParameters containeredParameters() {
+		return containeredParameters;
+	}
+
+	@Override
+	public String toString() {
+		return "MesosTaskManagerParameters{" +
+			"cpus=" + cpus +
+			", containeredParameters=" + containeredParameters +
+			'}';
+	}
+
+	/**
+	 * Create the Mesos TaskManager parameters.
+	 * @param flinkConfig the TM configuration.
+	 * @param containeredParameters additional containered parameters.
+     */
+	public static MesosTaskManagerParameters create(
+		Configuration flinkConfig,
+		ContaineredTaskManagerParameters containeredParameters) {
+
+		double cpus = flinkConfig.getDouble(ConfigConstants.MESOS_RESOURCEMANAGER_TASKS_CPUS,
+			Math.max(containeredParameters.numSlots(), 1.0));
+
+		return new MesosTaskManagerParameters(cpus, containeredParameters);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/d9b2be05/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManagerRunner.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManagerRunner.java b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManagerRunner.java
new file mode 100644
index 0000000..5dfc75e
--- /dev/null
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManagerRunner.java
@@ -0,0 +1,99 @@
+package org.apache.flink.mesos.runtime.clusterframework;
+
+import java.io.IOException;
+import java.security.PrivilegedAction;
+import java.util.Map;
+
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.mesos.cli.FlinkMesosSessionCli;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.taskmanager.TaskManager;
+import org.apache.flink.runtime.util.EnvironmentInformation;
+
+import org.apache.flink.util.Preconditions;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The entry point for running a TaskManager in a Mesos container.
+ */
+public class MesosTaskManagerRunner {
+
+	private static final Logger LOG = LoggerFactory.getLogger(MesosTaskManagerRunner.class);
+
+	/** The process environment variables */
+	private static final Map<String, String> ENV = System.getenv();
+
+	public static void runTaskManager(String[] args, final Class<? extends TaskManager> taskManager) throws IOException {
+		EnvironmentInformation.logEnvironmentInfo(LOG, taskManager.getSimpleName(), args);
+		org.apache.flink.runtime.util.SignalHandler.register(LOG);
+
+		// try to parse the command line arguments
+		final Configuration configuration;
+		try {
+			configuration = TaskManager.parseArgsAndLoadConfig(args);
+
+			// add dynamic properties to TaskManager configuration.
+			final Configuration dynamicProperties =
+				FlinkMesosSessionCli.decodeDynamicProperties(ENV.get(MesosConfigKeys.ENV_DYNAMIC_PROPERTIES));
+			LOG.debug("Mesos dynamic properties: {}", dynamicProperties);
+			configuration.addAll(dynamicProperties);
+		}
+		catch (Throwable t) {
+			LOG.error(t.getMessage(), t);
+			System.exit(TaskManager.STARTUP_FAILURE_RETURN_CODE());
+			return;
+		}
+
+		// read the environment variables
+		final Map<String, String> envs = System.getenv();
+		final String effectiveUsername = envs.get(MesosConfigKeys.ENV_CLIENT_USERNAME);
+		final String tmpDirs = envs.get(MesosConfigKeys.ENV_FLINK_TMP_DIR);
+
+		// configure local directory
+		String flinkTempDirs = configuration.getString(ConfigConstants.TASK_MANAGER_TMP_DIR_KEY, null);
+		if (flinkTempDirs != null) {
+			LOG.info("Overriding Mesos temporary file directories with those " +
+				"specified in the Flink config: " + flinkTempDirs);
+		}
+		else if (tmpDirs != null) {
+			LOG.info("Setting directories for temporary files to: " + tmpDirs);
+			configuration.setString(ConfigConstants.TASK_MANAGER_TMP_DIR_KEY, tmpDirs);
+		}
+
+		LOG.info("Mesos task runs as '" + UserGroupInformation.getCurrentUser().getShortUserName() +
+			"', setting user to execute Flink TaskManager to '" + effectiveUsername + "'");
+
+		// tell akka to die in case of an error
+		configuration.setBoolean(ConfigConstants.AKKA_JVM_EXIT_ON_FATAL_ERROR, true);
+
+		UserGroupInformation ugi = UserGroupInformation.createRemoteUser(effectiveUsername);
+		for (Token<? extends TokenIdentifier> toks : UserGroupInformation.getCurrentUser().getTokens()) {
+			ugi.addToken(toks);
+		}
+
+		// Infer the resource identifier from the environment variable
+		String containerID = Preconditions.checkNotNull(envs.get(MesosConfigKeys.ENV_FLINK_CONTAINER_ID));
+		final ResourceID resourceId = new ResourceID(containerID);
+		LOG.info("ResourceID assigned for this container: {}", resourceId);
+
+		ugi.doAs(new PrivilegedAction<Object>() {
+			@Override
+			public Object run() {
+				try {
+					TaskManager.selectNetworkInterfaceAndRunTaskManager(configuration, resourceId, taskManager);
+				}
+				catch (Throwable t) {
+					LOG.error("Error while starting the TaskManager", t);
+					System.exit(TaskManager.STARTUP_FAILURE_RETURN_CODE());
+				}
+				return null;
+			}
+		});
+	}
+}


[9/9] flink git commit: [FLINK-1984] port Mesos code to latest master

Posted by mx...@apache.org.
[FLINK-1984] port Mesos code to latest master

- move Scala code to /scala dir
- remove merge commits
- update version

This closes #2315


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

Branch: refs/heads/master
Commit: 842e3e7d13c821fccd599642a417b6328915a366
Parents: 38a9534
Author: Maximilian Michels <mx...@apache.org>
Authored: Mon Aug 29 17:08:01 2016 +0200
Committer: Maximilian Michels <mx...@apache.org>
Committed: Mon Aug 29 17:32:51 2016 +0200

----------------------------------------------------------------------
 .../flink/configuration/ConfigConstants.java    |   7 +-
 flink-mesos/pom.xml                             |   2 +-
 .../MesosApplicationMasterRunner.java           |   8 +-
 .../store/ZooKeeperMesosWorkerStore.java        |   2 +-
 .../ContaineredJobManager.scala                 | 172 -------------------
 .../ContaineredJobManager.scala                 | 172 +++++++++++++++++++
 6 files changed, 182 insertions(+), 181 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/842e3e7d/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
index 2fe27e0..f0f1b6b 100644
--- a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
+++ b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
@@ -742,6 +742,10 @@ public final class ConfigConstants {
 	@PublicEvolving
 	public static final String HA_ZOOKEEPER_CHECKPOINT_COUNTER_PATH = "high-availability.zookeeper.path.checkpoint-counter";
 
+	/** ZooKeeper root path (ZNode) for Mesos workers. */
+	@PublicEvolving
+	public static final String HA_ZOOKEEPER_MESOS_WORKERS_PATH = "recovery.zookeeper.path.mesos-workers";
+
 	@PublicEvolving
 	public static final String HA_ZOOKEEPER_SESSION_TIMEOUT = "high-availability.zookeeper.client.session-timeout";
 
@@ -790,9 +794,6 @@ public final class ConfigConstants {
 	@Deprecated
 	public static final String ZOOKEEPER_CHECKPOINT_COUNTER_PATH = "recovery.zookeeper.path.checkpoint-counter";
 
-	/** ZooKeeper root path (ZNode) for Mesos workers. */
-	public static final String ZOOKEEPER_MESOS_WORKERS_PATH = "recovery.zookeeper.path.mesos-workers";
-
 	/** Deprecated in favour of {@link #HA_ZOOKEEPER_SESSION_TIMEOUT}. */
 	@Deprecated
 	public static final String ZOOKEEPER_SESSION_TIMEOUT = "recovery.zookeeper.client.session-timeout";

http://git-wip-us.apache.org/repos/asf/flink/blob/842e3e7d/flink-mesos/pom.xml
----------------------------------------------------------------------
diff --git a/flink-mesos/pom.xml b/flink-mesos/pom.xml
index 43b3195..a6edc0b 100644
--- a/flink-mesos/pom.xml
+++ b/flink-mesos/pom.xml
@@ -23,7 +23,7 @@ under the License.
 	<parent>
 		<groupId>org.apache.flink</groupId>
 		<artifactId>flink-parent</artifactId>
-		<version>1.1-SNAPSHOT</version>
+		<version>1.2-SNAPSHOT</version>
 		<relativePath>..</relativePath>
 	</parent>
 	

http://git-wip-us.apache.org/repos/asf/flink/blob/842e3e7d/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosApplicationMasterRunner.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosApplicationMasterRunner.java b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosApplicationMasterRunner.java
index c13cdf9..9916a87 100644
--- a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosApplicationMasterRunner.java
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosApplicationMasterRunner.java
@@ -38,9 +38,9 @@ import org.apache.flink.mesos.util.ZooKeeperUtils;
 import org.apache.flink.runtime.akka.AkkaUtils;
 import org.apache.flink.runtime.clusterframework.BootstrapTools;
 import org.apache.flink.runtime.clusterframework.ContaineredTaskManagerParameters;
+import org.apache.flink.runtime.jobmanager.HighAvailabilityMode;
 import org.apache.flink.runtime.jobmanager.JobManager;
 import org.apache.flink.runtime.jobmanager.MemoryArchivist;
-import org.apache.flink.runtime.jobmanager.RecoveryMode;
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
 import org.apache.flink.runtime.process.ProcessReaper;
 import org.apache.flink.runtime.taskmanager.TaskManager;
@@ -481,11 +481,11 @@ public class MesosApplicationMasterRunner {
 
 	private static MesosWorkerStore createWorkerStore(Configuration flinkConfig) throws Exception {
 		MesosWorkerStore workerStore;
-		RecoveryMode recoveryMode = RecoveryMode.fromConfig(flinkConfig);
-		if (recoveryMode == RecoveryMode.STANDALONE) {
+		HighAvailabilityMode recoveryMode = HighAvailabilityMode.fromConfig(flinkConfig);
+		if (recoveryMode == HighAvailabilityMode.NONE) {
 			workerStore = new StandaloneMesosWorkerStore();
 		}
-		else if (recoveryMode == RecoveryMode.ZOOKEEPER) {
+		else if (recoveryMode == HighAvailabilityMode.ZOOKEEPER) {
 			// note: the store is responsible for closing the client.
 			CuratorFramework client = ZooKeeperUtils.startCuratorFramework(flinkConfig);
 			workerStore = ZooKeeperMesosWorkerStore.createMesosWorkerStore(client, flinkConfig);

http://git-wip-us.apache.org/repos/asf/flink/blob/842e3e7d/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/store/ZooKeeperMesosWorkerStore.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/store/ZooKeeperMesosWorkerStore.java b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/store/ZooKeeperMesosWorkerStore.java
index 45553d4..c5cef8e 100644
--- a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/store/ZooKeeperMesosWorkerStore.java
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/store/ZooKeeperMesosWorkerStore.java
@@ -292,7 +292,7 @@ public class ZooKeeperMesosWorkerStore implements MesosWorkerStore {
 			ZooKeeperUtils.createFileSystemStateStorage(configuration, "mesosWorkerStore");
 
 		String zooKeeperMesosWorkerStorePath = configuration.getString(
-			ConfigConstants.ZOOKEEPER_MESOS_WORKERS_PATH,
+			ConfigConstants.HA_ZOOKEEPER_MESOS_WORKERS_PATH,
 			ConfigConstants.DEFAULT_ZOOKEEPER_MESOS_WORKERS_PATH
 		);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/842e3e7d/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/ContaineredJobManager.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/ContaineredJobManager.scala b/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/ContaineredJobManager.scala
deleted file mode 100644
index 45b404a..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/clusterframework/ContaineredJobManager.scala
+++ /dev/null
@@ -1,172 +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 java.util.concurrent.ExecutorService
-
-import akka.actor.ActorRef
-import org.apache.flink.api.common.JobID
-import org.apache.flink.configuration.{Configuration => FlinkConfiguration}
-import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory
-import org.apache.flink.runtime.checkpoint.savepoint.SavepointStore
-import org.apache.flink.runtime.clusterframework.messages._
-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.jobgraph.JobStatus
-import org.apache.flink.runtime.jobmanager.scheduler.{Scheduler => FlinkScheduler}
-import org.apache.flink.runtime.jobmanager.{JobManager, SubmittedJobGraphStore}
-import org.apache.flink.runtime.leaderelection.LeaderElectionService
-import org.apache.flink.runtime.messages.JobManagerMessages.{CurrentJobStatus, JobNotFound, RequestJobStatus}
-import org.apache.flink.runtime.messages.Messages.Acknowledge
-import org.apache.flink.runtime.metrics.{MetricRegistry => FlinkMetricRegistry}
-
-import scala.concurrent.duration._
-import scala.language.postfixOps
-
-
-/** JobManager actor for execution on Yarn or Mesos. It enriches the [[JobManager]] with additional messages
-  * to start/administer/stop the session.
-  *
-  * @param flinkConfiguration Configuration object for the actor
-  * @param executorService Execution context which is used to execute concurrent tasks in the
-  *                         [[org.apache.flink.runtime.executiongraph.ExecutionGraph]]
-  * @param instanceManager Instance manager to manage the registered
-  *                        [[org.apache.flink.runtime.taskmanager.TaskManager]]
-  * @param scheduler Scheduler to schedule Flink jobs
-  * @param libraryCacheManager Manager to manage uploaded jar files
-  * @param archive Archive for finished Flink jobs
-  * @param restartStrategyFactory Restart strategy to be used in case of a job recovery
-  * @param timeout Timeout for futures
-  * @param leaderElectionService LeaderElectionService to participate in the leader election
-  */
-abstract class ContaineredJobManager(
-    flinkConfiguration: FlinkConfiguration,
-    executorService: ExecutorService,
-    instanceManager: InstanceManager,
-    scheduler: FlinkScheduler,
-    libraryCacheManager: BlobLibraryCacheManager,
-    archive: ActorRef,
-    restartStrategyFactory: RestartStrategyFactory,
-    timeout: FiniteDuration,
-    leaderElectionService: LeaderElectionService,
-    submittedJobGraphs : SubmittedJobGraphStore,
-    checkpointRecoveryFactory : CheckpointRecoveryFactory,
-    savepointStore: SavepointStore,
-    jobRecoveryTimeout: FiniteDuration,
-    metricsRegistry: Option[FlinkMetricRegistry])
-  extends JobManager(
-    flinkConfiguration,
-    executorService,
-    instanceManager,
-    scheduler,
-    libraryCacheManager,
-    archive,
-    restartStrategyFactory,
-    timeout,
-    leaderElectionService,
-    submittedJobGraphs,
-    checkpointRecoveryFactory,
-    savepointStore,
-    jobRecoveryTimeout,
-    metricsRegistry) {
-
-  val jobPollingInterval: FiniteDuration
-
-  // indicates if this JM has been started in a dedicated (per-job) mode.
-  var stopWhenJobFinished: JobID = null
-
-  override def handleMessage: Receive = {
-    handleContainerMessage orElse super.handleMessage
-  }
-
-  def handleContainerMessage: Receive = {
-
-    case msg @ (_: RegisterInfoMessageListener | _: UnRegisterInfoMessageListener) =>
-      // forward to ResourceManager
-      currentResourceManager match {
-        case Some(rm) =>
-          // we forward the message
-          rm.forward(decorateMessage(msg))
-        case None =>
-          // client has to try again
-      }
-
-    case msg: ShutdownClusterAfterJob =>
-      val jobId = msg.jobId()
-      log.info(s"ApplicationMaster will shut down session when job $jobId has finished.")
-      stopWhenJobFinished = jobId
-      // trigger regular job status messages (if this is a dedicated/per-job cluster)
-      if (stopWhenJobFinished != null) {
-        context.system.scheduler.schedule(0 seconds,
-          jobPollingInterval,
-          new Runnable {
-            override def run(): Unit = {
-              self ! decorateMessage(RequestJobStatus(stopWhenJobFinished))
-            }
-          }
-        )(context.dispatcher)
-      }
-
-      sender() ! decorateMessage(Acknowledge)
-
-    case msg: GetClusterStatus =>
-      sender() ! decorateMessage(
-        new GetClusterStatusResponse(
-          instanceManager.getNumberOfRegisteredTaskManagers,
-          instanceManager.getTotalNumberOfSlots)
-      )
-
-    case jnf: JobNotFound =>
-      log.debug(s"Job with ID ${jnf.jobID} not found in JobManager")
-      if (stopWhenJobFinished == null) {
-        log.warn("The ApplicationMaster didn't expect to receive this message")
-      }
-
-    case jobStatus: CurrentJobStatus =>
-      if (stopWhenJobFinished == null) {
-        log.warn(s"Received job status $jobStatus which wasn't requested.")
-      } else {
-        if (stopWhenJobFinished != jobStatus.jobID) {
-          log.warn(s"Received job status for job ${jobStatus.jobID} but expected status for " +
-            s"job $stopWhenJobFinished")
-        } else {
-          if (jobStatus.status.isGloballyTerminalState) {
-            log.info(s"Job with ID ${jobStatus.jobID} is in terminal state ${jobStatus.status}. " +
-              s"Shutting down session")
-            if (jobStatus.status == JobStatus.FINISHED) {
-              self ! decorateMessage(
-                new StopCluster(
-                  ApplicationStatus.SUCCEEDED,
-                  s"The monitored job with ID ${jobStatus.jobID} has finished.")
-              )
-            } else {
-              self ! decorateMessage(
-                new StopCluster(
-                  ApplicationStatus.FAILED,
-                  s"The monitored job with ID ${jobStatus.jobID} has failed to complete.")
-              )
-            }
-          } else {
-            log.debug(s"Monitored job with ID ${jobStatus.jobID} is in state ${jobStatus.status}")
-          }
-        }
-      }
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/842e3e7d/flink-runtime/src/main/scala/org/apache/flink/runtime/clusterframework/ContaineredJobManager.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/clusterframework/ContaineredJobManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/clusterframework/ContaineredJobManager.scala
new file mode 100644
index 0000000..5f965d2
--- /dev/null
+++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/clusterframework/ContaineredJobManager.scala
@@ -0,0 +1,172 @@
+/*
+ * 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 java.util.concurrent.ExecutorService
+
+import akka.actor.ActorRef
+import org.apache.flink.api.common.JobID
+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.clusterframework.messages._
+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.jobgraph.JobStatus
+import org.apache.flink.runtime.jobmanager.scheduler.{Scheduler => FlinkScheduler}
+import org.apache.flink.runtime.jobmanager.{JobManager, SubmittedJobGraphStore}
+import org.apache.flink.runtime.leaderelection.LeaderElectionService
+import org.apache.flink.runtime.messages.JobManagerMessages.{CurrentJobStatus, JobNotFound, RequestJobStatus}
+import org.apache.flink.runtime.messages.Messages.Acknowledge
+import org.apache.flink.runtime.metrics.{MetricRegistry => FlinkMetricRegistry}
+
+import scala.concurrent.duration._
+import scala.language.postfixOps
+
+
+/** JobManager actor for execution on Yarn or Mesos. It enriches the [[JobManager]] with additional messages
+  * to start/administer/stop the session.
+  *
+  * @param flinkConfiguration Configuration object for the actor
+  * @param executorService Execution context which is used to execute concurrent tasks in the
+  *                         [[org.apache.flink.runtime.executiongraph.ExecutionGraph]]
+  * @param instanceManager Instance manager to manage the registered
+  *                        [[org.apache.flink.runtime.taskmanager.TaskManager]]
+  * @param scheduler Scheduler to schedule Flink jobs
+  * @param libraryCacheManager Manager to manage uploaded jar files
+  * @param archive Archive for finished Flink jobs
+  * @param restartStrategyFactory Restart strategy to be used in case of a job recovery
+  * @param timeout Timeout for futures
+  * @param leaderElectionService LeaderElectionService to participate in the leader election
+  */
+abstract class ContaineredJobManager(
+    flinkConfiguration: Configuration,
+    executorService: ExecutorService,
+    instanceManager: InstanceManager,
+    scheduler: FlinkScheduler,
+    libraryCacheManager: BlobLibraryCacheManager,
+    archive: ActorRef,
+    restartStrategyFactory: RestartStrategyFactory,
+    timeout: FiniteDuration,
+    leaderElectionService: LeaderElectionService,
+    submittedJobGraphs : SubmittedJobGraphStore,
+    checkpointRecoveryFactory : CheckpointRecoveryFactory,
+    savepointStore: SavepointStore,
+    jobRecoveryTimeout: FiniteDuration,
+    metricsRegistry: Option[FlinkMetricRegistry])
+  extends JobManager(
+    flinkConfiguration,
+    executorService,
+    instanceManager,
+    scheduler,
+    libraryCacheManager,
+    archive,
+    restartStrategyFactory,
+    timeout,
+    leaderElectionService,
+    submittedJobGraphs,
+    checkpointRecoveryFactory,
+    savepointStore,
+    jobRecoveryTimeout,
+    metricsRegistry) {
+
+  val jobPollingInterval: FiniteDuration
+
+  // indicates if this JM has been started in a dedicated (per-job) mode.
+  var stopWhenJobFinished: JobID = null
+
+  override def handleMessage: Receive = {
+    handleContainerMessage orElse super.handleMessage
+  }
+
+  def handleContainerMessage: Receive = {
+
+    case msg @ (_: RegisterInfoMessageListener | _: UnRegisterInfoMessageListener) =>
+      // forward to ResourceManager
+      currentResourceManager match {
+        case Some(rm) =>
+          // we forward the message
+          rm.forward(decorateMessage(msg))
+        case None =>
+          // client has to try again
+      }
+
+    case msg: ShutdownClusterAfterJob =>
+      val jobId = msg.jobId()
+      log.info(s"ApplicationMaster will shut down session when job $jobId has finished.")
+      stopWhenJobFinished = jobId
+      // trigger regular job status messages (if this is a dedicated/per-job cluster)
+      if (stopWhenJobFinished != null) {
+        context.system.scheduler.schedule(0 seconds,
+          jobPollingInterval,
+          new Runnable {
+            override def run(): Unit = {
+              self ! decorateMessage(RequestJobStatus(stopWhenJobFinished))
+            }
+          }
+        )(context.dispatcher)
+      }
+
+      sender() ! decorateMessage(Acknowledge)
+
+    case msg: GetClusterStatus =>
+      sender() ! decorateMessage(
+        new GetClusterStatusResponse(
+          instanceManager.getNumberOfRegisteredTaskManagers,
+          instanceManager.getTotalNumberOfSlots)
+      )
+
+    case jnf: JobNotFound =>
+      log.debug(s"Job with ID ${jnf.jobID} not found in JobManager")
+      if (stopWhenJobFinished == null) {
+        log.warn("The ApplicationMaster didn't expect to receive this message")
+      }
+
+    case jobStatus: CurrentJobStatus =>
+      if (stopWhenJobFinished == null) {
+        log.warn(s"Received job status $jobStatus which wasn't requested.")
+      } else {
+        if (stopWhenJobFinished != jobStatus.jobID) {
+          log.warn(s"Received job status for job ${jobStatus.jobID} but expected status for " +
+            s"job $stopWhenJobFinished")
+        } else {
+          if (jobStatus.status.isGloballyTerminalState) {
+            log.info(s"Job with ID ${jobStatus.jobID} is in terminal state ${jobStatus.status}. " +
+              s"Shutting down session")
+            if (jobStatus.status == JobStatus.FINISHED) {
+              self ! decorateMessage(
+                new StopCluster(
+                  ApplicationStatus.SUCCEEDED,
+                  s"The monitored job with ID ${jobStatus.jobID} has finished.")
+              )
+            } else {
+              self ! decorateMessage(
+                new StopCluster(
+                  ApplicationStatus.FAILED,
+                  s"The monitored job with ID ${jobStatus.jobID} has failed to complete.")
+              )
+            }
+          } else {
+            log.debug(s"Monitored job with ID ${jobStatus.jobID} is in state ${jobStatus.status}")
+          }
+        }
+      }
+  }
+}


[3/9] flink git commit: [FLINK-1984] Mesos ResourceManager - T1 milestone

Posted by mx...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/d9b2be05/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/RegisteredMesosWorkerNode.scala
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/RegisteredMesosWorkerNode.scala b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/RegisteredMesosWorkerNode.scala
new file mode 100644
index 0000000..c519820
--- /dev/null
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/RegisteredMesosWorkerNode.scala
@@ -0,0 +1,15 @@
+package org.apache.flink.mesos.runtime.clusterframework
+
+import org.apache.flink.mesos.runtime.clusterframework.store.MesosWorkerStore
+import org.apache.flink.runtime.clusterframework.types.{ResourceID, ResourceIDRetrievable}
+
+/**
+  * A representation of a registered Mesos task managed by the {@link MesosFlinkResourceManager}.
+  */
+case class RegisteredMesosWorkerNode(task: MesosWorkerStore.Worker) extends ResourceIDRetrievable {
+
+  require(task.slaveID().isDefined)
+  require(task.hostname().isDefined)
+
+  override val getResourceID: ResourceID = MesosFlinkResourceManager.extractResourceID(task.taskID())
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/d9b2be05/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/store/MesosWorkerStore.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/store/MesosWorkerStore.java b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/store/MesosWorkerStore.java
new file mode 100644
index 0000000..061ed88
--- /dev/null
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/store/MesosWorkerStore.java
@@ -0,0 +1,134 @@
+package org.apache.flink.mesos.runtime.clusterframework.store;
+
+import org.apache.mesos.Protos;
+import scala.Option;
+
+import java.io.Serializable;
+import java.text.DecimalFormat;
+import java.util.List;
+import java.util.Objects;
+
+import static java.util.Objects.requireNonNull;
+
+/**
+ * A store of Mesos workers and associated framework information.
+ *
+ * Generates a framework ID as necessary.
+ */
+public interface MesosWorkerStore {
+
+	static final DecimalFormat TASKID_FORMAT = new DecimalFormat("taskmanager-00000");
+
+	void start() throws Exception;
+
+	void stop() throws Exception;
+
+	Option<Protos.FrameworkID> getFrameworkID() throws Exception;
+
+	void setFrameworkID(Option<Protos.FrameworkID> frameworkID) throws Exception;
+
+	List<Worker> recoverWorkers() throws Exception;
+
+	Protos.TaskID newTaskID() throws Exception;
+
+	void putWorker(Worker worker) throws Exception;
+
+	void removeWorker(Protos.TaskID taskID) throws Exception;
+
+	void cleanup() throws Exception;
+
+	/**
+	 * A stored task.
+	 *
+	 * The assigned slaveid/hostname is valid in Launched and Released states.  The hostname is needed
+	 * by Fenzo for optimization purposes.
+	 */
+	class Worker implements Serializable {
+		private Protos.TaskID taskID;
+
+		private Option<Protos.SlaveID> slaveID;
+
+		private Option<String> hostname;
+
+		private TaskState state;
+
+		public Worker(Protos.TaskID taskID, Option<Protos.SlaveID> slaveID, Option<String> hostname, TaskState state) {
+			requireNonNull(taskID, "taskID");
+			requireNonNull(slaveID, "slaveID");
+			requireNonNull(hostname, "hostname");
+			requireNonNull(state, "state");
+
+			this.taskID = taskID;
+			this.slaveID = slaveID;
+			this.hostname = hostname;
+			this.state = state;
+		}
+
+		public Protos.TaskID taskID() {
+			return taskID;
+		}
+
+		public Option<Protos.SlaveID> slaveID() {
+			return slaveID;
+		}
+
+		public Option<String> hostname() {
+			return hostname;
+		}
+
+		public TaskState state() {
+			return state;
+		}
+
+		// valid transition methods
+
+		public static Worker newTask(Protos.TaskID taskID) {
+			return new Worker(
+				taskID,
+				Option.<Protos.SlaveID>empty(), Option.<String>empty(),
+				TaskState.New);
+		}
+
+		public Worker launchTask(Protos.SlaveID slaveID, String hostname) {
+			return new Worker(taskID, Option.apply(slaveID), Option.apply(hostname), TaskState.Launched);
+		}
+
+		public Worker releaseTask() {
+			return new Worker(taskID, slaveID, hostname, TaskState.Released);
+		}
+
+		@Override
+		public boolean equals(Object o) {
+			if (this == o) {
+				return true;
+			}
+			if (o == null || getClass() != o.getClass()) {
+				return false;
+			}
+			Worker worker = (Worker) o;
+			return Objects.equals(taskID, worker.taskID) &&
+				Objects.equals(slaveID.isDefined() ? slaveID.get() : null, worker.slaveID.isDefined() ? worker.slaveID.get() : null) &&
+				Objects.equals(hostname.isDefined() ? hostname.get() : null, worker.hostname.isDefined() ? worker.hostname.get() : null) &&
+				state == worker.state;
+		}
+
+		@Override
+		public int hashCode() {
+			return Objects.hash(taskID, slaveID.isDefined() ? slaveID.get() : null, hostname.isDefined() ? hostname.get() : null, state);
+		}
+
+		@Override
+		public String toString() {
+			return "Worker{" +
+				"taskID=" + taskID +
+				", slaveID=" + slaveID +
+				", hostname=" + hostname +
+				", state=" + state +
+				'}';
+		}
+	}
+
+	enum TaskState {
+		New,Launched,Released
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/d9b2be05/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/store/StandaloneMesosWorkerStore.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/store/StandaloneMesosWorkerStore.java b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/store/StandaloneMesosWorkerStore.java
new file mode 100644
index 0000000..84b0ca1
--- /dev/null
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/store/StandaloneMesosWorkerStore.java
@@ -0,0 +1,69 @@
+package org.apache.flink.mesos.runtime.clusterframework.store;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.mesos.Protos;
+import scala.Option;
+
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * A standalone Mesos worker store.
+ */
+public class StandaloneMesosWorkerStore implements MesosWorkerStore {
+
+	private Option<Protos.FrameworkID> frameworkID = Option.empty();
+
+	private int taskCount = 0;
+
+	private Map<Protos.TaskID, Worker> storedWorkers = new LinkedHashMap<>();
+
+	public StandaloneMesosWorkerStore() {
+	}
+
+	@Override
+	public void start() throws Exception {
+
+	}
+
+	@Override
+	public void stop() throws Exception {
+
+	}
+
+	@Override
+	public Option<Protos.FrameworkID> getFrameworkID() throws Exception {
+		return frameworkID;
+	}
+
+	@Override
+	public void setFrameworkID(Option<Protos.FrameworkID> frameworkID) throws Exception {
+		this.frameworkID = frameworkID;
+	}
+
+	@Override
+	public List<Worker> recoverWorkers() throws Exception {
+		return ImmutableList.copyOf(storedWorkers.values());
+	}
+
+	@Override
+	public Protos.TaskID newTaskID() throws Exception {
+		Protos.TaskID taskID = Protos.TaskID.newBuilder().setValue(TASKID_FORMAT.format(++taskCount)).build();
+		return taskID;
+	}
+
+	@Override
+	public void putWorker(Worker worker) throws Exception {
+		storedWorkers.put(worker.taskID(), worker);
+	}
+
+	@Override
+	public void removeWorker(Protos.TaskID taskID) throws Exception {
+		storedWorkers.remove(taskID);
+	}
+
+	@Override
+	public void cleanup() throws Exception {
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/d9b2be05/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/store/ZooKeeperMesosWorkerStore.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/store/ZooKeeperMesosWorkerStore.java b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/store/ZooKeeperMesosWorkerStore.java
new file mode 100644
index 0000000..fe4c74d
--- /dev/null
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/store/ZooKeeperMesosWorkerStore.java
@@ -0,0 +1,272 @@
+package org.apache.flink.mesos.runtime.clusterframework.store;
+
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.recipes.shared.SharedCount;
+import org.apache.curator.framework.recipes.shared.SharedValue;
+import org.apache.curator.framework.recipes.shared.VersionedValue;
+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.state.StateHandle;
+import org.apache.flink.runtime.util.ZooKeeperUtils;
+import org.apache.flink.runtime.zookeeper.StateStorageHelper;
+import org.apache.flink.runtime.zookeeper.ZooKeeperStateHandleStore;
+import org.apache.mesos.Protos;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Option;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.ConcurrentModificationException;
+import java.util.List;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * A ZooKeeper-backed Mesos worker store.
+ */
+public class ZooKeeperMesosWorkerStore implements MesosWorkerStore {
+
+	private static final Logger LOG = LoggerFactory.getLogger(ZooKeeperMesosWorkerStore.class);
+
+	private final Object cacheLock = new Object();
+
+	/** Client (not a namespace facade) */
+	private final CuratorFramework client;
+
+	/** Flag indicating whether this instance is running. */
+	private boolean isRunning;
+
+	/** A persistent value of the assigned framework ID */
+	private final SharedValue frameworkIdInZooKeeper;
+
+	/** A persistent count of all tasks created, for generating unique IDs */
+	private final SharedCount totalTaskCountInZooKeeper;
+
+	/** A persistent store of serialized workers */
+	private final ZooKeeperStateHandleStore<MesosWorkerStore.Worker> workersInZooKeeper;
+
+	@SuppressWarnings("unchecked")
+	ZooKeeperMesosWorkerStore(
+		CuratorFramework client,
+		String storePath,
+		StateStorageHelper<MesosWorkerStore.Worker> stateStorage
+	) throws Exception {
+		checkNotNull(storePath, "storePath");
+		checkNotNull(stateStorage, "stateStorage");
+
+		// Keep a reference to the original client and not the namespace facade. The namespace
+		// facade cannot be closed.
+		this.client = checkNotNull(client, "client");
+
+		// All operations will have the given path as root
+		client.newNamespaceAwareEnsurePath(storePath).ensure(client.getZookeeperClient());
+		CuratorFramework facade = client.usingNamespace(client.getNamespace() + storePath);
+
+		// Track the assignd framework ID.
+		frameworkIdInZooKeeper = new SharedValue(facade, "/frameworkId", new byte[0]);
+
+		// Keep a count of all tasks created ever, as the basis for a unique ID.
+		totalTaskCountInZooKeeper = new SharedCount(facade, "/count", 0);
+
+		// Keep track of the workers in state handle storage.
+		facade.newNamespaceAwareEnsurePath("/workers").ensure(client.getZookeeperClient());
+		CuratorFramework storeFacade = client.usingNamespace(facade.getNamespace() + "/workers");
+
+		this.workersInZooKeeper = ZooKeeperStateHandleStore.class
+			.getConstructor(CuratorFramework.class, StateStorageHelper.class)
+			.newInstance(storeFacade, stateStorage);
+	}
+
+	@Override
+	public void start() throws Exception {
+		synchronized (cacheLock) {
+			if (!isRunning) {
+				isRunning = true;
+				frameworkIdInZooKeeper.start();
+				totalTaskCountInZooKeeper.start();
+			}
+		}
+	}
+
+	public void stop() throws Exception {
+		synchronized (cacheLock) {
+			if (isRunning) {
+				frameworkIdInZooKeeper.close();
+				totalTaskCountInZooKeeper.close();
+				client.close();
+				isRunning = false;
+			}
+		}
+	}
+
+	/**
+	 * Verifies that the state is running.
+	 */
+	private void verifyIsRunning() {
+		checkState(isRunning, "Not running. Forgot to call start()?");
+	}
+
+	/**
+	 * Get the persisted framework ID.
+	 * @return the current ID or empty if none is yet persisted.
+	 * @throws Exception on ZK failures, interruptions.
+     */
+	@Override
+	public Option<Protos.FrameworkID> getFrameworkID() throws Exception {
+		synchronized (cacheLock) {
+			verifyIsRunning();
+
+			Option<Protos.FrameworkID> frameworkID;
+			byte[] value = frameworkIdInZooKeeper.getValue();
+			if (value.length == 0) {
+				frameworkID = Option.empty();
+			} else {
+				frameworkID = Option.apply(Protos.FrameworkID.newBuilder().setValue(new String(value)).build());
+			}
+
+			return frameworkID;
+		}
+	}
+
+	/**
+	 * Update the persisted framework ID.
+	 * @param frameworkID the new ID or empty to remove the persisted ID.
+	 * @throws Exception on ZK failures, interruptions.
+     */
+	@Override
+	public void setFrameworkID(Option<Protos.FrameworkID> frameworkID) throws Exception {
+		synchronized (cacheLock) {
+			verifyIsRunning();
+
+			byte[] value = frameworkID.isDefined() ? frameworkID.get().getValue().getBytes() : new byte[0];
+			frameworkIdInZooKeeper.setValue(value);
+		}
+	}
+
+	/**
+	 * Generates a new task ID.
+	 */
+	@Override
+	public Protos.TaskID newTaskID() throws Exception {
+		synchronized (cacheLock) {
+			verifyIsRunning();
+
+			int nextCount;
+			boolean success;
+			do {
+				VersionedValue<Integer> count = totalTaskCountInZooKeeper.getVersionedValue();
+				nextCount = count.getValue() + 1;
+				success = totalTaskCountInZooKeeper.trySetCount(count, nextCount);
+			}
+			while (!success);
+
+			Protos.TaskID taskID = Protos.TaskID.newBuilder().setValue(TASKID_FORMAT.format(nextCount)).build();
+			return taskID;
+		}
+	}
+
+	@Override
+	public List<MesosWorkerStore.Worker> recoverWorkers() throws Exception {
+		synchronized (cacheLock) {
+			verifyIsRunning();
+
+			List<Tuple2<StateHandle<MesosWorkerStore.Worker>, String>> handles = workersInZooKeeper.getAll();
+
+			if(handles.size() != 0) {
+				List<MesosWorkerStore.Worker> workers = new ArrayList<>(handles.size());
+				for (Tuple2<StateHandle<MesosWorkerStore.Worker>, String> handle : handles) {
+					Worker worker = handle.f0.getState(ClassLoader.getSystemClassLoader());
+
+					workers.add(worker);
+				}
+
+				return workers;
+			}
+			else {
+				return Collections.emptyList();
+			}
+		}
+	}
+
+	@Override
+	public void putWorker(MesosWorkerStore.Worker worker) throws Exception {
+
+		checkNotNull(worker, "worker");
+		String path = getPathForWorker(worker.taskID());
+
+		synchronized (cacheLock) {
+			verifyIsRunning();
+
+			int currentVersion = workersInZooKeeper.exists(path);
+			if (currentVersion == -1) {
+				try {
+					workersInZooKeeper.add(path, worker);
+					LOG.debug("Added {} in ZooKeeper.", worker);
+				} catch (KeeperException.NodeExistsException ex) {
+					throw new ConcurrentModificationException("ZooKeeper unexpectedly modified", ex);
+				}
+			} else {
+				try {
+					workersInZooKeeper.replace(path, currentVersion, worker);
+					LOG.debug("Updated {} in ZooKeeper.", worker);
+				} catch (KeeperException.NoNodeException ex) {
+					throw new ConcurrentModificationException("ZooKeeper unexpectedly modified", ex);
+				}
+			}
+		}
+	}
+
+	@Override
+	public void removeWorker(Protos.TaskID taskID) throws Exception {
+		checkNotNull(taskID, "taskID");
+		String path = getPathForWorker(taskID);
+		synchronized (cacheLock) {
+			verifyIsRunning();
+
+			workersInZooKeeper.remove(path);
+			LOG.debug("Removed worker {} from ZooKeeper.", taskID);
+		}
+	}
+
+	@Override
+	public void cleanup() throws Exception {
+		// TODO
+	}
+
+	/**
+	 * Get the ZK path for the given task ID (with leading slash).
+	 */
+	private static String getPathForWorker(Protos.TaskID taskID) {
+		checkNotNull(taskID, "taskID");
+		return String.format("/%s", taskID.getValue());
+	}
+
+	/**
+	 * Create the ZooKeeper-backed Mesos worker store.
+	 * @param client the curator client.
+	 * @param configuration the Flink configuration.
+	 * @return a worker store.
+	 * @throws Exception
+     */
+	public static ZooKeeperMesosWorkerStore createMesosWorkerStore(
+			CuratorFramework client,
+			Configuration configuration) throws Exception {
+
+		checkNotNull(configuration, "Configuration");
+
+		StateStorageHelper<MesosWorkerStore.Worker> stateStorage =
+			ZooKeeperUtils.createFileSystemStateStorage(configuration, "mesosWorkerStore");
+
+		String zooKeeperMesosWorkerStorePath = configuration.getString(
+			ConfigConstants.ZOOKEEPER_MESOS_WORKERS_PATH,
+			ConfigConstants.DEFAULT_ZOOKEEPER_MESOS_WORKERS_PATH
+		);
+
+		return new ZooKeeperMesosWorkerStore(
+			client, zooKeeperMesosWorkerStorePath, stateStorage);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/d9b2be05/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/LaunchableTask.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/LaunchableTask.java b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/LaunchableTask.java
new file mode 100644
index 0000000..584cccb
--- /dev/null
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/LaunchableTask.java
@@ -0,0 +1,24 @@
+package org.apache.flink.mesos.scheduler;
+
+import com.netflix.fenzo.TaskAssignmentResult;
+import com.netflix.fenzo.TaskRequest;
+import org.apache.mesos.Protos;
+
+/**
+ * Specifies the task requirements and produces a Mesos TaskInfo description.
+ */
+public interface LaunchableTask {
+
+	/**
+	 * Get a representation of the task requirements as understood by Fenzo.
+     */
+	TaskRequest taskRequest();
+
+	/**
+	 * Prepare to launch the task by producing a Mesos TaskInfo record.
+	 * @param slaveId the slave assigned to the task.
+	 * @param taskAssignmentResult the task assignment details.
+     * @return a TaskInfo.
+     */
+	Protos.TaskInfo launch(Protos.SlaveID slaveId, TaskAssignmentResult taskAssignmentResult);
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/d9b2be05/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/SchedulerProxy.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/SchedulerProxy.java b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/SchedulerProxy.java
new file mode 100644
index 0000000..be07358
--- /dev/null
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/SchedulerProxy.java
@@ -0,0 +1,87 @@
+package org.apache.flink.mesos.scheduler;
+
+import akka.actor.ActorRef;
+
+import org.apache.flink.mesos.scheduler.messages.Disconnected;
+import org.apache.flink.mesos.scheduler.messages.Error;
+import org.apache.flink.mesos.scheduler.messages.Error;
+import org.apache.flink.mesos.scheduler.messages.OfferRescinded;
+import org.apache.flink.mesos.scheduler.messages.ReRegistered;
+import org.apache.flink.mesos.scheduler.messages.Registered;
+import org.apache.flink.mesos.scheduler.messages.ResourceOffers;
+import org.apache.flink.mesos.scheduler.messages.SlaveLost;
+import org.apache.flink.mesos.scheduler.messages.StatusUpdate;
+import org.apache.mesos.Protos;
+import org.apache.mesos.Scheduler;
+import org.apache.mesos.SchedulerDriver;
+
+import java.util.List;
+
+/**
+ * This class reacts to callbacks from the Mesos scheduler driver.
+ *
+ * In order to preserve actor concurrency safety, this class simply sends
+ * corresponding messages to the Mesos resource master actor.
+ *
+ * See https://mesos.apache.org/api/latest/java/org/apache/mesos/Scheduler.html
+ */
+public class SchedulerProxy implements Scheduler {
+
+	/** The actor to which we report the callbacks */
+	private ActorRef mesosActor;
+
+	public SchedulerProxy(ActorRef mesosActor) {
+		this.mesosActor = mesosActor;
+	}
+
+	@Override
+	public void registered(SchedulerDriver driver, Protos.FrameworkID frameworkId, Protos.MasterInfo masterInfo) {
+		mesosActor.tell(new Registered(frameworkId, masterInfo), ActorRef.noSender());
+	}
+
+	@Override
+	public void reregistered(SchedulerDriver driver, Protos.MasterInfo masterInfo) {
+		mesosActor.tell(new ReRegistered(masterInfo), ActorRef.noSender());
+	}
+
+	@Override
+	public void disconnected(SchedulerDriver driver) {
+		mesosActor.tell(new Disconnected(), ActorRef.noSender());
+	}
+
+
+	@Override
+	public void resourceOffers(SchedulerDriver driver, List<Protos.Offer> offers) {
+		mesosActor.tell(new ResourceOffers(offers), ActorRef.noSender());
+	}
+
+	@Override
+	public void offerRescinded(SchedulerDriver driver, Protos.OfferID offerId) {
+		mesosActor.tell(new OfferRescinded(offerId), ActorRef.noSender());
+	}
+
+	@Override
+	public void statusUpdate(SchedulerDriver driver, Protos.TaskStatus status) {
+		mesosActor.tell(new StatusUpdate(status), ActorRef.noSender());
+	}
+
+	@Override
+	public void frameworkMessage(SchedulerDriver driver, Protos.ExecutorID executorId, Protos.SlaveID slaveId, byte[] data) {
+		throw new UnsupportedOperationException("frameworkMessage is unexpected");
+	}
+
+	@Override
+	public void slaveLost(SchedulerDriver driver, Protos.SlaveID slaveId) {
+		mesosActor.tell(new SlaveLost(slaveId), ActorRef.noSender());
+	}
+
+	@Override
+	public void executorLost(SchedulerDriver driver, Protos.ExecutorID executorId, Protos.SlaveID slaveId, int status) {
+		throw new UnsupportedOperationException("executorLost is unexpected");
+	}
+
+	@Override
+	public void error(SchedulerDriver driver, String message) {
+		mesosActor.tell(new Error(message), ActorRef.noSender());
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/d9b2be05/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/TaskSchedulerBuilder.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/TaskSchedulerBuilder.java b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/TaskSchedulerBuilder.java
new file mode 100644
index 0000000..a4984de
--- /dev/null
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/TaskSchedulerBuilder.java
@@ -0,0 +1,16 @@
+package org.apache.flink.mesos.scheduler;
+
+import com.netflix.fenzo.TaskScheduler;
+import com.netflix.fenzo.VirtualMachineLease;
+import com.netflix.fenzo.functions.Action1;
+
+/**
+ * A builder for the Fenzo task scheduler.
+ *
+ * Note that the Fenzo-provided {@link TaskScheduler.Builder} cannot be mocked, which motivates this interface.
+ */
+public interface TaskSchedulerBuilder {
+	TaskSchedulerBuilder withLeaseRejectAction(Action1<VirtualMachineLease> action);
+
+	TaskScheduler build();
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/d9b2be05/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/AcceptOffers.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/AcceptOffers.java b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/AcceptOffers.java
new file mode 100644
index 0000000..b661474
--- /dev/null
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/AcceptOffers.java
@@ -0,0 +1,56 @@
+package org.apache.flink.mesos.scheduler.messages;
+
+import org.apache.mesos.Protos;
+
+import java.util.Collection;
+
+/**
+ * Local message sent by the launch coordinator to the scheduler to accept offers.
+ */
+public class AcceptOffers {
+
+	private String hostname;
+	private Collection<Protos.OfferID> offerIds;
+	private Collection<Protos.Offer.Operation> operations;
+	private Protos.Filters filters;
+
+	public AcceptOffers(String hostname, Collection<Protos.OfferID> offerIds, Collection<Protos.Offer.Operation> operations) {
+		this.hostname = hostname;
+		this.offerIds = offerIds;
+		this.operations = operations;
+		this.filters = Protos.Filters.newBuilder().build();
+	}
+
+	public AcceptOffers(String hostname, Collection<Protos.OfferID> offerIds, Collection<Protos.Offer.Operation> operations, Protos.Filters filters) {
+		this.hostname = hostname;
+		this.offerIds = offerIds;
+		this.operations = operations;
+		this.filters = filters;
+	}
+
+	public String hostname() {
+		return hostname;
+	}
+
+	public Collection<Protos.OfferID> offerIds() {
+		return offerIds;
+	}
+
+	public Collection<Protos.Offer.Operation> operations() {
+		return operations;
+	}
+
+	public Protos.Filters filters() {
+		return filters;
+	}
+
+	@Override
+	public String toString() {
+		return "AcceptOffers{" +
+			"hostname='" + hostname + '\'' +
+			", offerIds=" + offerIds +
+			", operations=" + operations +
+			", filters=" + filters +
+			'}';
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/d9b2be05/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/Connected.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/Connected.java b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/Connected.java
new file mode 100644
index 0000000..bd3aebe
--- /dev/null
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/Connected.java
@@ -0,0 +1,8 @@
+package org.apache.flink.mesos.scheduler.messages;
+
+/**
+ * Message sent by the callback handler to the scheduler actor
+ * when the scheduler becomes "connected" to the master.
+ */
+public abstract class Connected {
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/d9b2be05/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/Disconnected.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/Disconnected.java b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/Disconnected.java
new file mode 100644
index 0000000..be91c5c
--- /dev/null
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/Disconnected.java
@@ -0,0 +1,12 @@
+package org.apache.flink.mesos.scheduler.messages;
+
+/**
+ * Message sent by the callback handler to the scheduler actor
+ * when the scheduler becomes "disconnected" from the master (e.g., the master fails and another is taking over).
+ */
+public class Disconnected {
+	@Override
+	public String toString() {
+		return "Disconnected{}";
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/d9b2be05/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/Error.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/Error.java b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/Error.java
new file mode 100644
index 0000000..5e24bf8
--- /dev/null
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/Error.java
@@ -0,0 +1,24 @@
+package org.apache.flink.mesos.scheduler.messages;
+
+/**
+ * Message sent when there is an unrecoverable error in the scheduler or
+ * driver. The driver will be aborted BEFORE invoking this callback.
+ */
+public class Error {
+	private String message;
+
+	public Error(String message) {
+		this.message = message;
+	}
+
+	public String message() {
+		return message;
+	}
+
+	@Override
+	public String toString() {
+		return "Error{" +
+			"message='" + message + '\'' +
+			'}';
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/d9b2be05/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/OfferRescinded.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/OfferRescinded.java b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/OfferRescinded.java
new file mode 100644
index 0000000..9fac86b
--- /dev/null
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/OfferRescinded.java
@@ -0,0 +1,26 @@
+package org.apache.flink.mesos.scheduler.messages;
+
+import org.apache.mesos.Protos;
+
+/**
+ * Message sent by the callback handler to the scheduler actor
+ * when an offer is no longer valid (e.g., the slave was lost or another framework used resources in the offer).
+ */
+public class OfferRescinded {
+	private Protos.OfferID offerId;
+
+	public OfferRescinded(Protos.OfferID offerId) {
+		this.offerId = offerId;
+	}
+
+	public Protos.OfferID offerId() {
+		return offerId;
+	}
+
+	@Override
+	public String toString() {
+		return "OfferRescinded{" +
+			"offerId=" + offerId +
+			'}';
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/d9b2be05/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/ReRegistered.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/ReRegistered.java b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/ReRegistered.java
new file mode 100644
index 0000000..d4f1f76
--- /dev/null
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/ReRegistered.java
@@ -0,0 +1,30 @@
+package org.apache.flink.mesos.scheduler.messages;
+
+import org.apache.mesos.Protos;
+
+import static java.util.Objects.requireNonNull;
+
+/**
+ * Message sent by the callback handler to the scheduler actor
+ * when the scheduler re-registers with a newly elected Mesos master.
+ */
+public class ReRegistered extends Connected {
+	private Protos.MasterInfo masterInfo;
+
+	public ReRegistered(Protos.MasterInfo masterInfo) {
+		requireNonNull(masterInfo);
+
+		this.masterInfo = masterInfo;
+	}
+
+	public Protos.MasterInfo masterInfo() {
+		return masterInfo;
+	}
+
+	@Override
+	public String toString() {
+		return "ReRegistered{" +
+			"masterInfo=" + masterInfo +
+			'}';
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/d9b2be05/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/Registered.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/Registered.java b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/Registered.java
new file mode 100644
index 0000000..78ac880
--- /dev/null
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/Registered.java
@@ -0,0 +1,39 @@
+package org.apache.flink.mesos.scheduler.messages;
+
+import org.apache.mesos.Protos;
+
+import static java.util.Objects.requireNonNull;
+
+/**
+ * Message sent by the callback handler to the scheduler actor
+ * when the scheduler successfully registers with a Mesos master.
+ */
+public class Registered extends Connected {
+
+	private org.apache.mesos.Protos.FrameworkID frameworkId;
+	private Protos.MasterInfo masterInfo;
+
+	public Registered(Protos.FrameworkID frameworkId, Protos.MasterInfo masterInfo) {
+		requireNonNull(frameworkId);
+		requireNonNull(masterInfo);
+
+		this.frameworkId = frameworkId;
+		this.masterInfo = masterInfo;
+	}
+
+	public Protos.FrameworkID frameworkId() {
+		return frameworkId;
+	}
+
+	public Protos.MasterInfo masterInfo() {
+		return masterInfo;
+	}
+
+	@Override
+	public String toString() {
+		return "Registered{" +
+			"frameworkId=" + frameworkId +
+			", masterInfo=" + masterInfo +
+			'}';
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/d9b2be05/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/ResourceOffers.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/ResourceOffers.java b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/ResourceOffers.java
new file mode 100644
index 0000000..595ab83
--- /dev/null
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/ResourceOffers.java
@@ -0,0 +1,30 @@
+package org.apache.flink.mesos.scheduler.messages;
+
+import org.apache.mesos.Protos;
+
+import java.util.List;
+import static java.util.Objects.requireNonNull;
+
+/**
+ * Message sent by the callback handler to the scheduler actor
+ * when resources have been offered to this framework.
+ */
+public class ResourceOffers {
+	private List<Protos.Offer> offers;
+
+	public ResourceOffers(List<Protos.Offer> offers) {
+		requireNonNull(offers);
+		this.offers = offers;
+	}
+
+	public List<Protos.Offer> offers() {
+		return offers;
+	}
+
+	@Override
+	public String toString() {
+		return "ResourceOffers{" +
+			"offers=" + offers +
+			'}';
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/d9b2be05/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/SlaveLost.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/SlaveLost.java b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/SlaveLost.java
new file mode 100644
index 0000000..4ec9c0f
--- /dev/null
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/SlaveLost.java
@@ -0,0 +1,26 @@
+package org.apache.flink.mesos.scheduler.messages;
+
+import org.apache.mesos.Protos;
+
+/**
+ * Message sent by the callback handler to the scheduler actor
+ * when a slave has been determined unreachable (e.g., machine failure, network partition).
+ */
+public class SlaveLost {
+	private Protos.SlaveID slaveId;
+
+	public SlaveLost(Protos.SlaveID slaveId) {
+		this.slaveId = slaveId;
+	}
+
+	public Protos.SlaveID slaveId() {
+		return slaveId;
+	}
+
+	@Override
+	public String toString() {
+		return "SlaveLost{" +
+			"slaveId=" + slaveId +
+			'}';
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/d9b2be05/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/StatusUpdate.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/StatusUpdate.java b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/StatusUpdate.java
new file mode 100644
index 0000000..e64cac3
--- /dev/null
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/StatusUpdate.java
@@ -0,0 +1,27 @@
+package org.apache.flink.mesos.scheduler.messages;
+
+import org.apache.mesos.Protos;
+
+/**
+ * Message sent by the callback handler to the scheduler actor
+ * when the status of a task has changed (e.g., a slave is lost and so the task is lost,
+ * a task finishes and an executor sends a status update saying so, etc).
+ */
+public class StatusUpdate {
+	private Protos.TaskStatus status;
+
+	public StatusUpdate(Protos.TaskStatus status) {
+		this.status = status;
+	}
+
+	public Protos.TaskStatus status() {
+		return status;
+	}
+
+	@Override
+	public String toString() {
+		return "StatusUpdate{" +
+			"status=" + status +
+			'}';
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/d9b2be05/flink-mesos/src/main/java/org/apache/flink/mesos/util/MesosArtifactServer.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/util/MesosArtifactServer.java b/flink-mesos/src/main/java/org/apache/flink/mesos/util/MesosArtifactServer.java
new file mode 100644
index 0000000..96365c9
--- /dev/null
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/util/MesosArtifactServer.java
@@ -0,0 +1,286 @@
+package org.apache.flink.mesos.util;
+
+import io.netty.bootstrap.ServerBootstrap;
+import io.netty.buffer.Unpooled;
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelFuture;
+import io.netty.channel.ChannelFutureListener;
+import io.netty.channel.ChannelHandler;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.channel.ChannelInitializer;
+import io.netty.channel.DefaultFileRegion;
+import io.netty.channel.SimpleChannelInboundHandler;
+import io.netty.channel.nio.NioEventLoopGroup;
+import io.netty.channel.socket.SocketChannel;
+import io.netty.channel.socket.nio.NioServerSocketChannel;
+import io.netty.handler.codec.http.DefaultFullHttpResponse;
+import io.netty.handler.codec.http.DefaultHttpResponse;
+import io.netty.handler.codec.http.FullHttpResponse;
+import io.netty.handler.codec.http.HttpHeaders;
+import io.netty.handler.codec.http.HttpRequest;
+import io.netty.handler.codec.http.HttpResponse;
+import io.netty.handler.codec.http.HttpResponseStatus;
+import io.netty.handler.codec.http.HttpServerCodec;
+import io.netty.handler.codec.http.LastHttpContent;
+import io.netty.handler.codec.http.router.Handler;
+import io.netty.handler.codec.http.router.Routed;
+import io.netty.handler.codec.http.router.Router;
+import io.netty.util.CharsetUtil;
+import org.jets3t.service.utils.Mimetypes;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.RandomAccessFile;
+import java.net.InetSocketAddress;
+import java.net.MalformedURLException;
+import java.net.URL;
+
+import static io.netty.handler.codec.http.HttpHeaders.Names.CACHE_CONTROL;
+import static io.netty.handler.codec.http.HttpHeaders.Names.CONNECTION;
+import static io.netty.handler.codec.http.HttpHeaders.Names.CONTENT_TYPE;
+import static io.netty.handler.codec.http.HttpMethod.GET;
+import static io.netty.handler.codec.http.HttpMethod.HEAD;
+import static io.netty.handler.codec.http.HttpResponseStatus.GONE;
+import static io.netty.handler.codec.http.HttpResponseStatus.INTERNAL_SERVER_ERROR;
+import static io.netty.handler.codec.http.HttpResponseStatus.METHOD_NOT_ALLOWED;
+import static io.netty.handler.codec.http.HttpResponseStatus.NOT_FOUND;
+import static io.netty.handler.codec.http.HttpResponseStatus.OK;
+import static io.netty.handler.codec.http.HttpVersion.HTTP_1_1;
+
+
+/**
+ * A generic Mesos artifact server, designed specifically for use by the Mesos Fetcher.
+ *
+ * More information:
+ * http://mesos.apache.org/documentation/latest/fetcher/
+ * http://mesos.apache.org/documentation/latest/fetcher-cache-internals/
+ */
+public class MesosArtifactServer {
+
+	private static final Logger LOG = LoggerFactory.getLogger(MesosArtifactServer.class);
+
+	private final Router router;
+
+	private ServerBootstrap bootstrap;
+
+	private Channel serverChannel;
+
+	private URL baseURL;
+
+	public MesosArtifactServer(String sessionID, String serverHostname, int configuredPort) throws Exception {
+		if (configuredPort < 0 || configuredPort > 0xFFFF) {
+			throw new IllegalArgumentException("File server port is invalid: " + configuredPort);
+		}
+
+		router = new Router();
+
+		ChannelInitializer<SocketChannel> initializer = new ChannelInitializer<SocketChannel>() {
+
+			@Override
+			protected void initChannel(SocketChannel ch) {
+				Handler handler = new Handler(router);
+
+				ch.pipeline()
+					.addLast(new HttpServerCodec())
+					.addLast(handler.name(), handler)
+					.addLast(new UnknownFileHandler());
+			}
+		};
+
+		NioEventLoopGroup bossGroup   = new NioEventLoopGroup(1);
+		NioEventLoopGroup workerGroup = new NioEventLoopGroup();
+
+		this.bootstrap = new ServerBootstrap();
+		this.bootstrap
+			.group(bossGroup, workerGroup)
+			.channel(NioServerSocketChannel.class)
+			.childHandler(initializer);
+
+		Channel ch = this.bootstrap.bind(serverHostname, configuredPort).sync().channel();
+		this.serverChannel = ch;
+
+		InetSocketAddress bindAddress = (InetSocketAddress) ch.localAddress();
+		String address = bindAddress.getAddress().getHostAddress();
+		int port = bindAddress.getPort();
+
+		baseURL = new URL("http", serverHostname, port, "/" + sessionID + "/");
+
+		LOG.info("Mesos artifact server listening at " + address + ':' + port);
+	}
+
+	/**
+	 * Get the server port on which the artifact server is listening.
+     */
+	public synchronized int getServerPort() {
+		Channel server = this.serverChannel;
+		if (server != null) {
+			try {
+				return ((InetSocketAddress) server.localAddress()).getPort();
+			} catch (Exception e) {
+				LOG.error("Cannot access local server port", e);
+			}
+		}
+		return -1;
+	}
+
+	/**
+	 * Adds a file to the artifact server.
+	 * @param localFile the local file to serve.
+	 * @param remoteFile the remote path with which to locate the file.
+	 * @return the fully-qualified remote path to the file.
+	 * @throws MalformedURLException if the remote path is invalid.
+     */
+	public synchronized URL addFile(File localFile, String remoteFile) throws MalformedURLException {
+		URL fileURL = new URL(baseURL, remoteFile);
+		router.ANY(fileURL.getPath(), new VirtualFileServerHandler(localFile));
+		return fileURL;
+	}
+
+	/**
+	 * Stops the artifact server.
+	 * @throws Exception
+     */
+	public synchronized void stop() throws Exception {
+		if (this.serverChannel != null) {
+			this.serverChannel.close().awaitUninterruptibly();
+			this.serverChannel = null;
+		}
+		if (bootstrap != null) {
+			if (bootstrap.group() != null) {
+				bootstrap.group().shutdownGracefully();
+			}
+			bootstrap = null;
+		}
+	}
+
+	/**
+	 * Handle HEAD and GET requests for a specific file.
+	 */
+	@ChannelHandler.Sharable
+	public static class VirtualFileServerHandler extends SimpleChannelInboundHandler<Routed> {
+
+		private File file;
+
+		public VirtualFileServerHandler(File file) {
+			this.file = file;
+			if(!file.exists()) {
+				throw new IllegalArgumentException("no such file: " + file.getAbsolutePath());
+			}
+		}
+
+		@Override
+		protected void channelRead0(ChannelHandlerContext ctx, Routed routed) throws Exception {
+
+			HttpRequest request = routed.request();
+
+			if (LOG.isDebugEnabled()) {
+				LOG.debug(request.getMethod() + " request for file '" + file.getAbsolutePath() + '\'');
+			}
+
+			if(!(request.getMethod() == GET || request.getMethod() == HEAD)) {
+				sendMethodNotAllowed(ctx);
+				return;
+			}
+
+			final RandomAccessFile raf;
+			try {
+				raf = new RandomAccessFile(file, "r");
+			}
+			catch (FileNotFoundException e) {
+				sendError(ctx, GONE);
+				return;
+			}
+			try {
+				long fileLength = raf.length();
+
+				// compose the response
+				HttpResponse response = new DefaultHttpResponse(HTTP_1_1, OK);
+				if (HttpHeaders.isKeepAlive(request)) {
+					response.headers().set(CONNECTION, HttpHeaders.Values.KEEP_ALIVE);
+				}
+				HttpHeaders.setHeader(response, CACHE_CONTROL, "private");
+				HttpHeaders.setHeader(response, CONTENT_TYPE, Mimetypes.MIMETYPE_OCTET_STREAM);
+				HttpHeaders.setContentLength(response, fileLength);
+
+				ctx.write(response);
+
+				if (request.getMethod() == GET) {
+					// write the content.  Netty's DefaultFileRegion will close the file.
+					ctx.write(new DefaultFileRegion(raf.getChannel(), 0, fileLength), ctx.newProgressivePromise());
+				}
+				else {
+					// close the file immediately in HEAD case
+					raf.close();
+				}
+				ChannelFuture lastContentFuture = ctx.writeAndFlush(LastHttpContent.EMPTY_LAST_CONTENT);
+
+				// close the connection, if no keep-alive is needed
+				if (!HttpHeaders.isKeepAlive(request)) {
+					lastContentFuture.addListener(ChannelFutureListener.CLOSE);
+				}
+			}
+			catch(Exception ex) {
+				raf.close();
+				throw ex;
+			}
+		}
+
+		@Override
+		public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) {
+			if (ctx.channel().isActive()) {
+				LOG.error("Caught exception", cause);
+				sendError(ctx, INTERNAL_SERVER_ERROR);
+			}
+		}
+
+		/**
+		 * Send the "405 Method Not Allowed" response.
+		 *
+		 * @param ctx The channel context to write the response to.
+		 */
+		private static void sendMethodNotAllowed(ChannelHandlerContext ctx) {
+			FullHttpResponse response = new DefaultFullHttpResponse(HTTP_1_1, METHOD_NOT_ALLOWED);
+
+			// close the connection as soon as the error message is sent.
+			ctx.writeAndFlush(response).addListener(ChannelFutureListener.CLOSE);
+		}
+
+		/**
+		 * Writes a simple  error response message.
+		 *
+		 * @param ctx    The channel context to write the response to.
+		 * @param status The response status.
+		 */
+		private static void sendError(ChannelHandlerContext ctx, HttpResponseStatus status) {
+			FullHttpResponse response = new DefaultFullHttpResponse(
+				HTTP_1_1, status, Unpooled.copiedBuffer("Failure: " + status + "\r\n", CharsetUtil.UTF_8));
+			HttpHeaders.setHeader(response, CONTENT_TYPE, "text/plain; charset=UTF-8");
+
+			// close the connection as soon as the error message is sent.
+			ctx.writeAndFlush(response).addListener(ChannelFutureListener.CLOSE);
+		}
+	}
+
+
+	/**
+	 * Handle a request for a non-existent file.
+	 */
+	@ChannelHandler.Sharable
+	public static class UnknownFileHandler extends SimpleChannelInboundHandler<Object> {
+
+		@Override
+		protected void channelRead0(ChannelHandlerContext ctx, Object message) {
+			sendNotFound(ctx);
+		}
+
+		private static void sendNotFound(ChannelHandlerContext ctx) {
+			FullHttpResponse response = new DefaultFullHttpResponse(HTTP_1_1, NOT_FOUND);
+
+			// close the connection as soon as the error message is sent.
+			ctx.writeAndFlush(response).addListener(ChannelFutureListener.CLOSE);
+		}
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/d9b2be05/flink-mesos/src/main/java/org/apache/flink/mesos/util/MesosConfiguration.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/util/MesosConfiguration.java b/flink-mesos/src/main/java/org/apache/flink/mesos/util/MesosConfiguration.java
new file mode 100644
index 0000000..46a6c0b
--- /dev/null
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/util/MesosConfiguration.java
@@ -0,0 +1,108 @@
+package org.apache.flink.mesos.util;
+
+import org.apache.mesos.MesosSchedulerDriver;
+import org.apache.mesos.Protos;
+import org.apache.mesos.Scheduler;
+import org.apache.mesos.SchedulerDriver;
+import org.slf4j.Logger;
+import scala.Option;
+
+import java.util.Map;
+
+public class MesosConfiguration {
+
+	private String masterUrl;
+
+	private Protos.FrameworkInfo.Builder frameworkInfo;
+
+	private Option<Protos.Credential.Builder> credential = Option.empty();
+
+	public MesosConfiguration(
+		String masterUrl,
+		Protos.FrameworkInfo.Builder frameworkInfo,
+		Option<Protos.Credential.Builder> credential) {
+
+		this.masterUrl = masterUrl;
+		this.frameworkInfo = frameworkInfo;
+		this.credential = credential;
+	}
+
+	public String masterUrl() {
+		return masterUrl;
+	}
+
+	public Protos.FrameworkInfo.Builder frameworkInfo() {
+		return frameworkInfo;
+	}
+
+	public Option<Protos.Credential.Builder> credential() {
+		return credential;
+	}
+
+	/**
+	 * Revise the configuration with updated framework info.
+     */
+	public MesosConfiguration withFrameworkInfo(Protos.FrameworkInfo.Builder frameworkInfo) {
+		return new MesosConfiguration(masterUrl, frameworkInfo, credential);
+	}
+
+	/**
+	 * Create the Mesos scheduler driver based on this configuration.
+	 * @param scheduler the scheduler to use.
+	 * @param implicitAcknowledgements whether to configure the driver for implicit implicit acknowledgements.
+     * @return a scheduler driver.
+     */
+	public SchedulerDriver createDriver(Scheduler scheduler, boolean implicitAcknowledgements) {
+		MesosSchedulerDriver schedulerDriver;
+		if(this.credential().isDefined()) {
+			schedulerDriver =
+				new MesosSchedulerDriver(scheduler, frameworkInfo.build(), this.masterUrl(), false,
+					this.credential().get().build());
+		}
+		else {
+			schedulerDriver =
+				new MesosSchedulerDriver(scheduler, frameworkInfo.build(), this.masterUrl(), false);
+		}
+		return schedulerDriver;
+	}
+
+	@Override
+	public String toString() {
+		return "MesosConfiguration{" +
+			"masterUrl='" + masterUrl + '\'' +
+			", frameworkInfo=" + frameworkInfo +
+			", credential=" + (credential.isDefined() ? "(not shown)" : "(none)") +
+			'}';
+	}
+
+	/**
+	 * A utility method to log relevant Mesos connection info
+     */
+	public static void logMesosConfig(Logger log, MesosConfiguration config) {
+
+		Map<String,String> env = System.getenv();
+		Protos.FrameworkInfo.Builder info = config.frameworkInfo();
+
+		log.info("--------------------------------------------------------------------------------");
+		log.info(" Mesos Info:");
+		log.info("    Master URL: {}", config.masterUrl());
+
+		log.info(" Framework Info:");
+		log.info("    ID: {}", info.hasId() ? info.getId().getValue() : "(none)");
+		log.info("    Name: {}", info.hasName() ? info.getName() : "(none)");
+		log.info("    Failover Timeout (secs): {}", info.getFailoverTimeout());
+		log.info("    Role: {}", info.hasRole() ? info.getRole() : "(none)");
+		log.info("    Principal: {}", info.hasPrincipal() ? info.getPrincipal() : "(none)");
+		log.info("    Host: {}", info.hasHostname() ? info.getHostname() : "(none)");
+		if(env.containsKey("LIBPROCESS_IP")) {
+			log.info("    LIBPROCESS_IP: {}", env.get("LIBPROCESS_IP"));
+		}
+		if(env.containsKey("LIBPROCESS_PORT")) {
+			log.info("    LIBPROCESS_PORT: {}", env.get("LIBPROCESS_PORT"));
+		}
+		log.info("    Web UI: {}", info.hasWebuiUrl() ? info.getWebuiUrl() : "(none)");
+
+		log.info("--------------------------------------------------------------------------------");
+
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/d9b2be05/flink-mesos/src/main/java/org/apache/flink/mesos/util/ZooKeeperUtils.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/util/ZooKeeperUtils.java b/flink-mesos/src/main/java/org/apache/flink/mesos/util/ZooKeeperUtils.java
new file mode 100644
index 0000000..1cad8e7
--- /dev/null
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/util/ZooKeeperUtils.java
@@ -0,0 +1,22 @@
+package org.apache.flink.mesos.util;
+
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.flink.configuration.Configuration;
+
+public class ZooKeeperUtils {
+
+	/**
+	 * Starts a {@link CuratorFramework} instance and connects it to the given ZooKeeper
+	 * quorum.
+	 *
+	 * @param configuration {@link Configuration} object containing the configuration values
+	 * @return {@link CuratorFramework} instance
+	 */
+	@SuppressWarnings("unchecked")
+	public static CuratorFramework startCuratorFramework(Configuration configuration) {
+
+		// workaround for shaded curator dependency of flink-runtime
+		Object client = org.apache.flink.runtime.util.ZooKeeperUtils.startCuratorFramework(configuration);
+		return (CuratorFramework) client;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/d9b2be05/flink-mesos/src/main/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/resources/log4j.properties b/flink-mesos/src/main/resources/log4j.properties
new file mode 100644
index 0000000..61e5afe
--- /dev/null
+++ b/flink-mesos/src/main/resources/log4j.properties
@@ -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.
+################################################################################
+
+
+# Convenience file for local debugging of the JobManager/TaskManager.
+log4j.rootLogger=INFO, console
+log4j.appender.console=org.apache.log4j.ConsoleAppender
+log4j.appender.console.layout=org.apache.log4j.PatternLayout
+log4j.appender.console.layout.ConversionPattern=%d{HH:mm:ss,SSS} %-5p %-60c %x - %m%n
+
+log4j.logger.org.apache.flink.mesos=DEBUG
+log4j.logger.org.apache.flink.mesos.runtime.clusterframework.MesosFlinkResourceManager=INFO

http://git-wip-us.apache.org/repos/asf/flink/blob/d9b2be05/flink-mesos/src/main/scala/org/apache/flink/mesos/runtime/clusterframework/MesosJobManager.scala
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/scala/org/apache/flink/mesos/runtime/clusterframework/MesosJobManager.scala b/flink-mesos/src/main/scala/org/apache/flink/mesos/runtime/clusterframework/MesosJobManager.scala
new file mode 100644
index 0000000..f789163
--- /dev/null
+++ b/flink-mesos/src/main/scala/org/apache/flink/mesos/runtime/clusterframework/MesosJobManager.scala
@@ -0,0 +1,66 @@
+package org.apache.flink.mesos.runtime.clusterframework
+
+import java.util.concurrent.ExecutorService
+
+import akka.actor.ActorRef
+import org.apache.flink.configuration.{Configuration => FlinkConfiguration}
+import org.apache.flink.runtime.checkpoint.savepoint.SavepointStore
+import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory
+import org.apache.flink.runtime.clusterframework.ContaineredJobManager
+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.SubmittedJobGraphStore
+import org.apache.flink.runtime.jobmanager.scheduler.{Scheduler => FlinkScheduler}
+import org.apache.flink.runtime.leaderelection.LeaderElectionService
+import org.apache.flink.runtime.metrics.{MetricRegistry => FlinkMetricRegistry}
+
+import scala.concurrent.duration._
+
+/** JobManager actor for execution on Mesos. .
+  *
+  * @param flinkConfiguration Configuration object for the actor
+  * @param executorService Execution context which is used to execute concurrent tasks in the
+  *                         [[org.apache.flink.runtime.executiongraph.ExecutionGraph]]
+  * @param instanceManager Instance manager to manage the registered
+  *                        [[org.apache.flink.runtime.taskmanager.TaskManager]]
+  * @param scheduler Scheduler to schedule Flink jobs
+  * @param libraryCacheManager Manager to manage uploaded jar files
+  * @param archive Archive for finished Flink jobs
+  * @param restartStrategyFactory Restart strategy to be used in case of a job recovery
+  * @param timeout Timeout for futures
+  * @param leaderElectionService LeaderElectionService to participate in the leader election
+  */
+class MesosJobManager(flinkConfiguration: FlinkConfiguration,
+                      executorService: ExecutorService,
+                      instanceManager: InstanceManager,
+                      scheduler: FlinkScheduler,
+                      libraryCacheManager: BlobLibraryCacheManager,
+                      archive: ActorRef,
+                      restartStrategyFactory: RestartStrategyFactory,
+                      timeout: FiniteDuration,
+                      leaderElectionService: LeaderElectionService,
+                      submittedJobGraphs : SubmittedJobGraphStore,
+                      checkpointRecoveryFactory : CheckpointRecoveryFactory,
+                      savepointStore: SavepointStore,
+                      jobRecoveryTimeout: FiniteDuration,
+                      metricsRegistry: Option[FlinkMetricRegistry])
+  extends ContaineredJobManager(
+    flinkConfiguration,
+    executorService,
+    instanceManager,
+    scheduler,
+    libraryCacheManager,
+    archive,
+    restartStrategyFactory,
+    timeout,
+    leaderElectionService,
+    submittedJobGraphs,
+    checkpointRecoveryFactory,
+    savepointStore,
+    jobRecoveryTimeout,
+    metricsRegistry) {
+
+  val jobPollingInterval: FiniteDuration = 5 seconds
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/d9b2be05/flink-mesos/src/main/scala/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManager.scala
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/scala/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManager.scala b/flink-mesos/src/main/scala/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManager.scala
new file mode 100644
index 0000000..304e66d
--- /dev/null
+++ b/flink-mesos/src/main/scala/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManager.scala
@@ -0,0 +1,47 @@
+package org.apache.flink.mesos.runtime.clusterframework
+
+import org.apache.flink.runtime.clusterframework.types.ResourceID
+import org.apache.flink.runtime.instance.InstanceConnectionInfo
+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.{TaskManager, TaskManagerConfiguration}
+
+/** An extension of the TaskManager that listens for additional Mesos-related
+  * messages.
+  */
+class MesosTaskManager(
+                       config: TaskManagerConfiguration,
+                       resourceID: ResourceID,
+                       connectionInfo: InstanceConnectionInfo,
+                       memoryManager: MemoryManager,
+                       ioManager: IOManager,
+                       network: NetworkEnvironment,
+                       numberOfSlots: Int,
+                       leaderRetrievalService: LeaderRetrievalService)
+  extends TaskManager(
+    config,
+    resourceID,
+    connectionInfo,
+    memoryManager,
+    ioManager,
+    network,
+    numberOfSlots,
+    leaderRetrievalService) {
+
+  override def handleMessage: Receive = {
+    super.handleMessage
+  }
+}
+
+object MesosTaskManager {
+  /** Entry point (main method) to run the TaskManager on Mesos.
+    *
+    * @param args The command line arguments.
+    */
+  def main(args: Array[String]): Unit = {
+    MesosTaskManagerRunner.runTaskManager(args, classOf[MesosTaskManager])
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/d9b2be05/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/ConnectionMonitor.scala
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/ConnectionMonitor.scala b/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/ConnectionMonitor.scala
new file mode 100644
index 0000000..91cc66c
--- /dev/null
+++ b/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/ConnectionMonitor.scala
@@ -0,0 +1,108 @@
+package org.apache.flink.mesos.scheduler
+
+import akka.actor.{Actor, FSM, Props}
+import grizzled.slf4j.Logger
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.mesos.scheduler.ConnectionMonitor._
+import org.apache.flink.mesos.scheduler.messages._
+
+import scala.concurrent.duration._
+
+/**
+  * Actively monitors the Mesos connection.
+  */
+class ConnectionMonitor() extends Actor with FSM[FsmState, Unit] {
+
+  val LOG = Logger(getClass)
+
+  startWith(StoppedState, None)
+
+  when(StoppedState) {
+    case Event(msg: Start, _) =>
+      LOG.info(s"Connecting to Mesos...")
+      goto(ConnectingState)
+  }
+
+  when(ConnectingState, stateTimeout = CONNECT_RETRY_RATE) {
+    case Event(msg: Stop, _) =>
+      goto(StoppedState)
+
+    case Event(msg: Registered, _) =>
+      LOG.info(s"Connected to Mesos as framework ID ${msg.frameworkId.getValue}.")
+      LOG.debug(s"   Master Info: ${msg.masterInfo}")
+      goto(ConnectedState)
+
+    case Event(msg: ReRegistered, _) =>
+      LOG.info("Reconnected to a new Mesos master.")
+      LOG.debug(s"   Master Info: ${msg.masterInfo}")
+      goto(ConnectedState)
+
+    case Event(StateTimeout, _) =>
+      LOG.warn("Unable to connect to Mesos; still trying...")
+      stay()
+  }
+
+  when(ConnectedState) {
+    case Event(msg: Stop, _) =>
+      goto(StoppedState)
+
+    case Event(msg: Disconnected, _) =>
+      LOG.warn("Disconnected from the Mesos master.  Reconnecting...")
+      goto(ConnectingState)
+  }
+
+  onTransition {
+    case previousState -> nextState =>
+      LOG.debug(s"State change ($previousState -> $nextState) with data ${nextStateData}")
+  }
+
+  initialize()
+}
+
+object ConnectionMonitor {
+
+  val CONNECT_RETRY_RATE = (5 seconds)
+
+  // ------------------------------------------------------------------------
+  // State
+  // ------------------------------------------------------------------------
+
+  /**
+    * An FSM state of the connection monitor.
+    */
+  sealed trait FsmState
+  case object StoppedState extends FsmState
+  case object ConnectingState extends FsmState
+  case object ConnectedState extends FsmState
+
+  // ------------------------------------------------------------------------
+  //  Messages
+  // ------------------------------------------------------------------------
+
+  /**
+    * Starts the connection monitor.
+    */
+  case class Start()
+
+  /**
+    * Stops the connection monitor.
+    */
+  case class Stop()
+
+  // ------------------------------------------------------------------------
+  //  Utils
+  // ------------------------------------------------------------------------
+
+  /**
+    * Creates the properties for the ConnectionMonitor actor.
+    *
+    * @param actorClass the connection monitor actor class
+    * @param flinkConfig the Flink configuration.
+    * @tparam T the type of the connection monitor actor class
+    * @return the Props to create the connection monitor
+    */
+  def createActorProps[T <: ConnectionMonitor](actorClass: Class[T],
+     flinkConfig: Configuration): Props = {
+    Props.create(actorClass)
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/d9b2be05/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/LaunchCoordinator.scala
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/LaunchCoordinator.scala b/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/LaunchCoordinator.scala
new file mode 100644
index 0000000..a908ef9
--- /dev/null
+++ b/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/LaunchCoordinator.scala
@@ -0,0 +1,331 @@
+package org.apache.flink.mesos.scheduler
+
+import akka.actor.{Actor, ActorRef, FSM, Props}
+import com.netflix.fenzo._
+import com.netflix.fenzo.functions.Action1
+import com.netflix.fenzo.plugins.VMLeaseObject
+import grizzled.slf4j.Logger
+import org.apache.flink.api.java.tuple.{Tuple2=>FlinkTuple2}
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.mesos.scheduler.LaunchCoordinator._
+import org.apache.flink.mesos.scheduler.messages._
+import org.apache.mesos.Protos.TaskInfo
+import org.apache.mesos.{SchedulerDriver, Protos}
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable.{Map => MutableMap}
+import scala.concurrent.duration._
+
+/**
+  * The launch coordinator handles offer processing, including
+  * matching offers to tasks and making reservations.
+  *
+  * The coordinator uses Netflix Fenzo to optimize task placement.   During the GatheringOffers phase,
+  * offers are evaluated by Fenzo for suitability to the planned tasks.   Reservations are then placed
+  * against the best offers, leading to revised offers containing reserved resources with which to launch task(s).
+  */
+class LaunchCoordinator(
+    manager: ActorRef,
+    config: Configuration,
+    schedulerDriver: SchedulerDriver,
+    optimizerBuilder: TaskSchedulerBuilder
+  ) extends Actor with FSM[TaskState, GatherData] {
+
+  val LOG = Logger(getClass)
+
+  /**
+    * The task placement optimizer.
+    *
+    * The optimizer contains the following state:
+    *  - unused offers
+    *  - existing task placement (for fitness calculation involving task colocation)
+    */
+  private[mesos] val optimizer: TaskScheduler = {
+    optimizerBuilder
+      .withLeaseRejectAction(new Action1[VirtualMachineLease]() {
+        def call(lease: VirtualMachineLease) {
+          LOG.info(s"Declined offer ${lease.getId} from ${lease.hostname()} of ${lease.memoryMB()} MB, ${lease.cpuCores()} cpus.")
+          schedulerDriver.declineOffer(lease.getOffer.getId)
+        }
+      }).build
+  }
+
+  override def postStop(): Unit = {
+    optimizer.shutdown()
+    super.postStop()
+  }
+
+  /**
+    * Initial state
+    */
+  startWith(Suspended, GatherData(tasks = Nil, newLeases = Nil))
+
+  /**
+    * State: Suspended
+    *
+    * Wait for (re-)connection to Mesos.   No offers exist in this state, but outstanding tasks might.
+    */
+  when(Suspended) {
+    case Event(msg: Connected, data: GatherData) =>
+      if(data.tasks.nonEmpty) goto(GatheringOffers)
+      else goto(Idle)
+  }
+
+  /**
+    * State: Idle
+    *
+    * Wait for a task request to arrive, then transition into gathering offers.
+    */
+  onTransition {
+    case _ -> Idle => assert(nextStateData.tasks.isEmpty)
+  }
+
+  when(Idle) {
+    case Event(msg: Disconnected, data: GatherData) =>
+      goto(Suspended)
+
+    case Event(offers: ResourceOffers, data: GatherData) =>
+      // decline any offers that come in
+      schedulerDriver.suppressOffers()
+      for(offer <- offers.offers().asScala) { schedulerDriver.declineOffer(offer.getId) }
+      stay()
+
+    case Event(msg: Launch, data: GatherData) =>
+      goto(GatheringOffers) using data.copy(tasks = data.tasks ++ msg.tasks.asScala)
+  }
+
+  /**
+    * Transition logic to control the flow of offers.
+    */
+  onTransition {
+    case _ -> GatheringOffers =>
+      LOG.info(s"Now gathering offers for at least ${nextStateData.tasks.length} task(s).")
+      schedulerDriver.reviveOffers()
+
+    case GatheringOffers -> _ =>
+      // decline any outstanding offers and suppress future offers
+      LOG.info(s"No longer gathering offers; all requests fulfilled.")
+
+      assert(nextStateData.newLeases.isEmpty)
+      schedulerDriver.suppressOffers()
+      optimizer.expireAllLeases()
+  }
+
+  /**
+    * State: GatheringOffers
+    *
+    * Wait for offers to accumulate for a fixed length of time or from specific slaves.
+    *
+    * While gathering offers, other task requests may safely arrive.
+    */
+  when(GatheringOffers, stateTimeout = GATHER_DURATION) {
+
+    case Event(msg: Disconnected, data: GatherData) =>
+      // reconciliation spec: offers are implicitly declined upon disconnect
+      goto(Suspended) using data.copy(newLeases = Nil)
+
+    case Event(offers: ResourceOffers, data: GatherData) =>
+      val leases = offers.offers().asScala.map(new VMLeaseObject(_).asInstanceOf[VirtualMachineLease])
+      if(LOG.isInfoEnabled) {
+        val (cpus, mem) = leases.foldLeft((0.0,0.0)) { (z,o) => (z._1 + o.cpuCores(), z._2 + o.memoryMB()) }
+        LOG.info(s"Received offer(s) of $mem MB, $cpus cpus:")
+        for(lease <- leases) {
+          LOG.info(s"  ${lease.getId} from ${lease.hostname()} of ${lease.memoryMB()} MB, ${lease.cpuCores()} cpus")
+        }
+      }
+      stay using data.copy(newLeases = data.newLeases ++ leases)
+
+    case Event(StateTimeout, data: GatherData) =>
+      val remaining = MutableMap(data.tasks.map(t => t.taskRequest.getId -> t):_*)
+
+      LOG.info(s"Processing ${remaining.size} task(s) against ${data.newLeases.length} new offer(s) plus outstanding offers.")
+
+      // attempt to assign the outstanding tasks using the optimizer
+      val result = optimizer.scheduleOnce(
+        data.tasks.map(_.taskRequest).asJava, data.newLeases.asJava)
+
+      if(LOG.isInfoEnabled) {
+        // note that vmCurrentStates are computed before any actions taken (incl. expiration)
+        LOG.info("Resources considered: (note: expired offers not deducted from below)")
+        for(vm <- optimizer.getVmCurrentStates.asScala) {
+          val lease = vm.getCurrAvailableResources
+          LOG.info(s"  ${vm.getHostname} has ${lease.memoryMB()} MB, ${lease.cpuCores()} cpus")
+        }
+      }
+      log.debug(result.toString)
+
+      for((hostname, assignments) <- result.getResultMap.asScala) {
+
+        // process the assignments into a set of operations (reserve and/or launch)
+        val slaveId = assignments.getLeasesUsed.get(0).getOffer.getSlaveId
+        val offerIds = assignments.getLeasesUsed.asScala.map(_.getOffer.getId)
+        val operations = processAssignments(slaveId, assignments, remaining.toMap)
+
+        // update the state to reflect the launched tasks
+        val launchedTasks = operations
+          .filter(_.getType==Protos.Offer.Operation.Type.LAUNCH)
+          .flatMap(_.getLaunch.getTaskInfosList.asScala.map(_.getTaskId))
+        for(taskId <- launchedTasks) {
+          val task = remaining.remove(taskId.getValue).get
+          LOG.debug(s"Assigned task ${task.taskRequest().getId} to host ${hostname}.")
+          optimizer.getTaskAssigner.call(task.taskRequest, hostname)
+        }
+
+        // send the operations to Mesos via manager
+        manager ! new AcceptOffers(hostname, offerIds.asJava, operations.asJava)
+
+        if(LOG.isInfoEnabled) {
+          LOG.info(s"Launched ${launchedTasks.length} task(s) on ${hostname} using ${offerIds.length} offer(s):")
+          for(offerId <- offerIds) {
+            LOG.info(s"  ${offerId.getValue}")
+          }
+        }
+      }
+
+      // stay in GatheringOffers state if any tasks remain, otherwise transition to idle
+      if(remaining.isEmpty) {
+        goto(Idle) using data.copy(newLeases = Nil, tasks = Nil)
+      } else {
+        LOG.info(s"Waiting for more offers; ${remaining.size} task(s) are not yet launched.")
+
+        stay using data.copy(newLeases = Nil, tasks = remaining.values.toList) forMax SUBSEQUENT_GATHER_DURATION
+      }
+  }
+
+  /**
+    * Default handling of events.
+    */
+  whenUnhandled {
+    case Event(msg: Launch, data: GatherData) =>
+      // accumulate any tasks that come in
+      stay using data.copy(tasks = data.tasks ++ msg.tasks.asScala)
+
+    case Event(offer: OfferRescinded, data: GatherData) =>
+      // forget rescinded offers
+      LOG.info(s"Offer ${offer.offerId()} was rescinded.")
+      optimizer.expireLease(offer.offerId().getValue)
+      stay using data.copy(newLeases = data.newLeases.filterNot(_.getOffer.getId == offer.offerId()))
+
+    case Event(msg: Assign, _) =>
+      // recovering an earlier task assignment
+      for(task <- msg.tasks.asScala) {
+        LOG.debug(s"Assigned task ${task.f0.getId} to host ${task.f1}.")
+        optimizer.getTaskAssigner.call(task.f0, task.f1)
+      }
+      stay()
+
+    case Event(msg: Unassign, _) =>
+      // planning to terminate a task - unassign it from its host in the optimizer's state
+      LOG.debug(s"Unassigned task ${msg.taskID} from host ${msg.hostname}.")
+      optimizer.getTaskUnAssigner.call(msg.taskID.getValue, msg.hostname)
+      stay()
+  }
+
+  onTransition {
+    case previousState -> nextState =>
+      LOG.debug(s"State change ($previousState -> $nextState) with data $nextStateData")
+  }
+
+  initialize()
+}
+
+object LaunchCoordinator {
+
+  val GATHER_DURATION = 5.seconds
+  val SUBSEQUENT_GATHER_DURATION = 5.seconds
+
+  // ------------------------------------------------------------------------
+  //  FSM State
+  // ------------------------------------------------------------------------
+
+  /**
+    * An FSM state of the launch coordinator.
+    */
+  sealed trait TaskState
+  case object GatheringOffers extends TaskState
+  case object Idle extends TaskState
+  case object Suspended extends TaskState
+
+  /**
+    * FSM state data.
+    *
+    * @param tasks the tasks to launch.
+    * @param newLeases new leases not yet handed to the optimizer.
+    */
+  case class GatherData(tasks: Seq[LaunchableTask] = Nil, newLeases: Seq[VirtualMachineLease] = Nil)
+
+  // ------------------------------------------------------------------------
+  //  Messages
+  // ------------------------------------------------------------------------
+
+  /**
+    * Instructs the launch coordinator to launch some new task.
+    */
+  case class Launch(tasks: java.util.List[LaunchableTask]) {
+    require(tasks.size() >= 1, "Launch message must contain at least one task")
+  }
+
+  /**
+    * Informs the launch coordinator that some task(s) are assigned to a host (for planning purposes).
+    *
+    * This is sent by the RM in recovery procedures to recover the optimizer state.  In normal operation,
+    * the launch coordinator itself updates the optimizer state.
+    */
+  case class Assign(tasks: java.util.List[FlinkTuple2[TaskRequest, String]])
+
+  /**
+    * Informs the launch coordinator that some task is no longer assigned to a host (for planning purposes).
+    */
+  case class Unassign(taskID: Protos.TaskID, hostname: String)
+
+  // ------------------------------------------------------------------------
+  //  Utils
+  // ------------------------------------------------------------------------
+
+  /**
+    * Process the given task assignments into a set of Mesos operations.
+    *
+    * The operations may include reservations and task launches.
+    *
+    * @param slaveId the slave associated with the given assignments.
+    * @param assignments the task assignments as provided by the optimizer.
+    * @param allTasks all known tasks, keyed by taskId.
+    * @return the operations to perform.
+    */
+  private def processAssignments(
+      slaveId: Protos.SlaveID,
+      assignments: VMAssignmentResult,
+      allTasks: Map[String, LaunchableTask]): Seq[Protos.Offer.Operation] = {
+
+    def taskInfo(assignment: TaskAssignmentResult): Protos.TaskInfo = {
+      allTasks(assignment.getTaskId).launch(slaveId, assignment)
+    }
+
+    val launches = Protos.Offer.Operation.newBuilder().setType(Protos.Offer.Operation.Type.LAUNCH).setLaunch(
+      Protos.Offer.Operation.Launch.newBuilder().addAllTaskInfos(
+        assignments.getTasksAssigned.asScala.map(taskInfo).asJava
+      )
+    ).build()
+
+    Seq(launches)
+  }
+
+  /**
+    * Get the configuration properties for the launch coordinator.
+    *
+    * @param actorClass the launch coordinator actor class.
+    * @param flinkConfig the Flink configuration.
+    * @param schedulerDriver the Mesos scheduler driver.
+    * @tparam T the launch coordinator actor class.
+    * @return the Akka props to create the launch coordinator actor.
+    */
+  def createActorProps[T <: LaunchCoordinator](
+    actorClass: Class[T],
+    manager: ActorRef,
+    flinkConfig: Configuration,
+    schedulerDriver: SchedulerDriver,
+    optimizerBuilder: TaskSchedulerBuilder): Props = {
+
+    Props.create(actorClass, manager, flinkConfig, schedulerDriver, optimizerBuilder)
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/d9b2be05/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/ReconciliationCoordinator.scala
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/ReconciliationCoordinator.scala b/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/ReconciliationCoordinator.scala
new file mode 100644
index 0000000..f3edf40
--- /dev/null
+++ b/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/ReconciliationCoordinator.scala
@@ -0,0 +1,164 @@
+package org.apache.flink.mesos.scheduler
+
+import java.util.concurrent.ThreadLocalRandom
+
+import akka.actor.{Actor, FSM, Props}
+import grizzled.slf4j.Logger
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.mesos.scheduler.ReconciliationCoordinator._
+import org.apache.flink.mesos.scheduler.messages.{Connected, Disconnected, StatusUpdate}
+import org.apache.mesos.{SchedulerDriver, Protos}
+
+import scala.collection.JavaConverters._
+import scala.concurrent.duration._
+
+/**
+  * Coordinates task reconciliation between Mesos and the scheduler.
+  *
+  * Implements the reconciliation procedures as outlined here:
+  * http://mesos.apache.org/documentation/latest/reconciliation/
+  *
+  */
+class ReconciliationCoordinator(
+    flinkConfig: Configuration,
+    schedulerDriver: SchedulerDriver) extends Actor with FSM[TaskState,ReconciliationData] {
+
+  val LOG = Logger(getClass)
+
+  startWith(Suspended, ReconciliationData())
+
+  when(Suspended) {
+    case Event(reconcile: Reconcile, data: ReconciliationData) =>
+      val tasks = reconcile.tasks.map(task => (task.getTaskId,task))
+      stay using data.copy(remaining = if(reconcile.replace) tasks.toMap else data.remaining ++ tasks)
+
+    case Event(msg: Connected, data: ReconciliationData) =>
+      if(data.remaining.nonEmpty) goto(Reconciling)
+      else goto(Idle) using ReconciliationData()
+  }
+
+  when(Idle) {
+    case Event(reconcile: Reconcile, _) =>
+      goto(Reconciling) using {
+        val tasks = reconcile.tasks.map(task => (task.getTaskId,task))
+        ReconciliationData(remaining = tasks.toMap)
+      }
+  }
+
+  onTransition {
+    case _ -> Reconciling =>
+      log.info(s"Reconciliation requested for ${nextStateData.remaining.size} task(s)")
+      schedulerDriver.reconcileTasks(nextStateData.remaining.values.asJavaCollection)
+  }
+
+  when(Reconciling, stateTimeout = INITIAL_RECONCILIATION_TIMEOUT) {
+
+    case Event(reconcile: Reconcile, data: ReconciliationData) =>
+      // initiate reconciliation for additional tasks (even while reconciliation is ongoing)
+      schedulerDriver.reconcileTasks(reconcile.tasks.asJavaCollection)
+      val tasks = reconcile.tasks.map(task => (task.getTaskId,task))
+      stay using data.copy(remaining = if(reconcile.replace) tasks.toMap else data.remaining ++ tasks)
+
+    case Event(update: StatusUpdate, data: ReconciliationData) =>
+      // status information arrived for a task
+      val remaining = data.remaining - update.status().getTaskId
+      if(remaining.isEmpty) {
+        log.info("Reconciliation completed")
+        goto(Idle) using ReconciliationData()
+      } else {
+        stay using data.copy(remaining = remaining)
+      }
+
+    case Event(StateTimeout, data: ReconciliationData) =>
+      // timeout waiting for task status information
+      log.warning("Reconciliation is proceeding slowly; re-sending the reconciliation request.")
+      schedulerDriver.reconcileTasks(data.remaining.values.asJavaCollection)
+      stay using data.copy(retries = data.retries + 1) forMax(backoff(data.retries))
+  }
+
+  whenUnhandled {
+    case Event(update: StatusUpdate, _) =>
+      // discard status updates when not in reconciliation state
+      stay()
+
+    case Event(msg: Disconnected, data: ReconciliationData) =>
+      goto(Suspended) using data.copy(retries = 0)
+  }
+
+  onTransition {
+    case previousState -> nextState =>
+      LOG.debug(s"State change ($previousState -> $nextState) with data ${nextStateData}")
+  }
+
+  initialize()
+}
+
+object ReconciliationCoordinator {
+
+  val INITIAL_RECONCILIATION_TIMEOUT = 1 minutes
+  val RECONCILIATION_MIN_BACKOFF = 5 seconds
+  val RECONCILIATION_MAX_BACKOFF = 1 minute
+
+  /**
+    * An abstract FSM state.
+    */
+  sealed trait TaskState
+
+  /**
+    * The state of active reconciliation.
+    */
+  case object Reconciling extends TaskState
+
+  /**
+    * The state of idling when reconciliation is not underway.
+    */
+  case object Idle extends TaskState
+
+  /**
+    * The state of being disconnected from Mesos.
+    */
+  case object Suspended extends TaskState
+
+  /**
+    * The state data of the reconciliation coordinator.
+    *
+    * @param remaining
+    * @param retries
+    */
+  case class ReconciliationData(remaining: Map[Protos.TaskID,Protos.TaskStatus] = Map(), retries: Int = 0)
+
+  /**
+    * Initiates the task reconciliation process.
+    *
+    * @param tasks
+    */
+  case class Reconcile(tasks: Seq[Protos.TaskStatus], replace: Boolean = false) {
+    require(tasks.length >= 1, "Reconcile message must contain at least one task")
+  }
+
+  /**
+    * Calculate an exponential backoff duration.
+    */
+  private def backoff(
+      retries: Int,
+      minBackoff: FiniteDuration = RECONCILIATION_MIN_BACKOFF,
+      maxBackoff: FiniteDuration = RECONCILIATION_MAX_BACKOFF,
+      randomFactor: Double = 0.2): FiniteDuration = {
+    val rnd = 1.0 + ThreadLocalRandom.current().nextDouble() * randomFactor
+    maxBackoff.min(minBackoff * math.pow(2, math.min(retries, 30))) * rnd match {
+      case f: FiniteDuration => f
+      case _ => maxBackoff
+    }
+  }
+
+  /**
+    * Create the properties for a reconciliation coordinator.
+    */
+  def createActorProps[T <: ReconciliationCoordinator](
+      actorClass: Class[T],
+      flinkConfig: Configuration,
+      schedulerDriver: SchedulerDriver): Props = {
+
+    Props.create(actorClass, flinkConfig, schedulerDriver)
+  }
+}


[5/9] flink git commit: [FLINK-1984] Mesos ResourceManager - T1 milestone (2)

Posted by mx...@apache.org.
[FLINK-1984] Mesos ResourceManager - T1 milestone (2)

Add license information


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

Branch: refs/heads/master
Commit: 67be9e074e2eb327dfe9e817b625a0f6350d8da2
Parents: d9b2be0
Author: wrighe3 <er...@emc.com>
Authored: Mon Aug 1 11:09:47 2016 -0700
Committer: Maximilian Michels <mx...@apache.org>
Committed: Mon Aug 29 17:32:46 2016 +0200

----------------------------------------------------------------------
 .../main/java/org/apache/flink/mesos/Utils.java | 18 +++++++++
 .../flink/mesos/cli/FlinkMesosSessionCli.java   | 18 +++++++++
 .../clusterframework/LaunchableMesosWorker.java | 18 +++++++++
 .../MesosApplicationMasterRunner.java           | 18 +++++++++
 .../clusterframework/MesosConfigKeys.java       | 18 +++++++++
 .../MesosFlinkResourceManager.java              | 18 +++++++++
 .../MesosTaskManagerParameters.java             | 18 +++++++++
 .../MesosTaskManagerRunner.java                 | 18 +++++++++
 .../RegisteredMesosWorkerNode.scala             | 18 +++++++++
 .../store/MesosWorkerStore.java                 | 18 +++++++++
 .../store/StandaloneMesosWorkerStore.java       | 18 +++++++++
 .../store/ZooKeeperMesosWorkerStore.java        | 18 +++++++++
 .../flink/mesos/scheduler/LaunchableTask.java   | 18 +++++++++
 .../flink/mesos/scheduler/SchedulerProxy.java   | 18 +++++++++
 .../mesos/scheduler/TaskSchedulerBuilder.java   | 18 +++++++++
 .../mesos/scheduler/messages/AcceptOffers.java  | 18 +++++++++
 .../mesos/scheduler/messages/Connected.java     | 18 +++++++++
 .../mesos/scheduler/messages/Disconnected.java  | 18 +++++++++
 .../flink/mesos/scheduler/messages/Error.java   | 18 +++++++++
 .../scheduler/messages/OfferRescinded.java      | 18 +++++++++
 .../mesos/scheduler/messages/ReRegistered.java  | 18 +++++++++
 .../mesos/scheduler/messages/Registered.java    | 18 +++++++++
 .../scheduler/messages/ResourceOffers.java      | 18 +++++++++
 .../mesos/scheduler/messages/SlaveLost.java     | 18 +++++++++
 .../mesos/scheduler/messages/StatusUpdate.java  | 18 +++++++++
 .../flink/mesos/util/MesosArtifactServer.java   | 18 +++++++++
 .../flink/mesos/util/MesosConfiguration.java    | 40 ++++++++++++++++++++
 .../apache/flink/mesos/util/ZooKeeperUtils.java | 18 +++++++++
 .../clusterframework/MesosJobManager.scala      | 18 +++++++++
 .../clusterframework/MesosTaskManager.scala     | 18 +++++++++
 .../mesos/scheduler/ConnectionMonitor.scala     | 18 +++++++++
 .../mesos/scheduler/LaunchCoordinator.scala     | 18 +++++++++
 .../scheduler/ReconciliationCoordinator.scala   | 18 +++++++++
 .../flink/mesos/scheduler/TaskMonitor.scala     | 18 +++++++++
 .../apache/flink/mesos/scheduler/Tasks.scala    | 18 +++++++++
 .../MesosFlinkResourceManagerTest.java          | 18 +++++++++
 .../scala/org/apache/flink/mesos/Utils.scala    | 18 +++++++++
 .../mesos/scheduler/LaunchCoordinatorTest.scala | 18 +++++++++
 .../ReconciliationCoordinatorTest.scala         | 18 +++++++++
 .../flink/mesos/scheduler/TaskMonitorTest.scala | 18 +++++++++
 .../org/apache/flink/runtime/akka/FSMSpec.scala | 18 +++++++++
 41 files changed, 760 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/67be9e07/flink-mesos/src/main/java/org/apache/flink/mesos/Utils.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/Utils.java b/flink-mesos/src/main/java/org/apache/flink/mesos/Utils.java
index 2509465..bd5ef50 100644
--- a/flink-mesos/src/main/java/org/apache/flink/mesos/Utils.java
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/Utils.java
@@ -1,3 +1,21 @@
+/*
+ * 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.mesos;
 
 import org.apache.mesos.Protos;

http://git-wip-us.apache.org/repos/asf/flink/blob/67be9e07/flink-mesos/src/main/java/org/apache/flink/mesos/cli/FlinkMesosSessionCli.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/cli/FlinkMesosSessionCli.java b/flink-mesos/src/main/java/org/apache/flink/mesos/cli/FlinkMesosSessionCli.java
index b767344..278f1a2 100644
--- a/flink-mesos/src/main/java/org/apache/flink/mesos/cli/FlinkMesosSessionCli.java
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/cli/FlinkMesosSessionCli.java
@@ -1,3 +1,21 @@
+/*
+ * 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.mesos.cli;
 
 import com.fasterxml.jackson.core.JsonProcessingException;

http://git-wip-us.apache.org/repos/asf/flink/blob/67be9e07/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorker.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorker.java b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorker.java
index 8abd79a..d416ec1 100644
--- a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorker.java
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorker.java
@@ -1,3 +1,21 @@
+/*
+ * 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.mesos.runtime.clusterframework;
 
 import com.netflix.fenzo.ConstraintEvaluator;

http://git-wip-us.apache.org/repos/asf/flink/blob/67be9e07/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosApplicationMasterRunner.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosApplicationMasterRunner.java b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosApplicationMasterRunner.java
index 30f2258..1bd02c4 100644
--- a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosApplicationMasterRunner.java
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosApplicationMasterRunner.java
@@ -1,3 +1,21 @@
+/*
+ * 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.mesos.runtime.clusterframework;
 
 import akka.actor.ActorRef;

http://git-wip-us.apache.org/repos/asf/flink/blob/67be9e07/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosConfigKeys.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosConfigKeys.java b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosConfigKeys.java
index 3173286..9413c68 100644
--- a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosConfigKeys.java
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosConfigKeys.java
@@ -1,3 +1,21 @@
+/*
+ * 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.mesos.runtime.clusterframework;
 
 /**

http://git-wip-us.apache.org/repos/asf/flink/blob/67be9e07/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosFlinkResourceManager.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosFlinkResourceManager.java b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosFlinkResourceManager.java
index 483c7b7..66fff7d 100644
--- a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosFlinkResourceManager.java
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosFlinkResourceManager.java
@@ -1,3 +1,21 @@
+/*
+ * 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.mesos.runtime.clusterframework;
 
 import akka.actor.ActorRef;

http://git-wip-us.apache.org/repos/asf/flink/blob/67be9e07/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManagerParameters.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManagerParameters.java b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManagerParameters.java
index b3956aa..5321ab6 100644
--- a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManagerParameters.java
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManagerParameters.java
@@ -1,3 +1,21 @@
+/*
+ * 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.mesos.runtime.clusterframework;
 
 import org.apache.flink.configuration.ConfigConstants;

http://git-wip-us.apache.org/repos/asf/flink/blob/67be9e07/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManagerRunner.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManagerRunner.java b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManagerRunner.java
index 5dfc75e..e34def6 100644
--- a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManagerRunner.java
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManagerRunner.java
@@ -1,3 +1,21 @@
+/*
+ * 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.mesos.runtime.clusterframework;
 
 import java.io.IOException;

http://git-wip-us.apache.org/repos/asf/flink/blob/67be9e07/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/RegisteredMesosWorkerNode.scala
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/RegisteredMesosWorkerNode.scala b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/RegisteredMesosWorkerNode.scala
index c519820..59764ef 100644
--- a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/RegisteredMesosWorkerNode.scala
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/RegisteredMesosWorkerNode.scala
@@ -1,3 +1,21 @@
+/*
+ * 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.mesos.runtime.clusterframework
 
 import org.apache.flink.mesos.runtime.clusterframework.store.MesosWorkerStore

http://git-wip-us.apache.org/repos/asf/flink/blob/67be9e07/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/store/MesosWorkerStore.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/store/MesosWorkerStore.java b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/store/MesosWorkerStore.java
index 061ed88..2671fc1 100644
--- a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/store/MesosWorkerStore.java
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/store/MesosWorkerStore.java
@@ -1,3 +1,21 @@
+/*
+ * 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.mesos.runtime.clusterframework.store;
 
 import org.apache.mesos.Protos;

http://git-wip-us.apache.org/repos/asf/flink/blob/67be9e07/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/store/StandaloneMesosWorkerStore.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/store/StandaloneMesosWorkerStore.java b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/store/StandaloneMesosWorkerStore.java
index 84b0ca1..9dcc6c0 100644
--- a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/store/StandaloneMesosWorkerStore.java
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/store/StandaloneMesosWorkerStore.java
@@ -1,3 +1,21 @@
+/*
+ * 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.mesos.runtime.clusterframework.store;
 
 import com.google.common.collect.ImmutableList;

http://git-wip-us.apache.org/repos/asf/flink/blob/67be9e07/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/store/ZooKeeperMesosWorkerStore.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/store/ZooKeeperMesosWorkerStore.java b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/store/ZooKeeperMesosWorkerStore.java
index fe4c74d..7c689b8 100644
--- a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/store/ZooKeeperMesosWorkerStore.java
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/store/ZooKeeperMesosWorkerStore.java
@@ -1,3 +1,21 @@
+/*
+ * 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.mesos.runtime.clusterframework.store;
 
 import org.apache.curator.framework.CuratorFramework;

http://git-wip-us.apache.org/repos/asf/flink/blob/67be9e07/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/LaunchableTask.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/LaunchableTask.java b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/LaunchableTask.java
index 584cccb..203708b 100644
--- a/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/LaunchableTask.java
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/LaunchableTask.java
@@ -1,3 +1,21 @@
+/*
+ * 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.mesos.scheduler;
 
 import com.netflix.fenzo.TaskAssignmentResult;

http://git-wip-us.apache.org/repos/asf/flink/blob/67be9e07/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/SchedulerProxy.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/SchedulerProxy.java b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/SchedulerProxy.java
index be07358..3411771 100644
--- a/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/SchedulerProxy.java
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/SchedulerProxy.java
@@ -1,3 +1,21 @@
+/*
+ * 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.mesos.scheduler;
 
 import akka.actor.ActorRef;

http://git-wip-us.apache.org/repos/asf/flink/blob/67be9e07/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/TaskSchedulerBuilder.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/TaskSchedulerBuilder.java b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/TaskSchedulerBuilder.java
index a4984de..80e572d 100644
--- a/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/TaskSchedulerBuilder.java
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/TaskSchedulerBuilder.java
@@ -1,3 +1,21 @@
+/*
+ * 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.mesos.scheduler;
 
 import com.netflix.fenzo.TaskScheduler;

http://git-wip-us.apache.org/repos/asf/flink/blob/67be9e07/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/AcceptOffers.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/AcceptOffers.java b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/AcceptOffers.java
index b661474..0404f07 100644
--- a/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/AcceptOffers.java
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/AcceptOffers.java
@@ -1,3 +1,21 @@
+/*
+ * 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.mesos.scheduler.messages;
 
 import org.apache.mesos.Protos;

http://git-wip-us.apache.org/repos/asf/flink/blob/67be9e07/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/Connected.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/Connected.java b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/Connected.java
index bd3aebe..b51e761 100644
--- a/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/Connected.java
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/Connected.java
@@ -1,3 +1,21 @@
+/*
+ * 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.mesos.scheduler.messages;
 
 /**

http://git-wip-us.apache.org/repos/asf/flink/blob/67be9e07/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/Disconnected.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/Disconnected.java b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/Disconnected.java
index be91c5c..22664b7 100644
--- a/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/Disconnected.java
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/Disconnected.java
@@ -1,3 +1,21 @@
+/*
+ * 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.mesos.scheduler.messages;
 
 /**

http://git-wip-us.apache.org/repos/asf/flink/blob/67be9e07/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/Error.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/Error.java b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/Error.java
index 5e24bf8..83508c1 100644
--- a/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/Error.java
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/Error.java
@@ -1,3 +1,21 @@
+/*
+ * 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.mesos.scheduler.messages;
 
 /**

http://git-wip-us.apache.org/repos/asf/flink/blob/67be9e07/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/OfferRescinded.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/OfferRescinded.java b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/OfferRescinded.java
index 9fac86b..6ae74c4 100644
--- a/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/OfferRescinded.java
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/OfferRescinded.java
@@ -1,3 +1,21 @@
+/*
+ * 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.mesos.scheduler.messages;
 
 import org.apache.mesos.Protos;

http://git-wip-us.apache.org/repos/asf/flink/blob/67be9e07/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/ReRegistered.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/ReRegistered.java b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/ReRegistered.java
index d4f1f76..116fc89 100644
--- a/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/ReRegistered.java
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/ReRegistered.java
@@ -1,3 +1,21 @@
+/*
+ * 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.mesos.scheduler.messages;
 
 import org.apache.mesos.Protos;

http://git-wip-us.apache.org/repos/asf/flink/blob/67be9e07/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/Registered.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/Registered.java b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/Registered.java
index 78ac880..c2ce214 100644
--- a/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/Registered.java
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/Registered.java
@@ -1,3 +1,21 @@
+/*
+ * 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.mesos.scheduler.messages;
 
 import org.apache.mesos.Protos;

http://git-wip-us.apache.org/repos/asf/flink/blob/67be9e07/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/ResourceOffers.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/ResourceOffers.java b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/ResourceOffers.java
index 595ab83..f9d21ea 100644
--- a/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/ResourceOffers.java
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/ResourceOffers.java
@@ -1,3 +1,21 @@
+/*
+ * 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.mesos.scheduler.messages;
 
 import org.apache.mesos.Protos;

http://git-wip-us.apache.org/repos/asf/flink/blob/67be9e07/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/SlaveLost.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/SlaveLost.java b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/SlaveLost.java
index 4ec9c0f..9255e87 100644
--- a/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/SlaveLost.java
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/SlaveLost.java
@@ -1,3 +1,21 @@
+/*
+ * 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.mesos.scheduler.messages;
 
 import org.apache.mesos.Protos;

http://git-wip-us.apache.org/repos/asf/flink/blob/67be9e07/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/StatusUpdate.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/StatusUpdate.java b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/StatusUpdate.java
index e64cac3..18c9a00 100644
--- a/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/StatusUpdate.java
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/StatusUpdate.java
@@ -1,3 +1,21 @@
+/*
+ * 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.mesos.scheduler.messages;
 
 import org.apache.mesos.Protos;

http://git-wip-us.apache.org/repos/asf/flink/blob/67be9e07/flink-mesos/src/main/java/org/apache/flink/mesos/util/MesosArtifactServer.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/util/MesosArtifactServer.java b/flink-mesos/src/main/java/org/apache/flink/mesos/util/MesosArtifactServer.java
index 96365c9..63bb7b1 100644
--- a/flink-mesos/src/main/java/org/apache/flink/mesos/util/MesosArtifactServer.java
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/util/MesosArtifactServer.java
@@ -1,3 +1,21 @@
+/*
+ * 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.mesos.util;
 
 import io.netty.bootstrap.ServerBootstrap;

http://git-wip-us.apache.org/repos/asf/flink/blob/67be9e07/flink-mesos/src/main/java/org/apache/flink/mesos/util/MesosConfiguration.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/util/MesosConfiguration.java b/flink-mesos/src/main/java/org/apache/flink/mesos/util/MesosConfiguration.java
index 46a6c0b..7961995 100644
--- a/flink-mesos/src/main/java/org/apache/flink/mesos/util/MesosConfiguration.java
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/util/MesosConfiguration.java
@@ -1,3 +1,21 @@
+/*
+ * 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.mesos.util;
 
 import org.apache.mesos.MesosSchedulerDriver;
@@ -9,6 +27,9 @@ import scala.Option;
 
 import java.util.Map;
 
+/**
+ * The typed configuration settings associated with a Mesos scheduler.
+ */
 public class MesosConfiguration {
 
 	private String masterUrl;
@@ -27,14 +48,33 @@ public class MesosConfiguration {
 		this.credential = credential;
 	}
 
+	/**
+	 * The Mesos connection string.
+	 *
+	 * The value should be in one of the following forms:
+	 * <pre>
+	 * {@code
+	 *     host:port
+	 *     zk://host1:port1,host2:port2,.../path
+	 *     zk://username:password@host1:port1,host2:port2,.../path
+	 *     file:///path/to/file (where file contains one of the above)
+	 * }
+	 * </pre>
+     */
 	public String masterUrl() {
 		return masterUrl;
 	}
 
+	/**
+	 * The framework registration info.
+     */
 	public Protos.FrameworkInfo.Builder frameworkInfo() {
 		return frameworkInfo;
 	}
 
+	/**
+	 * The credential to authenticate the framework principal.
+     */
 	public Option<Protos.Credential.Builder> credential() {
 		return credential;
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/67be9e07/flink-mesos/src/main/java/org/apache/flink/mesos/util/ZooKeeperUtils.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/util/ZooKeeperUtils.java b/flink-mesos/src/main/java/org/apache/flink/mesos/util/ZooKeeperUtils.java
index 1cad8e7..ced4a11 100644
--- a/flink-mesos/src/main/java/org/apache/flink/mesos/util/ZooKeeperUtils.java
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/util/ZooKeeperUtils.java
@@ -1,3 +1,21 @@
+/*
+ * 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.mesos.util;
 
 import org.apache.curator.framework.CuratorFramework;

http://git-wip-us.apache.org/repos/asf/flink/blob/67be9e07/flink-mesos/src/main/scala/org/apache/flink/mesos/runtime/clusterframework/MesosJobManager.scala
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/scala/org/apache/flink/mesos/runtime/clusterframework/MesosJobManager.scala b/flink-mesos/src/main/scala/org/apache/flink/mesos/runtime/clusterframework/MesosJobManager.scala
index f789163..3513210 100644
--- a/flink-mesos/src/main/scala/org/apache/flink/mesos/runtime/clusterframework/MesosJobManager.scala
+++ b/flink-mesos/src/main/scala/org/apache/flink/mesos/runtime/clusterframework/MesosJobManager.scala
@@ -1,3 +1,21 @@
+/*
+ * 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.mesos.runtime.clusterframework
 
 import java.util.concurrent.ExecutorService

http://git-wip-us.apache.org/repos/asf/flink/blob/67be9e07/flink-mesos/src/main/scala/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManager.scala
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/scala/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManager.scala b/flink-mesos/src/main/scala/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManager.scala
index 304e66d..3e1054c 100644
--- a/flink-mesos/src/main/scala/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManager.scala
+++ b/flink-mesos/src/main/scala/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManager.scala
@@ -1,3 +1,21 @@
+/*
+ * 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.mesos.runtime.clusterframework
 
 import org.apache.flink.runtime.clusterframework.types.ResourceID

http://git-wip-us.apache.org/repos/asf/flink/blob/67be9e07/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/ConnectionMonitor.scala
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/ConnectionMonitor.scala b/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/ConnectionMonitor.scala
index 91cc66c..ffd65a8 100644
--- a/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/ConnectionMonitor.scala
+++ b/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/ConnectionMonitor.scala
@@ -1,3 +1,21 @@
+/*
+ * 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.mesos.scheduler
 
 import akka.actor.{Actor, FSM, Props}

http://git-wip-us.apache.org/repos/asf/flink/blob/67be9e07/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/LaunchCoordinator.scala
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/LaunchCoordinator.scala b/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/LaunchCoordinator.scala
index a908ef9..bee25d3 100644
--- a/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/LaunchCoordinator.scala
+++ b/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/LaunchCoordinator.scala
@@ -1,3 +1,21 @@
+/*
+ * 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.mesos.scheduler
 
 import akka.actor.{Actor, ActorRef, FSM, Props}

http://git-wip-us.apache.org/repos/asf/flink/blob/67be9e07/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/ReconciliationCoordinator.scala
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/ReconciliationCoordinator.scala b/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/ReconciliationCoordinator.scala
index f3edf40..239e040 100644
--- a/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/ReconciliationCoordinator.scala
+++ b/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/ReconciliationCoordinator.scala
@@ -1,3 +1,21 @@
+/*
+ * 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.mesos.scheduler
 
 import java.util.concurrent.ThreadLocalRandom

http://git-wip-us.apache.org/repos/asf/flink/blob/67be9e07/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/TaskMonitor.scala
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/TaskMonitor.scala b/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/TaskMonitor.scala
index 43b6260..ec79585 100644
--- a/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/TaskMonitor.scala
+++ b/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/TaskMonitor.scala
@@ -1,3 +1,21 @@
+/*
+ * 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.mesos.scheduler
 
 import grizzled.slf4j.Logger

http://git-wip-us.apache.org/repos/asf/flink/blob/67be9e07/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/Tasks.scala
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/Tasks.scala b/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/Tasks.scala
index 1ebe0b2..14147a5 100644
--- a/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/Tasks.scala
+++ b/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/Tasks.scala
@@ -1,3 +1,21 @@
+/*
+ * 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.mesos.scheduler
 
 import akka.actor.{Actor, ActorRef, Props}

http://git-wip-us.apache.org/repos/asf/flink/blob/67be9e07/flink-mesos/src/test/java/org/apache/flink/mesos/runtime/clusterframework/MesosFlinkResourceManagerTest.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/test/java/org/apache/flink/mesos/runtime/clusterframework/MesosFlinkResourceManagerTest.java b/flink-mesos/src/test/java/org/apache/flink/mesos/runtime/clusterframework/MesosFlinkResourceManagerTest.java
index 12159f8..05fb033 100644
--- a/flink-mesos/src/test/java/org/apache/flink/mesos/runtime/clusterframework/MesosFlinkResourceManagerTest.java
+++ b/flink-mesos/src/test/java/org/apache/flink/mesos/runtime/clusterframework/MesosFlinkResourceManagerTest.java
@@ -1,3 +1,21 @@
+/*
+ * 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.mesos.runtime.clusterframework;
 
 

http://git-wip-us.apache.org/repos/asf/flink/blob/67be9e07/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 18ff0b6..3ab72cd 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
@@ -1,3 +1,21 @@
+/*
+ * 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.mesos
 
 import java.util.concurrent.atomic.AtomicLong

http://git-wip-us.apache.org/repos/asf/flink/blob/67be9e07/flink-mesos/src/test/scala/org/apache/flink/mesos/scheduler/LaunchCoordinatorTest.scala
----------------------------------------------------------------------
diff --git a/flink-mesos/src/test/scala/org/apache/flink/mesos/scheduler/LaunchCoordinatorTest.scala b/flink-mesos/src/test/scala/org/apache/flink/mesos/scheduler/LaunchCoordinatorTest.scala
index 3b2bee6..be7d788 100644
--- a/flink-mesos/src/test/scala/org/apache/flink/mesos/scheduler/LaunchCoordinatorTest.scala
+++ b/flink-mesos/src/test/scala/org/apache/flink/mesos/scheduler/LaunchCoordinatorTest.scala
@@ -1,3 +1,21 @@
+/*
+ * 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.mesos.scheduler
 
 import java.util.{Collections, UUID}

http://git-wip-us.apache.org/repos/asf/flink/blob/67be9e07/flink-mesos/src/test/scala/org/apache/flink/mesos/scheduler/ReconciliationCoordinatorTest.scala
----------------------------------------------------------------------
diff --git a/flink-mesos/src/test/scala/org/apache/flink/mesos/scheduler/ReconciliationCoordinatorTest.scala b/flink-mesos/src/test/scala/org/apache/flink/mesos/scheduler/ReconciliationCoordinatorTest.scala
index ee13316..fb64115 100644
--- a/flink-mesos/src/test/scala/org/apache/flink/mesos/scheduler/ReconciliationCoordinatorTest.scala
+++ b/flink-mesos/src/test/scala/org/apache/flink/mesos/scheduler/ReconciliationCoordinatorTest.scala
@@ -1,3 +1,21 @@
+/*
+ * 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.mesos.scheduler
 
 import java.util.UUID

http://git-wip-us.apache.org/repos/asf/flink/blob/67be9e07/flink-mesos/src/test/scala/org/apache/flink/mesos/scheduler/TaskMonitorTest.scala
----------------------------------------------------------------------
diff --git a/flink-mesos/src/test/scala/org/apache/flink/mesos/scheduler/TaskMonitorTest.scala b/flink-mesos/src/test/scala/org/apache/flink/mesos/scheduler/TaskMonitorTest.scala
index 6bd7178..ff32116 100644
--- a/flink-mesos/src/test/scala/org/apache/flink/mesos/scheduler/TaskMonitorTest.scala
+++ b/flink-mesos/src/test/scala/org/apache/flink/mesos/scheduler/TaskMonitorTest.scala
@@ -1,3 +1,21 @@
+/*
+ * 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.mesos.scheduler
 
 import java.util.UUID

http://git-wip-us.apache.org/repos/asf/flink/blob/67be9e07/flink-mesos/src/test/scala/org/apache/flink/runtime/akka/FSMSpec.scala
----------------------------------------------------------------------
diff --git a/flink-mesos/src/test/scala/org/apache/flink/runtime/akka/FSMSpec.scala b/flink-mesos/src/test/scala/org/apache/flink/runtime/akka/FSMSpec.scala
index b33e9aa..3252f94 100644
--- a/flink-mesos/src/test/scala/org/apache/flink/runtime/akka/FSMSpec.scala
+++ b/flink-mesos/src/test/scala/org/apache/flink/runtime/akka/FSMSpec.scala
@@ -1,3 +1,21 @@
+/*
+ * 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.akka
 
 import java.util


[2/9] flink git commit: [FLINK-1984] Mesos ResourceManager - T1 milestone

Posted by mx...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/d9b2be05/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/TaskMonitor.scala
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/TaskMonitor.scala b/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/TaskMonitor.scala
new file mode 100644
index 0000000..43b6260
--- /dev/null
+++ b/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/TaskMonitor.scala
@@ -0,0 +1,240 @@
+package org.apache.flink.mesos.scheduler
+
+import grizzled.slf4j.Logger
+
+import akka.actor.{Actor, FSM, Props}
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.mesos.scheduler.ReconciliationCoordinator.Reconcile
+import org.apache.flink.mesos.scheduler.TaskMonitor._
+import org.apache.flink.mesos.scheduler.messages.{Connected, Disconnected, StatusUpdate}
+import org.apache.mesos.Protos.TaskState._
+import org.apache.mesos.{SchedulerDriver, Protos}
+
+import scala.PartialFunction.empty
+import scala.concurrent.duration._
+
+/**
+  * Monitors a Mesos task throughout its lifecycle.
+  *
+  * Models a task with a state machine reflecting the perceived state of the task in Mesos.   The state
+  * is primarily updated when task status information arrives from Mesos.
+  *
+  * The associated state data primarily tracks the task's goal (intended) state, as persisted by the scheduler.
+  * Keep in mind that goal state is persisted before actions are taken.    The goal state strictly transitions
+  * thru New->Launched->Released.
+  *
+  * Unlike most exchanges with Mesos, task status is delivered at-least-once, so status handling should be idempotent.
+  */
+class TaskMonitor(
+    flinkConfig: Configuration,
+    schedulerDriver: SchedulerDriver,
+    goalState: TaskGoalState) extends Actor with FSM[TaskMonitorState,StateData] {
+
+  val LOG = Logger(getClass)
+
+  startWith(Suspended, StateData(goalState))
+
+  // ------------------------------------------------------------------------
+  //  Suspended State
+  // ------------------------------------------------------------------------
+
+  when(Suspended) {
+    case Event(update: TaskGoalStateUpdated, _) =>
+      stay() using StateData(update.state)
+    case Event(msg: StatusUpdate, _) =>
+      stay()
+    case Event(msg: Connected, StateData(goal: New)) =>
+      goto(New)
+    case Event(msg: Connected, StateData(goal: Launched)) =>
+      goto(Reconciling)
+    case Event(msg: Connected, StateData(goal: Released)) =>
+      goto(Killing)
+  }
+
+  // ------------------------------------------------------------------------
+  //  New State
+  // ------------------------------------------------------------------------
+
+  when(New) {
+    case Event(TaskGoalStateUpdated(goal: Launched), _) =>
+      goto(Staging) using StateData(goal)
+  }
+
+  // ------------------------------------------------------------------------
+  //  Reconciliation State
+  // ------------------------------------------------------------------------
+
+  onTransition {
+    case _ -> Reconciling =>
+      nextStateData.goal match {
+        case goal: Launched =>
+          val taskStatus = Protos.TaskStatus.newBuilder()
+            .setTaskId(goal.taskID).setSlaveId(goal.slaveID).setState(TASK_STAGING).build()
+          context.parent ! Reconcile(Seq(taskStatus))
+        case _ =>
+      }
+  }
+
+  when(Reconciling) {
+    empty
+  }
+
+  // ------------------------------------------------------------------------
+  //  Staging State
+  // ------------------------------------------------------------------------
+
+  when(Staging, stateTimeout = LAUNCH_TIMEOUT) {
+    case Event(StateTimeout, _) =>
+      LOG.warn(s"Mesos task ${stateData.goal.taskID.getValue} didn't launch as expected; reconciling.")
+
+      // likely cause: the task launch message was dropped - docs suggest reconciliation
+      goto(Reconciling)
+  }
+
+  // ------------------------------------------------------------------------
+  //  Running State
+  // ------------------------------------------------------------------------
+
+  when(Running) {
+    empty
+  }
+
+  // ------------------------------------------------------------------------
+  //  Killing State
+  // ------------------------------------------------------------------------
+
+  onTransition {
+    case _ -> Killing =>
+      schedulerDriver.killTask(nextStateData.goal.taskID)
+  }
+
+  when(Killing, stateTimeout = RETRY_INTERVAL) {
+
+    case Event(TaskGoalStateUpdated(goal: Released), _) =>
+      stay() using StateData(goal)
+
+    case Event(StateTimeout, _) =>
+      // retry kill command
+      LOG.info(s"Re-attempting to kill Mesos task ${stateData.goal.taskID.getValue}.")
+      schedulerDriver.killTask(stateData.goal.taskID)
+      stay()
+  }
+
+  // ------------------------------------------------------------------------
+  //  Common Handling
+  // ------------------------------------------------------------------------
+
+  whenUnhandled {
+
+    case Event(msg: Disconnected, _) =>
+      goto(Suspended)
+
+    case Event(TaskGoalStateUpdated(goal: Released), _) =>
+      goto(Killing) using StateData(goal)
+
+    case Event(msg: StatusUpdate, StateData(goal: Launched)) =>
+      LOG.debug(s"Status update received for Mesos task ${goal.taskID.getValue}: $msg")
+      msg.status().getState match {
+        case TASK_STAGING | TASK_STARTING => goto(Staging)
+        case TASK_RUNNING =>
+          LOG.info(s"Mesos task ${goal.taskID.getValue} is running.")
+          goto(Running)
+        case TASK_FINISHED | TASK_LOST | TASK_FAILED | TASK_KILLED | TASK_ERROR =>
+          LOG.warn(s"Mesos task ${goal.taskID.getValue} failed unexpectedly.")
+          context.parent ! TaskTerminated(goal.taskID, msg.status())
+          stop()
+      }
+
+    case Event(msg: StatusUpdate, StateData(goal: Released)) =>
+      LOG.debug(s"Status update received for Mesos task ${goal.taskID.getValue}: $msg")
+      msg.status().getState match {
+        case TASK_STAGING | TASK_STARTING | TASK_RUNNING =>
+          LOG.info(s"Mesos task ${goal.taskID.getValue} is running unexpectedly; killing.")
+          goto(Killing)
+        case TASK_FINISHED | TASK_LOST | TASK_FAILED | TASK_KILLED | TASK_ERROR =>
+          LOG.info(s"Mesos task ${goal.taskID.getValue} exited as planned.")
+          context.parent ! TaskTerminated(goal.taskID, msg.status())
+          stop()
+      }
+  }
+
+  onTransition {
+    case previousState -> nextState =>
+      LOG.debug(s"State change ($previousState -> $nextState) with data ${nextStateData}")
+  }
+
+  initialize()
+}
+
+object TaskMonitor {
+
+  val RETRY_INTERVAL = (5 seconds)
+  val LAUNCH_TIMEOUT = (30 seconds)
+
+  // ------------------------------------------------------------------------
+  // State
+  // ------------------------------------------------------------------------
+
+  /**
+    * An FSM state of the task monitor, roughly corresponding to the task status.
+    */
+  sealed trait TaskMonitorState
+  case object Suspended extends TaskMonitorState
+  case object New extends TaskMonitorState
+  case object Reconciling extends TaskMonitorState
+  case object Staging extends TaskMonitorState
+  case object Running extends TaskMonitorState
+  case object Killing extends TaskMonitorState
+
+  /**
+    * The task monitor state data.
+    * @param goal the goal (intentional) state of the task.
+    */
+  case class StateData(goal:TaskGoalState)
+
+  /**
+    * Indicates the goal (intentional) state of a Mesos task; behavior varies accordingly.
+    */
+  sealed trait TaskGoalState {
+    val taskID: Protos.TaskID
+  }
+  case class New(taskID: Protos.TaskID) extends TaskGoalState
+  case class Launched(taskID: Protos.TaskID, slaveID: Protos.SlaveID) extends TaskGoalState
+  case class Released(taskID: Protos.TaskID, slaveID: Protos.SlaveID) extends TaskGoalState
+
+
+  // ------------------------------------------------------------------------
+  //  Messages
+  // ------------------------------------------------------------------------
+
+  /**
+    * Conveys an update to the goal (intentional) state of a given task.
+    */
+  case class TaskGoalStateUpdated(state: TaskGoalState)
+
+  /**
+    * Indicates that the Mesos task has terminated for whatever reason.
+    */
+  case class TaskTerminated(taskID: Protos.TaskID, status: Protos.TaskStatus)
+
+  // ------------------------------------------------------------------------
+  //  Utils
+  // ------------------------------------------------------------------------
+
+  /**
+    * Creates the properties for the TaskMonitor actor.
+    *
+    * @param actorClass the task monitor actor class
+    * @param flinkConfig the Flink configuration
+    * @param schedulerDriver the Mesos scheduler driver
+    * @param goalState the task's goal state
+    * @tparam T the type of the task monitor actor class
+    * @return the Props to create the task monitor
+    */
+  def createActorProps[T <: TaskMonitor](actorClass: Class[T],
+                                         flinkConfig: Configuration,
+                                         schedulerDriver: SchedulerDriver,
+                                         goalState: TaskGoalState): Props = {
+    Props.create(actorClass, flinkConfig, schedulerDriver, goalState)
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/d9b2be05/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/Tasks.scala
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/Tasks.scala b/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/Tasks.scala
new file mode 100644
index 0000000..1ebe0b2
--- /dev/null
+++ b/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/Tasks.scala
@@ -0,0 +1,96 @@
+package org.apache.flink.mesos.scheduler
+
+import akka.actor.{Actor, ActorRef, Props}
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.mesos.scheduler.ReconciliationCoordinator.Reconcile
+import org.apache.flink.mesos.scheduler.TaskMonitor.{TaskGoalState, TaskGoalStateUpdated, TaskTerminated}
+import org.apache.flink.mesos.scheduler.Tasks._
+import org.apache.flink.mesos.scheduler.messages._
+import org.apache.mesos.{SchedulerDriver, Protos}
+
+import scala.collection.mutable.{Map => MutableMap}
+
+/**
+  * Aggregate of monitored tasks.
+  *
+  * Routes messages between the scheduler and individual task monitor actors.
+  */
+class Tasks[M <: TaskMonitor](
+     flinkConfig: Configuration,
+     schedulerDriver: SchedulerDriver,
+     taskMonitorClass: Class[M]) extends Actor {
+
+  /**
+    * A map of task monitors by task ID.
+    */
+  private val taskMap: MutableMap[Protos.TaskID,ActorRef] = MutableMap()
+
+  /**
+    * Cache of current connection state.
+    */
+  private var registered: Option[Any] = None
+
+  override def preStart(): Unit = {
+    // TODO subscribe to context.system.deadLetters for messages to nonexistent tasks
+  }
+
+  override def receive: Receive = {
+
+    case msg: Disconnected =>
+      registered = None
+      context.actorSelection("*").tell(msg, self)
+
+    case msg : Connected =>
+      registered = Some(msg)
+      context.actorSelection("*").tell(msg, self)
+
+    case msg: TaskGoalStateUpdated =>
+      val taskID = msg.state.taskID
+
+      // ensure task monitor exists
+      if(!taskMap.contains(taskID)) {
+        val actorRef = createTask(msg.state)
+        registered.foreach(actorRef ! _)
+      }
+
+      taskMap(taskID) ! msg
+
+    case msg: StatusUpdate =>
+      taskMap(msg.status().getTaskId) ! msg
+
+    case msg: Reconcile =>
+      context.parent.forward(msg)
+
+    case msg: TaskTerminated =>
+      context.parent.forward(msg)
+  }
+
+  private def createTask(task: TaskGoalState): ActorRef = {
+    val actorProps = TaskMonitor.createActorProps(taskMonitorClass, flinkConfig, schedulerDriver, task)
+    val actorRef = context.actorOf(actorProps, name = actorName(task.taskID))
+    taskMap.put(task.taskID, actorRef)
+    actorRef
+  }
+}
+
+object Tasks {
+
+  /**
+    * Extract the actor name for the given task ID.
+    */
+  def actorName(taskID: Protos.TaskID): String = {
+    taskID.getValue
+  }
+
+  /**
+    * Create a tasks actor.
+    */
+  def createActorProps[T <: Tasks[M], M <: TaskMonitor](
+      actorClass: Class[T],
+      flinkConfig: Configuration,
+      schedulerDriver: SchedulerDriver,
+      taskMonitorClass: Class[M]): Props = {
+
+    Props.create(actorClass, flinkConfig, schedulerDriver, taskMonitorClass)
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/d9b2be05/flink-mesos/src/main/scala/org/apache/flink/runtime/clusterframework/ContaineredJobManager.scala
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/scala/org/apache/flink/runtime/clusterframework/ContaineredJobManager.scala b/flink-mesos/src/main/scala/org/apache/flink/runtime/clusterframework/ContaineredJobManager.scala
new file mode 100644
index 0000000..49c86b5
--- /dev/null
+++ b/flink-mesos/src/main/scala/org/apache/flink/runtime/clusterframework/ContaineredJobManager.scala
@@ -0,0 +1,174 @@
+/*
+ * 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 java.util.concurrent.{TimeUnit, ExecutorService}
+
+import akka.actor.ActorRef
+
+import org.apache.flink.api.common.JobID
+import org.apache.flink.configuration.{Configuration => FlinkConfiguration, ConfigConstants}
+import org.apache.flink.runtime.checkpoint.savepoint.SavepointStore
+import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory
+import org.apache.flink.runtime.clusterframework.ApplicationStatus
+import org.apache.flink.runtime.executiongraph.restart.RestartStrategyFactory
+import org.apache.flink.runtime.clusterframework.messages._
+import org.apache.flink.runtime.jobgraph.JobStatus
+import org.apache.flink.runtime.jobmanager.{SubmittedJobGraphStore, JobManager}
+import org.apache.flink.runtime.leaderelection.LeaderElectionService
+import org.apache.flink.runtime.messages.JobManagerMessages.{RequestJobStatus, CurrentJobStatus, JobNotFound}
+import org.apache.flink.runtime.messages.Messages.Acknowledge
+import org.apache.flink.runtime.metrics.{MetricRegistry => FlinkMetricRegistry}
+import org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager
+import org.apache.flink.runtime.instance.InstanceManager
+import org.apache.flink.runtime.jobmanager.scheduler.{Scheduler => FlinkScheduler}
+
+import scala.concurrent.duration._
+import scala.language.postfixOps
+
+
+/** JobManager actor for execution on Yarn or Mesos. It enriches the [[JobManager]] with additional messages
+  * to start/administer/stop the session.
+  *
+  * @param flinkConfiguration Configuration object for the actor
+  * @param executorService Execution context which is used to execute concurrent tasks in the
+  *                         [[org.apache.flink.runtime.executiongraph.ExecutionGraph]]
+  * @param instanceManager Instance manager to manage the registered
+  *                        [[org.apache.flink.runtime.taskmanager.TaskManager]]
+  * @param scheduler Scheduler to schedule Flink jobs
+  * @param libraryCacheManager Manager to manage uploaded jar files
+  * @param archive Archive for finished Flink jobs
+  * @param restartStrategyFactory Restart strategy to be used in case of a job recovery
+  * @param timeout Timeout for futures
+  * @param leaderElectionService LeaderElectionService to participate in the leader election
+  */
+abstract class ContaineredJobManager(
+                      flinkConfiguration: FlinkConfiguration,
+                      executorService: ExecutorService,
+                      instanceManager: InstanceManager,
+                      scheduler: FlinkScheduler,
+                      libraryCacheManager: BlobLibraryCacheManager,
+                      archive: ActorRef,
+                      restartStrategyFactory: RestartStrategyFactory,
+                      timeout: FiniteDuration,
+                      leaderElectionService: LeaderElectionService,
+                      submittedJobGraphs : SubmittedJobGraphStore,
+                      checkpointRecoveryFactory : CheckpointRecoveryFactory,
+                      savepointStore: SavepointStore,
+                      jobRecoveryTimeout: FiniteDuration,
+                      metricsRegistry: Option[FlinkMetricRegistry])
+  extends JobManager(
+    flinkConfiguration,
+    executorService,
+    instanceManager,
+    scheduler,
+    libraryCacheManager,
+    archive,
+    restartStrategyFactory,
+    timeout,
+    leaderElectionService,
+    submittedJobGraphs,
+    checkpointRecoveryFactory,
+    savepointStore,
+    jobRecoveryTimeout,
+    metricsRegistry) {
+
+  val jobPollingInterval: FiniteDuration
+
+  // indicates if this JM has been started in a dedicated (per-job) mode.
+  var stopWhenJobFinished: JobID = null
+
+  override def handleMessage: Receive = {
+    handleContainerMessage orElse super.handleMessage
+  }
+
+  def handleContainerMessage: Receive = {
+
+    case msg @ (_: RegisterInfoMessageListener | _: UnRegisterInfoMessageListener) =>
+      // forward to ResourceManager
+      currentResourceManager match {
+        case Some(rm) =>
+          // we forward the message
+          rm.forward(decorateMessage(msg))
+        case None =>
+        // client has to try again
+      }
+
+    case msg: ShutdownClusterAfterJob =>
+      val jobId = msg.jobId()
+      log.info(s"ApplicationMaster will shut down session when job $jobId has finished.")
+      stopWhenJobFinished = jobId
+      // trigger regular job status messages (if this is a dedicated/per-job cluster)
+      if (stopWhenJobFinished != null) {
+        context.system.scheduler.schedule(0 seconds,
+          jobPollingInterval,
+          new Runnable {
+            override def run(): Unit = {
+              self ! decorateMessage(RequestJobStatus(stopWhenJobFinished))
+            }
+          }
+        )(context.dispatcher)
+      }
+
+      sender() ! decorateMessage(Acknowledge)
+
+    case msg: GetClusterStatus =>
+      sender() ! decorateMessage(
+        new GetClusterStatusResponse(
+          instanceManager.getNumberOfRegisteredTaskManagers,
+          instanceManager.getTotalNumberOfSlots)
+      )
+
+    case jnf: JobNotFound =>
+      log.debug(s"Job with ID ${jnf.jobID} not found in JobManager")
+      if (stopWhenJobFinished == null) {
+        log.warn("The ApplicationMaster didn't expect to receive this message")
+      }
+
+    case jobStatus: CurrentJobStatus =>
+      if (stopWhenJobFinished == null) {
+        log.warn(s"Received job status $jobStatus which wasn't requested.")
+      } else {
+        if (stopWhenJobFinished != jobStatus.jobID) {
+          log.warn(s"Received job status for job ${jobStatus.jobID} but expected status for " +
+            s"job $stopWhenJobFinished")
+        } else {
+          if (jobStatus.status.isTerminalState) {
+            log.info(s"Job with ID ${jobStatus.jobID} is in terminal state ${jobStatus.status}. " +
+              s"Shutting down session")
+            if (jobStatus.status == JobStatus.FINISHED) {
+              self ! decorateMessage(
+                new StopCluster(
+                  ApplicationStatus.SUCCEEDED,
+                  s"The monitored job with ID ${jobStatus.jobID} has finished.")
+              )
+            } else {
+              self ! decorateMessage(
+                new StopCluster(
+                  ApplicationStatus.FAILED,
+                  s"The monitored job with ID ${jobStatus.jobID} has failed to complete.")
+              )
+            }
+          } else {
+            log.debug(s"Monitored job with ID ${jobStatus.jobID} is in state ${jobStatus.status}")
+          }
+        }
+      }
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/d9b2be05/flink-mesos/src/test/java/org/apache/flink/mesos/runtime/clusterframework/MesosFlinkResourceManagerTest.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/test/java/org/apache/flink/mesos/runtime/clusterframework/MesosFlinkResourceManagerTest.java b/flink-mesos/src/test/java/org/apache/flink/mesos/runtime/clusterframework/MesosFlinkResourceManagerTest.java
new file mode 100644
index 0000000..12159f8
--- /dev/null
+++ b/flink-mesos/src/test/java/org/apache/flink/mesos/runtime/clusterframework/MesosFlinkResourceManagerTest.java
@@ -0,0 +1,697 @@
+package org.apache.flink.mesos.runtime.clusterframework;
+
+
+import akka.actor.ActorRef;
+import akka.actor.ActorSystem;
+import akka.testkit.JavaTestKit;
+import akka.testkit.TestActorRef;
+import akka.testkit.TestProbe;
+import junit.framework.AssertionFailedError;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.mesos.runtime.clusterframework.store.MesosWorkerStore;
+import org.apache.flink.mesos.scheduler.ConnectionMonitor;
+import org.apache.flink.mesos.scheduler.LaunchCoordinator;
+import org.apache.flink.mesos.scheduler.TaskMonitor;
+import org.apache.flink.mesos.scheduler.messages.*;
+import org.apache.flink.mesos.scheduler.messages.Error;
+import org.apache.flink.mesos.util.MesosConfiguration;
+import org.apache.flink.runtime.akka.AkkaUtils;
+import org.apache.flink.runtime.clusterframework.ApplicationStatus;
+import org.apache.flink.runtime.clusterframework.ContaineredTaskManagerParameters;
+import org.apache.flink.runtime.clusterframework.messages.*;
+import org.apache.flink.runtime.clusterframework.types.ResourceID;
+import org.apache.flink.runtime.instance.ActorGateway;
+import org.apache.flink.runtime.instance.AkkaActorGateway;
+import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.runtime.util.LeaderRetrievalUtils;
+import org.apache.mesos.SchedulerDriver;
+import org.apache.mesos.Protos;
+import org.apache.mesos.Scheduler;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Option;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import static java.util.Collections.singletonList;
+import java.util.HashMap;
+
+import static org.apache.flink.mesos.runtime.clusterframework.MesosFlinkResourceManager.extractGoalState;
+import static org.apache.flink.mesos.runtime.clusterframework.MesosFlinkResourceManager.extractResourceID;
+import static org.hamcrest.Matchers.*;
+import static org.junit.Assert.assertThat;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyBoolean;
+import static org.mockito.Mockito.*;
+
+/**
+ * General tests for the Mesos resource manager component.
+ */
+public class MesosFlinkResourceManagerTest {
+
+	private static final Logger LOG = LoggerFactory.getLogger(MesosFlinkResourceManagerTest.class);
+
+	private static ActorSystem system;
+
+	private static Configuration config = new Configuration() {{
+		setInteger(ConfigConstants.MESOS_MAX_FAILED_TASKS, -1);
+	}};
+
+	@BeforeClass
+	public static void setup() {
+		system = AkkaUtils.createLocalActorSystem(config);
+	}
+
+	@AfterClass
+	public static void teardown() {
+		JavaTestKit.shutdownActorSystem(system);
+	}
+
+	/**
+	 * The RM with some test-specific behavior.
+	 */
+	static class TestingMesosFlinkResourceManager extends MesosFlinkResourceManager {
+
+		public TestProbe connectionMonitor = new TestProbe(system);
+		public TestProbe taskRouter = new TestProbe(system);
+		public TestProbe launchCoordinator = new TestProbe(system);
+		public TestProbe reconciliationCoordinator = new TestProbe(system);
+
+		public TestingMesosFlinkResourceManager(
+			Configuration flinkConfig,
+			MesosConfiguration mesosConfig,
+			MesosWorkerStore workerStore,
+			LeaderRetrievalService leaderRetrievalService,
+			MesosTaskManagerParameters taskManagerParameters,
+			Protos.TaskInfo.Builder taskManagerLaunchContext,
+			int maxFailedTasks,
+			int numInitialTaskManagers) {
+
+			super(flinkConfig, mesosConfig, workerStore, leaderRetrievalService, taskManagerParameters,
+				taskManagerLaunchContext, maxFailedTasks, numInitialTaskManagers);
+		}
+
+		@Override
+		protected ActorRef createConnectionMonitor() { return connectionMonitor.ref(); }
+		@Override
+		protected ActorRef createTaskRouter() { return taskRouter.ref(); }
+		@Override
+		protected ActorRef createLaunchCoordinator() { return launchCoordinator.ref(); }
+		@Override
+		protected ActorRef createReconciliationCoordinator() { return reconciliationCoordinator.ref(); }
+
+		@Override
+		protected void fatalError(String message, Throwable error) {
+			// override the super's behavior of exiting the process
+			context().stop(self());
+		}
+	}
+
+	/**
+	 * The context fixture.
+	 */
+	static class Context extends JavaTestKit {
+
+		// mocks
+		public ActorGateway jobManager;
+		public LeaderRetrievalService retrievalService;
+		public MesosConfiguration mesosConfig;
+		public MesosWorkerStore workerStore;
+		public SchedulerDriver schedulerDriver;
+		public TestingMesosFlinkResourceManager resourceManagerInstance;
+		public ActorGateway resourceManager;
+
+		// domain objects for test purposes
+		Protos.FrameworkID framework1 = Protos.FrameworkID.newBuilder().setValue("framework1").build();
+		public Protos.SlaveID slave1 = Protos.SlaveID.newBuilder().setValue("slave1").build();
+		public String slave1host = "localhost";
+		public Protos.OfferID offer1 = Protos.OfferID.newBuilder().setValue("offer1").build();
+		public Protos.TaskID task1 = Protos.TaskID.newBuilder().setValue("taskmanager-00001").build();
+		public Protos.TaskID task2 = Protos.TaskID.newBuilder().setValue("taskmanager-00002").build();
+		public Protos.TaskID task3 = Protos.TaskID.newBuilder().setValue("taskmanager-00003").build();
+
+		/**
+		 * Create mock RM dependencies.
+		 */
+		public Context() {
+			super(system);
+
+			try {
+				jobManager = TestingUtils.createForwardingActor(system, getTestActor(), Option.<String>empty());
+				retrievalService = LeaderRetrievalUtils.createLeaderRetrievalService(config, jobManager.actor());
+
+				// scheduler driver
+				schedulerDriver = mock(SchedulerDriver.class);
+
+				// config
+				mesosConfig = mock(MesosConfiguration.class);
+				when(mesosConfig.frameworkInfo()).thenReturn(Protos.FrameworkInfo.newBuilder());
+				when(mesosConfig.withFrameworkInfo(any(Protos.FrameworkInfo.Builder.class))).thenReturn(mesosConfig);
+				when(mesosConfig.createDriver(any(Scheduler.class), anyBoolean())).thenReturn(schedulerDriver);
+
+				// worker store
+				workerStore = mock(MesosWorkerStore.class);
+				when(workerStore.getFrameworkID()).thenReturn(Option.<Protos.FrameworkID>empty());
+			} catch (Exception ex) {
+				throw new RuntimeException(ex);
+			}
+		}
+
+		/**
+		 * Initialize the resource manager.
+		 */
+		public void initialize() {
+			ContaineredTaskManagerParameters containeredParams =
+				new ContaineredTaskManagerParameters(1024, 768, 256, 4, new HashMap<String, String>());
+			MesosTaskManagerParameters tmParams = new MesosTaskManagerParameters(1.0, containeredParams);
+			Protos.TaskInfo.Builder taskInfo = Protos.TaskInfo.newBuilder();
+
+			TestActorRef<TestingMesosFlinkResourceManager> resourceManagerRef =
+				TestActorRef.create(system, MesosFlinkResourceManager.createActorProps(
+					TestingMesosFlinkResourceManager.class,
+					config, mesosConfig, workerStore, retrievalService, tmParams, taskInfo, 0, LOG));
+			resourceManagerInstance = resourceManagerRef.underlyingActor();
+			resourceManager = new AkkaActorGateway(resourceManagerRef, null);
+
+			verify(schedulerDriver).start();
+			resourceManagerInstance.connectionMonitor.expectMsgClass(ConnectionMonitor.Start.class);
+		}
+
+		/**
+		 * Send a RegisterResourceManagerSuccessful message to the RM.
+		 * @param currentlyRegisteredTaskManagers the already-registered workers.
+         */
+		public void register(Collection<ResourceID> currentlyRegisteredTaskManagers) {
+			// register with JM
+			expectMsgClass(RegisterResourceManager.class);
+			resourceManager.tell(
+				new RegisterResourceManagerSuccessful(jobManager.actor(), currentlyRegisteredTaskManagers),
+				jobManager);
+		}
+
+		/**
+		 * Prepares a launch operation.
+         */
+		public Protos.Offer.Operation launch(Protos.TaskInfo... taskInfo) {
+			return Protos.Offer.Operation.newBuilder()
+				.setType(Protos.Offer.Operation.Type.LAUNCH)
+				.setLaunch(Protos.Offer.Operation.Launch.newBuilder().addAllTaskInfos(Arrays.asList(taskInfo))
+				).build();
+		}
+	}
+
+	/**
+	 * Test recovery of persistent workers.
+	 */
+	@Test
+	public void testRecoverWorkers() {
+		new Context() {{
+			new Within(duration("10 seconds")) {
+				@Override
+				protected void run() {
+					try {
+						// set the initial persistent state then initialize the RM
+						MesosWorkerStore.Worker worker1 = MesosWorkerStore.Worker.newTask(task1);
+						MesosWorkerStore.Worker worker2 = MesosWorkerStore.Worker.newTask(task2).launchTask(slave1, slave1host);
+						MesosWorkerStore.Worker worker3 = MesosWorkerStore.Worker.newTask(task3).launchTask(slave1, slave1host).releaseTask();
+						when(workerStore.getFrameworkID()).thenReturn(Option.apply(framework1));
+						when(workerStore.recoverWorkers()).thenReturn(Arrays.asList(worker1, worker2, worker3));
+						initialize();
+
+						// verify that the internal state was updated, the task router was notified,
+						// and the launch coordinator was asked to launch a task
+						assertThat(resourceManagerInstance.workersInNew, hasEntry(extractResourceID(task1), worker1));
+						assertThat(resourceManagerInstance.workersInLaunch, hasEntry(extractResourceID(task2), worker2));
+						assertThat(resourceManagerInstance.workersBeingReturned, hasEntry(extractResourceID(task3), worker3));
+						resourceManagerInstance.taskRouter.expectMsgClass(TaskMonitor.TaskGoalStateUpdated.class);
+						LaunchCoordinator.Assign actualAssign =
+							resourceManagerInstance.launchCoordinator.expectMsgClass(LaunchCoordinator.Assign.class);
+						assertThat(actualAssign.tasks(), hasSize(1));
+						assertThat(actualAssign.tasks().get(0).f0.getId(), equalTo(task2.getValue()));
+						assertThat(actualAssign.tasks().get(0).f1, equalTo(slave1host));
+						resourceManagerInstance.launchCoordinator.expectMsgClass(LaunchCoordinator.Launch.class);
+
+						register(Collections.<ResourceID>emptyList());
+					}
+					catch(Exception ex) {
+						throw new RuntimeException(ex);
+					}
+				}
+			};
+		}};
+	}
+
+	/**
+	 * Test re-acceptance of registered workers upon JM registration.
+	 */
+	@Test
+	public void testReacceptRegisteredWorkers() {
+		new Context() {{
+			new Within(duration("10 seconds")) {
+				@Override
+				protected void run() {
+					try {
+						// set the initial persistent state then initialize the RM
+						MesosWorkerStore.Worker worker1launched = MesosWorkerStore.Worker.newTask(task1).launchTask(slave1, slave1host);
+						when(workerStore.getFrameworkID()).thenReturn(Option.apply(framework1));
+						when(workerStore.recoverWorkers()).thenReturn(singletonList(worker1launched));
+						initialize();
+
+						// send RegisterResourceManagerSuccessful to the RM with some 'known' workers.
+						// This will cause the RM to reaccept the workers.
+						assertThat(resourceManagerInstance.workersInLaunch, hasEntry(extractResourceID(task1), worker1launched));
+						register(singletonList(extractResourceID(task1)));
+						assertThat(resourceManagerInstance.workersInLaunch.entrySet(), empty());
+					}
+					catch(Exception ex) {
+						throw new RuntimeException(ex);
+					}
+				}
+			};
+		}};
+	}
+
+	/**
+	 * Test normal worker registration.
+	 */
+	@Test
+	public void testWorkerRegistered() {
+		new Context() {{
+			new Within(duration("10 seconds")) {
+				@Override
+				protected void run() {
+					try {
+						// set the initial state with a (recovered) launched worker
+						MesosWorkerStore.Worker worker1launched = MesosWorkerStore.Worker.newTask(task1).launchTask(slave1, slave1host);
+						when(workerStore.getFrameworkID()).thenReturn(Option.apply(framework1));
+						when(workerStore.recoverWorkers()).thenReturn(singletonList(worker1launched));
+						initialize();
+						assertThat(resourceManagerInstance.workersInLaunch, hasEntry(extractResourceID(task1), worker1launched));
+						register(Collections.<ResourceID>emptyList());
+
+						// send registration message
+						NotifyResourceStarted msg = new NotifyResourceStarted(extractResourceID(task1));
+						resourceManager.tell(msg);
+
+						// verify that the internal state was updated
+						assertThat(resourceManagerInstance.workersInLaunch.entrySet(), empty());
+					}
+					catch(Exception ex) {
+						throw new RuntimeException(ex);
+					}
+				}
+			};
+		}};
+	}
+
+	/**
+	 * Test release of registered workers.
+	 */
+	@Test
+	public void testReleaseRegisteredWorker() {
+		new Context() {{
+			new Within(duration("10 seconds")) {
+				@Override
+				protected void run() {
+					try {
+						// set the initial persistent state, initialize the RM, then register with task1 as a registered worker
+						MesosWorkerStore.Worker worker1 = MesosWorkerStore.Worker.newTask(task1).launchTask(slave1, slave1host);
+						when(workerStore.getFrameworkID()).thenReturn(Option.apply(framework1));
+						when(workerStore.recoverWorkers()).thenReturn(singletonList(worker1));
+						initialize();
+						resourceManagerInstance.launchCoordinator.expectMsgClass(LaunchCoordinator.Assign.class);
+						register(singletonList(extractResourceID(task1)));
+
+						// release the registered worker
+						resourceManager.tell(new RemoveResource(extractResourceID(task1)));
+
+						// verify that the worker was persisted, the internal state was updated, the task router was notified,
+						// and the launch coordinator was notified about the host assignment change
+						MesosWorkerStore.Worker worker2Released = worker1.releaseTask();
+						verify(workerStore).putWorker(worker2Released);
+						assertThat(resourceManagerInstance.workersBeingReturned, hasEntry(extractResourceID(task1), worker2Released));
+						resourceManagerInstance.launchCoordinator.expectMsg(new LaunchCoordinator.Unassign(task1, slave1host));
+
+						// send the subsequent terminated message
+						resourceManager.tell(new TaskMonitor.TaskTerminated(task1, Protos.TaskStatus.newBuilder()
+							.setTaskId(task1).setSlaveId(slave1).setState(Protos.TaskState.TASK_FINISHED).build()));
+
+						// verify that the instance state was updated
+						assertThat(resourceManagerInstance.workersBeingReturned.entrySet(), empty());
+						verify(workerStore).removeWorker(task1);
+					}
+					catch(Exception ex) {
+						throw new RuntimeException(ex);
+					}
+				}
+			};
+		}};
+	}
+
+	/**
+	 * Test request for new workers.
+	 */
+	@Test
+	public void testRequestNewWorkers() {
+		new Context() {{
+			new Within(duration("10 seconds")) {
+				@Override
+				protected void run() {
+					try {
+						initialize();
+						register(Collections.<ResourceID>emptyList());
+
+						// set the target pool size
+						when(workerStore.newTaskID()).thenReturn(task1).thenThrow(new AssertionFailedError());
+						resourceManager.tell(new SetWorkerPoolSize(1), jobManager);
+
+						// verify that a new worker was persisted, the internal state was updated, the task router was notified,
+						// and the launch coordinator was asked to launch a task
+						MesosWorkerStore.Worker expected = MesosWorkerStore.Worker.newTask(task1);
+						verify(workerStore).putWorker(expected);
+						assertThat(resourceManagerInstance.workersInNew, hasEntry(extractResourceID(task1), expected));
+						resourceManagerInstance.taskRouter.expectMsgClass(TaskMonitor.TaskGoalStateUpdated.class);
+						resourceManagerInstance.launchCoordinator.expectMsgClass(LaunchCoordinator.Launch.class);
+					}
+					catch(Exception ex) {
+						throw new RuntimeException(ex);
+					}
+				}
+			};
+		}};
+	}
+
+	/**
+	 * Test offer handling.
+	 */
+	@Test
+	public void testOfferHandling() {
+		new Context() {{
+			new Within(duration("10 seconds")) {
+				@Override
+				protected void run() {
+					initialize();
+					register(Collections.<ResourceID>emptyList());
+
+					// Verify that the RM forwards offers to the launch coordinator.
+					resourceManager.tell(new ResourceOffers(Collections.<Protos.Offer>emptyList()));
+					resourceManagerInstance.launchCoordinator.expectMsgClass(ResourceOffers.class);
+					resourceManager.tell(new OfferRescinded(offer1));
+					resourceManagerInstance.launchCoordinator.expectMsgClass(OfferRescinded.class);
+				}
+			};
+		}};
+	}
+
+	/**
+	 * Test offer acceptance.
+	 */
+	@Test
+	public void testAcceptOffers() {
+		new Context() {{
+			new Within(duration("10 seconds")) {
+				@Override
+				protected void run() {
+					try {
+						// set the initial persistent state with a new task then initialize the RM
+						MesosWorkerStore.Worker worker1 = MesosWorkerStore.Worker.newTask(task1);
+						when(workerStore.getFrameworkID()).thenReturn(Option.apply(framework1));
+						when(workerStore.recoverWorkers()).thenReturn(singletonList(worker1));
+						initialize();
+						assertThat(resourceManagerInstance.workersInNew, hasEntry(extractResourceID(task1), worker1));
+						resourceManagerInstance.taskRouter.expectMsgClass(TaskMonitor.TaskGoalStateUpdated.class);
+						register(Collections.<ResourceID>emptyList());
+
+						// send an AcceptOffers message as the LaunchCoordinator would
+						// to launch task1 onto slave1 with offer1
+						Protos.TaskInfo task1info = Protos.TaskInfo.newBuilder()
+							.setTaskId(task1).setName("").setSlaveId(slave1).build();
+						AcceptOffers msg = new AcceptOffers(slave1host, singletonList(offer1), singletonList(launch(task1info)));
+						resourceManager.tell(msg);
+
+						// verify that the worker was persisted, the internal state was updated,
+						// Mesos was asked to launch task1, and the task router was notified
+						MesosWorkerStore.Worker worker1launched = worker1.launchTask(slave1, slave1host);
+						verify(workerStore).putWorker(worker1launched);
+						assertThat(resourceManagerInstance.workersInNew.entrySet(), empty());
+						assertThat(resourceManagerInstance.workersInLaunch, hasEntry(extractResourceID(task1), worker1launched));
+						resourceManagerInstance.taskRouter.expectMsg(
+							new TaskMonitor.TaskGoalStateUpdated(extractGoalState(worker1launched)));
+						verify(schedulerDriver).acceptOffers(msg.offerIds(), msg.operations(), msg.filters());
+					}
+					catch(Exception ex) {
+						throw new RuntimeException(ex);
+					}
+				}
+			};
+		}};
+	}
+
+	/**
+	 * Test status handling.
+	 */
+	@Test
+	public void testStatusHandling() {
+		new Context() {{
+			new Within(duration("10 seconds")) {
+				@Override
+				protected void run() {
+					initialize();
+					register(Collections.<ResourceID>emptyList());
+
+					// Verify that the RM forwards status updates to the launch coordinator and task router.
+					resourceManager.tell(new StatusUpdate(Protos.TaskStatus.newBuilder()
+						.setTaskId(task1).setSlaveId(slave1).setState(Protos.TaskState.TASK_LOST).build()),
+						resourceManager);
+					resourceManagerInstance.reconciliationCoordinator.expectMsgClass(StatusUpdate.class);
+					resourceManagerInstance.taskRouter.expectMsgClass(StatusUpdate.class);
+				}
+			};
+		}};
+	}
+
+	/**
+	 * Test unplanned task failure of a pending worker.
+	 */
+	@Test
+	public void testPendingWorkerFailed() {
+		new Context() {{
+			new Within(duration("10 seconds")) {
+				@Override
+				protected void run() {
+					try {
+						// set the initial persistent state with a launched worker that hasn't yet registered
+						MesosWorkerStore.Worker worker1launched = MesosWorkerStore.Worker.newTask(task1).launchTask(slave1, slave1host);
+						when(workerStore.getFrameworkID()).thenReturn(Option.apply(framework1));
+						when(workerStore.recoverWorkers()).thenReturn(singletonList(worker1launched));
+						initialize();
+						register(Collections.<ResourceID>emptyList());
+
+						// tell the RM that a task failed (and prepare a replacement task)
+						when(workerStore.newTaskID()).thenReturn(task2);
+						resourceManager.tell(new SetWorkerPoolSize(1), jobManager);
+						resourceManager.tell(new TaskMonitor.TaskTerminated(task1, Protos.TaskStatus.newBuilder()
+							.setTaskId(task1).setSlaveId(slave1).setState(Protos.TaskState.TASK_FAILED).build()));
+
+						// verify that the instance state was updated
+						assertThat(resourceManagerInstance.workersInLaunch.entrySet(), empty());
+						verify(workerStore).newTaskID();
+					}
+					catch(Exception ex) {
+						throw new RuntimeException(ex);
+					}
+				}
+			};
+		}};
+	}
+
+	/**
+	 * Test unplanned task failure of a registered worker.
+	 */
+	@Test
+	public void testRegisteredWorkerFailed() {
+		new Context() {{
+			new Within(duration("10 seconds")) {
+				@Override
+				protected void run() {
+					try {
+						// set the initial persistent state with a launched & registered worker
+						MesosWorkerStore.Worker worker1launched = MesosWorkerStore.Worker.newTask(task1).launchTask(slave1, slave1host);
+						when(workerStore.getFrameworkID()).thenReturn(Option.apply(framework1));
+						when(workerStore.recoverWorkers()).thenReturn(singletonList(worker1launched));
+						initialize();
+						register(singletonList(extractResourceID(task1)));
+
+						// tell the RM that a task failed (and prepare a replacement task)
+						when(workerStore.newTaskID()).thenReturn(task2);
+						resourceManager.tell(new SetWorkerPoolSize(1), jobManager);
+						resourceManager.tell(new TaskMonitor.TaskTerminated(task1, Protos.TaskStatus.newBuilder()
+							.setTaskId(task1).setSlaveId(slave1).setState(Protos.TaskState.TASK_FAILED).build()));
+
+						// verify that the instance state was updated and a replacement was created
+						assertThat(resourceManagerInstance.workersInLaunch.entrySet(), empty());
+						expectMsgClass(ResourceRemoved.class);
+						verify(workerStore).newTaskID();
+					}
+					catch(Exception ex) {
+						throw new RuntimeException(ex);
+					}
+				}
+			};
+		}};
+	}
+
+	/**
+	 * Test cluster stop handling.
+	 */
+	@Test
+	public void testStopApplication() {
+		new Context() {{
+			new Within(duration("10 seconds")) {
+				@Override
+				protected void run() {
+					try {
+						initialize();
+						register(Collections.<ResourceID>emptyList());
+						watch(resourceManager.actor());
+						resourceManager.tell(new StopCluster(ApplicationStatus.SUCCEEDED, ""), resourceManager);
+
+						// verify that the Mesos framework is shutdown
+						verify(schedulerDriver).stop(false);
+						verify(workerStore).cleanup();
+						expectTerminated(resourceManager.actor());
+					}
+					catch(Exception ex) {
+						throw new RuntimeException(ex);
+					}
+				}
+			};
+		}};
+	}
+
+	// ------------- connectivity tests -----------------------------
+
+	/**
+	 * Test Mesos registration handling.
+	 */
+	@Test
+	public void testRegistered() {
+		new Context() {{
+			new Within(duration("10 seconds")) {
+				@Override
+				protected void run() {
+					try {
+						initialize();
+						register(Collections.<ResourceID>emptyList());
+
+						Protos.MasterInfo masterInfo = Protos.MasterInfo.newBuilder()
+							.setId("master1").setIp(0).setPort(5050).build();
+						resourceManager.tell(new Registered(framework1, masterInfo), resourceManager);
+
+						verify(workerStore).setFrameworkID(Option.apply(framework1));
+						resourceManagerInstance.connectionMonitor.expectMsgClass(Registered.class);
+						resourceManagerInstance.reconciliationCoordinator.expectMsgClass(Registered.class);
+						resourceManagerInstance.launchCoordinator.expectMsgClass(Registered.class);
+						resourceManagerInstance.taskRouter.expectMsgClass(Registered.class);
+					}
+					catch(Exception ex) {
+						throw new RuntimeException(ex);
+					}
+				}
+			};
+		}};
+	}
+
+
+	/**
+	 * Test Mesos re-registration handling.
+	 */
+	@Test
+	public void testReRegistered() {
+		new Context() {{
+			new Within(duration("10 seconds")) {
+				@Override
+				protected void run() {
+					try {
+						when(workerStore.getFrameworkID()).thenReturn(Option.apply(framework1));
+						initialize();
+						register(Collections.<ResourceID>emptyList());
+
+						Protos.MasterInfo masterInfo = Protos.MasterInfo.newBuilder()
+							.setId("master1").setIp(0).setPort(5050).build();
+						resourceManager.tell(new ReRegistered(masterInfo), resourceManager);
+
+						resourceManagerInstance.connectionMonitor.expectMsgClass(ReRegistered.class);
+						resourceManagerInstance.reconciliationCoordinator.expectMsgClass(ReRegistered.class);
+						resourceManagerInstance.launchCoordinator.expectMsgClass(ReRegistered.class);
+						resourceManagerInstance.taskRouter.expectMsgClass(ReRegistered.class);
+					}
+					catch(Exception ex) {
+						throw new RuntimeException(ex);
+					}
+				}
+			};
+		}};
+	}
+
+	/**
+	 * Test Mesos re-registration handling.
+	 */
+	@Test
+	public void testDisconnected() {
+		new Context() {{
+			new Within(duration("10 seconds")) {
+				@Override
+				protected void run() {
+					try {
+						when(workerStore.getFrameworkID()).thenReturn(Option.apply(framework1));
+						initialize();
+						register(Collections.<ResourceID>emptyList());
+
+						resourceManager.tell(new Disconnected(), resourceManager);
+
+						resourceManagerInstance.connectionMonitor.expectMsgClass(Disconnected.class);
+						resourceManagerInstance.reconciliationCoordinator.expectMsgClass(Disconnected.class);
+						resourceManagerInstance.launchCoordinator.expectMsgClass(Disconnected.class);
+						resourceManagerInstance.taskRouter.expectMsgClass(Disconnected.class);
+					}
+					catch(Exception ex) {
+						throw new RuntimeException(ex);
+					}
+				}
+			};
+		}};
+	}
+
+	/**
+	 * Test Mesos scheduler error.
+	 */
+	@Test
+	public void testError() {
+		new Context() {{
+			new Within(duration("10 seconds")) {
+				@Override
+				protected void run() {
+					try {
+						when(workerStore.getFrameworkID()).thenReturn(Option.apply(framework1));
+						initialize();
+						register(Collections.<ResourceID>emptyList());
+
+						watch(resourceManager.actor());
+						resourceManager.tell(new Error("test"), resourceManager);
+						expectTerminated(resourceManager.actor());
+					}
+					catch(Exception ex) {
+						throw new RuntimeException(ex);
+					}
+				}
+			};
+		}};
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/d9b2be05/flink-mesos/src/test/resources/log4j-test.properties
----------------------------------------------------------------------
diff --git a/flink-mesos/src/test/resources/log4j-test.properties b/flink-mesos/src/test/resources/log4j-test.properties
new file mode 100644
index 0000000..3476c19
--- /dev/null
+++ b/flink-mesos/src/test/resources/log4j-test.properties
@@ -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.
+################################################################################
+
+log4j.rootLogger=INFO, console
+
+# Log all infos in the given file
+log4j.appender.console=org.apache.log4j.ConsoleAppender
+log4j.appender.console.layout=org.apache.log4j.PatternLayout
+log4j.appender.console.layout.ConversionPattern=%d{HH:mm:ss,SSS} %-5p %-60c %x - %m%n
+
+# suppress the irrelevant (wrong) warnings from the netty channel handler
+log4j.logger.org.jboss.netty.channel.DefaultChannelPipeline=ERROR, console
+
+# log whats going on between the tests
+log4j.logger.org.apache.flink.runtime.leaderelection=INFO
+log4j.logger.org.apache.flink.runtime.leaderretrieval=INFO
+

http://git-wip-us.apache.org/repos/asf/flink/blob/d9b2be05/flink-mesos/src/test/resources/logback-test.xml
----------------------------------------------------------------------
diff --git a/flink-mesos/src/test/resources/logback-test.xml b/flink-mesos/src/test/resources/logback-test.xml
new file mode 100644
index 0000000..48e4374
--- /dev/null
+++ b/flink-mesos/src/test/resources/logback-test.xml
@@ -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.
+  -->
+
+<configuration>
+    <appender name="STDOUT" class="ch.qos.logback.core.ConsoleAppender">
+        <encoder>
+            <pattern>%d{HH:mm:ss.SSS} %-5level [%thread] %logger{60} - %msg%n</pattern>
+        </encoder>
+    </appender>
+
+    <root level="WARN">
+        <appender-ref ref="STDOUT"/>
+    </root>
+
+    <!--<logger name="org.apache.flink.runtime.operators.BatchTask" level="OFF"/>-->
+    <!--<logger name="org.apache.flink.runtime.client.JobClient" level="OFF"/>-->
+    <!--<logger name="org.apache.flink.runtime.taskmanager.Task" level="OFF"/>-->
+    <!--<logger name="org.apache.flink.runtime.jobmanager.JobManager" level="OFF"/>-->
+    <!--<logger name="org.apache.flink.runtime.taskmanager.TaskManager" level="OFF"/>-->
+    <!--<logger name="org.apache.flink.runtime.executiongraph.ExecutionGraph" level="OFF"/>-->
+    <!--<logger name="org.apache.flink.runtime.jobmanager.EventCollector" level="OFF"/>-->
+</configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/flink/blob/d9b2be05/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
new file mode 100644
index 0000000..18ff0b6
--- /dev/null
+++ b/flink-mesos/src/test/scala/org/apache/flink/mesos/Utils.scala
@@ -0,0 +1,34 @@
+package org.apache.flink.mesos
+
+import java.util.concurrent.atomic.AtomicLong
+
+import akka.actor._
+import akka.testkit.{TestActorRef, TestFSMRef}
+import org.mockito.ArgumentMatcher
+
+import scala.collection.JavaConverters._
+import scala.reflect.ClassTag
+
+object Matchers {
+  def contentsMatch[T](plan: Seq[T]): java.util.Collection[T] = {
+    org.mockito.Matchers.argThat(new ArgumentMatcher[java.util.Collection[T]] {
+      override def matches(o: scala.Any): Boolean = o match {
+        case actual: java.util.Collection[T] => actual.size() == plan.size && actual.containsAll(plan.asJava)
+        case _ => false
+      }
+    })
+  }
+}
+
+object TestFSMUtils {
+
+  val number = new AtomicLong
+  def randomName: String = {
+    val l = number.getAndIncrement()
+    "$" + akka.util.Helpers.base64(l)
+  }
+
+  def testFSMRef[S, D, T <: Actor: ClassTag](factory: \u21d2 T, supervisor: ActorRef)(implicit ev: T <:< FSM[S, D], system: ActorSystem): TestFSMRef[S, D, T] = {
+    new TestFSMRef(system, Props(factory), supervisor, TestFSMUtils.randomName)
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/d9b2be05/flink-mesos/src/test/scala/org/apache/flink/mesos/scheduler/LaunchCoordinatorTest.scala
----------------------------------------------------------------------
diff --git a/flink-mesos/src/test/scala/org/apache/flink/mesos/scheduler/LaunchCoordinatorTest.scala b/flink-mesos/src/test/scala/org/apache/flink/mesos/scheduler/LaunchCoordinatorTest.scala
new file mode 100644
index 0000000..3b2bee6
--- /dev/null
+++ b/flink-mesos/src/test/scala/org/apache/flink/mesos/scheduler/LaunchCoordinatorTest.scala
@@ -0,0 +1,421 @@
+package org.apache.flink.mesos.scheduler
+
+import java.util.{Collections, UUID}
+import java.util.concurrent.atomic.AtomicReference
+
+import akka.actor.FSM.StateTimeout
+import akka.testkit._
+import com.netflix.fenzo.TaskRequest.{AssignedResources, NamedResourceSetRequest}
+import com.netflix.fenzo._
+import com.netflix.fenzo.functions.{Action1, Action2}
+import com.netflix.fenzo.plugins.VMLeaseObject
+import org.apache.flink.api.java.tuple.{Tuple2=>FlinkTuple2}
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.mesos.scheduler.LaunchCoordinator._
+import org.apache.flink.mesos.scheduler.messages._
+import org.apache.flink.runtime.akka.AkkaUtils
+import org.apache.mesos.Protos.{SlaveID, TaskInfo}
+import org.apache.mesos.{SchedulerDriver, Protos}
+import org.junit.runner.RunWith
+import org.mockito.Mockito.{verify, _}
+import org.mockito.invocation.InvocationOnMock
+import org.mockito.stubbing.Answer
+import org.mockito.{Matchers => MM, Mockito}
+import org.scalatest.junit.JUnitRunner
+import org.scalatest.{BeforeAndAfterAll, Matchers, WordSpecLike}
+
+import scala.collection.JavaConverters._
+
+import org.apache.flink.mesos.Utils.range
+import org.apache.flink.mesos.Utils.ranges
+import org.apache.flink.mesos.Utils.scalar
+
+@RunWith(classOf[JUnitRunner])
+class LaunchCoordinatorTest
+  extends TestKitBase
+    with ImplicitSender
+    with WordSpecLike
+    with Matchers
+    with BeforeAndAfterAll {
+
+  lazy val config = new Configuration()
+  implicit lazy val system = AkkaUtils.createLocalActorSystem(config)
+
+  override def afterAll(): Unit = {
+    TestKit.shutdownActorSystem(system)
+  }
+
+  def randomFramework = {
+    Protos.FrameworkID.newBuilder().setValue(UUID.randomUUID.toString).build
+  }
+
+  def randomTask = {
+    val taskID = Protos.TaskID.newBuilder.setValue(UUID.randomUUID.toString).build
+
+    def generateTaskRequest = {
+      new TaskRequest() {
+        private[mesos] val assignedResources = new AtomicReference[TaskRequest.AssignedResources]
+        override def getId: String = taskID.getValue
+        override def taskGroupName: String = ""
+        override def getCPUs: Double = 1.0
+        override def getMemory: Double = 1024.0
+        override def getNetworkMbps: Double = 0.0
+        override def getDisk: Double = 0.0
+        override def getPorts: Int = 1
+        override def getCustomNamedResources: java.util.Map[String, NamedResourceSetRequest] =
+          Collections.emptyMap[String, NamedResourceSetRequest]
+        override def getSoftConstraints: java.util.List[_ <: VMTaskFitnessCalculator] = null
+        override def getHardConstraints: java.util.List[_ <: ConstraintEvaluator] = null
+        override def getAssignedResources: AssignedResources = assignedResources.get()
+        override def setAssignedResources(assignedResources: AssignedResources): Unit = {
+          this.assignedResources.set(assignedResources)
+        }
+      }
+    }
+
+    val task: LaunchableTask = new LaunchableTask() {
+      override def taskRequest: TaskRequest = generateTaskRequest
+      override def launch(slaveId: SlaveID, taskAssignment: TaskAssignmentResult): Protos.TaskInfo = {
+        Protos.TaskInfo.newBuilder
+          .setTaskId(taskID).setName(taskID.getValue)
+          .setCommand(Protos.CommandInfo.newBuilder.setValue("whoami"))
+          .setSlaveId(slaveId)
+          .build()
+      }
+      override def toString = taskRequest.getId
+    }
+
+    (taskID, task)
+  }
+
+  def randomSlave = {
+    val slaveID = Protos.SlaveID.newBuilder.setValue(UUID.randomUUID.toString).build
+    val hostname = s"host-${slaveID.getValue}"
+    (slaveID, hostname)
+  }
+
+  def randomOffer(frameworkID: Protos.FrameworkID, slave: (Protos.SlaveID, String)) = {
+    val offerID = Protos.OfferID.newBuilder().setValue(UUID.randomUUID.toString)
+    Protos.Offer.newBuilder()
+      .setFrameworkId(frameworkID)
+      .setId(offerID)
+      .setSlaveId(slave._1)
+      .setHostname(slave._2)
+      .addResources(scalar("cpus", 0.75))
+      .addResources(scalar("mem", 4096.0))
+      .addResources(scalar("disk", 1024.0))
+      .addResources(ranges("ports", range(9000, 9001)))
+      .build()
+  }
+
+  def lease(offer: Protos.Offer) = {
+    new VMLeaseObject(offer)
+  }
+
+  /**
+    * Mock a successful task assignment result matching a task to an offer.
+    */
+  def taskAssignmentResult(lease: VirtualMachineLease, task: TaskRequest): TaskAssignmentResult = {
+    val ports = lease.portRanges().get(0)
+    val r = mock(classOf[TaskAssignmentResult])
+    when(r.getTaskId).thenReturn(task.getId)
+    when(r.getHostname).thenReturn(lease.hostname())
+    when(r.getAssignedPorts).thenReturn(
+      (ports.getBeg to ports.getBeg + task.getPorts).toList.asJava.asInstanceOf[java.util.List[Integer]])
+    when(r.getRequest).thenReturn(task)
+    when(r.isSuccessful).thenReturn(true)
+    when(r.getFitness).thenReturn(1.0)
+    r
+  }
+
+  /**
+    * Mock a VM assignment result with the given leases and tasks.
+    */
+  def vmAssignmentResult(hostname: String,
+                         leasesUsed: Seq[VirtualMachineLease],
+                         tasksAssigned: Set[TaskAssignmentResult]): VMAssignmentResult = {
+    new VMAssignmentResult(hostname, leasesUsed.asJava, tasksAssigned.asJava)
+  }
+
+  /**
+    * Mock a scheduling result with the given successes and failures.
+    */
+  def schedulingResult(successes: Seq[VMAssignmentResult],
+                       failures: Seq[TaskAssignmentResult] = Nil,
+                       exceptions: Seq[Exception] = Nil,
+                       leasesAdded: Int = 0,
+                       leasesRejected: Int = 0): SchedulingResult = {
+    val r = mock(classOf[SchedulingResult])
+    when(r.getResultMap).thenReturn(successes.map(r => r.getHostname -> r).toMap.asJava)
+    when(r.getExceptions).thenReturn(exceptions.asJava)
+    val groupedFailures = failures.groupBy(_.getRequest).mapValues(_.asJava)
+    when(r.getFailures).thenReturn(groupedFailures.asJava)
+    when(r.getLeasesAdded).thenReturn(leasesAdded)
+    when(r.getLeasesRejected).thenReturn(leasesRejected)
+    when(r.getRuntime).thenReturn(0)
+    when(r.getNumAllocations).thenThrow(new NotImplementedError())
+    when(r.getTotalVMsCount).thenThrow(new NotImplementedError())
+    when(r.getIdleVMsCount).thenThrow(new NotImplementedError())
+    r
+  }
+
+
+  /**
+    * Mock a task scheduler.
+    * The task assigner/unassigner is pre-wired.
+    */
+  def taskScheduler() = {
+    val optimizer = mock(classOf[TaskScheduler])
+    val taskAssigner = mock(classOf[Action2[TaskRequest, String]])
+    when[Action2[TaskRequest, String]](optimizer.getTaskAssigner).thenReturn(taskAssigner)
+    val taskUnassigner = mock(classOf[Action2[String, String]])
+    when[Action2[String, String]](optimizer.getTaskUnAssigner).thenReturn(taskUnassigner)
+    optimizer
+  }
+
+  /**
+    * Create a task scheduler builder.
+    */
+  def taskSchedulerBuilder(optimizer: TaskScheduler) = new TaskSchedulerBuilder {
+    var leaseRejectAction: Action1[VirtualMachineLease] = null
+    override def withLeaseRejectAction(action: Action1[VirtualMachineLease]): TaskSchedulerBuilder = {
+      leaseRejectAction = action
+      this
+    }
+    override def build(): TaskScheduler = optimizer
+  }
+
+  /**
+    * Process a call to scheduleOnce with the given function.
+    */
+  def scheduleOnce(f: (Seq[TaskRequest],Seq[VirtualMachineLease]) => SchedulingResult) = {
+    new Answer[SchedulingResult] {
+      override def answer(invocationOnMock: InvocationOnMock): SchedulingResult = {
+        val args = invocationOnMock.getArguments
+        val requests = args(0).asInstanceOf[java.util.List[TaskRequest]]
+        val newLeases = args(1).asInstanceOf[java.util.List[VirtualMachineLease]]
+        f(requests.asScala, newLeases.asScala)
+      }
+    }
+  }
+
+  /**
+    * The context fixture.
+    */
+  class Context {
+    val optimizer = taskScheduler()
+    val optimizerBuilder = taskSchedulerBuilder(optimizer)
+    val schedulerDriver = mock(classOf[SchedulerDriver])
+    val trace = Mockito.inOrder(schedulerDriver)
+    val fsm = TestFSMRef(new LaunchCoordinator(testActor, config, schedulerDriver, optimizerBuilder))
+
+    val framework = randomFramework
+    val task1 = randomTask
+    val task2 = randomTask
+    val task3 = randomTask
+
+    val slave1 = {
+      val slave = randomSlave
+      (slave._1, slave._2, randomOffer(framework, slave), randomOffer(framework, slave), randomOffer(framework, slave))
+    }
+
+    val slave2 = {
+      val slave = randomSlave
+      (slave._1, slave._2, randomOffer(framework, slave), randomOffer(framework, slave), randomOffer(framework, slave))
+    }
+  }
+
+  def inState = afterWord("in state")
+  def handle = afterWord("handle")
+
+  def handlesAssignments(state: TaskState) = {
+    "Unassign" which {
+      s"stays in $state with updated optimizer state" in new Context {
+        optimizer.getTaskAssigner.call(task1._2.taskRequest, slave1._2)
+        fsm.setState(state)
+        fsm ! Unassign(task1._1, slave1._2)
+        verify(optimizer.getTaskUnAssigner).call(task1._1.getValue, slave1._2)
+        fsm.stateName should be (state)
+      }
+    }
+    "Assign" which {
+      s"stays in $state with updated optimizer state" in new Context {
+        fsm.setState(state)
+        fsm ! Assign(Seq(new FlinkTuple2(task1._2.taskRequest, slave1._2)).asJava)
+        verify(optimizer.getTaskAssigner).call(MM.any(), MM.any())
+        fsm.stateName should be (state)
+      }
+    }
+  }
+
+  "The LaunchCoordinator" when inState {
+
+    "Suspended" should handle {
+      "Connected" which {
+        "transitions to Idle when the task queue is empty" in new Context {
+          fsm.setState(Suspended)
+          fsm ! new Connected {}
+          fsm.stateName should be (Idle)
+        }
+        "transitions to GatheringOffers when the task queue is non-empty" in new Context {
+          fsm.setState(Suspended, GatherData(tasks = Seq(task1._2), newLeases = Nil))
+          fsm ! new Connected {}
+          fsm.stateName should be (GatheringOffers)
+          fsm.stateData.tasks should contain only (task1._2)
+        }
+      }
+      "Launch" which {
+        "stays in Suspended with updated task queue" in new Context {
+          fsm.setState(Suspended, GatherData(tasks = Seq(task1._2), newLeases = Nil))
+          fsm ! Launch(Seq(task2._2).asJava)
+          fsm.stateName should be (Suspended)
+          fsm.stateData.tasks should contain only (task1._2, task2._2)
+        }
+      }
+
+      behave like handlesAssignments(Suspended)
+    }
+
+    "Idle" should handle {
+      "Disconnected" which {
+        "transitions to Suspended" in new Context {
+          fsm.setState(Idle)
+          fsm ! new Disconnected()
+          fsm.stateName should be (Suspended)
+        }
+      }
+      "ResourceOffers" which {
+        "stays in Idle with offers declined" in new Context {
+          fsm.setState(Idle)
+          fsm ! new ResourceOffers(Seq(slave1._3, slave1._4).asJava)
+          verify(schedulerDriver).declineOffer(slave1._3.getId)
+          verify(schedulerDriver).declineOffer(slave1._4.getId)
+          fsm.stateName should be (Idle)
+        }
+      }
+      "Launch" which {
+        "transitions to GatheringOffers with updated task queue" in new Context {
+          fsm.setState(Idle)
+          fsm ! Launch(Seq(task1._2, task2._2).asJava)
+          fsm.stateName should be (GatheringOffers)
+          fsm.stateData.tasks should contain only (task1._2, task2._2)
+        }
+      }
+
+      behave like handlesAssignments(Idle)
+    }
+
+    "GatheringOffers" should handle {
+      "(enter)" which {
+        "revives offers" in new Context {
+          fsm.setState(GatheringOffers, GatherData())
+          verify(schedulerDriver).reviveOffers()
+        }
+      }
+      "(exit)" which {
+        "suppresses offers" in new Context {
+          fsm.setState(GatheringOffers, GatherData())
+          fsm ! new Disconnected()
+          verify(schedulerDriver).suppressOffers()
+        }
+        "declines any outstanding offers" in new Context {
+          //fsm.setState(GatheringOffers, GatherData(newOffers = Seq(new VMLeaseObject(slave1._3))))
+          fsm.setState(GatheringOffers, GatherData())
+          fsm ! new Disconnected()
+          verify(optimizer).expireAllLeases()
+        }
+      }
+      "Disconnected" which {
+        "transitions to Suspended with task queue intact" in new Context {
+          fsm.setState(GatheringOffers, GatherData(tasks = Seq(task1._2)))
+          fsm ! new Disconnected()
+          fsm.stateName should be (Suspended)
+          fsm.stateData.tasks should contain only (task1._2)
+        }
+        "transitions to Suspended with offer queue emptied" in new Context {
+          fsm.setState(GatheringOffers, GatherData(tasks = Seq(task1._2), newLeases = Seq(lease(slave1._3))))
+          fsm ! new Disconnected()
+          fsm.stateName should be (Suspended)
+          fsm.stateData.newLeases should be (empty)
+        }
+      }
+      "Launch" which {
+        "stays in GatheringOffers with updated task queue" in new Context {
+          fsm.setState(GatheringOffers, GatherData(tasks = Seq(task1._2), newLeases = Seq(lease(slave1._3))))
+          fsm ! Launch(Seq(task2._2).asJava)
+          fsm.stateName should be (GatheringOffers)
+          fsm.stateData.tasks should contain only (task1._2, task2._2)
+          fsm.stateData.newLeases.map(_.getOffer) should contain only (slave1._3)
+        }
+      }
+      "ResourceOffers" which {
+        "stays in GatheringOffers with offer queue updated" in new Context {
+          fsm.setState(GatheringOffers, GatherData(tasks = Seq(task1._2), newLeases = Seq(lease(slave1._3))))
+          fsm ! new ResourceOffers(Seq(slave1._4, slave2._3).asJava)
+          fsm.stateName should be (GatheringOffers)
+          fsm.stateData.tasks should contain only (task1._2)
+          fsm.stateData.newLeases.map(_.getOffer) should contain only (slave1._3, slave1._4, slave2._3)
+        }
+      }
+      "OfferRescinded" which {
+        "stays in GatheringOffers with offer queue updated" in new Context {
+          fsm.setState(GatheringOffers, GatherData(tasks = Seq(task1._2), newLeases = Seq(lease(slave1._3))))
+          fsm ! new OfferRescinded(slave1._3.getId)
+          verify(optimizer).expireLease(slave1._3.getId.getValue)
+          fsm.stateName should be (GatheringOffers)
+          fsm.stateData.tasks should contain only (task1._2)
+          fsm.stateData.newLeases should be (empty)
+        }
+      }
+      "StateTimeout" which {
+        "sends AcceptOffers message for matched tasks" in new Context {
+          when(optimizer.scheduleOnce(MM.any(), MM.any())) thenAnswer scheduleOnce { (requests, newLeases) =>
+            val (lease, task) = (newLeases.head, requests.head)
+            schedulingResult(
+              successes = Seq(vmAssignmentResult(lease.hostname(), Seq(lease), Set(taskAssignmentResult(lease, task)))))
+          }
+          fsm.setState(GatheringOffers, GatherData(tasks = Seq(task1._2), newLeases = Seq(lease(slave1._3))))
+          fsm ! StateTimeout
+          val offers = expectMsgType[AcceptOffers]
+          offers.hostname() should be (slave1._2)
+          offers.offerIds() should contain only (slave1._3.getId)
+        }
+        "transitions to Idle when task queue is empty" in new Context {
+          when(optimizer.scheduleOnce(MM.any(), MM.any())) thenAnswer scheduleOnce { (requests, newLeases) =>
+            val (lease, task) = (newLeases.head, requests.head)
+            schedulingResult(
+              successes = Seq(vmAssignmentResult(lease.hostname(), Seq(lease), Set(taskAssignmentResult(lease, task)))))
+          }
+          fsm.setState(GatheringOffers, GatherData(tasks = Seq(task1._2), newLeases = Seq(lease(slave1._3))))
+          fsm ! StateTimeout
+          fsm.stateName should be (Idle)
+          fsm.stateData.tasks should be (empty)
+          fsm.stateData.newLeases should be (empty)
+        }
+        "stays in GatheringOffers when task queue is non-empty" in new Context {
+          when(optimizer.scheduleOnce(MM.any(), MM.any())) thenAnswer scheduleOnce { (requests, newLeases) =>
+            schedulingResult(successes = Nil)
+          }
+          fsm.setState(GatheringOffers, GatherData(tasks = Seq(task1._2), newLeases = Seq(lease(slave1._3))))
+          fsm ! StateTimeout
+          fsm.stateName should be (GatheringOffers)
+          fsm.stateData.tasks should contain only (task1._2)
+          fsm.stateData.newLeases should be (empty)
+        }
+        "declines old offers" in new Context {
+          when(optimizer.scheduleOnce(MM.any(), MM.any())) thenAnswer scheduleOnce { (requests, newLeases) =>
+            optimizerBuilder.leaseRejectAction.call(newLeases.head)
+            schedulingResult(successes = Nil)
+          }
+          fsm.setState(GatheringOffers, GatherData(tasks = Seq(task1._2), newLeases = Seq(lease(slave1._3))))
+          fsm ! StateTimeout
+          verify(schedulerDriver).declineOffer(slave1._3.getId)
+        }
+      }
+
+      behave like handlesAssignments(GatheringOffers)
+    }
+  }
+
+  override def toString = s"LaunchCoordinatorTest()"
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/d9b2be05/flink-mesos/src/test/scala/org/apache/flink/mesos/scheduler/ReconciliationCoordinatorTest.scala
----------------------------------------------------------------------
diff --git a/flink-mesos/src/test/scala/org/apache/flink/mesos/scheduler/ReconciliationCoordinatorTest.scala b/flink-mesos/src/test/scala/org/apache/flink/mesos/scheduler/ReconciliationCoordinatorTest.scala
new file mode 100644
index 0000000..ee13316
--- /dev/null
+++ b/flink-mesos/src/test/scala/org/apache/flink/mesos/scheduler/ReconciliationCoordinatorTest.scala
@@ -0,0 +1,214 @@
+package org.apache.flink.mesos.scheduler
+
+import java.util.UUID
+
+import akka.actor.FSM
+import akka.testkit._
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.mesos.Matchers._
+import org.apache.flink.mesos.scheduler.messages.{Connected, Disconnected, StatusUpdate}
+import org.apache.flink.runtime.akka.AkkaUtils
+import org.apache.mesos.Protos.TaskState._
+import org.apache.mesos.{SchedulerDriver, Protos}
+import org.junit.runner.RunWith
+import org.mockito.Mockito
+import org.mockito.Mockito._
+import org.scalatest.junit.JUnitRunner
+import org.scalatest.{BeforeAndAfterAll, Matchers, WordSpecLike}
+
+@RunWith(classOf[JUnitRunner])
+class ReconciliationCoordinatorTest
+    extends TestKitBase
+    with ImplicitSender
+    with WordSpecLike
+    with Matchers
+    with BeforeAndAfterAll {
+
+  import ReconciliationCoordinator._
+
+  lazy val config = new Configuration()
+  implicit lazy val system = AkkaUtils.createLocalActorSystem(config)
+
+  override def afterAll(): Unit = {
+    TestKit.shutdownActorSystem(system)
+  }
+
+  def randomTask = {
+    val slaveID = Protos.SlaveID.newBuilder.setValue(UUID.randomUUID.toString).build()
+    val taskID = Protos.TaskID.newBuilder.setValue(UUID.randomUUID.toString).build
+    val taskStatus = Protos.TaskStatus.newBuilder().setTaskId(taskID).setSlaveId(slaveID).setState(TASK_STAGING).build()
+    (taskID, taskStatus)
+  }
+
+  class Context {
+    val schedulerDriver = mock(classOf[SchedulerDriver])
+    val trace = Mockito.inOrder(schedulerDriver)
+    val fsm = TestFSMRef(new ReconciliationCoordinator(config, schedulerDriver))
+    val task1 = randomTask
+    val task2 = randomTask
+    val task3 = randomTask
+  }
+
+  def inState = afterWord("in state")
+  def handle = afterWord("handle")
+
+  "The ReconciliationCoordinator" when inState {
+
+    "Suspended" should handle {
+      "Connected" which {
+        "transitions to Idle when the queue is empty" in new Context {
+          fsm.setState(Suspended)
+          fsm ! new Connected {}
+          fsm.stateName should be (Idle)
+          fsm.stateData.remaining should be (empty)
+        }
+
+        "transitions to Reconciling when the queue is non-empty" in new Context {
+          fsm.setState(Suspended, ReconciliationData(Seq(randomTask).toMap))
+          fsm ! new Connected {}
+          fsm.stateName should be (Reconciling)
+        }
+      }
+
+      "Reconcile" which {
+        "queues requests" in new Context {
+          fsm.setState(Suspended)
+          fsm ! new Reconcile(Seq(task1).unzip._2)
+          fsm.stateData.remaining should contain only (task1)
+        }
+      }
+
+      "StatusUpdate" which {
+        "is disregarded" in new Context {
+          fsm.setState(Suspended)
+          fsm ! new StatusUpdate(task1._2)
+          fsm.stateName should be (Suspended)
+        }
+      }
+    }
+
+    "Idle" should handle {
+
+      "Disconnected" which {
+        "transitions to Suspended" in new Context {
+          fsm ! new Disconnected()
+          fsm.stateName should be (Suspended)
+        }
+      }
+
+      "Reconcile" which {
+        "transitions to Reconciling with the given tasks" in new Context {
+          fsm.setState(Idle)
+          fsm ! new Reconcile(Seq(task1._2))
+
+          verify(schedulerDriver).reconcileTasks(contentsMatch(Seq(task1._2)))
+          fsm.stateName should be (Reconciling)
+          fsm.stateData.remaining should contain only (task1)
+        }
+      }
+
+      "StatusUpdate" which {
+        "is disregarded" in new Context {
+          fsm.setState(Idle)
+          fsm ! new StatusUpdate(task1._2)
+          fsm.stateName should be (Idle)
+          fsm.stateData.remaining should be (empty)
+        }
+      }
+    }
+
+    "Reconciling" should handle {
+
+      "(enter)" which {
+        "initiates reconciliation" in new Context {
+          fsm.setState(Reconciling, ReconciliationData(Seq(task1).toMap))
+          verify(schedulerDriver).reconcileTasks(contentsMatch(Seq(task1._2)))
+        }
+      }
+
+      "Disconnected" which {
+        "transitions to Suspended with queue intact" in new Context {
+          fsm.setState(Reconciling, ReconciliationData(Seq(task1).toMap))
+          reset(schedulerDriver)
+          fsm ! FSM.StateTimeout
+          fsm ! new Disconnected()
+
+          fsm.stateName should be (Suspended)
+          fsm.stateData.remaining should contain only (task1)
+        }
+
+        "transitions to Suspended with retry counter reset" in new Context {
+          fsm.setState(Reconciling, ReconciliationData(Seq(task1).toMap))
+          reset(schedulerDriver)
+          fsm ! FSM.StateTimeout
+          fsm.stateData.retries should be (1)
+          fsm ! new Disconnected()
+
+          fsm.stateName should be (Suspended)
+          fsm.stateData.retries should be (0)
+        }
+      }
+
+      "Reconcile" which {
+        "stays in Reconciling with additional tasks" in new Context {
+          // testing the aggregation logic of: (A,B)+(B,C)=(A,B,C)
+          fsm.setState(Reconciling, ReconciliationData(Seq(task1, task2).toMap))
+          reset(schedulerDriver)
+          fsm ! new Reconcile(Seq(task2, task3).unzip._2)
+
+          // the reconcileTasks message should mention only the new tasks
+          verify(schedulerDriver).reconcileTasks(contentsMatch(Seq(task2, task3).unzip._2))
+          fsm.stateName should be (Reconciling)
+          fsm.stateData.remaining should contain only (task1, task2, task3)
+        }
+      }
+
+      "StatusUpdate" which {
+        "transitions to Idle when queue is empty" in new Context {
+          fsm.setState(Reconciling, ReconciliationData(Seq(task1).toMap))
+          reset(schedulerDriver)
+          val taskStatus = task1._2.toBuilder.setState(TASK_LOST).build()
+          fsm ! FSM.StateTimeout
+          fsm ! new StatusUpdate(taskStatus)
+
+          fsm.stateName should be (Idle)
+          fsm.stateData.remaining should be (empty)
+          fsm.stateData.retries should be (0)
+        }
+
+        "stays in Reconciling when queue is non-empty" in new Context {
+          fsm.setState(Reconciling, ReconciliationData(Seq(task1, task2).toMap))
+          reset(schedulerDriver)
+          val taskStatus = task1._2.toBuilder.setState(TASK_LOST).build()
+          fsm ! new StatusUpdate(taskStatus)
+
+          fsm.stateName should be (Reconciling)
+          fsm.stateData.remaining should contain only (task2)
+        }
+
+        "ignores updates for unrecognized tasks" in new Context {
+          fsm.setState(Reconciling, ReconciliationData(Seq(task1).toMap))
+          reset(schedulerDriver)
+          val taskStatus = task2._2.toBuilder.setState(TASK_LOST).build()
+          fsm ! new StatusUpdate(taskStatus)
+
+          fsm.stateName should be (Reconciling)
+          fsm.stateData.remaining should contain only (task1)
+        }
+      }
+
+      "StateTimeout" which {
+        "stays in Reconciling with reconciliation retry" in new Context {
+          fsm.setState(Reconciling, ReconciliationData(Seq(task1,task2).toMap))
+          reset(schedulerDriver)
+          fsm ! FSM.StateTimeout
+
+          verify(schedulerDriver).reconcileTasks(contentsMatch(Seq(task1,task2).unzip._2))
+          fsm.stateName should be (Reconciling)
+          fsm.stateData.remaining should contain only (task1,task2)
+          fsm.stateData.retries should be (1)
+        }
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/d9b2be05/flink-mesos/src/test/scala/org/apache/flink/mesos/scheduler/TaskMonitorTest.scala
----------------------------------------------------------------------
diff --git a/flink-mesos/src/test/scala/org/apache/flink/mesos/scheduler/TaskMonitorTest.scala b/flink-mesos/src/test/scala/org/apache/flink/mesos/scheduler/TaskMonitorTest.scala
new file mode 100644
index 0000000..6bd7178
--- /dev/null
+++ b/flink-mesos/src/test/scala/org/apache/flink/mesos/scheduler/TaskMonitorTest.scala
@@ -0,0 +1,237 @@
+package org.apache.flink.mesos.scheduler
+
+import java.util.UUID
+
+import akka.actor.FSM.StateTimeout
+import akka.testkit._
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.mesos.TestFSMUtils
+import org.apache.flink.mesos.scheduler.ReconciliationCoordinator.Reconcile
+import org.apache.flink.mesos.scheduler.messages.{Disconnected, Connected, StatusUpdate}
+import org.apache.flink.runtime.akka.AkkaUtils
+import org.apache.mesos.{SchedulerDriver, Protos}
+import org.apache.mesos.Protos.TaskState._
+import org.junit.runner.RunWith
+import org.mockito.Mockito
+import org.mockito.Mockito._
+import org.scalatest.junit.JUnitRunner
+import org.scalatest.{BeforeAndAfterAll, Matchers, WordSpecLike}
+
+@RunWith(classOf[JUnitRunner])
+class TaskMonitorTest
+  extends TestKitBase
+    with ImplicitSender
+    with WordSpecLike
+    with Matchers
+    with BeforeAndAfterAll {
+
+  import TaskMonitor._
+
+  lazy val config = new Configuration()
+  implicit lazy val system = AkkaUtils.createLocalActorSystem(config)
+
+  override def afterAll(): Unit = {
+    TestKit.shutdownActorSystem(system)
+  }
+
+  def randomSlave = {
+    val slaveID = Protos.SlaveID.newBuilder.setValue(UUID.randomUUID.toString).build
+    val hostname = s"host-${slaveID.getValue}"
+    (slaveID, hostname)
+  }
+
+  def randomTask(slaveID: Protos.SlaveID) = {
+    val taskID = Protos.TaskID.newBuilder.setValue(UUID.randomUUID.toString).build
+    val taskStatus = Protos.TaskStatus.newBuilder().setTaskId(taskID).setSlaveId(slaveID).setState(TASK_STAGING).build()
+    (taskID, taskStatus)
+  }
+
+  class Context {
+    val schedulerDriver = mock(classOf[SchedulerDriver])
+    val trace = Mockito.inOrder(schedulerDriver)
+    val slave = randomSlave
+    val task = randomTask(slave._1)
+    val parent = TestProbe()
+    val fsm = TestFSMUtils.testFSMRef(new TaskMonitor(config, schedulerDriver, New(task._1)), parent.ref)
+    parent.watch(fsm)
+  }
+
+  def inState = afterWord("in state")
+  def handle = afterWord("handle")
+
+  def handlesStatusUpdate(state: TaskMonitorState) = {
+    "StatusUpdate" which {
+      "transitions to Staging when goal state is Launched and status is TASK_STAGING|TASK_STARTING" in new Context {
+        fsm.setState(state, StateData(Launched(task._1, slave._1)))
+        fsm ! new StatusUpdate(task._2.toBuilder.setState(TASK_STAGING).build())
+        fsm.stateName should be (Staging)
+        fsm.stateData should be (StateData(Launched(task._1, slave._1)))
+      }
+      "transitions to Running when goal state is Launched and status is TASK_RUNNING" in new Context {
+        fsm.setState(state, StateData(Launched(task._1, slave._1)))
+        fsm ! new StatusUpdate(task._2.toBuilder.setState(TASK_RUNNING).build())
+        fsm.stateName should be (Running)
+        fsm.stateData should be (StateData(Launched(task._1, slave._1)))
+      }
+      "stops when goal state is Launched and status is TASK_KILLED" in new Context {
+        fsm.setState(state, StateData(Launched(task._1, slave._1)))
+        fsm ! new StatusUpdate(task._2.toBuilder.setState(TASK_KILLED).build())
+        parent.fishForMessage() {
+          case msg: Reconcile => false // initial state might have been Reconciling
+          case msg: TaskTerminated => true
+        }
+        parent.expectTerminated(fsm)
+      }
+      "transitions to Killing when goal state is Released and status is TASK_STAGING|TASK_STARTING|TASK_RUNNING" in new Context {
+        fsm.setState(state, StateData(Released(task._1, slave._1)))
+        fsm ! new StatusUpdate(task._2.toBuilder.setState(TASK_RUNNING).build())
+        fsm.stateName should be (Killing)
+        fsm.stateData should be (StateData(Released(task._1, slave._1)))
+      }
+      "stops when goal state is Released and status is TASK_KILLED" in new Context {
+        fsm.setState(state, StateData(Released(task._1, slave._1)))
+        fsm ! new StatusUpdate(task._2.toBuilder.setState(TASK_KILLED).build())
+        parent.fishForMessage() {
+          case msg: Reconcile => false // initial state might have been Reconciling
+          case msg: TaskTerminated => true
+        }
+        parent.expectTerminated(fsm)
+      }
+    }
+  }
+
+  def handlesDisconnect(state: TaskMonitorState) = {
+    "Disconnected" which {
+      "transitions to Suspended" in new Context {
+        fsm.setState(state, StateData(New(task._1)))
+        fsm ! new Disconnected()
+        fsm.stateName should be (Suspended)
+        fsm.stateData should be (StateData(New(task._1)))
+      }
+    }
+  }
+
+  def handlesRelease(state: TaskMonitorState) = {
+    "TaskGoalStateUpdated" which {
+      "transitions to Killing when the new goal state is Released" in new Context {
+        fsm.setState(state, StateData(Launched(task._1, slave._1)))
+        fsm ! TaskGoalStateUpdated(Released(task._1, slave._1))
+        fsm.stateName should be (Killing)
+        fsm.stateData.goal should be (Released(task._1, slave._1))
+      }
+    }
+  }
+
+  "TaskMonitor" when inState {
+
+    "Suspended" should handle {
+      "TaskGoalStateUpdated" which {
+        "stays in Suspended with updated state data" in new Context {
+          fsm.setState(Suspended, StateData(Launched(task._1, slave._1)))
+          fsm ! TaskGoalStateUpdated(Released(task._1, slave._1))
+          fsm.stateName should be (Suspended)
+          fsm.stateData.goal should be (Released(task._1, slave._1))
+        }
+      }
+      "StatusUpdate" which {
+        "is disregarded" in new Context {
+          fsm.setState(Suspended, StateData(Launched(task._1, slave._1)))
+          fsm ! new StatusUpdate(task._2)
+          fsm.stateName should be (Suspended)
+        }
+      }
+      "Connected" which {
+        "transitions to New when the goal state is New" in new Context {
+          fsm.setState(Suspended, StateData(New(task._1)))
+          fsm ! new Connected() {}
+          fsm.stateName should be (New)
+          fsm.stateData should be (StateData(New(task._1)))
+        }
+        "transitions to Reconciling when the goal state is Launched" in new Context {
+          fsm.setState(Suspended, StateData(Launched(task._1, slave._1)))
+          fsm ! new Connected() {}
+          fsm.stateName should be (Reconciling)
+          fsm.stateData should be (StateData(Launched(task._1, slave._1)))
+        }
+        "transitions to Killing when the goal state is Released" in new Context {
+          fsm.setState(Suspended, StateData(Released(task._1, slave._1)))
+          fsm ! new Connected() {}
+          fsm.stateName should be (Killing)
+          fsm.stateData should be (StateData(Released(task._1, slave._1)))
+        }
+      }
+    }
+
+    "New" should handle {
+      "TaskGoalStateUpdated" which {
+        "transitions to Staging when the new goal state is Launched" in new Context {
+          fsm.setState(New, StateData(New(task._1)))
+          fsm ! TaskGoalStateUpdated(Launched(task._1, slave._1))
+          fsm.stateName should be (Staging)
+          fsm.stateData should be (StateData(Launched(task._1, slave._1)))
+        }
+      }
+      behave like handlesDisconnect(New)
+    }
+
+    "Reconciling" should handle {
+      "(enter)" which {
+        "initiates reconciliation when goal state is Launched" in new Context {
+          fsm.setState(Reconciling, StateData(Launched(task._1, slave._1)))
+          parent.expectMsgType[Reconcile]
+        }
+      }
+      behave like handlesStatusUpdate(Reconciling)
+      behave like handlesDisconnect(Reconciling)
+      behave like handlesRelease(Reconciling)
+    }
+
+    "Staging" should handle {
+      "StateTimeout" which {
+        "transitions to Reconciling" in new Context {
+          fsm.setState(Staging, StateData(Launched(task._1, slave._1)))
+          fsm ! StateTimeout
+          fsm.stateName should be (Reconciling)
+          fsm.stateData should be (StateData(Launched(task._1, slave._1)))
+        }
+      }
+      behave like handlesStatusUpdate(Staging)
+      behave like handlesDisconnect(Staging)
+      behave like handlesRelease(Staging)
+    }
+
+    "Running" should handle {
+      behave like handlesStatusUpdate(Running)
+      behave like handlesDisconnect(Running)
+      behave like handlesRelease(Running)
+    }
+
+    "Killing" should handle {
+      "(enter)" which {
+        "sends kill message to Mesos" in new Context {
+          fsm.setState(Killing, StateData(Released(task._1, slave._1)))
+          verify(schedulerDriver).killTask(task._1)
+        }
+      }
+      "TaskGoalStateUpdated" which {
+        "stays in Killing when the new goal state is Released" in new Context {
+          fsm.setState(Killing, StateData(Launched(task._1, slave._1)))
+          fsm ! TaskGoalStateUpdated(Released(task._1, slave._1))
+          fsm.stateName should be (Killing)
+          fsm.stateData.goal should be (Released(task._1, slave._1))
+        }
+      }
+      "StateTimeout" which {
+        "stays in Killing with another kill message sent" in new Context {
+          fsm.setState(Killing, StateData(Released(task._1, slave._1)))
+          reset(schedulerDriver)
+          fsm ! StateTimeout
+          verify(schedulerDriver).killTask(task._1)
+          fsm.stateName should be (Killing)
+        }
+      }
+      behave like handlesStatusUpdate(Killing)
+      behave like handlesDisconnect(Killing)
+    }
+  }
+}


[7/9] flink git commit: [FLINK-1984] Mesos ResourceManager - T1 milstone (3)

Posted by mx...@apache.org.
[FLINK-1984] Mesos ResourceManager - T1 milstone (3)

- Fenzo usage fix - always call scheduleOnce after expireAllLeases.
- increased aggressiveness of task scheduler
- factored YarnJobManager and MesosJobManager to share base class
`ContaineredJobManager`
- improved supervision for task actors, unit tests
- support for zombie tasks (i.e. non-strict slave registry)
- improved javadocs
- fix for style violations (e.g. line length)
- completed the SchedulerProxy
- final fields
- improved preconditions
- log lines to use {}
- cleanup ZK state
- serializable messages


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

Branch: refs/heads/master
Commit: 59eeea4ce4fb0361c2f02618f5f4dbcfc6c0467d
Parents: 67be9e0
Author: wrighe3 <er...@emc.com>
Authored: Wed Aug 24 09:27:26 2016 -0700
Committer: Maximilian Michels <mx...@apache.org>
Committed: Mon Aug 29 17:32:51 2016 +0200

----------------------------------------------------------------------
 .../main/java/org/apache/flink/mesos/Utils.java |  13 +-
 .../flink/mesos/cli/FlinkMesosSessionCli.java   |  13 ++
 .../clusterframework/LaunchableMesosWorker.java |   9 +-
 .../MesosApplicationMasterRunner.java           |  61 ++----
 .../MesosFlinkResourceManager.java              |  43 ++--
 .../MesosTaskManagerParameters.java             |  12 ++
 .../MesosTaskManagerRunner.java                 |  10 +-
 .../store/MesosWorkerStore.java                 | 119 ++++++++---
 .../store/StandaloneMesosWorkerStore.java       |  11 +-
 .../store/ZooKeeperMesosWorkerStore.java        |  52 +++--
 .../flink/mesos/scheduler/SchedulerProxy.java   |  10 +-
 .../mesos/scheduler/TaskSchedulerBuilder.java   |   7 +
 .../mesos/scheduler/messages/AcceptOffers.java  |  13 +-
 .../mesos/scheduler/messages/Connected.java     |   5 +-
 .../mesos/scheduler/messages/Disconnected.java  |   7 +-
 .../flink/mesos/scheduler/messages/Error.java   |   9 +-
 .../mesos/scheduler/messages/ExecutorLost.java  |  66 ++++++
 .../scheduler/messages/FrameworkMessage.java    |  67 ++++++
 .../scheduler/messages/OfferRescinded.java      |   9 +-
 .../mesos/scheduler/messages/ReRegistered.java  |   5 +-
 .../mesos/scheduler/messages/Registered.java    |   6 +-
 .../scheduler/messages/ResourceOffers.java      |   8 +-
 .../mesos/scheduler/messages/SlaveLost.java     |   9 +-
 .../mesos/scheduler/messages/StatusUpdate.java  |   9 +-
 .../flink/mesos/util/MesosArtifactServer.java   |  12 +-
 .../flink/mesos/util/MesosConfiguration.java    |  23 +-
 .../apache/flink/mesos/util/ZooKeeperUtils.java |   2 +-
 .../clusterframework/MesosTaskManager.scala     |  16 +-
 .../mesos/scheduler/LaunchCoordinator.scala     |  63 +++---
 .../scheduler/ReconciliationCoordinator.scala   |  10 +-
 .../flink/mesos/scheduler/TaskMonitor.scala     |  16 +-
 .../apache/flink/mesos/scheduler/Tasks.scala    |  67 +++---
 .../ContaineredJobManager.scala                 | 174 ---------------
 .../MesosFlinkResourceManagerTest.java          |  29 +--
 .../scala/org/apache/flink/mesos/Utils.scala    |   6 +-
 .../mesos/scheduler/LaunchCoordinatorTest.scala |  91 +++++---
 .../ReconciliationCoordinatorTest.scala         |   3 +-
 .../flink/mesos/scheduler/TaskMonitorTest.scala |  14 +-
 .../flink/mesos/scheduler/TasksTest.scala       | 210 +++++++++++++++++++
 .../org/apache/flink/runtime/akka/FSMSpec.scala |   3 +-
 .../ContaineredJobManager.scala                 | 172 +++++++++++++++
 .../flink/yarn/YarnApplicationMasterRunner.java |  16 +-
 .../org/apache/flink/yarn/YarnJobManager.scala  | 104 +--------
 43 files changed, 1040 insertions(+), 564 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/59eeea4c/flink-mesos/src/main/java/org/apache/flink/mesos/Utils.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/Utils.java b/flink-mesos/src/main/java/org/apache/flink/mesos/Utils.java
index bd5ef50..173ae33 100644
--- a/flink-mesos/src/main/java/org/apache/flink/mesos/Utils.java
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/Utils.java
@@ -26,7 +26,7 @@ import java.util.Arrays;
 public class Utils {
 	/**
 	 * Construct a Mesos environment variable.
-     */
+	 */
 	public static Protos.Environment.Variable variable(String name, String value) {
 		return Protos.Environment.Variable.newBuilder()
 			.setName(name)
@@ -36,7 +36,7 @@ public class Utils {
 
 	/**
 	 * Construct a Mesos URI.
-     */
+	 */
 	public static Protos.CommandInfo.URI uri(URL url, boolean cacheable) {
 		return Protos.CommandInfo.URI.newBuilder()
 			.setValue(url.toExternalForm())
@@ -45,6 +45,9 @@ public class Utils {
 			.build();
 	}
 
+	/**
+	 * Construct a scalar resource value.
+	 */
 	public static Protos.Resource scalar(String name, double value) {
 		return Protos.Resource.newBuilder()
 			.setName(name)
@@ -53,10 +56,16 @@ public class Utils {
 			.build();
 	}
 
+	/**
+	 * Construct a range value.
+	 */
 	public static Protos.Value.Range range(long begin, long end) {
 		return Protos.Value.Range.newBuilder().setBegin(begin).setEnd(end).build();
 	}
 
+	/**
+	 * Construct a ranges resource value.
+	 */
 	public static Protos.Resource ranges(String name, Protos.Value.Range... ranges) {
 		return Protos.Resource.newBuilder()
 			.setName(name)

http://git-wip-us.apache.org/repos/asf/flink/blob/59eeea4c/flink-mesos/src/main/java/org/apache/flink/mesos/cli/FlinkMesosSessionCli.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/cli/FlinkMesosSessionCli.java b/flink-mesos/src/main/java/org/apache/flink/mesos/cli/FlinkMesosSessionCli.java
index 278f1a2..dcce0b8 100644
--- a/flink-mesos/src/main/java/org/apache/flink/mesos/cli/FlinkMesosSessionCli.java
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/cli/FlinkMesosSessionCli.java
@@ -26,10 +26,18 @@ import org.apache.flink.configuration.Configuration;
 import java.io.IOException;
 import java.util.Map;
 
+/**
+ * Class handling the command line interface to the Mesos session.
+ */
 public class FlinkMesosSessionCli {
 
 	private static final ObjectMapper mapper = new ObjectMapper();
 
+	/**
+	 * Decode encoded dynamic properties.
+	 * @param dynamicPropertiesEncoded encoded properties produced by the encoding method.
+	 * @return a configuration instance to be merged with the static configuration.
+	 */
 	public static Configuration decodeDynamicProperties(String dynamicPropertiesEncoded) {
 		try {
 			Configuration configuration = new Configuration();
@@ -47,6 +55,11 @@ public class FlinkMesosSessionCli {
 		}
 	}
 
+	/**
+	 * Encode dynamic properties as a string to be transported as an environment variable.
+	 * @param configuration the dynamic properties to encode.
+	 * @return a string to be decoded later.
+	 */
 	public static String encodeDynamicProperties(Configuration configuration) {
 		try {
 			String dynamicPropertiesEncoded = mapper.writeValueAsString(configuration.toMap());

http://git-wip-us.apache.org/repos/asf/flink/blob/59eeea4c/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorker.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorker.java b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorker.java
index d416ec1..5f940b5 100644
--- a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorker.java
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/LaunchableMesosWorker.java
@@ -59,7 +59,7 @@ public class LaunchableMesosWorker implements LaunchableTask {
 	 * @param params the TM parameters such as memory, cpu to acquire.
 	 * @param template a template for the TaskInfo to be constructed at launch time.
 	 * @param taskID the taskID for this worker.
-     */
+	 */
 	public LaunchableMesosWorker(MesosTaskManagerParameters params, Protos.TaskInfo.Builder template, Protos.TaskID taskID) {
 		this.params = params;
 		this.template = template;
@@ -152,8 +152,8 @@ public class LaunchableMesosWorker implements LaunchableTask {
 	 * Construct the TaskInfo needed to launch the worker.
 	 * @param slaveId the assigned slave.
 	 * @param assignment the assignment details.
-     * @return a fully-baked TaskInfo.
-     */
+	 * @return a fully-baked TaskInfo.
+	 */
 	@Override
 	public Protos.TaskInfo launch(Protos.SlaveID slaveId, TaskAssignmentResult assignment) {
 
@@ -167,10 +167,9 @@ public class LaunchableMesosWorker implements LaunchableTask {
 			.setName(taskID.getValue())
 			.addResources(scalar("cpus", assignment.getRequest().getCPUs()))
 			.addResources(scalar("mem", assignment.getRequest().getMemory()));
-		//.addResources(scalar("disk", assignment.getRequest.getDisk).setRole("Flink"))
 
 		// use the assigned ports for the TM
-		if (assignment.getAssignedPorts().size() != TM_PORT_KEYS.length) {
+		if (assignment.getAssignedPorts().size() < TM_PORT_KEYS.length) {
 			throw new IllegalArgumentException("unsufficient # of ports assigned");
 		}
 		for (int i = 0; i < TM_PORT_KEYS.length; i++) {

http://git-wip-us.apache.org/repos/asf/flink/blob/59eeea4c/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosApplicationMasterRunner.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosApplicationMasterRunner.java b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosApplicationMasterRunner.java
index 1bd02c4..c13cdf9 100644
--- a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosApplicationMasterRunner.java
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosApplicationMasterRunner.java
@@ -20,6 +20,7 @@ package org.apache.flink.mesos.runtime.clusterframework;
 
 import akka.actor.ActorRef;
 import akka.actor.ActorSystem;
+import akka.actor.Address;
 import akka.actor.Props;
 
 import org.apache.curator.framework.CuratorFramework;
@@ -69,6 +70,8 @@ import java.util.concurrent.TimeUnit;
 import static org.apache.flink.mesos.Utils.uri;
 import static org.apache.flink.mesos.Utils.variable;
 
+import static org.apache.flink.util.Preconditions.checkState;
+
 /**
  * This class is the executable entry point for the Mesos Application Master.
  * It starts actor system and the actors for {@link org.apache.flink.runtime.jobmanager.JobManager}
@@ -170,10 +173,10 @@ public class MesosApplicationMasterRunner {
 			// configuration problem occurs
 
 			final String workingDir = ENV.get(MesosConfigKeys.ENV_MESOS_SANDBOX);
-			require(workingDir != null, "Sandbox directory variable (%s) not set", MesosConfigKeys.ENV_MESOS_SANDBOX);
+			checkState(workingDir != null, "Sandbox directory variable (%s) not set", MesosConfigKeys.ENV_MESOS_SANDBOX);
 
 			final String sessionID = ENV.get(MesosConfigKeys.ENV_SESSION_ID);
-			require(sessionID != null, "Session ID (%s) not set", MesosConfigKeys.ENV_SESSION_ID);
+			checkState(sessionID != null, "Session ID (%s) not set", MesosConfigKeys.ENV_SESSION_ID);
 
 			// Note that we use the "appMasterHostname" given by the system, to make sure
 			// we use the hostnames consistently throughout akka.
@@ -232,7 +235,7 @@ public class MesosApplicationMasterRunner {
 			// JM endpoint, which should be explicitly configured by the dispatcher (based on acquired net resources)
 			final int listeningPort = config.getInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY,
 				ConfigConstants.DEFAULT_JOB_MANAGER_IPC_PORT);
-			require(listeningPort >= 0 && listeningPort <= 65536, "Config parameter \"" +
+			checkState(listeningPort >= 0 && listeningPort <= 65536, "Config parameter \"" +
 				ConfigConstants.JOB_MANAGER_IPC_PORT_KEY + "\" is invalid, it must be between 0 and 65536");
 
 			// ----------------- (2) start the actor system -------------------
@@ -241,8 +244,9 @@ public class MesosApplicationMasterRunner {
 			// using the configured address and ports
 			actorSystem = BootstrapTools.startActorSystem(config, appMasterHostname, listeningPort, LOG);
 
-			final String akkaHostname = AkkaUtils.getAddress(actorSystem).host().get();
-			final int akkaPort = (Integer) AkkaUtils.getAddress(actorSystem).port().get();
+			Address address = AkkaUtils.getAddress(actorSystem);
+			final String akkaHostname = address.host().get();
+			final int akkaPort = (Integer) address.port().get();
 
 			LOG.info("Actor system bound to hostname {}.", akkaHostname);
 
@@ -259,7 +263,7 @@ public class MesosApplicationMasterRunner {
 			LOG.debug("TaskManager configuration: {}", taskManagerConfig);
 
 			final Protos.TaskInfo.Builder taskManagerContext = createTaskManagerContext(
-				config, mesosConfig, ENV,
+				config, ENV,
 				taskManagerParameters, taskManagerConfig,
 				workingDir, getTaskManagerClass(), artifactServer, LOG);
 
@@ -315,7 +319,6 @@ public class MesosApplicationMasterRunner {
 
 			ActorRef resourceMaster = actorSystem.actorOf(resourceMasterProps, "Mesos_Resource_Master");
 
-
 			// 4: Process reapers
 			// The process reapers ensure that upon unexpected actor death, the process exits
 			// and does not stay lingering around unresponsive
@@ -334,14 +337,6 @@ public class MesosApplicationMasterRunner {
 			// make sure that everything whatever ends up in the log
 			LOG.error("Mesos JobManager initialization failed", t);
 
-			if (actorSystem != null) {
-				try {
-					actorSystem.shutdown();
-				} catch (Throwable tt) {
-					LOG.error("Error shutting down actor system", tt);
-				}
-			}
-
 			if (webMonitor != null) {
 				try {
 					webMonitor.stop();
@@ -358,6 +353,14 @@ public class MesosApplicationMasterRunner {
 				}
 			}
 
+			if (actorSystem != null) {
+				try {
+					actorSystem.shutdown();
+				} catch (Throwable tt) {
+					LOG.error("Error shutting down actor system", tt);
+				}
+			}
+
 			return INIT_ERROR_EXIT_CODE;
 		}
 
@@ -407,29 +410,14 @@ public class MesosApplicationMasterRunner {
 	}
 
 	/**
-	 * Validates a condition, throwing a RuntimeException if the condition is violated.
-	 *
-	 * @param condition The condition.
-	 * @param message The message for the runtime exception, with format variables as defined by
-	 *                {@link String#format(String, Object...)}.
-	 * @param values The format arguments.
-	 */
-	private static void require(boolean condition, String message, Object... values) {
-		if (!condition) {
-			throw new RuntimeException(String.format(message, values));
-		}
-	}
-
-	/**
 	 *
 	 * @param baseDirectory
 	 * @param additional
 	 *
 	 * @return The configuration to be used by the TaskManagers.
 	 */
-	@SuppressWarnings("deprecation")
 	private static Configuration createConfiguration(String baseDirectory, Configuration additional) {
-		LOG.info("Loading config from directory " + baseDirectory);
+		LOG.info("Loading config from directory {}", baseDirectory);
 
 		Configuration configuration = GlobalConfiguration.loadConfiguration(baseDirectory);
 
@@ -491,7 +479,7 @@ public class MesosApplicationMasterRunner {
 		return mesos;
 	}
 
-	private MesosWorkerStore createWorkerStore(Configuration flinkConfig) throws Exception {
+	private static MesosWorkerStore createWorkerStore(Configuration flinkConfig) throws Exception {
 		MesosWorkerStore workerStore;
 		RecoveryMode recoveryMode = RecoveryMode.fromConfig(flinkConfig);
 		if (recoveryMode == RecoveryMode.STANDALONE) {
@@ -523,8 +511,6 @@ public class MesosApplicationMasterRunner {
 	 *
 	 * @param flinkConfig
 	 *         The Flink configuration object.
-	 * @param mesosConfig
-	 *         The Mesos configuration object.
 	 * @param env
 	 *         The environment variables.
 	 * @param tmParams
@@ -547,7 +533,6 @@ public class MesosApplicationMasterRunner {
 	 */
 	public static Protos.TaskInfo.Builder createTaskManagerContext(
 		Configuration flinkConfig,
-		MesosConfiguration mesosConfig,
 		Map<String, String> env,
 		MesosTaskManagerParameters tmParams,
 		Configuration taskManagerConfig,
@@ -563,13 +548,13 @@ public class MesosApplicationMasterRunner {
 		log.info("Setting up artifacts for TaskManagers");
 
 		String shipListString = env.get(MesosConfigKeys.ENV_CLIENT_SHIP_FILES);
-		require(shipListString != null, "Environment variable %s not set", MesosConfigKeys.ENV_CLIENT_SHIP_FILES);
+		checkState(shipListString != null, "Environment variable %s not set", MesosConfigKeys.ENV_CLIENT_SHIP_FILES);
 
 		String clientUsername = env.get(MesosConfigKeys.ENV_CLIENT_USERNAME);
-		require(clientUsername != null, "Environment variable %s not set", MesosConfigKeys.ENV_CLIENT_USERNAME);
+		checkState(clientUsername != null, "Environment variable %s not set", MesosConfigKeys.ENV_CLIENT_USERNAME);
 
 		String classPathString = env.get(MesosConfigKeys.ENV_FLINK_CLASSPATH);
-		require(classPathString != null, "Environment variable %s not set", MesosConfigKeys.ENV_FLINK_CLASSPATH);
+		checkState(classPathString != null, "Environment variable %s not set", MesosConfigKeys.ENV_FLINK_CLASSPATH);
 
 		// register the Flink jar
 		final File flinkJarFile = new File(workingDirectory, "flink.jar");

http://git-wip-us.apache.org/repos/asf/flink/blob/59eeea4c/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosFlinkResourceManager.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosFlinkResourceManager.java b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosFlinkResourceManager.java
index 66fff7d..6b24ee8 100644
--- a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosFlinkResourceManager.java
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosFlinkResourceManager.java
@@ -96,8 +96,9 @@ public class MesosFlinkResourceManager extends FlinkResourceManager<RegisteredMe
 
 	private ActorRef reconciliationCoordinator;
 
-	private MesosWorkerStore workerStore;
+	private final MesosWorkerStore workerStore;
 
+	/** planning state related to workers - package private for unit test purposes */
 	final Map<ResourceID, MesosWorkerStore.Worker> workersInNew;
 	final Map<ResourceID, MesosWorkerStore.Worker> workersInLaunch;
 	final Map<ResourceID, MesosWorkerStore.Worker> workersBeingReturned;
@@ -250,7 +251,7 @@ public class MesosFlinkResourceManager extends FlinkResourceManager<RegisteredMe
 	 *
 	 * @param finalStatus The application status to report.
 	 * @param optionalDiagnostics An optional diagnostics message.
-     */
+	 */
 	@Override
 	protected void shutdownApplication(ApplicationStatus finalStatus, String optionalDiagnostics) {
 
@@ -264,10 +265,10 @@ public class MesosFlinkResourceManager extends FlinkResourceManager<RegisteredMe
 		}
 
 		try {
-			workerStore.cleanup();
+			workerStore.stop(true);
 		}
 		catch(Exception ex) {
-			LOG.warn("unable to cleanup the ZooKeeper state", ex);
+			LOG.warn("unable to stop the worker state store", ex);
 		}
 
 		context().stop(self());
@@ -336,7 +337,7 @@ public class MesosFlinkResourceManager extends FlinkResourceManager<RegisteredMe
 	 * Plan for some additional workers to be launched.
 	 *
 	 * @param numWorkers The number of workers to allocate.
-     */
+	 */
 	@Override
 	protected void requestNewWorkers(int numWorkers) {
 
@@ -346,7 +347,7 @@ public class MesosFlinkResourceManager extends FlinkResourceManager<RegisteredMe
 
 			// generate new workers into persistent state and launch associated actors
 			for (int i = 0; i < numWorkers; i++) {
-				MesosWorkerStore.Worker worker = MesosWorkerStore.Worker.newTask(workerStore.newTaskID());
+				MesosWorkerStore.Worker worker = MesosWorkerStore.Worker.newWorker(workerStore.newTaskID());
 				workerStore.putWorker(worker);
 				workersInNew.put(extractResourceID(worker.taskID()), worker);
 
@@ -379,7 +380,7 @@ public class MesosFlinkResourceManager extends FlinkResourceManager<RegisteredMe
 	 *
 	 * Acceptance is routed through the RM to update the persistent state before
 	 * forwarding the message to Mesos.
-     */
+	 */
 	private void acceptOffers(AcceptOffers msg) {
 
 		try {
@@ -394,7 +395,7 @@ public class MesosFlinkResourceManager extends FlinkResourceManager<RegisteredMe
 					MesosWorkerStore.Worker worker = workersInNew.remove(extractResourceID(info.getTaskId()));
 					assert (worker != null);
 
-					worker = worker.launchTask(info.getSlaveId(), msg.hostname());
+					worker = worker.launchWorker(info.getSlaveId(), msg.hostname());
 					workerStore.putWorker(worker);
 					workersInLaunch.put(extractResourceID(worker.taskID()), worker);
 
@@ -450,7 +451,7 @@ public class MesosFlinkResourceManager extends FlinkResourceManager<RegisteredMe
 	 *
 	 * @param toConsolidate The worker IDs known previously to the JobManager.
 	 * @return A collection of registered worker node records.
-     */
+	 */
 	@Override
 	protected Collection<RegisteredMesosWorkerNode> reacceptRegisteredWorkers(Collection<ResourceID> toConsolidate) {
 
@@ -497,13 +498,13 @@ public class MesosFlinkResourceManager extends FlinkResourceManager<RegisteredMe
 
 	/**
 	 * Plan for the removal of the given worker.
-     */
+	 */
 	private void releaseWorker(MesosWorkerStore.Worker worker) {
 		try {
 			LOG.info("Releasing worker {}", worker.taskID());
 
 			// update persistent state of worker to Released
-			worker = worker.releaseTask();
+			worker = worker.releaseWorker();
 			workerStore.putWorker(worker);
 			workersBeingReturned.put(extractResourceID(worker.taskID()), worker);
 			taskRouter.tell(new TaskMonitor.TaskGoalStateUpdated(extractGoalState(worker)), self());
@@ -580,20 +581,26 @@ public class MesosFlinkResourceManager extends FlinkResourceManager<RegisteredMe
 
 	/**
 	 * Invoked when a Mesos task reaches a terminal status.
-     */
+	 */
 	private void taskTerminated(Protos.TaskID taskID, Protos.TaskStatus status) {
 		// this callback occurs for failed containers and for released containers alike
 
 		final ResourceID id = extractResourceID(taskID);
 
+		boolean existed;
 		try {
-			workerStore.removeWorker(taskID);
+			existed = workerStore.removeWorker(taskID);
 		}
 		catch(Exception ex) {
 			fatalError("unable to remove worker", ex);
 			return;
 		}
 
+		if(!existed) {
+			LOG.info("Received a termination notice for an unrecognized worker: {}", id);
+			return;
+		}
+
 		// check if this is a failed task or a released task
 		if (workersBeingReturned.remove(id) != null) {
 			// regular finished worker that we released
@@ -624,7 +631,7 @@ public class MesosFlinkResourceManager extends FlinkResourceManager<RegisteredMe
 			sendInfoMessage(diagMessage);
 
 			LOG.info(diagMessage);
-			LOG.info("Total number of failed tasks so far: " + failedTasksSoFar);
+			LOG.info("Total number of failed tasks so far: {}", failedTasksSoFar);
 
 			// maxFailedTasks == -1 is infinite number of retries.
 			if (maxFailedTasks >= 0 && failedTasksSoFar > maxFailedTasks) {
@@ -672,20 +679,20 @@ public class MesosFlinkResourceManager extends FlinkResourceManager<RegisteredMe
 	 * Extracts the Mesos task goal state from the worker information.
 	 * @param worker the persistent worker information.
 	 * @return goal state information for the {@Link TaskMonitor}.
-     */
+	 */
 	static TaskMonitor.TaskGoalState extractGoalState(MesosWorkerStore.Worker worker) {
 		switch(worker.state()) {
 			case New: return new TaskMonitor.New(worker.taskID());
 			case Launched: return new TaskMonitor.Launched(worker.taskID(), worker.slaveID().get());
 			case Released: return new TaskMonitor.Released(worker.taskID(), worker.slaveID().get());
-			default: throw new IllegalArgumentException();
+			default: throw new IllegalArgumentException("unsupported worker state");
 		}
 	}
 
 	/**
 	 * Creates the Fenzo optimizer (builder).
-	 * The builder is an indirection to faciliate unit testing of the Launch Coordinator.
-     */
+	 * The builder is an indirection to facilitate unit testing of the Launch Coordinator.
+	 */
 	private static TaskSchedulerBuilder createOptimizer() {
 		return new TaskSchedulerBuilder() {
 			TaskScheduler.Builder builder = new TaskScheduler.Builder();

http://git-wip-us.apache.org/repos/asf/flink/blob/59eeea4c/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManagerParameters.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManagerParameters.java b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManagerParameters.java
index 5321ab6..1b19d08 100644
--- a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManagerParameters.java
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManagerParameters.java
@@ -24,6 +24,12 @@ import org.apache.flink.runtime.clusterframework.ContaineredTaskManagerParameter
 
 import static java.util.Objects.requireNonNull;
 
+/**
+ * This class describes the Mesos-specific parameters for launching a TaskManager process.
+ *
+ * These parameters are in addition to the common parameters
+ * provided by {@link ContaineredTaskManagerParameters}.
+ */
 public class MesosTaskManagerParameters {
 
 	private double cpus;
@@ -36,10 +42,16 @@ public class MesosTaskManagerParameters {
 		this.containeredParameters = containeredParameters;
 	}
 
+	/**
+	 * Get the CPU units to use for the TaskManager process.
+     */
 	public double cpus() {
 		return cpus;
 	}
 
+	/**
+	 * Get the common containered parameters.
+     */
 	public ContaineredTaskManagerParameters containeredParameters() {
 		return containeredParameters;
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/59eeea4c/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManagerRunner.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManagerRunner.java b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManagerRunner.java
index e34def6..94a9e99 100644
--- a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManagerRunner.java
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManagerRunner.java
@@ -63,7 +63,7 @@ public class MesosTaskManagerRunner {
 			configuration.addAll(dynamicProperties);
 		}
 		catch (Throwable t) {
-			LOG.error(t.getMessage(), t);
+			LOG.error("Failed to load the TaskManager configuration and dynamic properties.", t);
 			System.exit(TaskManager.STARTUP_FAILURE_RETURN_CODE());
 			return;
 		}
@@ -77,15 +77,15 @@ public class MesosTaskManagerRunner {
 		String flinkTempDirs = configuration.getString(ConfigConstants.TASK_MANAGER_TMP_DIR_KEY, null);
 		if (flinkTempDirs != null) {
 			LOG.info("Overriding Mesos temporary file directories with those " +
-				"specified in the Flink config: " + flinkTempDirs);
+				"specified in the Flink config: {}", flinkTempDirs);
 		}
 		else if (tmpDirs != null) {
-			LOG.info("Setting directories for temporary files to: " + tmpDirs);
+			LOG.info("Setting directories for temporary files to: {}", tmpDirs);
 			configuration.setString(ConfigConstants.TASK_MANAGER_TMP_DIR_KEY, tmpDirs);
 		}
 
-		LOG.info("Mesos task runs as '" + UserGroupInformation.getCurrentUser().getShortUserName() +
-			"', setting user to execute Flink TaskManager to '" + effectiveUsername + "'");
+		LOG.info("Mesos task runs as '{}', setting user to execute Flink TaskManager to '{}'",
+			UserGroupInformation.getCurrentUser().getShortUserName(), effectiveUsername);
 
 		// tell akka to die in case of an error
 		configuration.setBoolean(ConfigConstants.AKKA_JVM_EXIT_ON_FATAL_ERROR, true);

http://git-wip-us.apache.org/repos/asf/flink/blob/59eeea4c/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/store/MesosWorkerStore.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/store/MesosWorkerStore.java b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/store/MesosWorkerStore.java
index 2671fc1..d6ff6bc 100644
--- a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/store/MesosWorkerStore.java
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/store/MesosWorkerStore.java
@@ -30,47 +30,75 @@ import static java.util.Objects.requireNonNull;
 
 /**
  * A store of Mesos workers and associated framework information.
- *
- * Generates a framework ID as necessary.
  */
 public interface MesosWorkerStore {
 
-	static final DecimalFormat TASKID_FORMAT = new DecimalFormat("taskmanager-00000");
+	/**
+	 * The template for naming the worker.
+	 */
+	DecimalFormat TASKID_FORMAT = new DecimalFormat("taskmanager-00000");
 
+	/**
+	 * Start the worker store.
+	 */
 	void start() throws Exception;
 
-	void stop() throws Exception;
+	/**
+	 * Stop the worker store.
+	 * @param cleanup if true, cleanup any stored information.
+	 */
+	void stop(boolean cleanup) throws Exception;
 
+	/**
+	 * Get the stored Mesos framework ID.
+	 */
 	Option<Protos.FrameworkID> getFrameworkID() throws Exception;
 
+	/**
+	 * Set the stored Mesos framework ID.
+	 */
 	void setFrameworkID(Option<Protos.FrameworkID> frameworkID) throws Exception;
 
+	/**
+	 * Recover the stored workers.
+	 */
 	List<Worker> recoverWorkers() throws Exception;
 
+	/**
+	 * Generate a new task ID for a worker.
+	 */
 	Protos.TaskID newTaskID() throws Exception;
 
+	/**
+	 * Put a worker into storage.
+	 */
 	void putWorker(Worker worker) throws Exception;
 
-	void removeWorker(Protos.TaskID taskID) throws Exception;
-
-	void cleanup() throws Exception;
+	/**
+	 * Remove a worker from storage.
+	 * @return true if the worker existed.
+	 */
+	boolean removeWorker(Protos.TaskID taskID) throws Exception;
 
 	/**
-	 * A stored task.
+	 * A stored worker.
 	 *
 	 * The assigned slaveid/hostname is valid in Launched and Released states.  The hostname is needed
 	 * by Fenzo for optimization purposes.
 	 */
 	class Worker implements Serializable {
-		private Protos.TaskID taskID;
 
-		private Option<Protos.SlaveID> slaveID;
+		private static final long serialVersionUID = 1L;
+
+		private final Protos.TaskID taskID;
 
-		private Option<String> hostname;
+		private final Option<Protos.SlaveID> slaveID;
 
-		private TaskState state;
+		private final Option<String> hostname;
 
-		public Worker(Protos.TaskID taskID, Option<Protos.SlaveID> slaveID, Option<String> hostname, TaskState state) {
+		private final WorkerState state;
+
+		private Worker(Protos.TaskID taskID, Option<Protos.SlaveID> slaveID, Option<String> hostname, WorkerState state) {
 			requireNonNull(taskID, "taskID");
 			requireNonNull(slaveID, "slaveID");
 			requireNonNull(hostname, "hostname");
@@ -82,37 +110,61 @@ public interface MesosWorkerStore {
 			this.state = state;
 		}
 
+		/**
+		 * Get the worker's task ID.
+         */
 		public Protos.TaskID taskID() {
 			return taskID;
 		}
 
+		/**
+		 * Get the worker's assigned slave ID.
+         */
 		public Option<Protos.SlaveID> slaveID() {
 			return slaveID;
 		}
 
+		/**
+		 * Get the worker's assigned hostname.
+         */
 		public Option<String> hostname() {
 			return hostname;
 		}
 
-		public TaskState state() {
+		/**
+		 * Get the worker's state.
+         */
+		public WorkerState state() {
 			return state;
 		}
 
 		// valid transition methods
 
-		public static Worker newTask(Protos.TaskID taskID) {
+		/**
+		 * Create a new worker with the given taskID.
+		 * @return a new worker instance.
+		 */
+		public static Worker newWorker(Protos.TaskID taskID) {
 			return new Worker(
 				taskID,
 				Option.<Protos.SlaveID>empty(), Option.<String>empty(),
-				TaskState.New);
+				WorkerState.New);
 		}
 
-		public Worker launchTask(Protos.SlaveID slaveID, String hostname) {
-			return new Worker(taskID, Option.apply(slaveID), Option.apply(hostname), TaskState.Launched);
+		/**
+		 * Transition the worker to a launched state.
+		 * @return a new worker instance (does not mutate the current instance).
+		 */
+		public Worker launchWorker(Protos.SlaveID slaveID, String hostname) {
+			return new Worker(taskID, Option.apply(slaveID), Option.apply(hostname), WorkerState.Launched);
 		}
 
-		public Worker releaseTask() {
-			return new Worker(taskID, slaveID, hostname, TaskState.Released);
+		/**
+		 * Transition the worker to a released state.
+		 * @return a new worker instance (does not mutate the current instance).
+		 */
+		public Worker releaseWorker() {
+			return new Worker(taskID, slaveID, hostname, WorkerState.Released);
 		}
 
 		@Override
@@ -125,14 +177,14 @@ public interface MesosWorkerStore {
 			}
 			Worker worker = (Worker) o;
 			return Objects.equals(taskID, worker.taskID) &&
-				Objects.equals(slaveID.isDefined() ? slaveID.get() : null, worker.slaveID.isDefined() ? worker.slaveID.get() : null) &&
-				Objects.equals(hostname.isDefined() ? hostname.get() : null, worker.hostname.isDefined() ? worker.hostname.get() : null) &&
+				Objects.equals(slaveID, worker.slaveID) &&
+				Objects.equals(hostname, worker.hostname) &&
 				state == worker.state;
 		}
 
 		@Override
 		public int hashCode() {
-			return Objects.hash(taskID, slaveID.isDefined() ? slaveID.get() : null, hostname.isDefined() ? hostname.get() : null, state);
+			return Objects.hash(taskID, slaveID, hostname, state);
 		}
 
 		@Override
@@ -146,7 +198,24 @@ public interface MesosWorkerStore {
 		}
 	}
 
-	enum TaskState {
-		New,Launched,Released
+	/**
+	 * The (planned) state of the worker.
+	 */
+	enum WorkerState {
+
+		/**
+		 * Indicates that the worker is new (not yet launched).
+		 */
+		New,
+
+		/**
+		 * Indicates that the worker is launched.
+		 */
+		Launched,
+
+		/**
+		 * Indicates that the worker is released.
+		 */
+		Released
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/59eeea4c/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/store/StandaloneMesosWorkerStore.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/store/StandaloneMesosWorkerStore.java b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/store/StandaloneMesosWorkerStore.java
index 9dcc6c0..55e7e28 100644
--- a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/store/StandaloneMesosWorkerStore.java
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/store/StandaloneMesosWorkerStore.java
@@ -46,7 +46,7 @@ public class StandaloneMesosWorkerStore implements MesosWorkerStore {
 	}
 
 	@Override
-	public void stop() throws Exception {
+	public void stop(boolean cleanup) throws Exception {
 
 	}
 
@@ -77,11 +77,8 @@ public class StandaloneMesosWorkerStore implements MesosWorkerStore {
 	}
 
 	@Override
-	public void removeWorker(Protos.TaskID taskID) throws Exception {
-		storedWorkers.remove(taskID);
-	}
-
-	@Override
-	public void cleanup() throws Exception {
+	public boolean removeWorker(Protos.TaskID taskID) throws Exception {
+		Worker prior = storedWorkers.remove(taskID);
+		return prior != null;
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/59eeea4c/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/store/ZooKeeperMesosWorkerStore.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/store/ZooKeeperMesosWorkerStore.java b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/store/ZooKeeperMesosWorkerStore.java
index 7c689b8..45553d4 100644
--- a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/store/ZooKeeperMesosWorkerStore.java
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/store/ZooKeeperMesosWorkerStore.java
@@ -50,7 +50,10 @@ public class ZooKeeperMesosWorkerStore implements MesosWorkerStore {
 
 	private static final Logger LOG = LoggerFactory.getLogger(ZooKeeperMesosWorkerStore.class);
 
-	private final Object cacheLock = new Object();
+	private final Object startStopLock = new Object();
+
+	/** Root store path in ZK. */
+	private final String storePath;
 
 	/** Client (not a namespace facade) */
 	private final CuratorFramework client;
@@ -79,6 +82,7 @@ public class ZooKeeperMesosWorkerStore implements MesosWorkerStore {
 		// Keep a reference to the original client and not the namespace facade. The namespace
 		// facade cannot be closed.
 		this.client = checkNotNull(client, "client");
+		this.storePath = storePath;
 
 		// All operations will have the given path as root
 		client.newNamespaceAwareEnsurePath(storePath).ensure(client.getZookeeperClient());
@@ -94,6 +98,7 @@ public class ZooKeeperMesosWorkerStore implements MesosWorkerStore {
 		facade.newNamespaceAwareEnsurePath("/workers").ensure(client.getZookeeperClient());
 		CuratorFramework storeFacade = client.usingNamespace(facade.getNamespace() + "/workers");
 
+		// using late-binding as a workaround for shaded curator dependency of flink-runtime.
 		this.workersInZooKeeper = ZooKeeperStateHandleStore.class
 			.getConstructor(CuratorFramework.class, StateStorageHelper.class)
 			.newInstance(storeFacade, stateStorage);
@@ -101,7 +106,7 @@ public class ZooKeeperMesosWorkerStore implements MesosWorkerStore {
 
 	@Override
 	public void start() throws Exception {
-		synchronized (cacheLock) {
+		synchronized (startStopLock) {
 			if (!isRunning) {
 				isRunning = true;
 				frameworkIdInZooKeeper.start();
@@ -110,11 +115,17 @@ public class ZooKeeperMesosWorkerStore implements MesosWorkerStore {
 		}
 	}
 
-	public void stop() throws Exception {
-		synchronized (cacheLock) {
+	public void stop(boolean cleanup) throws Exception {
+		synchronized (startStopLock) {
 			if (isRunning) {
 				frameworkIdInZooKeeper.close();
 				totalTaskCountInZooKeeper.close();
+
+				if(cleanup) {
+					workersInZooKeeper.removeAndDiscardAllState();
+					client.delete().deletingChildrenIfNeeded().forPath(storePath);
+				}
+
 				client.close();
 				isRunning = false;
 			}
@@ -132,10 +143,10 @@ public class ZooKeeperMesosWorkerStore implements MesosWorkerStore {
 	 * Get the persisted framework ID.
 	 * @return the current ID or empty if none is yet persisted.
 	 * @throws Exception on ZK failures, interruptions.
-     */
+	 */
 	@Override
 	public Option<Protos.FrameworkID> getFrameworkID() throws Exception {
-		synchronized (cacheLock) {
+		synchronized (startStopLock) {
 			verifyIsRunning();
 
 			Option<Protos.FrameworkID> frameworkID;
@@ -154,10 +165,10 @@ public class ZooKeeperMesosWorkerStore implements MesosWorkerStore {
 	 * Update the persisted framework ID.
 	 * @param frameworkID the new ID or empty to remove the persisted ID.
 	 * @throws Exception on ZK failures, interruptions.
-     */
+	 */
 	@Override
 	public void setFrameworkID(Option<Protos.FrameworkID> frameworkID) throws Exception {
-		synchronized (cacheLock) {
+		synchronized (startStopLock) {
 			verifyIsRunning();
 
 			byte[] value = frameworkID.isDefined() ? frameworkID.get().getValue().getBytes() : new byte[0];
@@ -170,7 +181,7 @@ public class ZooKeeperMesosWorkerStore implements MesosWorkerStore {
 	 */
 	@Override
 	public Protos.TaskID newTaskID() throws Exception {
-		synchronized (cacheLock) {
+		synchronized (startStopLock) {
 			verifyIsRunning();
 
 			int nextCount;
@@ -189,7 +200,7 @@ public class ZooKeeperMesosWorkerStore implements MesosWorkerStore {
 
 	@Override
 	public List<MesosWorkerStore.Worker> recoverWorkers() throws Exception {
-		synchronized (cacheLock) {
+		synchronized (startStopLock) {
 			verifyIsRunning();
 
 			List<Tuple2<StateHandle<MesosWorkerStore.Worker>, String>> handles = workersInZooKeeper.getAll();
@@ -216,7 +227,7 @@ public class ZooKeeperMesosWorkerStore implements MesosWorkerStore {
 		checkNotNull(worker, "worker");
 		String path = getPathForWorker(worker.taskID());
 
-		synchronized (cacheLock) {
+		synchronized (startStopLock) {
 			verifyIsRunning();
 
 			int currentVersion = workersInZooKeeper.exists(path);
@@ -239,22 +250,23 @@ public class ZooKeeperMesosWorkerStore implements MesosWorkerStore {
 	}
 
 	@Override
-	public void removeWorker(Protos.TaskID taskID) throws Exception {
+	public boolean removeWorker(Protos.TaskID taskID) throws Exception {
 		checkNotNull(taskID, "taskID");
 		String path = getPathForWorker(taskID);
-		synchronized (cacheLock) {
+		synchronized (startStopLock) {
 			verifyIsRunning();
 
-			workersInZooKeeper.remove(path);
+			if(workersInZooKeeper.exists(path) == -1) {
+				LOG.debug("No such worker {} in ZooKeeper.", taskID);
+				return false;
+			}
+
+			workersInZooKeeper.removeAndDiscardState(path);
 			LOG.debug("Removed worker {} from ZooKeeper.", taskID);
+			return true;
 		}
 	}
 
-	@Override
-	public void cleanup() throws Exception {
-		// TODO
-	}
-
 	/**
 	 * Get the ZK path for the given task ID (with leading slash).
 	 */
@@ -269,7 +281,7 @@ public class ZooKeeperMesosWorkerStore implements MesosWorkerStore {
 	 * @param configuration the Flink configuration.
 	 * @return a worker store.
 	 * @throws Exception
-     */
+	 */
 	public static ZooKeeperMesosWorkerStore createMesosWorkerStore(
 			CuratorFramework client,
 			Configuration configuration) throws Exception {

http://git-wip-us.apache.org/repos/asf/flink/blob/59eeea4c/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/SchedulerProxy.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/SchedulerProxy.java b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/SchedulerProxy.java
index 3411771..b6d3383 100644
--- a/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/SchedulerProxy.java
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/SchedulerProxy.java
@@ -22,7 +22,8 @@ import akka.actor.ActorRef;
 
 import org.apache.flink.mesos.scheduler.messages.Disconnected;
 import org.apache.flink.mesos.scheduler.messages.Error;
-import org.apache.flink.mesos.scheduler.messages.Error;
+import org.apache.flink.mesos.scheduler.messages.ExecutorLost;
+import org.apache.flink.mesos.scheduler.messages.FrameworkMessage;
 import org.apache.flink.mesos.scheduler.messages.OfferRescinded;
 import org.apache.flink.mesos.scheduler.messages.ReRegistered;
 import org.apache.flink.mesos.scheduler.messages.Registered;
@@ -46,7 +47,7 @@ import java.util.List;
 public class SchedulerProxy implements Scheduler {
 
 	/** The actor to which we report the callbacks */
-	private ActorRef mesosActor;
+	private final ActorRef mesosActor;
 
 	public SchedulerProxy(ActorRef mesosActor) {
 		this.mesosActor = mesosActor;
@@ -67,7 +68,6 @@ public class SchedulerProxy implements Scheduler {
 		mesosActor.tell(new Disconnected(), ActorRef.noSender());
 	}
 
-
 	@Override
 	public void resourceOffers(SchedulerDriver driver, List<Protos.Offer> offers) {
 		mesosActor.tell(new ResourceOffers(offers), ActorRef.noSender());
@@ -85,7 +85,7 @@ public class SchedulerProxy implements Scheduler {
 
 	@Override
 	public void frameworkMessage(SchedulerDriver driver, Protos.ExecutorID executorId, Protos.SlaveID slaveId, byte[] data) {
-		throw new UnsupportedOperationException("frameworkMessage is unexpected");
+		mesosActor.tell(new FrameworkMessage(executorId, slaveId, data), ActorRef.noSender());
 	}
 
 	@Override
@@ -95,7 +95,7 @@ public class SchedulerProxy implements Scheduler {
 
 	@Override
 	public void executorLost(SchedulerDriver driver, Protos.ExecutorID executorId, Protos.SlaveID slaveId, int status) {
-		throw new UnsupportedOperationException("executorLost is unexpected");
+		mesosActor.tell(new ExecutorLost(executorId, slaveId, status), ActorRef.noSender());
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/59eeea4c/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/TaskSchedulerBuilder.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/TaskSchedulerBuilder.java b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/TaskSchedulerBuilder.java
index 80e572d..c841e22 100644
--- a/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/TaskSchedulerBuilder.java
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/TaskSchedulerBuilder.java
@@ -28,7 +28,14 @@ import com.netflix.fenzo.functions.Action1;
  * Note that the Fenzo-provided {@link TaskScheduler.Builder} cannot be mocked, which motivates this interface.
  */
 public interface TaskSchedulerBuilder {
+
+	/**
+	 * Set the callback action for rejecting a lease.
+	 */
 	TaskSchedulerBuilder withLeaseRejectAction(Action1<VirtualMachineLease> action);
 
+	/**
+	 * Build a Fenzo task scheduler.
+	 */
 	TaskScheduler build();
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/59eeea4c/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/AcceptOffers.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/AcceptOffers.java b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/AcceptOffers.java
index 0404f07..dac2803 100644
--- a/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/AcceptOffers.java
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/AcceptOffers.java
@@ -20,17 +20,20 @@ package org.apache.flink.mesos.scheduler.messages;
 
 import org.apache.mesos.Protos;
 
+import java.io.Serializable;
 import java.util.Collection;
 
 /**
  * Local message sent by the launch coordinator to the scheduler to accept offers.
  */
-public class AcceptOffers {
+public class AcceptOffers implements Serializable {
 
-	private String hostname;
-	private Collection<Protos.OfferID> offerIds;
-	private Collection<Protos.Offer.Operation> operations;
-	private Protos.Filters filters;
+	private static final long serialVersionUID = 1L;
+
+	private final String hostname;
+	private final Collection<Protos.OfferID> offerIds;
+	private final Collection<Protos.Offer.Operation> operations;
+	private final Protos.Filters filters;
 
 	public AcceptOffers(String hostname, Collection<Protos.OfferID> offerIds, Collection<Protos.Offer.Operation> operations) {
 		this.hostname = hostname;

http://git-wip-us.apache.org/repos/asf/flink/blob/59eeea4c/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/Connected.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/Connected.java b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/Connected.java
index b51e761..20cd059 100644
--- a/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/Connected.java
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/Connected.java
@@ -18,9 +18,12 @@
 
 package org.apache.flink.mesos.scheduler.messages;
 
+import java.io.Serializable;
+
 /**
  * Message sent by the callback handler to the scheduler actor
  * when the scheduler becomes "connected" to the master.
  */
-public abstract class Connected {
+public abstract class Connected implements Serializable {
+	private static final long serialVersionUID = 1L;
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/59eeea4c/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/Disconnected.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/Disconnected.java b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/Disconnected.java
index 22664b7..df0f0d9 100644
--- a/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/Disconnected.java
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/Disconnected.java
@@ -18,11 +18,16 @@
 
 package org.apache.flink.mesos.scheduler.messages;
 
+import java.io.Serializable;
+
 /**
  * Message sent by the callback handler to the scheduler actor
  * when the scheduler becomes "disconnected" from the master (e.g., the master fails and another is taking over).
  */
-public class Disconnected {
+public class Disconnected implements Serializable {
+
+	private static final long serialVersionUID = 1L;
+
 	@Override
 	public String toString() {
 		return "Disconnected{}";

http://git-wip-us.apache.org/repos/asf/flink/blob/59eeea4c/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/Error.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/Error.java b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/Error.java
index 83508c1..83896fd 100644
--- a/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/Error.java
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/Error.java
@@ -18,12 +18,17 @@
 
 package org.apache.flink.mesos.scheduler.messages;
 
+import java.io.Serializable;
+
 /**
  * Message sent when there is an unrecoverable error in the scheduler or
  * driver. The driver will be aborted BEFORE invoking this callback.
  */
-public class Error {
-	private String message;
+public class Error implements Serializable {
+
+	private static final long serialVersionUID = 1L;
+
+	private final String message;
 
 	public Error(String message) {
 		this.message = message;

http://git-wip-us.apache.org/repos/asf/flink/blob/59eeea4c/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/ExecutorLost.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/ExecutorLost.java b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/ExecutorLost.java
new file mode 100644
index 0000000..9cb2e30
--- /dev/null
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/ExecutorLost.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.mesos.scheduler.messages;
+
+import org.apache.mesos.Protos;
+
+import java.io.Serializable;
+
+/**
+ * Message sent when an executor has exited/terminated. Note that any
+ * tasks running will have TASK_LOST status updates automagically
+ * generated.
+ */
+public class ExecutorLost implements Serializable {
+
+	private static final long serialVersionUID = 1L;
+
+	private final Protos.ExecutorID executorId;
+
+	private final Protos.SlaveID slaveId;
+
+	private final int status;
+
+	public ExecutorLost(Protos.ExecutorID executorId, Protos.SlaveID slaveId, int status) {
+		this.executorId = executorId;
+		this.slaveId = slaveId;
+		this.status = status;
+	}
+
+	public Protos.ExecutorID executorId() {
+		return executorId;
+	}
+
+	public Protos.SlaveID slaveId() {
+		return slaveId;
+	}
+
+	public int status() {
+		return status;
+	}
+
+	@Override
+	public String toString() {
+		return "ExecutorLost{" +
+			"executorId=" + executorId +
+			", slaveId=" + slaveId +
+			", status=" + status +
+			'}';
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/59eeea4c/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/FrameworkMessage.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/FrameworkMessage.java b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/FrameworkMessage.java
new file mode 100644
index 0000000..ed74300
--- /dev/null
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/FrameworkMessage.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.mesos.scheduler.messages;
+
+import org.apache.mesos.Protos;
+
+import java.io.Serializable;
+import java.util.Arrays;
+
+/**
+ * Message sent when an executor sends a message. These messages are best
+ * effort; do not expect a framework message to be retransmitted in
+ * any reliable fashion.
+ */
+public class FrameworkMessage implements Serializable {
+
+	private static final long serialVersionUID = 1L;
+
+	private final Protos.ExecutorID executorId;
+
+	private final Protos.SlaveID slaveId;
+
+	private final byte[] data;
+
+	public FrameworkMessage(Protos.ExecutorID executorId, Protos.SlaveID slaveId, byte[] data) {
+		this.executorId = executorId;
+		this.slaveId = slaveId;
+		this.data = data;
+	}
+
+	public Protos.ExecutorID executorId() {
+		return executorId;
+	}
+
+	public Protos.SlaveID slaveId() {
+		return slaveId;
+	}
+
+	public byte[] data() {
+		return data;
+	}
+
+	@Override
+	public String toString() {
+		return "FrameworkMessage{" +
+			"executorId=" + executorId +
+			", slaveId=" + slaveId +
+			", data=" + Arrays.toString(data) +
+			'}';
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/59eeea4c/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/OfferRescinded.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/OfferRescinded.java b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/OfferRescinded.java
index 6ae74c4..1b39b66 100644
--- a/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/OfferRescinded.java
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/OfferRescinded.java
@@ -20,12 +20,17 @@ package org.apache.flink.mesos.scheduler.messages;
 
 import org.apache.mesos.Protos;
 
+import java.io.Serializable;
+
 /**
  * Message sent by the callback handler to the scheduler actor
  * when an offer is no longer valid (e.g., the slave was lost or another framework used resources in the offer).
  */
-public class OfferRescinded {
-	private Protos.OfferID offerId;
+public class OfferRescinded implements Serializable {
+
+	private static final long serialVersionUID = 1L;
+
+	private final Protos.OfferID offerId;
 
 	public OfferRescinded(Protos.OfferID offerId) {
 		this.offerId = offerId;

http://git-wip-us.apache.org/repos/asf/flink/blob/59eeea4c/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/ReRegistered.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/ReRegistered.java b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/ReRegistered.java
index 116fc89..71aa8d6 100644
--- a/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/ReRegistered.java
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/ReRegistered.java
@@ -27,7 +27,10 @@ import static java.util.Objects.requireNonNull;
  * when the scheduler re-registers with a newly elected Mesos master.
  */
 public class ReRegistered extends Connected {
-	private Protos.MasterInfo masterInfo;
+
+	private static final long serialVersionUID = 1L;
+
+	private final Protos.MasterInfo masterInfo;
 
 	public ReRegistered(Protos.MasterInfo masterInfo) {
 		requireNonNull(masterInfo);

http://git-wip-us.apache.org/repos/asf/flink/blob/59eeea4c/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/Registered.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/Registered.java b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/Registered.java
index c2ce214..f38e68a 100644
--- a/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/Registered.java
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/Registered.java
@@ -28,8 +28,10 @@ import static java.util.Objects.requireNonNull;
  */
 public class Registered extends Connected {
 
-	private org.apache.mesos.Protos.FrameworkID frameworkId;
-	private Protos.MasterInfo masterInfo;
+	private static final long serialVersionUID = 1L;
+
+	private final Protos.FrameworkID frameworkId;
+	private final Protos.MasterInfo masterInfo;
 
 	public Registered(Protos.FrameworkID frameworkId, Protos.MasterInfo masterInfo) {
 		requireNonNull(frameworkId);

http://git-wip-us.apache.org/repos/asf/flink/blob/59eeea4c/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/ResourceOffers.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/ResourceOffers.java b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/ResourceOffers.java
index f9d21ea..dc5111d 100644
--- a/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/ResourceOffers.java
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/ResourceOffers.java
@@ -20,6 +20,7 @@ package org.apache.flink.mesos.scheduler.messages;
 
 import org.apache.mesos.Protos;
 
+import java.io.Serializable;
 import java.util.List;
 import static java.util.Objects.requireNonNull;
 
@@ -27,8 +28,11 @@ import static java.util.Objects.requireNonNull;
  * Message sent by the callback handler to the scheduler actor
  * when resources have been offered to this framework.
  */
-public class ResourceOffers {
-	private List<Protos.Offer> offers;
+public class ResourceOffers implements Serializable {
+
+	private static final long serialVersionUID = 1L;
+
+	private final List<Protos.Offer> offers;
 
 	public ResourceOffers(List<Protos.Offer> offers) {
 		requireNonNull(offers);

http://git-wip-us.apache.org/repos/asf/flink/blob/59eeea4c/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/SlaveLost.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/SlaveLost.java b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/SlaveLost.java
index 9255e87..a2730c4 100644
--- a/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/SlaveLost.java
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/SlaveLost.java
@@ -20,12 +20,17 @@ package org.apache.flink.mesos.scheduler.messages;
 
 import org.apache.mesos.Protos;
 
+import java.io.Serializable;
+
 /**
  * Message sent by the callback handler to the scheduler actor
  * when a slave has been determined unreachable (e.g., machine failure, network partition).
  */
-public class SlaveLost {
-	private Protos.SlaveID slaveId;
+public class SlaveLost implements Serializable {
+
+	private static final long serialVersionUID = 1L;
+
+	private final Protos.SlaveID slaveId;
 
 	public SlaveLost(Protos.SlaveID slaveId) {
 		this.slaveId = slaveId;

http://git-wip-us.apache.org/repos/asf/flink/blob/59eeea4c/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/StatusUpdate.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/StatusUpdate.java b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/StatusUpdate.java
index 18c9a00..65a2f1b 100644
--- a/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/StatusUpdate.java
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/scheduler/messages/StatusUpdate.java
@@ -20,13 +20,18 @@ package org.apache.flink.mesos.scheduler.messages;
 
 import org.apache.mesos.Protos;
 
+import java.io.Serializable;
+
 /**
  * Message sent by the callback handler to the scheduler actor
  * when the status of a task has changed (e.g., a slave is lost and so the task is lost,
  * a task finishes and an executor sends a status update saying so, etc).
  */
-public class StatusUpdate {
-	private Protos.TaskStatus status;
+public class StatusUpdate implements Serializable {
+
+	private static final long serialVersionUID = 1L;
+
+	private final Protos.TaskStatus status;
 
 	public StatusUpdate(Protos.TaskStatus status) {
 		this.status = status;

http://git-wip-us.apache.org/repos/asf/flink/blob/59eeea4c/flink-mesos/src/main/java/org/apache/flink/mesos/util/MesosArtifactServer.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/util/MesosArtifactServer.java b/flink-mesos/src/main/java/org/apache/flink/mesos/util/MesosArtifactServer.java
index 63bb7b1..6547cb3 100644
--- a/flink-mesos/src/main/java/org/apache/flink/mesos/util/MesosArtifactServer.java
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/util/MesosArtifactServer.java
@@ -125,12 +125,12 @@ public class MesosArtifactServer {
 
 		baseURL = new URL("http", serverHostname, port, "/" + sessionID + "/");
 
-		LOG.info("Mesos artifact server listening at " + address + ':' + port);
+		LOG.info("Mesos artifact server listening at {}:{}", address, port);
 	}
 
 	/**
 	 * Get the server port on which the artifact server is listening.
-     */
+	 */
 	public synchronized int getServerPort() {
 		Channel server = this.serverChannel;
 		if (server != null) {
@@ -149,7 +149,7 @@ public class MesosArtifactServer {
 	 * @param remoteFile the remote path with which to locate the file.
 	 * @return the fully-qualified remote path to the file.
 	 * @throws MalformedURLException if the remote path is invalid.
-     */
+	 */
 	public synchronized URL addFile(File localFile, String remoteFile) throws MalformedURLException {
 		URL fileURL = new URL(baseURL, remoteFile);
 		router.ANY(fileURL.getPath(), new VirtualFileServerHandler(localFile));
@@ -159,7 +159,7 @@ public class MesosArtifactServer {
 	/**
 	 * Stops the artifact server.
 	 * @throws Exception
-     */
+	 */
 	public synchronized void stop() throws Exception {
 		if (this.serverChannel != null) {
 			this.serverChannel.close().awaitUninterruptibly();
@@ -179,7 +179,7 @@ public class MesosArtifactServer {
 	@ChannelHandler.Sharable
 	public static class VirtualFileServerHandler extends SimpleChannelInboundHandler<Routed> {
 
-		private File file;
+		private final File file;
 
 		public VirtualFileServerHandler(File file) {
 			this.file = file;
@@ -194,7 +194,7 @@ public class MesosArtifactServer {
 			HttpRequest request = routed.request();
 
 			if (LOG.isDebugEnabled()) {
-				LOG.debug(request.getMethod() + " request for file '" + file.getAbsolutePath() + '\'');
+				LOG.debug("{} request for file '{}'", request.getMethod(), file.getAbsolutePath());
 			}
 
 			if(!(request.getMethod() == GET || request.getMethod() == HEAD)) {

http://git-wip-us.apache.org/repos/asf/flink/blob/59eeea4c/flink-mesos/src/main/java/org/apache/flink/mesos/util/MesosConfiguration.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/util/MesosConfiguration.java b/flink-mesos/src/main/java/org/apache/flink/mesos/util/MesosConfiguration.java
index 7961995..40dc41c 100644
--- a/flink-mesos/src/main/java/org/apache/flink/mesos/util/MesosConfiguration.java
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/util/MesosConfiguration.java
@@ -27,25 +27,27 @@ import scala.Option;
 
 import java.util.Map;
 
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
 /**
  * The typed configuration settings associated with a Mesos scheduler.
  */
 public class MesosConfiguration {
 
-	private String masterUrl;
+	private final String masterUrl;
 
-	private Protos.FrameworkInfo.Builder frameworkInfo;
+	private final Protos.FrameworkInfo.Builder frameworkInfo;
 
-	private Option<Protos.Credential.Builder> credential = Option.empty();
+	private final Option<Protos.Credential.Builder> credential;
 
 	public MesosConfiguration(
 		String masterUrl,
 		Protos.FrameworkInfo.Builder frameworkInfo,
 		Option<Protos.Credential.Builder> credential) {
 
-		this.masterUrl = masterUrl;
-		this.frameworkInfo = frameworkInfo;
-		this.credential = credential;
+		this.masterUrl = checkNotNull(masterUrl);
+		this.frameworkInfo = checkNotNull(frameworkInfo);
+		this.credential = checkNotNull(credential);
 	}
 
 	/**
@@ -89,19 +91,20 @@ public class MesosConfiguration {
 	/**
 	 * Create the Mesos scheduler driver based on this configuration.
 	 * @param scheduler the scheduler to use.
-	 * @param implicitAcknowledgements whether to configure the driver for implicit implicit acknowledgements.
+	 * @param implicitAcknowledgements whether to configure the driver for implicit acknowledgements.
      * @return a scheduler driver.
      */
 	public SchedulerDriver createDriver(Scheduler scheduler, boolean implicitAcknowledgements) {
 		MesosSchedulerDriver schedulerDriver;
 		if(this.credential().isDefined()) {
 			schedulerDriver =
-				new MesosSchedulerDriver(scheduler, frameworkInfo.build(), this.masterUrl(), false,
-					this.credential().get().build());
+				new MesosSchedulerDriver(scheduler, frameworkInfo.build(), this.masterUrl(),
+					implicitAcknowledgements, this.credential().get().build());
 		}
 		else {
 			schedulerDriver =
-				new MesosSchedulerDriver(scheduler, frameworkInfo.build(), this.masterUrl(), false);
+				new MesosSchedulerDriver(scheduler, frameworkInfo.build(), this.masterUrl(),
+					implicitAcknowledgements);
 		}
 		return schedulerDriver;
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/59eeea4c/flink-mesos/src/main/java/org/apache/flink/mesos/util/ZooKeeperUtils.java
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/util/ZooKeeperUtils.java b/flink-mesos/src/main/java/org/apache/flink/mesos/util/ZooKeeperUtils.java
index ced4a11..6892a65 100644
--- a/flink-mesos/src/main/java/org/apache/flink/mesos/util/ZooKeeperUtils.java
+++ b/flink-mesos/src/main/java/org/apache/flink/mesos/util/ZooKeeperUtils.java
@@ -33,7 +33,7 @@ public class ZooKeeperUtils {
 	@SuppressWarnings("unchecked")
 	public static CuratorFramework startCuratorFramework(Configuration configuration) {
 
-		// workaround for shaded curator dependency of flink-runtime
+		// using late-binding as a workaround for shaded curator dependency of flink-runtime
 		Object client = org.apache.flink.runtime.util.ZooKeeperUtils.startCuratorFramework(configuration);
 		return (CuratorFramework) client;
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/59eeea4c/flink-mesos/src/main/scala/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManager.scala
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/scala/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManager.scala b/flink-mesos/src/main/scala/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManager.scala
index 3e1054c..d8b6775 100644
--- a/flink-mesos/src/main/scala/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManager.scala
+++ b/flink-mesos/src/main/scala/org/apache/flink/mesos/runtime/clusterframework/MesosTaskManager.scala
@@ -30,14 +30,14 @@ import org.apache.flink.runtime.taskmanager.{TaskManager, TaskManagerConfigurati
   * messages.
   */
 class MesosTaskManager(
-                       config: TaskManagerConfiguration,
-                       resourceID: ResourceID,
-                       connectionInfo: InstanceConnectionInfo,
-                       memoryManager: MemoryManager,
-                       ioManager: IOManager,
-                       network: NetworkEnvironment,
-                       numberOfSlots: Int,
-                       leaderRetrievalService: LeaderRetrievalService)
+    config: TaskManagerConfiguration,
+    resourceID: ResourceID,
+    connectionInfo: InstanceConnectionInfo,
+    memoryManager: MemoryManager,
+    ioManager: IOManager,
+    network: NetworkEnvironment,
+    numberOfSlots: Int,
+    leaderRetrievalService: LeaderRetrievalService)
   extends TaskManager(
     config,
     resourceID,

http://git-wip-us.apache.org/repos/asf/flink/blob/59eeea4c/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/LaunchCoordinator.scala
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/LaunchCoordinator.scala b/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/LaunchCoordinator.scala
index bee25d3..1024b5c 100644
--- a/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/LaunchCoordinator.scala
+++ b/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/LaunchCoordinator.scala
@@ -18,6 +18,8 @@
 
 package org.apache.flink.mesos.scheduler
 
+import java.util.Collections
+
 import akka.actor.{Actor, ActorRef, FSM, Props}
 import com.netflix.fenzo._
 import com.netflix.fenzo.functions.Action1
@@ -27,7 +29,6 @@ import org.apache.flink.api.java.tuple.{Tuple2=>FlinkTuple2}
 import org.apache.flink.configuration.Configuration
 import org.apache.flink.mesos.scheduler.LaunchCoordinator._
 import org.apache.flink.mesos.scheduler.messages._
-import org.apache.mesos.Protos.TaskInfo
 import org.apache.mesos.{SchedulerDriver, Protos}
 
 import scala.collection.JavaConverters._
@@ -38,9 +39,10 @@ import scala.concurrent.duration._
   * The launch coordinator handles offer processing, including
   * matching offers to tasks and making reservations.
   *
-  * The coordinator uses Netflix Fenzo to optimize task placement.   During the GatheringOffers phase,
-  * offers are evaluated by Fenzo for suitability to the planned tasks.   Reservations are then placed
-  * against the best offers, leading to revised offers containing reserved resources with which to launch task(s).
+  * The coordinator uses Netflix Fenzo to optimize task placement.   During the GatheringOffers
+  * phase, offers are evaluated by Fenzo for suitability to the planned tasks.   Reservations are
+  * then placed against the best offers, leading to revised offers containing reserved resources
+  * with which to launch task(s).
   */
 class LaunchCoordinator(
     manager: ActorRef,
@@ -62,7 +64,8 @@ class LaunchCoordinator(
     optimizerBuilder
       .withLeaseRejectAction(new Action1[VirtualMachineLease]() {
         def call(lease: VirtualMachineLease) {
-          LOG.info(s"Declined offer ${lease.getId} from ${lease.hostname()} of ${lease.memoryMB()} MB, ${lease.cpuCores()} cpus.")
+          LOG.info(s"Declined offer ${lease.getId} from ${lease.hostname()} "
+            + s"of ${lease.memoryMB()} MB, ${lease.cpuCores()} cpus.")
           schedulerDriver.declineOffer(lease.getOffer.getId)
         }
       }).build
@@ -81,7 +84,7 @@ class LaunchCoordinator(
   /**
     * State: Suspended
     *
-    * Wait for (re-)connection to Mesos.   No offers exist in this state, but outstanding tasks might.
+    * Wait for (re-)connection to Mesos. No offers exist in this state, but outstanding tasks might.
     */
   when(Suspended) {
     case Event(msg: Connected, data: GatherData) =>
@@ -127,6 +130,7 @@ class LaunchCoordinator(
       assert(nextStateData.newLeases.isEmpty)
       schedulerDriver.suppressOffers()
       optimizer.expireAllLeases()
+      optimizer.scheduleOnce(Collections.emptyList(), Collections.emptyList())
   }
 
   /**
@@ -143,20 +147,24 @@ class LaunchCoordinator(
       goto(Suspended) using data.copy(newLeases = Nil)
 
     case Event(offers: ResourceOffers, data: GatherData) =>
-      val leases = offers.offers().asScala.map(new VMLeaseObject(_).asInstanceOf[VirtualMachineLease])
+      val leases = offers.offers().asScala.map(
+        new VMLeaseObject(_).asInstanceOf[VirtualMachineLease])
       if(LOG.isInfoEnabled) {
-        val (cpus, mem) = leases.foldLeft((0.0,0.0)) { (z,o) => (z._1 + o.cpuCores(), z._2 + o.memoryMB()) }
+        val (cpus, mem) = leases.foldLeft((0.0,0.0)) {
+          (z,o) => (z._1 + o.cpuCores(), z._2 + o.memoryMB())
+        }
         LOG.info(s"Received offer(s) of $mem MB, $cpus cpus:")
-        for(lease <- leases) {
-          LOG.info(s"  ${lease.getId} from ${lease.hostname()} of ${lease.memoryMB()} MB, ${lease.cpuCores()} cpus")
+        for(l <- leases) {
+          LOG.info(s"  ${l.getId} from ${l.hostname()} of ${l.memoryMB()} MB, ${l.cpuCores()} cpus")
         }
       }
-      stay using data.copy(newLeases = data.newLeases ++ leases)
+      stay using data.copy(newLeases = data.newLeases ++ leases) forMax (1 seconds)
 
     case Event(StateTimeout, data: GatherData) =>
       val remaining = MutableMap(data.tasks.map(t => t.taskRequest.getId -> t):_*)
 
-      LOG.info(s"Processing ${remaining.size} task(s) against ${data.newLeases.length} new offer(s) plus outstanding offers.")
+      LOG.info(s"Processing ${remaining.size} task(s) against ${data.newLeases.length}"
+        + s" new offer(s) plus outstanding offers.")
 
       // attempt to assign the outstanding tasks using the optimizer
       val result = optimizer.scheduleOnce(
@@ -193,7 +201,8 @@ class LaunchCoordinator(
         manager ! new AcceptOffers(hostname, offerIds.asJava, operations.asJava)
 
         if(LOG.isInfoEnabled) {
-          LOG.info(s"Launched ${launchedTasks.length} task(s) on ${hostname} using ${offerIds.length} offer(s):")
+          LOG.info(s"Launched ${launchedTasks.length} task(s) on ${hostname}"
+            + s" using ${offerIds.length} offer(s):")
           for(offerId <- offerIds) {
             LOG.info(s"  ${offerId.getValue}")
           }
@@ -206,7 +215,7 @@ class LaunchCoordinator(
       } else {
         LOG.info(s"Waiting for more offers; ${remaining.size} task(s) are not yet launched.")
 
-        stay using data.copy(newLeases = Nil, tasks = remaining.values.toList) forMax SUBSEQUENT_GATHER_DURATION
+        stay using data.copy(newLeases = Nil, tasks = remaining.values.toList)
       }
   }
 
@@ -222,7 +231,8 @@ class LaunchCoordinator(
       // forget rescinded offers
       LOG.info(s"Offer ${offer.offerId()} was rescinded.")
       optimizer.expireLease(offer.offerId().getValue)
-      stay using data.copy(newLeases = data.newLeases.filterNot(_.getOffer.getId == offer.offerId()))
+      stay using data.copy(
+        newLeases = data.newLeases.filterNot(_.getOffer.getId == offer.offerId()))
 
     case Event(msg: Assign, _) =>
       // recovering an earlier task assignment
@@ -250,7 +260,6 @@ class LaunchCoordinator(
 object LaunchCoordinator {
 
   val GATHER_DURATION = 5.seconds
-  val SUBSEQUENT_GATHER_DURATION = 5.seconds
 
   // ------------------------------------------------------------------------
   //  FSM State
@@ -284,15 +293,17 @@ object LaunchCoordinator {
   }
 
   /**
-    * Informs the launch coordinator that some task(s) are assigned to a host (for planning purposes).
+    * Informs the launch coordinator that some task(s) are assigned
+    * to a host (for planning purposes).
     *
-    * This is sent by the RM in recovery procedures to recover the optimizer state.  In normal operation,
-    * the launch coordinator itself updates the optimizer state.
+    * This is sent by the RM in recovery procedures to recover the optimizer state.
+    * In normal operation, the launch coordinator itself updates the optimizer state.
     */
   case class Assign(tasks: java.util.List[FlinkTuple2[TaskRequest, String]])
 
   /**
-    * Informs the launch coordinator that some task is no longer assigned to a host (for planning purposes).
+    * Informs the launch coordinator that some task is no longer assigned
+    * to a host (for planning purposes).
     */
   case class Unassign(taskID: Protos.TaskID, hostname: String)
 
@@ -319,11 +330,13 @@ object LaunchCoordinator {
       allTasks(assignment.getTaskId).launch(slaveId, assignment)
     }
 
-    val launches = Protos.Offer.Operation.newBuilder().setType(Protos.Offer.Operation.Type.LAUNCH).setLaunch(
-      Protos.Offer.Operation.Launch.newBuilder().addAllTaskInfos(
-        assignments.getTasksAssigned.asScala.map(taskInfo).asJava
-      )
-    ).build()
+    val launches = Protos.Offer.Operation.newBuilder()
+      .setType(Protos.Offer.Operation.Type.LAUNCH)
+      .setLaunch(
+        Protos.Offer.Operation.Launch.newBuilder().addAllTaskInfos(
+          assignments.getTasksAssigned.asScala.map(taskInfo).asJava
+        ))
+      .build()
 
     Seq(launches)
   }

http://git-wip-us.apache.org/repos/asf/flink/blob/59eeea4c/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/ReconciliationCoordinator.scala
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/ReconciliationCoordinator.scala b/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/ReconciliationCoordinator.scala
index 239e040..2ce7225 100644
--- a/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/ReconciliationCoordinator.scala
+++ b/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/ReconciliationCoordinator.scala
@@ -48,7 +48,8 @@ class ReconciliationCoordinator(
   when(Suspended) {
     case Event(reconcile: Reconcile, data: ReconciliationData) =>
       val tasks = reconcile.tasks.map(task => (task.getTaskId,task))
-      stay using data.copy(remaining = if(reconcile.replace) tasks.toMap else data.remaining ++ tasks)
+      stay using data.copy(
+        remaining = if(reconcile.replace) tasks.toMap else data.remaining ++ tasks)
 
     case Event(msg: Connected, data: ReconciliationData) =>
       if(data.remaining.nonEmpty) goto(Reconciling)
@@ -75,7 +76,8 @@ class ReconciliationCoordinator(
       // initiate reconciliation for additional tasks (even while reconciliation is ongoing)
       schedulerDriver.reconcileTasks(reconcile.tasks.asJavaCollection)
       val tasks = reconcile.tasks.map(task => (task.getTaskId,task))
-      stay using data.copy(remaining = if(reconcile.replace) tasks.toMap else data.remaining ++ tasks)
+      stay using data.copy(
+        remaining = if(reconcile.replace) tasks.toMap else data.remaining ++ tasks)
 
     case Event(update: StatusUpdate, data: ReconciliationData) =>
       // status information arrived for a task
@@ -143,7 +145,9 @@ object ReconciliationCoordinator {
     * @param remaining
     * @param retries
     */
-  case class ReconciliationData(remaining: Map[Protos.TaskID,Protos.TaskStatus] = Map(), retries: Int = 0)
+  case class ReconciliationData(
+      remaining: Map[Protos.TaskID,Protos.TaskStatus] = Map(),
+      retries: Int = 0)
 
   /**
     * Initiates the task reconciliation process.

http://git-wip-us.apache.org/repos/asf/flink/blob/59eeea4c/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/TaskMonitor.scala
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/TaskMonitor.scala b/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/TaskMonitor.scala
index ec79585..7840fd4 100644
--- a/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/TaskMonitor.scala
+++ b/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/TaskMonitor.scala
@@ -34,14 +34,15 @@ import scala.concurrent.duration._
 /**
   * Monitors a Mesos task throughout its lifecycle.
   *
-  * Models a task with a state machine reflecting the perceived state of the task in Mesos.   The state
-  * is primarily updated when task status information arrives from Mesos.
+  * Models a task with a state machine reflecting the perceived state of the task in Mesos.
+  * The state is primarily updated when task status information arrives from Mesos.
   *
-  * The associated state data primarily tracks the task's goal (intended) state, as persisted by the scheduler.
-  * Keep in mind that goal state is persisted before actions are taken.    The goal state strictly transitions
-  * thru New->Launched->Released.
+  * The associated state data primarily tracks the task's goal (intended) state, as
+  * persisted by the scheduler. Keep in mind that goal state is persisted before actions are taken.
+  * The goal state strictly transitions thru New->Launched->Released.
   *
-  * Unlike most exchanges with Mesos, task status is delivered at-least-once, so status handling should be idempotent.
+  * Unlike most exchanges with Mesos, task status is delivered at-least-once,
+  * so status handling should be idempotent.
   */
 class TaskMonitor(
     flinkConfig: Configuration,
@@ -103,7 +104,8 @@ class TaskMonitor(
 
   when(Staging, stateTimeout = LAUNCH_TIMEOUT) {
     case Event(StateTimeout, _) =>
-      LOG.warn(s"Mesos task ${stateData.goal.taskID.getValue} didn't launch as expected; reconciling.")
+      LOG.warn(s"Mesos task ${stateData.goal.taskID.getValue} didn't launch as expected;"
+        + s" reconciling.")
 
       // likely cause: the task launch message was dropped - docs suggest reconciliation
       goto(Reconciling)

http://git-wip-us.apache.org/repos/asf/flink/blob/59eeea4c/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/Tasks.scala
----------------------------------------------------------------------
diff --git a/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/Tasks.scala b/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/Tasks.scala
index 14147a5..4f49c16 100644
--- a/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/Tasks.scala
+++ b/flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/Tasks.scala
@@ -18,11 +18,11 @@
 
 package org.apache.flink.mesos.scheduler
 
-import akka.actor.{Actor, ActorRef, Props}
+import akka.actor.SupervisorStrategy.Escalate
+import akka.actor._
 import org.apache.flink.configuration.Configuration
 import org.apache.flink.mesos.scheduler.ReconciliationCoordinator.Reconcile
-import org.apache.flink.mesos.scheduler.TaskMonitor.{TaskGoalState, TaskGoalStateUpdated, TaskTerminated}
-import org.apache.flink.mesos.scheduler.Tasks._
+import org.apache.flink.mesos.scheduler.TaskMonitor._
 import org.apache.flink.mesos.scheduler.messages._
 import org.apache.mesos.{SchedulerDriver, Protos}
 
@@ -33,23 +33,23 @@ import scala.collection.mutable.{Map => MutableMap}
   *
   * Routes messages between the scheduler and individual task monitor actors.
   */
-class Tasks[M <: TaskMonitor](
+class Tasks(
      flinkConfig: Configuration,
      schedulerDriver: SchedulerDriver,
-     taskMonitorClass: Class[M]) extends Actor {
+     taskMonitorCreator: (ActorRefFactory,TaskGoalState) => ActorRef) extends Actor {
 
   /**
     * A map of task monitors by task ID.
     */
-  private val taskMap: MutableMap[Protos.TaskID,ActorRef] = MutableMap()
+  private[scheduler] val taskMap: MutableMap[Protos.TaskID,ActorRef] = MutableMap()
 
   /**
     * Cache of current connection state.
     */
-  private var registered: Option[Any] = None
+  private[scheduler] var registered: Option[Any] = None
 
-  override def preStart(): Unit = {
-    // TODO subscribe to context.system.deadLetters for messages to nonexistent tasks
+  override def supervisorStrategy: SupervisorStrategy = AllForOneStrategy() {
+    case _ => Escalate
   }
 
   override def receive: Receive = {
@@ -65,27 +65,42 @@ class Tasks[M <: TaskMonitor](
     case msg: TaskGoalStateUpdated =>
       val taskID = msg.state.taskID
 
-      // ensure task monitor exists
-      if(!taskMap.contains(taskID)) {
-        val actorRef = createTask(msg.state)
-        registered.foreach(actorRef ! _)
-      }
+      taskMap.get(taskID) match {
+        case None =>
+          // create a new actor to monitor the task, with the appropriate initial goal state
+          val actorRef = createTask(msg.state)
+          registered.foreach(actorRef ! _)
 
-      taskMap(taskID) ! msg
+        case Some(actor) =>
+          // tell the actor to update its goal state
+          actor ! msg
+      }
 
     case msg: StatusUpdate =>
-      taskMap(msg.status().getTaskId) ! msg
+      taskMap.get(msg.status().getTaskId) match {
+        case Some(ref) =>
+          // tell the actor about the status change
+          ref ! msg
+        case None =>
+          // a status update was received for an unrecognized task, which may occur
+          // when a task is resurrected (i.e. the Mesos master isn't using a strict registry).
+          // see the Mesos reconciliation guide for more information.
+
+          // create a monitor to reliably terminate the resurrected task
+          val actorRef = createTask(Released(msg.status().getTaskId, msg.status().getSlaveId))
+          registered.foreach(actorRef ! _)
+      }
 
     case msg: Reconcile =>
       context.parent.forward(msg)
 
     case msg: TaskTerminated =>
+      taskMap.remove(msg.taskID)
       context.parent.forward(msg)
   }
 
   private def createTask(task: TaskGoalState): ActorRef = {
-    val actorProps = TaskMonitor.createActorProps(taskMonitorClass, flinkConfig, schedulerDriver, task)
-    val actorRef = context.actorOf(actorProps, name = actorName(task.taskID))
+    val actorRef = taskMonitorCreator(context, task)
     taskMap.put(task.taskID, actorRef)
     actorRef
   }
@@ -94,21 +109,19 @@ class Tasks[M <: TaskMonitor](
 object Tasks {
 
   /**
-    * Extract the actor name for the given task ID.
-    */
-  def actorName(taskID: Protos.TaskID): String = {
-    taskID.getValue
-  }
-
-  /**
     * Create a tasks actor.
     */
-  def createActorProps[T <: Tasks[M], M <: TaskMonitor](
+  def createActorProps[T <: Tasks, M <: TaskMonitor](
       actorClass: Class[T],
       flinkConfig: Configuration,
       schedulerDriver: SchedulerDriver,
       taskMonitorClass: Class[M]): Props = {
 
-    Props.create(actorClass, flinkConfig, schedulerDriver, taskMonitorClass)
+    val taskMonitorCreator = (factory: ActorRefFactory, task: TaskGoalState) => {
+      val props = TaskMonitor.createActorProps(taskMonitorClass, flinkConfig, schedulerDriver, task)
+      factory.actorOf(props)
+    }
+
+    Props.create(actorClass, flinkConfig, schedulerDriver, taskMonitorCreator)
   }
 }