Github user wangmiao1981 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/15541#discussion_r84422908
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskAssigner.scala 
---
    @@ -0,0 +1,218 @@
    +/*
    + * 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.spark.scheduler
    +
    +import scala.collection.mutable.ArrayBuffer
    +import scala.collection.mutable.PriorityQueue
    +import scala.util.Random
    +
    +import org.apache.spark.internal.{config, Logging}
    +import org.apache.spark.SparkConf
    +import org.apache.spark.util.Utils
    +
    +/** Tracks the current state of the workers with available cores and 
assigned task list. */
    +class OfferState(val workOffer: WorkerOffer) {
    +  /** The current remaining cores that can be allocated to tasks. */
    +  var coresAvailable: Int = workOffer.cores
    +  /** The list of tasks that are assigned to this WorkerOffer. */
    +  val tasks = new ArrayBuffer[TaskDescription](coresAvailable)
    +}
    +
    +/**
    + * TaskAssigner is the base class for all task assigner implementations, 
and can be
    + * extended to implement different task scheduling algorithms.
    + * Together with [[org.apache.spark.scheduler.TaskScheduler 
TaskScheduler]], TaskAssigner
    + * is used to assign tasks to workers with available cores. Internally, 
when TaskScheduler
    + * perform task assignment given available workers, it first sorts the 
candidate tasksets,
    + * and then for each taskset, it takes a number of rounds to request 
TaskAssigner for task
    + * assignment with different locality restrictions until there is either 
no qualified
    + * workers or no valid tasks to be assigned.
    + *
    + * TaskAssigner is responsible to maintain the worker availability state 
and task assignment
    + * information. The contract between 
[[org.apache.spark.scheduler.TaskScheduler TaskScheduler]]
    + * and TaskAssigner is as follows.
    + *
    + * First, TaskScheduler invokes construct() of TaskAssigner to initialize 
the its internal
    + * worker states at the beginning of resource offering.
    + *
    + * Second, before each round of task assignment for a taskset, 
TaskScheduler invoke the init()
    + * of TaskAssigner to initialize the data structure for the round.
    + *
    + * Third, when performing real task assignment, hasNext()/getNext() is 
used by TaskScheduler
    + * to check the worker availability and retrieve current offering from 
TaskAssigner.
    + *
    + * Fourth, then offerAccepted is used by TaskScheduler to notify the 
TaskAssigner so that
    + * TaskAssigner can decide whether the current offer is valid or not for 
the next request.
    + *
    + * Fifth, After task assignment is done, TaskScheduler invokes the tasks() 
to
    + * retrieve all the task assignment information.
    + */
    +
    +private[scheduler] abstract class TaskAssigner {
    +  protected var offer: Seq[OfferState] = _
    +  protected var cpuPerTask = 1
    +
    +  protected def withCpuPerTask(cpuPerTask: Int): Unit = {
    +    this.cpuPerTask = cpuPerTask
    +  }
    +
    +  /** The final assigned offer returned to TaskScheduler. */
    +  final def tasks: Seq[ArrayBuffer[TaskDescription]] = offer.map(_.tasks)
    +
    +  /** Invoked at the beginning of resource offering to construct the offer 
with the workoffers. */
    +  def construct(workOffer: Seq[WorkerOffer]): Unit = {
    +    offer = Random.shuffle(workOffer.map(o => new OfferState(o)))
    +  }
    +
    +  /** Invoked at each round of Taskset assignment to initialize the 
internal structure. */
    +  def init(): Unit
    +
    +  /**
    +   * Tests Whether there is offer available to be used inside of one round 
of Taskset assignment.
    +   *  @return  `true` if a subsequent call to `next` will yield an element,
    +   *           `false` otherwise.
    +   */
    +  def hasNext: Boolean
    +
    +  /**
    +   * Produces next worker offer based on the task assignment strategy.
    +   * @return  the next available offer, if `hasNext` is `true`,
    +   *          undefined behavior otherwise.
    +   */
    +  def next(): OfferState
    +
    +  /**
    +   * Invoked by the TaskScheduler to indicate whether the current offer is 
accepted or not so that
    +   * the assigner can decide whether the current worker is valid for the 
next offering.
    +   */
    +  def offerAccepted(assigned: Boolean): Unit
    +}
    +
    +object TaskAssigner extends Logging {
    +  private val roundrobin = classOf[RoundRobinAssigner].getCanonicalName
    +  private val packed = classOf[PackedAssigner].getCanonicalName
    +  private val balanced = classOf[BalancedAssigner].getCanonicalName
    +  private val assignerMap: Map[String, String] =
    +    Map("roundrobin" -> roundrobin,
    +      "packed" -> packed,
    +      "balanced" -> balanced)
    +
    +  def init(conf: SparkConf): TaskAssigner = {
    +    val assignerName = conf.get(config.SPARK_SCHEDULER_TASK_ASSIGNER.key, 
"roundrobin")
    +    val className = {
    +      val name = assignerMap.get(assignerName.toLowerCase())
    +      name.getOrElse {
    +        logWarning(s"$assignerName cannot be constructed, fallback to 
default $roundrobin.")
    +        roundrobin
    +      }
    +    }
    +    // The className is valid. No need to catch exceptions.
    +    logInfo(s"Constructing TaskAssigner as $className")
    +    val assigner = Utils.classForName(className)
    +      .getConstructor().newInstance().asInstanceOf[TaskAssigner]
    +    assigner.withCpuPerTask(cpuPerTask = conf.getInt("spark.task.cpus", 1))
    +    assigner
    +  }
    +}
    +
    +/**
    + * Assign the task to workers with available cores in roundrobin manner.
    + */
    +class RoundRobinAssigner extends TaskAssigner {
    +  private var currentOfferIndex = 0
    +
    +  override def init(): Unit = {
    +    currentOfferIndex = 0
    +  }
    +
    +  override def hasNext: Boolean = currentOfferIndex < offer.size
    +
    +  override def next(): OfferState = {
    +    offer(currentOfferIndex)
    +  }
    +
    +  override def offerAccepted(assigned: Boolean): Unit = {
    +    currentOfferIndex += 1
    +  }
    +}
    +
    +/**
    + * Assign the task to workers with the most available cores. It other 
words, BalancedAssigner tries
    --- End diff --
    
    `It` -> `In`


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org

Reply via email to