Skip to content

Commit

Permalink
[SPARK-31267][SQL] Flaky test: WholeStageCodegenSparkSubmitSuite.Gene…
Browse files Browse the repository at this point in the history
…rated code on driver should not embed platform-specific constant

### What changes were proposed in this pull request?

Allow customized timeouts for `runSparkSubmit`, which will make flaky tests more likely to pass by using a larger timeout value.

I was able to reproduce the test failure on my laptop, which took 1.5 - 2 minutes to finish the test. After increasing the timeout, the test now can pass locally.

### Why are the changes needed?

This allows slow tests to use a larger timeout, so they are more likely to succeed.

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

No

### How was this patch tested?

The test was able to pass on my local env after the change.

Closes apache#28438 from tianshizz/SPARK-31267.

Authored-by: Tianshi Zhu <zhutianshirea@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
  • Loading branch information
tianshizz authored and huaxingao committed May 4, 2020
1 parent d8a2fa0 commit da32137
Show file tree
Hide file tree
Showing 2 changed files with 5 additions and 3 deletions.
Expand Up @@ -31,6 +31,7 @@ import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{FileStatus, FSDataInputStream, Path}
import org.scalatest.{BeforeAndAfterEach, Matchers}
import org.scalatest.concurrent.{Signaler, ThreadSignaler, TimeLimits}
import org.scalatest.time.Span
import org.scalatest.time.SpanSugar._

import org.apache.spark._
Expand Down Expand Up @@ -1419,7 +1420,7 @@ object SparkSubmitSuite extends SparkFunSuite with TimeLimits {
implicit val defaultSignaler: Signaler = ThreadSignaler

// NOTE: This is an expensive operation in terms of time (10 seconds+). Use sparingly.
def runSparkSubmit(args: Seq[String], root: String = ".."): Unit = {
def runSparkSubmit(args: Seq[String], root: String = "..", timeout: Span = 1.minute): Unit = {
val sparkHome = sys.props.getOrElse("spark.test.home", fail("spark.test.home is not set!"))
val sparkSubmitFile = if (Utils.isWindows) {
new File(s"$root\\bin\\spark-submit.cmd")
Expand All @@ -1432,7 +1433,7 @@ object SparkSubmitSuite extends SparkFunSuite with TimeLimits {
Map("SPARK_TESTING" -> "1", "SPARK_HOME" -> sparkHome))

try {
val exitCode = failAfter(1.minute) { process.waitFor() }
val exitCode = failAfter(timeout) { process.waitFor() }
if (exitCode != 0) {
fail(s"Process returned with exit code $exitCode. See the log4j logs for more detail.")
}
Expand Down
Expand Up @@ -19,6 +19,7 @@ package org.apache.spark.sql.execution

import org.scalatest.{Assertions, BeforeAndAfterEach, Matchers}
import org.scalatest.concurrent.TimeLimits
import org.scalatest.time.SpanSugar._

import org.apache.spark.{SparkFunSuite, TestUtils}
import org.apache.spark.deploy.SparkSubmitSuite
Expand Down Expand Up @@ -50,7 +51,7 @@ class WholeStageCodegenSparkSubmitSuite extends SparkFunSuite
"--conf", "spark.executor.extraJavaOptions=-XX:+UseCompressedOops",
"--conf", "spark.sql.adaptive.enabled=false",
unusedJar.toString)
SparkSubmitSuite.runSparkSubmit(argsForSparkSubmit, "../..")
SparkSubmitSuite.runSparkSubmit(argsForSparkSubmit, "../..", 3.minutes)
}
}

Expand Down

0 comments on commit da32137

Please sign in to comment.