flink-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From EronWright <...@git.apache.org>
Subject [GitHub] flink pull request #2315: [FLINK-1984] Integrate Flink with Apache Mesos (T1...
Date Wed, 17 Aug 2016 16:06:56 GMT
Github user EronWright commented on a diff in the pull request:

    https://github.com/apache/flink/pull/2315#discussion_r75153237
  
    --- 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 --
    
    Usually one waits for offers to maximize the overall fitness; there's a fundamental latency/fitness
tradeoff.   I agree that offer quality (aside from the hard constraints of cpu/ram) is not
an important consideration at this time.    This overall design allows for tuning along many
dimensions, and I didn't want to engage in premature optimization in this first-cut.    Nonetheless
I will change the timeout to be much more aggressive.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

Mime
View raw message