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
Original file line number Diff line number Diff line change
Expand Up @@ -172,7 +172,7 @@ private[spark] class TaskSchedulerImpl private[scheduler](

if (!isLocal && conf.getBoolean("spark.speculation", false)) {
logInfo("Starting speculative execution thread")
speculationScheduler.scheduleAtFixedRate(new Runnable {
speculationScheduler.scheduleWithFixedDelay(new Runnable {
override def run(): Unit = Utils.tryOrStopSparkContext(sc) {
checkSpeculatableTasks()
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,18 +19,19 @@ package org.apache.spark.scheduler

import java.io.NotSerializableException
import java.nio.ByteBuffer
import java.util.Arrays
import java.util.concurrent.ConcurrentLinkedQueue
import java.util.concurrent.atomic.AtomicLong

import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet}
import scala.math.{max, min}
import scala.math.max
import scala.util.control.NonFatal

import org.apache.spark._
import org.apache.spark.internal.Logging
import org.apache.spark.scheduler.SchedulingMode._
import org.apache.spark.TaskState.TaskState
import org.apache.spark.util.{AccumulatorV2, Clock, SystemClock, Utils}
import org.apache.spark.util.collection.MedianHeap

/**
* Schedules the tasks within a single TaskSet in the TaskSchedulerImpl. This class keeps track of
Expand All @@ -52,7 +53,8 @@ private[spark] class TaskSetManager(
val taskSet: TaskSet,
val maxTaskFailures: Int,
blacklistTracker: Option[BlacklistTracker] = None,
clock: Clock = new SystemClock()) extends Schedulable with Logging {
clock: Clock = new SystemClock(),
newAlgorithm: Boolean = false) extends Schedulable with Logging {

private val conf = sched.sc.conf

Expand All @@ -63,6 +65,8 @@ private[spark] class TaskSetManager(
// Limit of bytes for total size of results (default is 1GB)
val maxResultSize = Utils.getMaxResultSize(conf)

val speculationEnabled = conf.getBoolean("spark.speculation", false)

// Serializer for closures and tasks.
val env = SparkEnv.get
val ser = env.closureSerializer.newInstance()
Expand Down Expand Up @@ -141,6 +145,11 @@ private[spark] class TaskSetManager(
// Task index, start and finish time for each task attempt (indexed by task ID)
val taskInfos = new HashMap[Long, TaskInfo]

// Use a MedianHeap to record durations of successful tasks so we know when to launch
// speculative tasks. This is only used when speculation is enabled, to avoid the overhead
// of inserting into the heap when the heap won't be used.
val successfulTaskDurations = new MedianHeap()

// How frequently to reprint duplicate exceptions in full, in milliseconds
val EXCEPTION_PRINT_INTERVAL =
conf.getLong("spark.logging.exceptionPrintInterval", 10000)
Expand Down Expand Up @@ -692,10 +701,19 @@ private[spark] class TaskSetManager(
/**
* Marks a task as successful and notifies the DAGScheduler that the task has ended.
*/
val handleSuccessfulTasksCost = new AtomicLong(0L)

def handleSuccessfulTask(tid: Long, result: DirectTaskResult[_]): Unit = {
val info = taskInfos(tid)
val index = info.index

info.markFinished(TaskState.FINISHED, clock.getTimeMillis())
if (speculationEnabled) {
val startTime = System.currentTimeMillis()
successfulTaskDurations.insert(info.duration)
handleSuccessfulTasksCost.addAndGet(System.currentTimeMillis() - startTime)
}

removeRunningTask(tid)
// This method is called by "TaskSchedulerImpl.handleSuccessfulTask" which holds the
// "TaskSchedulerImpl" lock until exiting. To avoid the SPARK-7655 issue, we should not
Expand Down Expand Up @@ -917,11 +935,11 @@ private[spark] class TaskSetManager(
var foundTasks = false
val minFinishedForSpeculation = (SPECULATION_QUANTILE * numTasks).floor.toInt
logDebug("Checking for speculative tasks: minFinished = " + minFinishedForSpeculation)

if (tasksSuccessful >= minFinishedForSpeculation && tasksSuccessful > 0) {
val time = clock.getTimeMillis()
val durations = taskInfos.values.filter(_.successful).map(_.duration).toArray
Arrays.sort(durations)
val medianDuration = durations(min((0.5 * tasksSuccessful).round.toInt, durations.length - 1))
var medianDuration = successfulTaskDurations.median

val threshold = max(SPECULATION_MULTIPLIER * medianDuration, minTimeToSpeculation)
// TODO: Threshold should also look at standard deviation of task durations and have a lower
// bound based on that.
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,95 @@
/*
* 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.util.collection

import scala.collection.mutable.PriorityQueue

/**
* MedianHeap is designed to be used to quickly track the median of a group of numbers
* that may contain duplicates. Inserting a new number has O(log n) time complexity and
* determining the median has O(1) time complexity.
* The basic idea is to maintain two heaps: a smallerHalf and a largerHalf. The smallerHalf
* stores the smaller half of all numbers while the largerHalf stores the larger half.
* The sizes of two heaps need to be balanced each time when a new number is inserted so
* that their sizes will not be different by more than 1. Therefore each time when
* findMedian() is called we check if two heaps have the same size. If they do, we should
* return the average of the two top values of heaps. Otherwise we return the top of the
* heap which has one more element.
*/

private[spark] class MedianHeap(implicit val ord: Ordering[Double]) {

// Stores all the numbers less than the current median in a smallerHalf,
// i.e median is the maximum, at the root
private[this] var smallerHalf = PriorityQueue.empty[Double](ord)

// Stores all the numbers greater than the current median in a largerHalf,
// i.e median is the minimum, at the root
private[this] var largerHalf = PriorityQueue.empty[Double](ord.reverse)

// Returns if there is no element in MedianHeap.
def isEmpty(): Boolean = {
smallerHalf.isEmpty && largerHalf.isEmpty
}

// Size of MedianHeap.
def size(): Int = {
smallerHalf.size + largerHalf.size
}

// Insert a new number into MedianHeap.
def insert(x: Double): Unit = {
// If both heaps are empty, we arbitrarily insert it into a heap, let's say, the largerHalf.
if (isEmpty) {
largerHalf.enqueue(x)
} else {
// If the number is larger than current median, it should be inserted into largerHalf,
// otherwise smallerHalf.
if (x > median) {
largerHalf.enqueue(x)
} else {
smallerHalf.enqueue(x)
}
}
rebalance()
}

// Re-balance the heaps.
private[this] def rebalance(): Unit = {
if (largerHalf.size - smallerHalf.size > 1) {
smallerHalf.enqueue(largerHalf.dequeue())
}
if (smallerHalf.size - largerHalf.size > 1) {
largerHalf.enqueue(smallerHalf.dequeue)
}
}

// Returns the median of the numbers.
def median: Double = {
if (isEmpty) {
throw new NoSuchElementException("MedianHeap is empty.")
}
if (largerHalf.size == smallerHalf.size) {
(largerHalf.head + smallerHalf.head) / 2.0
} else if (largerHalf.size > smallerHalf.size) {
largerHalf.head
} else {
smallerHalf.head
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -178,7 +178,6 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg
}
}


test("TaskSet with no preferences") {
sc = new SparkContext("local", "test")
sched = new FakeTaskScheduler(sc, ("exec1", "host1"))
Expand Down Expand Up @@ -893,6 +892,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg
val taskSet = FakeTask.createTaskSet(4)
// Set the speculation multiplier to be 0 so speculative tasks are launched immediately
sc.conf.set("spark.speculation.multiplier", "0.0")
sc.conf.set("spark.speculation", "true")
val clock = new ManualClock()
val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock)
val accumUpdatesByTask: Array[Seq[AccumulatorV2[_, _]]] = taskSet.tasks.map { task =>
Expand Down Expand Up @@ -941,13 +941,15 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg
assert(sched.endedTasks(3) === Success)
}


test("Killing speculative tasks does not count towards aborting the taskset") {
sc = new SparkContext("local", "test")
sched = new FakeTaskScheduler(sc, ("exec1", "host1"), ("exec2", "host2"))
val taskSet = FakeTask.createTaskSet(5)
// Set the speculation multiplier to be 0 so speculative tasks are launched immediately
sc.conf.set("spark.speculation.multiplier", "0.0")
sc.conf.set("spark.speculation.quantile", "0.6")
sc.conf.set("spark.speculation", "true")
val clock = new ManualClock()
val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock)
val accumUpdatesByTask: Array[Seq[AccumulatorV2[_, _]]] = taskSet.tasks.map { task =>
Expand Down Expand Up @@ -1109,6 +1111,36 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg
.updateBlacklistForFailedTask(anyString(), anyString(), anyInt())
}

test("Measurement for SPARK-16929.") {
val tasksNum = 100000
val conf = new SparkConf().set("spark.speculation", "true")
sc = new SparkContext("local", "test", conf)
sched = new FakeTaskScheduler(sc)
val taskSet = FakeTask.createTaskSet(tasksNum)
val clock = new ManualClock()
val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock,
newAlgorithm = true)

for(i <- 0 until tasksNum) {
manager.resourceOffer("exec", "host", NO_PREF)
}

val accumUpdatesByTask: Array[Seq[AccumulatorV2[_, _]]] = taskSet.tasks.map { task =>
task.metrics.internalAccums
}
val random = new Random()
for (id <- 0 until tasksNum - 1) {
clock.setTime(random.nextInt(360000) + 1)
manager.handleSuccessfulTask(id, createTaskResult(id, accumUpdatesByTask(id)))
assert(sched.endedTasks(id) === Success)
}
// scalastyle:off
println("handleSuccessfulTasks cost: " + manager.handleSuccessfulTasksCost.get())
val startTime = System.currentTimeMillis()
manager.checkSpeculatableTasks(600000)
println("checkSpeculatableTasks: " + (System.currentTimeMillis() - startTime))
}

private def createTaskResult(
id: Int,
accumUpdates: Seq[AccumulatorV2[_, _]] = Seq.empty): DirectTaskResult[Int] = {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,70 @@
/*
* 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.util.collection

import java.util.Arrays
import java.util.NoSuchElementException

import scala.collection.mutable.ArrayBuffer
import scala.util.Random

import org.apache.spark.SparkFunSuite

class MedianHeapSuite extends SparkFunSuite {

test("If no numbers in MedianHeap, NoSuchElementException is thrown.") {
val medianHeap = new MedianHeap()
intercept[NoSuchElementException] {
medianHeap.median
}
}

test("Median should be correct when size of MedianHeap is even") {
val array = Array(0, 1, 2, 3, 4, 5, 6, 7, 8, 9)
val medianHeap = new MedianHeap()
array.foreach(medianHeap.insert(_))
assert(medianHeap.size() === 10)
assert(medianHeap.median === ((array(4) + array(5)) / 2.0))
}

test("Median should be correct when size of MedianHeap is odd") {
val array = Array(0, 1, 2, 3, 4, 5, 6, 7, 8)
val medianHeap = new MedianHeap()
array.foreach(medianHeap.insert(_))
assert(medianHeap.size() === 9)
assert(medianHeap.median === (array(4)))
}

test("Median should be correct though there are duplicated numbers inside.") {
val array = Array(0, 0, 1, 1, 2, 2, 3, 3, 4, 4)
val medianHeap = new MedianHeap()
array.foreach(medianHeap.insert(_))
assert(medianHeap.size === 10)
assert(medianHeap.median === ((array(4) + array(5)) / 2.0))
}

test("Median should be correct when skew situations.") {
val medianHeap = new MedianHeap()
(0 until 10).foreach(_ => medianHeap.insert(5))
assert(medianHeap.median === 5)
(0 until 100).foreach(_ => medianHeap.insert(10))
assert(medianHeap.median === 10)
(0 until 1000).foreach(_ => medianHeap.insert(0))
assert(medianHeap.median === 0)
}
}