Skip to content
Snippets Groups Projects
Commit 73e64f7d authored by erenavsarogullari's avatar erenavsarogullari Committed by Imran Rashid
Browse files

[SPARK-19662][SCHEDULER][TEST] Add Fair Scheduler Unit Test coverage for different build cases

## What changes were proposed in this pull request?
Fair Scheduler can be built via one of the following options:
- By setting a `spark.scheduler.allocation.file` property,
- By setting `fairscheduler.xml` into classpath.

These options are checked **in order** and fair-scheduler is built via first found option. If invalid path is found, `FileNotFoundException` will be expected.

This PR aims unit test coverage of these use cases and a minor documentation change has been added for second option(`fairscheduler.xml` into classpath) to inform the users.

Also, this PR was related with #16813 and has been created separately to keep patch content as isolated and to help the reviewers.

## How was this patch tested?
Added new Unit Tests.

Author: erenavsarogullari <erenavsarogullari@gmail.com>

Closes #16992 from erenavsarogullari/SPARK-19662.
parent 24e6c187
No related branches found
No related tags found
No related merge requests found
<?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>
\ No newline at end of file
......@@ -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}
......@@ -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)
......
......@@ -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 %}
......
0% Loading or .
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment