[ https://issues.apache.org/jira/browse/FLINK-1984?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15424729#comment-15424729 ]
ASF GitHub Bot commented on FLINK-1984: --------------------------------------- Github user tillrohrmann commented on a diff in the pull request: https://github.com/apache/flink/pull/2315#discussion_r75147221 --- Diff: flink-mesos/src/main/scala/org/apache/flink/mesos/scheduler/LaunchCoordinator.scala --- @@ -0,0 +1,349 @@ +/* + * 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} +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. --- End diff -- Why do we wait for offers being accumulated? Why not starting tasks as soon as we get offers? > Integrate Flink with Apache Mesos > --------------------------------- > > Key: FLINK-1984 > URL: https://issues.apache.org/jira/browse/FLINK-1984 > Project: Flink > Issue Type: New Feature > Components: Cluster Management > Reporter: Robert Metzger > Assignee: Eron Wright > Priority: Minor > Attachments: 251.patch > > > There are some users asking for an integration of Flink into Mesos. > -There also is a pending pull request for adding Mesos support for Flink-: > https://github.com/apache/flink/pull/251 > Update (May '16): a new effort is now underway, building on the recent > ResourceManager work. > Design document: ([google > doc|https://docs.google.com/document/d/1WItafBmGbjlaBbP8Of5PAFOH9GUJQxf5S4hjEuPchuU/edit?usp=sharing]) -- This message was sent by Atlassian JIRA (v6.3.4#6332)