Skip to content

Commit

Permalink
[SPARK-44054][CORE][TESTS] Make test cases inherit SparkFunSuite ha…
Browse files Browse the repository at this point in the history
…ve a default timeout

### What changes were proposed in this pull request?
This pr use `failAfter` to wrap the `testBody` of `SparkFunSuite#test` to control the test timeout, and add an un-document config `spark.test.timeout` with default value 20 minutes in this pr, the test inherit `SparkFunSuite` will fail with `TestFailedDueToTimeoutException` when test timeout.

### Why are the changes needed?
Avoid GA task times out due to test case blocks.

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
- Pass Github Actions
- manual checked.

Closes apache#41590 from LuciferYang/add-failAfter.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
  • Loading branch information
LuciferYang authored and dongjoon-hyun committed Jun 19, 2023
1 parent 1008d64 commit 74185cf
Show file tree
Hide file tree
Showing 2 changed files with 7 additions and 2 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,6 @@ import org.apache.kafka.clients.producer.ProducerConfig
import org.apache.kafka.clients.producer.internals.DefaultPartitioner
import org.apache.kafka.common.Cluster
import org.apache.kafka.common.serialization.ByteArraySerializer
import org.scalatest.concurrent.TimeLimits.failAfter
import org.scalatest.time.SpanSugar._

import org.apache.spark.{SparkConf, SparkContext, SparkException, TestUtils}
Expand Down
8 changes: 7 additions & 1 deletion core/src/test/scala/org/apache/spark/SparkFunSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,9 @@ import org.apache.logging.log4j.core.appender.AbstractAppender
import org.apache.logging.log4j.core.config.Property
import org.scalactic.source.Position
import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, BeforeAndAfterEach, Failed, Outcome, Tag}
import org.scalatest.concurrent.TimeLimits
import org.scalatest.funsuite.AnyFunSuite // scalastyle:ignore funsuite
import org.scalatest.time._ // scalastyle:ignore

import org.apache.spark.deploy.LocalSparkCluster
import org.apache.spark.internal.Logging
Expand Down Expand Up @@ -69,6 +71,7 @@ abstract class SparkFunSuite
with BeforeAndAfterAll
with BeforeAndAfterEach
with ThreadAudit
with TimeLimits
with Logging {
// scalastyle:on

Expand Down Expand Up @@ -147,7 +150,10 @@ abstract class SparkFunSuite
if (excluded.contains(testName)) {
ignore(s"$testName (excluded)")(testBody)
} else {
super.test(testName, testTags: _*)(testBody)
val timeout = sys.props.getOrElse("spark.test.timeout", "20").toLong
super.test(testName, testTags: _*)(
failAfter(Span(timeout, Minutes))(testBody)
)
}
}

Expand Down

0 comments on commit 74185cf

Please sign in to comment.