Skip to content

Commit

Permalink
[SPARK-23775][TEST] Make DataFrameRangeSuite not flaky
Browse files Browse the repository at this point in the history
## What changes were proposed in this pull request?

DataFrameRangeSuite.test("Cancelling stage in a query with Range.") stays sometimes in an infinite loop and times out the build.

There were multiple issues with the test:

1. The first valid stageId is zero when the test started alone and not in a suite and the following code waits until timeout:

```
eventually(timeout(10.seconds), interval(1.millis)) {
  assert(DataFrameRangeSuite.stageToKill > 0)
}
```

2. The `DataFrameRangeSuite.stageToKill` was overwritten by the task's thread after the reset which ended up in canceling the same stage 2 times. This caused the infinite wait.

This PR solves this mentioned flakyness by removing the shared `DataFrameRangeSuite.stageToKill` and using `wait` and `CountDownLatch` for synhronization.

## How was this patch tested?

Existing unit test.

Author: Gabor Somogyi <[email protected]>

Closes #20888 from gaborgsomogyi/SPARK-23775.
  • Loading branch information
gaborgsomogyi authored and Marcelo Vanzin committed Apr 18, 2018
1 parent a906647 commit 0c94e48
Showing 1 changed file with 45 additions and 33 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -17,14 +17,16 @@

package org.apache.spark.sql

import java.util.concurrent.{CountDownLatch, TimeUnit}

import scala.concurrent.duration._
import scala.math.abs
import scala.util.Random

import org.scalatest.concurrent.Eventually

import org.apache.spark.{SparkException, TaskContext}
import org.apache.spark.scheduler.{SparkListener, SparkListenerJobStart}
import org.apache.spark.{SparkContext, SparkException}
import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskStart}
import org.apache.spark.sql.functions._
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.test.SharedSQLContext
Expand Down Expand Up @@ -152,39 +154,53 @@ class DataFrameRangeSuite extends QueryTest with SharedSQLContext with Eventuall
}

test("Cancelling stage in a query with Range.") {
val listener = new SparkListener {
override def onJobStart(jobStart: SparkListenerJobStart): Unit = {
eventually(timeout(10.seconds), interval(1.millis)) {
assert(DataFrameRangeSuite.stageToKill > 0)
// Save and restore the value because SparkContext is shared
val savedInterruptOnCancel = sparkContext
.getLocalProperty(SparkContext.SPARK_JOB_INTERRUPT_ON_CANCEL)

try {
sparkContext.setLocalProperty(SparkContext.SPARK_JOB_INTERRUPT_ON_CANCEL, "true")

for (codegen <- Seq(true, false)) {
// This countdown latch used to make sure with all the stages cancelStage called in listener
val latch = new CountDownLatch(2)

val listener = new SparkListener {
override def onTaskStart(taskStart: SparkListenerTaskStart): Unit = {
sparkContext.cancelStage(taskStart.stageId)
latch.countDown()
}
}
sparkContext.cancelStage(DataFrameRangeSuite.stageToKill)
}
}

sparkContext.addSparkListener(listener)
for (codegen <- Seq(true, false)) {
withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> codegen.toString()) {
DataFrameRangeSuite.stageToKill = -1
val ex = intercept[SparkException] {
spark.range(0, 100000000000L, 1, 1).map { x =>
DataFrameRangeSuite.stageToKill = TaskContext.get().stageId()
x
}.toDF("id").agg(sum("id")).collect()
sparkContext.addSparkListener(listener)
withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> codegen.toString()) {
val ex = intercept[SparkException] {
sparkContext.range(0, 10000L, numSlices = 10).mapPartitions { x =>
x.synchronized {
x.wait()
}
x
}.toDF("id").agg(sum("id")).collect()
}
ex.getCause() match {
case null =>
assert(ex.getMessage().contains("cancelled"))
case cause: SparkException =>
assert(cause.getMessage().contains("cancelled"))
case cause: Throwable =>
fail("Expected the cause to be SparkException, got " + cause.toString() + " instead.")
}
}
ex.getCause() match {
case null =>
assert(ex.getMessage().contains("cancelled"))
case cause: SparkException =>
assert(cause.getMessage().contains("cancelled"))
case cause: Throwable =>
fail("Expected the cause to be SparkException, got " + cause.toString() + " instead.")
latch.await(20, TimeUnit.SECONDS)
eventually(timeout(20.seconds)) {
assert(sparkContext.statusTracker.getExecutorInfos.map(_.numRunningTasks()).sum == 0)
}
sparkContext.removeSparkListener(listener)
}
eventually(timeout(20.seconds)) {
assert(sparkContext.statusTracker.getExecutorInfos.map(_.numRunningTasks()).sum == 0)
}
} finally {
sparkContext.setLocalProperty(SparkContext.SPARK_JOB_INTERRUPT_ON_CANCEL,
savedInterruptOnCancel)
}
sparkContext.removeSparkListener(listener)
}

test("SPARK-20430 Initialize Range parameters in a driver side") {
Expand All @@ -204,7 +220,3 @@ class DataFrameRangeSuite extends QueryTest with SharedSQLContext with Eventuall
}
}
}

object DataFrameRangeSuite {
@volatile var stageToKill = -1
}

0 comments on commit 0c94e48

Please sign in to comment.