And, no, Spark's scheduler will not preempt already running Tasks.  In
fact, just killing running Tasks for any reason is trickier than we'd like
it to be, so it isn't done by default:
https://issues.apache.org/jira/browse/SPARK-17064

On Fri, Sep 2, 2016 at 11:34 AM, Mark Hamstra <m...@clearstorydata.com>
wrote:

> The comparator is used in `Pool#getSortedTaskSetQueue`.  The
> `TaskSchedulerImpl` calls that on the rootPool when the TaskScheduler needs
> to handle `resourceOffers` for available Executor cores.  Creation of the
> `sortedTaskSets` is a recursive, nested sorting of the `Schedulable`
> entities -- you can have pools within pools within pools within... if you
> really want to, but they eventually bottom out in TaskSetManagers.  The
> `sortedTaskSets` is a flattened queue of the TaskSets, and the available
> cores are offered to those TaskSets in that queued order until the next
> time the scheduler backend handles the available resource offers and a new
> `sortedTaskSets` is generated.
>
> On Fri, Sep 2, 2016 at 2:37 AM, enrico d'urso <e.du...@live.com> wrote:
>
>> Thank you.
>>
>> May I know when that comparator is called?
>> It looks like spark scheduler has not any form of preemption, am I right?
>>
>> Thank you
>> ------------------------------
>> *From:* Mark Hamstra <m...@clearstorydata.com>
>> *Sent:* Thursday, September 1, 2016 8:44:10 PM
>>
>> *To:* enrico d'urso
>> *Cc:* user@spark.apache.org
>> *Subject:* Re: Spark scheduling mode
>>
>> Spark's FairSchedulingAlgorithm is not round robin:
>> https://github.com/apache/spark/blob/master/core/src/
>> main/scala/org/apache/spark/scheduler/SchedulingAlgorithm.scala#L43
>>
>> When at the scope of fair scheduling Jobs within a single Pool, the
>> Schedulable entities being handled (s1 and s2) are TaskSetManagers, which
>> are at the granularity of Stages, not Jobs.  Since weight is 1 and minShare
>> is 0 for TaskSetManagers, the FairSchedulingAlgorithm for TaskSetManagers
>> just boils down to prioritizing TaskSets (i.e. Stages) with the fewest
>> number of runningTasks.
>>
>> On Thu, Sep 1, 2016 at 11:23 AM, enrico d'urso <e.du...@live.com> wrote:
>>
>>> I tried it before, but still I am not able to see a proper round robin
>>> across the jobs I submit.
>>> Given this:
>>>
>>> <pool name="production">
>>>     <schedulingMode>FAIR</schedulingMode>
>>>     <weight>1</weight>
>>>     <minShare>2</minShare>
>>>   </pool>
>>>
>>> Each jobs inside production pool should be scheduled in round robin way,
>>> am I right?
>>>
>>> ------------------------------
>>> *From:* Mark Hamstra <m...@clearstorydata.com>
>>> *Sent:* Thursday, September 1, 2016 8:19:44 PM
>>> *To:* enrico d'urso
>>> *Cc:* user@spark.apache.org
>>> *Subject:* Re: Spark scheduling mode
>>>
>>> The default pool (`<pool name = "default">`) can be configured like any
>>> other pool: https://spark.apache.org/docs/latest/job-scheduling.ht
>>> ml#configuring-pool-properties
>>>
>>> On Thu, Sep 1, 2016 at 11:11 AM, enrico d'urso <e.du...@live.com> wrote:
>>>
>>>> Is there a way to force scheduling to be fair *inside* the default
>>>> pool?
>>>> I mean, round robin for the jobs that belong to the default pool.
>>>>
>>>> Cheers,
>>>> ------------------------------
>>>> *From:* Mark Hamstra <m...@clearstorydata.com>
>>>> *Sent:* Thursday, September 1, 2016 7:24:54 PM
>>>> *To:* enrico d'urso
>>>> *Cc:* user@spark.apache.org
>>>> *Subject:* Re: Spark scheduling mode
>>>>
>>>> Just because you've flipped spark.scheduler.mode to FAIR, that doesn't
>>>> mean that Spark can magically configure and start multiple scheduling pools
>>>> for you, nor can it know to which pools you want jobs assigned.  Without
>>>> doing any setup of additional scheduling pools or assigning of jobs to
>>>> pools, you're just dumping all of your jobs into the one available default
>>>> pool (which is now being fair scheduled with an empty set of other pools)
>>>> and the scheduling of jobs within that pool is still the default intra-pool
>>>> scheduling, FIFO -- i.e., you've effectively accomplished nothing by only
>>>> flipping spark.scheduler.mode to FAIR.
>>>>
>>>> On Thu, Sep 1, 2016 at 7:10 AM, enrico d'urso <e.du...@live.com> wrote:
>>>>
>>>>> I am building a Spark App, in which I submit several jobs (pyspark). I
>>>>> am using threads to run them in parallel, and also I am setting:
>>>>> conf.set("spark.scheduler.mode", "FAIR") Still, I see the jobs run
>>>>> serially in FIFO way. Am I missing something?
>>>>>
>>>>> Cheers,
>>>>>
>>>>>
>>>>> Enrico
>>>>>
>>>>
>>>>
>>>
>>
>

Reply via email to