From 527614681b74dd591f4cef1b3471e6d00240e6a1 Mon Sep 17 00:00:00 2001 From: Tianshi Zhu Date: Sat, 2 May 2020 20:10:55 -0400 Subject: [PATCH] add a timeout option for runSparkSubmit with a default of 1 minute --- .../scala/org/apache/spark/deploy/SparkSubmitSuite.scala | 5 +++-- .../sql/execution/WholeStageCodegenSparkSubmitSuite.scala | 3 ++- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index 1f3243400a91..fd2d1f56ed9b 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -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._ @@ -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") @@ -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.") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSparkSubmitSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSparkSubmitSuite.scala index f6814d8ff8a3..c5a01de91196 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSparkSubmitSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSparkSubmitSuite.scala @@ -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 @@ -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) } }