Skip to content
Closed
Show file tree
Hide file tree
Changes from 1 commit
Commits
Show all changes
22 commits
Select commit Hold shift + click to select a range
9f5b195
[SPARK-26329][CORE] Faster polling of executor memory metrics.
wypoon Jan 4, 2019
03e41a8
[SPARK-26329][CORE] Fix test compilation error in sql.
wypoon Feb 12, 2019
7f6bd74
[SPARK-26329][CORE] Fix Mima issues.
wypoon Feb 13, 2019
7397897
[SPARK-26329][CORE] Fix possible NPE.
wypoon Feb 13, 2019
e1aeafc
[SPARK-26329][CORE] Fix JsonProtocolSuite post-rebase to account for …
wypoon Mar 6, 2019
75ba39d
[SPARK-26329][CORE] Extract polling logic into a separate class.
wypoon Mar 8, 2019
ea2ff0d
[SPARK-26329][CORE] On task failure, send executor metrics in the Tas…
wypoon Mar 18, 2019
0cbfc04
[SPARK-26329][CORE] Unit tests for sending executor metrics in TaskRe…
wypoon Mar 21, 2019
8cb30a8
[SPARK-26329][CORE] Add driver updates to test for executor metrics a…
wypoon Mar 22, 2019
077abb0
[SPARK-26329][CORE] Add SparkListenerTaskEnd events to test for execu…
wypoon Mar 23, 2019
7a3c90d
[SPARK-26329][CORE] Address feedback from irashid.
wypoon Mar 27, 2019
3ed583a
[SPARK-26329][CORE] Fix ExecutorSuite failures.
wypoon Mar 28, 2019
0a4828a
[SPARK-26329][CORE] Delete a comment on irashid's suggestion.
wypoon Mar 28, 2019
9530b75
[SPARK-26329][CORE] Change executorUpdates to be a scala.collection.m…
wypoon Mar 28, 2019
38a397c
[SPARK-26329][CORE] Update HistoryServerSuite.
wypoon Apr 9, 2019
e062e60
[SPARK-26329][CORE] Get executor updates and reset the peaks in a sin…
wypoon May 15, 2019
20b4b7e
[SPARK-26329][CORE] Test fixes after rebase on master.
wypoon Jul 3, 2019
b898ad2
[SPARK-26329][CORE] Adopt some suggestions from attilapiros.
wypoon Jul 4, 2019
fbb55bf
[SPARK-26329][CORE] Address feedback from Imran Rashid.
wypoon Jul 19, 2019
99addf1
[SPARK-26329][CORE] Make TCMP case class private.
wypoon Jul 19, 2019
7331b27
[SPARK-26329][CORE] Fix a test post-rebase.
wypoon Jul 29, 2019
7556d6a
[SPARK-26329][CORE] Update a doc comment based on feedback from Imran…
wypoon Jul 31, 2019
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
Prev Previous commit
Next Next commit
[SPARK-26329][CORE] Test fixes after rebase on master.
  • Loading branch information
wypoon committed Jul 29, 2019
commit 20b4b7e2efdfcadf4f8ace132087924539fbe3cb
Original file line number Diff line number Diff line change
Expand Up @@ -1032,7 +1032,7 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite {
private def onExecutorIdle(manager: ExecutorAllocationManager, id: String): Unit = {
val info = new TaskInfo(1, 1, 1, 0, id, "foo.example.com", TaskLocality.PROCESS_LOCAL, false)
info.markFinished(TaskState.FINISHED, 1)
post(SparkListenerTaskEnd(1, 1, "foo", Success, info, null))
post(SparkListenerTaskEnd(1, 1, "foo", Success, info, new ExecutorMetrics, null))
}

private def removeExecutor(manager: ExecutorAllocationManager, executorId: String): Boolean = {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@ import org.mockito.ArgumentMatchers.any
import org.mockito.Mockito.{doAnswer, mock, when}

import org.apache.spark._
import org.apache.spark.executor.ExecutorMetrics
import org.apache.spark.internal.config._
import org.apache.spark.scheduler._
import org.apache.spark.storage._
Expand Down Expand Up @@ -102,11 +103,13 @@ class ExecutorMonitorSuite extends SparkFunSuite {
monitor.onTaskStart(SparkListenerTaskStart(i, 1, taskInfo("1", 1)))
assert(!monitor.isExecutorIdle("1"))

monitor.onTaskEnd(SparkListenerTaskEnd(i, 1, "foo", Success, taskInfo("1", 1), null))
monitor.onTaskEnd(SparkListenerTaskEnd(i, 1, "foo", Success, taskInfo("1", 1),
new ExecutorMetrics, null))
assert(!monitor.isExecutorIdle("1"))
}

monitor.onTaskEnd(SparkListenerTaskEnd(1, 1, "foo", Success, taskInfo("1", 1), null))
monitor.onTaskEnd(SparkListenerTaskEnd(1, 1, "foo", Success, taskInfo("1", 1),
new ExecutorMetrics, null))
assert(monitor.isExecutorIdle("1"))
assert(monitor.timedOutExecutors(clock.getTimeMillis()).isEmpty)
assert(monitor.timedOutExecutors(clock.getTimeMillis() + idleTimeoutMs + 1) === Seq("1"))
Expand Down Expand Up @@ -256,7 +259,8 @@ class ExecutorMonitorSuite extends SparkFunSuite {
monitor.executorsKilled(Seq("3"))
assert(monitor.pendingRemovalCount === 2)

monitor.onTaskEnd(SparkListenerTaskEnd(1, 1, "foo", Success, taskInfo("2", 1), null))
monitor.onTaskEnd(SparkListenerTaskEnd(1, 1, "foo", Success, taskInfo("2", 1),
new ExecutorMetrics, null))
assert(monitor.timedOutExecutors().isEmpty)
clock.advance(idleDeadline)
assert(monitor.timedOutExecutors().toSet === Set("2"))
Expand Down