@@ -1900,7 +1900,7 @@ class TaskSetManagerSuite
19001900 sc.conf.set(config.SPECULATION_ENABLED , true )
19011901 sc.conf.set(config.SPECULATION_MULTIPLIER , 1.5 )
19021902 sc.conf.set(config.SPECULATION_QUANTILE , 0.5 )
1903- sc.conf.set(config.EXECUTOR_DECOMMISSION_KILL_INTERVAL .key, " 5ms " )
1903+ sc.conf.set(config.EXECUTOR_DECOMMISSION_KILL_INTERVAL .key, " 5s " )
19041904 val clock = new ManualClock ()
19051905 val manager = new TaskSetManager (sched, taskSet, MAX_TASK_FAILURES , clock = clock)
19061906 val accumUpdatesByTask : Array [Seq [AccumulatorV2 [_, _]]] = taskSet.tasks.map { task =>
@@ -1917,15 +1917,15 @@ class TaskSetManagerSuite
19171917 assert(taskOption2.isDefined)
19181918 assert(taskOption2.get.executorId === " exec2" )
19191919
1920- clock.advance(6 ) // time = 6ms
1920+ clock.advance(6 * 1000 ) // time = 6s
19211921 // Start TASK 3 on exec2 after some delay
19221922 val taskOption3 = manager.resourceOffer(" exec2" , " host2" , NO_PREF )._1
19231923 assert(taskOption3.isDefined)
19241924 assert(taskOption3.get.executorId === " exec2" )
19251925
19261926 assert(sched.startedTasks.toSet === Set (0 , 1 , 2 , 3 ))
19271927
1928- clock.advance(4 ) // time = 10ms
1928+ clock.advance(4 * 1000 ) // time = 10s
19291929 // Complete the first 2 tasks and leave the other 2 tasks in running
19301930 for (id <- Set (0 , 1 )) {
19311931 manager.handleSuccessfulTask(id, createTaskResult(id, accumUpdatesByTask(id)))
@@ -1934,7 +1934,7 @@ class TaskSetManagerSuite
19341934
19351935 // checkSpeculatableTasks checks that the task runtime is greater than the threshold for
19361936 // speculating. Since we use a SPECULATION_MULTIPLIER of 1.5, So tasks need to be running for
1937- // > 15ms for speculation
1937+ // > 15s for speculation
19381938 assert(! manager.checkSpeculatableTasks(0 ))
19391939 assert(sched.speculativeTasks.toSet === Set ())
19401940
@@ -1943,12 +1943,12 @@ class TaskSetManagerSuite
19431943 // (TASK2 -> 15, TASK3 -> 15)
19441944 manager.executorDecommission(" exec2" )
19451945 assert(manager.tidToExecutorKillTimeMapping.keySet === Set (2 , 3 ))
1946- assert(manager.tidToExecutorKillTimeMapping(2 ) === 15 )
1947- assert(manager.tidToExecutorKillTimeMapping(3 ) === 15 )
1946+ assert(manager.tidToExecutorKillTimeMapping(2 ) === 15 * 1000 )
1947+ assert(manager.tidToExecutorKillTimeMapping(3 ) === 15 * 1000 )
19481948
19491949 assert(manager.checkSpeculatableTasks(0 ))
1950- // Task2 started at t=0 , so it can still finish before t=15 (Median task runtime = 10 )
1951- // Task3 started at t=6 , so it might not finish before t=15 . So Task 3 should be part
1950+ // Task2 started at t=0s , so it can still finish before t=15s (Median task runtime = 10s )
1951+ // Task3 started at t=6s , so it might not finish before t=15s . So Task 3 should be part
19521952 // of speculativeTasks
19531953 assert(sched.speculativeTasks.toSet === Set (3 ))
19541954 assert(manager.copiesRunning(3 ) === 1 )
@@ -1966,15 +1966,15 @@ class TaskSetManagerSuite
19661966 assert(speculativeTask3.executorId === " exec3" )
19671967 assert(speculativeTask3.attemptNumber === 1 )
19681968
1969- clock.advance(1 ) // time = 11 ms
1969+ clock.advance(1 * 1000 ) // time = 11s
19701970 // Running checkSpeculatableTasks again should return false
19711971 assert(! manager.checkSpeculatableTasks(0 ))
19721972 assert(manager.copiesRunning(2 ) === 1 )
19731973 assert(manager.copiesRunning(3 ) === 2 )
19741974
1975- clock.advance(5 ) // time = 16 ms
1976- // At t=16 ms , Task 2 has been running for 16 ms . It is more than the
1977- // SPECULATION_MULTIPLIER * medianRuntime = 1.5 * 10 = 15 ms . So now Task 2 will
1975+ clock.advance(5 * 1000 ) // time = 16s
1976+ // At t=16s , Task 2 has been running for 16s . It is more than the
1977+ // SPECULATION_MULTIPLIER * medianRuntime = 1.5 * 10 = 15s . So now Task 2 will
19781978 // be selected for speculation. Here we are verifying that regular speculation configs
19791979 // should still take effect even when a EXECUTOR_DECOMMISSION_KILL_INTERVAL is provided and
19801980 // corresponding executor is decommissioned
@@ -1998,51 +1998,6 @@ class TaskSetManagerSuite
19981998 assert(manager.resourceOffer(" exec1" , " host1" , ANY )._1.isEmpty)
19991999 }
20002000
2001- test(" SPARK-21040: Check speculative tasks are not launched when an executor" +
2002- " is decommissioned and the tasks running on it can finish within" +
2003- " the EXECUTOR_DECOMMISSION_KILL_INTERVAL" ) {
2004- sc = new SparkContext (" local" , " test" )
2005- sched = new FakeTaskScheduler (sc, (" exec1" , " host1" ), (" exec2" , " host2" ))
2006- val taskSet = FakeTask .createTaskSet(4 )
2007- sc.conf.set(config.SPECULATION_ENABLED , true )
2008- sc.conf.set(config.SPECULATION_MULTIPLIER , 1.5 )
2009- sc.conf.set(config.SPECULATION_QUANTILE , 0.5 )
2010- // Set high value for EXECUTOR_DECOMMISSION_KILL_INTERVAL
2011- sc.conf.set(config.EXECUTOR_DECOMMISSION_KILL_INTERVAL .key, " 50ms" )
2012- val clock = new ManualClock ()
2013- val manager = new TaskSetManager (sched, taskSet, MAX_TASK_FAILURES , clock = clock)
2014- val accumUpdatesByTask : Array [Seq [AccumulatorV2 [_, _]]] = taskSet.tasks.map { task =>
2015- task.metrics.internalAccums
2016- }
2017-
2018- // Offer resources for 4 tasks to start, 2 on each exec
2019- Seq (" exec1" -> " host1" , " exec2" -> " host2" ).foreach { case (exec, host) =>
2020- (0 until 2 ).foreach { _ =>
2021- val taskOption = manager.resourceOffer(exec, host, NO_PREF )._1
2022- assert(taskOption.isDefined)
2023- assert(taskOption.get.executorId === exec)
2024- }
2025- }
2026- assert(sched.startedTasks.toSet === Set (0 , 1 , 2 , 3 ))
2027-
2028- clock.advance(10 ) // time = 10ms
2029- // Complete the first 2 tasks and leave the other 2 tasks in running
2030- for (id <- Set (0 , 1 )) {
2031- manager.handleSuccessfulTask(id, createTaskResult(id, accumUpdatesByTask(id)))
2032- assert(sched.endedTasks(id) === Success )
2033- }
2034- // decommission exec-2. All tasks running on exec-2 (i.e. TASK 2,3) will be added to
2035- // executorDecommissionSpeculationTriggerTimeoutOpt
2036- // (TASK2 -> 60, TASK3 -> 60)
2037- manager.executorDecommission(" exec2" )
2038-
2039- // Since the EXECUTOR_DECOMMISSION_KILL_INTERVAL was high, so the already running tasks
2040- // on executor 2 still have chance to finish. So they should not be speculated.
2041- assert(! manager.checkSpeculatableTasks(0 ))
2042- clock.advance(5 ) // time = 15ms
2043- assert(! manager.checkSpeculatableTasks(0 ))
2044- }
2045-
20462001 test(" SPARK-29976 Regular speculation configs should still take effect even when a " +
20472002 " threshold is provided" ) {
20482003 val (manager, clock) = testSpeculationDurationSetup(
0 commit comments