Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[SPARK-21939][TEST] Use TimeLimits instead of Timeouts #19150

Closed
wants to merge 2 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
Expand Up @@ -18,7 +18,7 @@
package org.apache.spark

import org.scalatest.Matchers
import org.scalatest.concurrent.Timeouts._
import org.scalatest.concurrent.TimeLimits._
import org.scalatest.time.{Millis, Span}

import org.apache.spark.security.EncryptionFunSuite
Expand Down
4 changes: 2 additions & 2 deletions core/src/test/scala/org/apache/spark/DriverSuite.scala
Expand Up @@ -19,13 +19,13 @@ package org.apache.spark

import java.io.File

import org.scalatest.concurrent.Timeouts
import org.scalatest.concurrent.TimeLimits
import org.scalatest.prop.TableDrivenPropertyChecks._
import org.scalatest.time.SpanSugar._

import org.apache.spark.util.Utils

class DriverSuite extends SparkFunSuite with Timeouts {
class DriverSuite extends SparkFunSuite with TimeLimits {

ignore("driver should exit after finishing without cleanup (SPARK-530)") {
val sparkHome = sys.props.getOrElse("spark.test.home", fail("spark.test.home is not set!"))
Expand Down
2 changes: 1 addition & 1 deletion core/src/test/scala/org/apache/spark/UnpersistSuite.scala
Expand Up @@ -17,7 +17,7 @@

package org.apache.spark

import org.scalatest.concurrent.Timeouts._
import org.scalatest.concurrent.TimeLimits._
import org.scalatest.time.{Millis, Span}

class UnpersistSuite extends SparkFunSuite with LocalSparkContext {
Expand Down
Expand Up @@ -31,7 +31,7 @@ import org.apache.commons.io.FileUtils
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{FileStatus, FSDataInputStream, Path}
import org.scalatest.{BeforeAndAfterEach, Matchers}
import org.scalatest.concurrent.Timeouts
import org.scalatest.concurrent.TimeLimits
import org.scalatest.time.SpanSugar._

import org.apache.spark._
Expand Down Expand Up @@ -97,7 +97,7 @@ class SparkSubmitSuite
with Matchers
with BeforeAndAfterEach
with ResetSystemProperties
with Timeouts
with TimeLimits
with TestPrematureExit {

override def beforeEach() {
Expand Down
Expand Up @@ -24,13 +24,13 @@ import scala.concurrent.ExecutionContext.Implicits.global
import scala.concurrent.duration.Duration

import org.scalatest.BeforeAndAfterAll
import org.scalatest.concurrent.Timeouts
import org.scalatest.concurrent.TimeLimits
import org.scalatest.time.SpanSugar._

import org.apache.spark._
import org.apache.spark.util.ThreadUtils

class AsyncRDDActionsSuite extends SparkFunSuite with BeforeAndAfterAll with Timeouts {
class AsyncRDDActionsSuite extends SparkFunSuite with BeforeAndAfterAll with TimeLimits {

@transient private var sc: SparkContext = _

Expand Down
Expand Up @@ -25,7 +25,7 @@ import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Map}
import scala.language.reflectiveCalls
import scala.util.control.NonFatal

import org.scalatest.concurrent.Timeouts
import org.scalatest.concurrent.TimeLimits
import org.scalatest.time.SpanSugar._

import org.apache.spark._
Expand Down Expand Up @@ -98,7 +98,7 @@ class MyRDD(

class DAGSchedulerSuiteDummyException extends Exception

class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with Timeouts {
class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLimits {

import DAGSchedulerSuite._

Expand Down
Expand Up @@ -18,7 +18,7 @@
package org.apache.spark.scheduler

import org.apache.hadoop.mapred.{FileOutputCommitter, TaskAttemptContext}
import org.scalatest.concurrent.Timeouts
import org.scalatest.concurrent.TimeLimits
import org.scalatest.time.{Seconds, Span}

import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkFunSuite, TaskContext}
Expand All @@ -32,7 +32,7 @@ import org.apache.spark.util.Utils
class OutputCommitCoordinatorIntegrationSuite
extends SparkFunSuite
with LocalSparkContext
with Timeouts {
with TimeLimits {

override def beforeAll(): Unit = {
super.beforeAll()
Expand Down
Expand Up @@ -33,7 +33,7 @@ import org.mockito.{Matchers => mc}
import org.mockito.Mockito.{mock, times, verify, when}
import org.scalatest._
import org.scalatest.concurrent.Eventually._
import org.scalatest.concurrent.Timeouts._
import org.scalatest.concurrent.TimeLimits._

import org.apache.spark._
import org.apache.spark.broadcast.BroadcastManager
Expand Down
Expand Up @@ -24,11 +24,11 @@ import scala.concurrent.duration._
import scala.language.postfixOps

import org.scalatest.concurrent.Eventually._
import org.scalatest.concurrent.Timeouts
import org.scalatest.concurrent.TimeLimits

import org.apache.spark.SparkFunSuite

class EventLoopSuite extends SparkFunSuite with Timeouts {
class EventLoopSuite extends SparkFunSuite with TimeLimits {

test("EventLoop") {
val buffer = new ConcurrentLinkedQueue[Int]
Expand Down
Expand Up @@ -24,7 +24,7 @@ import scala.collection.mutable
import org.eclipse.jetty.util.ConcurrentHashSet
import org.scalatest.concurrent.Eventually
import org.scalatest.concurrent.PatienceConfiguration.Timeout
import org.scalatest.concurrent.Timeouts._
import org.scalatest.concurrent.TimeLimits._
import org.scalatest.time.SpanSugar._

import org.apache.spark.SparkFunSuite
Expand Down
Expand Up @@ -27,7 +27,7 @@ import scala.util.Random
import scala.util.control.NonFatal

import org.scalatest.{Assertions, BeforeAndAfterAll}
import org.scalatest.concurrent.{Eventually, Timeouts}
import org.scalatest.concurrent.{Eventually, Signaler, ThreadSignaler, TimeLimits}
import org.scalatest.concurrent.PatienceConfiguration.Timeout
import org.scalatest.exceptions.TestFailedDueToTimeoutException
import org.scalatest.time.Span
Expand Down Expand Up @@ -67,8 +67,9 @@ import org.apache.spark.util.{Clock, SystemClock, Utils}
* avoid hanging forever in the case of failures. However, individual suites can change this
* by overriding `streamingTimeout`.
*/
trait StreamTest extends QueryTest with SharedSQLContext with Timeouts with BeforeAndAfterAll {
trait StreamTest extends QueryTest with SharedSQLContext with TimeLimits with BeforeAndAfterAll {

implicit val defaultSignaler: Signaler = ThreadSignaler
override def afterAll(): Unit = {
super.afterAll()
StateStore.stop() // stop the state store maintenance thread and unload store providers
Expand Down
Expand Up @@ -26,7 +26,7 @@ import scala.tools.nsc.Properties

import org.apache.hadoop.fs.Path
import org.scalatest.{BeforeAndAfterEach, Matchers}
import org.scalatest.concurrent.Timeouts
import org.scalatest.concurrent.TimeLimits
import org.scalatest.exceptions.TestFailedDueToTimeoutException
import org.scalatest.time.SpanSugar._

Expand All @@ -50,7 +50,7 @@ class HiveSparkSubmitSuite
with Matchers
with BeforeAndAfterEach
with ResetSystemProperties
with Timeouts {
with TimeLimits {

// TODO: rewrite these or mark them as slow tests to be run sparingly

Expand Down
Expand Up @@ -24,8 +24,9 @@ import scala.collection.mutable

import org.scalatest.BeforeAndAfter
import org.scalatest.Matchers._
import org.scalatest.concurrent.{Signaler, ThreadSignaler}
import org.scalatest.concurrent.Eventually._
import org.scalatest.concurrent.Timeouts._
import org.scalatest.concurrent.TimeLimits._
import org.scalatest.time.SpanSugar._

import org.apache.spark.{SparkConf, SparkException, SparkFunSuite}
Expand All @@ -34,6 +35,7 @@ import org.apache.spark.util.ManualClock

class BlockGeneratorSuite extends SparkFunSuite with BeforeAndAfter {

implicit val defaultSignaler: Signaler = ThreadSignaler
private val blockIntervalMs = 10
private val conf = new SparkConf().set("spark.streaming.blockInterval", s"${blockIntervalMs}ms")
@volatile private var blockGenerator: BlockGenerator = null
Expand Down