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

    https://github.com/apache/spark/pull/15541#discussion_r83995590
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskAssigner.scala 
---
    @@ -0,0 +1,233 @@
    +/*
    + * 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
    +
    +/** Tracking 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 worker.
    +  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, 
TaskScheduler, requested
    + * to perform task assignment given available workers, first sorts the 
candidate tasksets,
    + * and then for each taskset, it takes a number of rounds to request 
TaskAssigner for task
    + * assignment with different the 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. Before each
    + * round of task assignment for a taskset, TaskScheduler invoke the init() 
of TaskAssigner to
    + * initialize the data structure for the round. When performing real task 
assignment,
    + * hasNext()/getNext() is used by TaskScheduler to check the worker 
availability and retrieve
    + * current offering from TaskAssigner. 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. After task assignment is done, TaskScheduler invokes 
the tasks() to
    + * retrieve all the task assignment information, and eventually, invokes 
reset() method so that
    + * TaskAssigner can cleanup its internal maintained resources.
    + */
    +
    +private[scheduler] abstract class TaskAssigner {
    +  var offer: Seq[OfferState] = _
    +  var CPUS_PER_TASK = 1
    --- End diff --
    
    - nit: use `protected`
    - nit: Why camel case ? This looks like an regular class var and not a 
constant. I know you want this to be treated as a constant but probably should 
have better guarding against that



---
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