Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
35 changes: 35 additions & 0 deletions core/src/test/resources/fairscheduler-with-valid-data.xml
Original file line number Diff line number Diff line change
@@ -0,0 +1,35 @@
<?xml version="1.0"?>
<!--
~ 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.
-->

<allocations>
<pool name="pool1">
<minShare>3</minShare>
<weight>1</weight>
<schedulingMode>FIFO</schedulingMode>
</pool>
<pool name="pool2">
<minShare>4</minShare>
<weight>2</weight>
<schedulingMode>FAIR</schedulingMode>
</pool>
<pool name="pool3">
<minShare>2</minShare>
<weight>3</weight>
<schedulingMode>FAIR</schedulingMode>
</pool>
</allocations>
44 changes: 44 additions & 0 deletions core/src/test/scala/org/apache/spark/scheduler/PoolSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@

package org.apache.spark.scheduler

import java.io.FileNotFoundException
import java.util.Properties

import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkFunSuite}
Expand Down Expand Up @@ -292,6 +293,49 @@ class PoolSuite extends SparkFunSuite with LocalSparkContext {
}
}

test("Fair Scheduler should build fair scheduler when " +
"valid spark.scheduler.allocation.file property is set") {
val xmlPath = getClass.getClassLoader.getResource("fairscheduler-with-valid-data.xml").getFile()
val conf = new SparkConf().set(SCHEDULER_ALLOCATION_FILE_PROPERTY, xmlPath)
sc = new SparkContext(LOCAL, APP_NAME, conf)

val rootPool = new Pool("", SchedulingMode.FAIR, 0, 0)
val schedulableBuilder = new FairSchedulableBuilder(rootPool, sc.conf)
schedulableBuilder.buildPools()

verifyPool(rootPool, schedulableBuilder.DEFAULT_POOL_NAME, 0, 1, FIFO)
verifyPool(rootPool, "pool1", 3, 1, FIFO)
verifyPool(rootPool, "pool2", 4, 2, FAIR)
verifyPool(rootPool, "pool3", 2, 3, FAIR)
}

test("Fair Scheduler should use default file(fairscheduler.xml) if it exists in classpath " +
"and spark.scheduler.allocation.file property is not set") {
val conf = new SparkConf()
sc = new SparkContext(LOCAL, APP_NAME, conf)

val rootPool = new Pool("", SchedulingMode.FAIR, 0, 0)
val schedulableBuilder = new FairSchedulableBuilder(rootPool, sc.conf)
schedulableBuilder.buildPools()

verifyPool(rootPool, schedulableBuilder.DEFAULT_POOL_NAME, 0, 1, FIFO)
verifyPool(rootPool, "1", 2, 1, FIFO)
verifyPool(rootPool, "2", 3, 1, FIFO)
verifyPool(rootPool, "3", 0, 1, FIFO)
}

test("Fair Scheduler should throw FileNotFoundException " +
"when invalid spark.scheduler.allocation.file property is set") {
val conf = new SparkConf().set(SCHEDULER_ALLOCATION_FILE_PROPERTY, "INVALID_FILE_PATH")
sc = new SparkContext(LOCAL, APP_NAME, conf)

val rootPool = new Pool("", SchedulingMode.FAIR, 0, 0)
val schedulableBuilder = new FairSchedulableBuilder(rootPool, sc.conf)
intercept[FileNotFoundException] {
schedulableBuilder.buildPools()
}
}

private def verifyPool(rootPool: Pool, poolName: String, expectedInitMinShare: Int,
expectedInitWeight: Int, expectedSchedulingMode: SchedulingMode): Unit = {
val selectedPool = rootPool.getSchedulableByName(poolName)
Expand Down
2 changes: 1 addition & 1 deletion docs/job-scheduling.md
Original file line number Diff line number Diff line change
Expand Up @@ -235,7 +235,7 @@ properties:
of the cluster. By default, each pool's `minShare` is 0.

The pool properties can be set by creating an XML file, similar to `conf/fairscheduler.xml.template`,
and setting a `spark.scheduler.allocation.file` property in your
and either putting a file named `fairscheduler.xml` on the classpath, or setting `spark.scheduler.allocation.file` property in your
[SparkConf](configuration.html#spark-properties).

{% highlight scala %}
Expand Down