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-7558] Demarcate tests in unit-tests.log #6441

Closed
wants to merge 14 commits into from
Closed
  •  
  •  
  •  
7 changes: 7 additions & 0 deletions bagel/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,13 @@
<artifactId>spark-core_${scala.binary.version}</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-core_${scala.binary.version}</artifactId>
<version>${project.version}</version>
<type>test-jar</type>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.scalacheck</groupId>
<artifactId>scalacheck_${scala.binary.version}</artifactId>
Expand Down
4 changes: 2 additions & 2 deletions bagel/src/test/scala/org/apache/spark/bagel/BagelSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@

package org.apache.spark.bagel

import org.scalatest.{BeforeAndAfter, FunSuite, Assertions}
import org.scalatest.{BeforeAndAfter, Assertions}
import org.scalatest.concurrent.Timeouts
import org.scalatest.time.SpanSugar._

Expand All @@ -27,7 +27,7 @@ import org.apache.spark.storage.StorageLevel
class TestVertex(val active: Boolean, val age: Int) extends Vertex with Serializable
class TestMessage(val targetId: String) extends Message[String] with Serializable

class BagelSuite extends FunSuite with Assertions with BeforeAndAfter with Timeouts {
class BagelSuite extends SparkFunSuite with Assertions with BeforeAndAfter with Timeouts {

var sc: SparkContext = _

Expand Down
6 changes: 6 additions & 0 deletions core/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -338,6 +338,12 @@
<dependency>
<groupId>org.seleniumhq.selenium</groupId>
<artifactId>selenium-java</artifactId>
<exclusions>
<exclusion>
<groupId>com.google.guava</groupId>
<artifactId>guava</artifactId>
</exclusion>
</exclusions>
<scope>test</scope>
</dependency>
<!-- Added for selenium: -->
Expand Down
3 changes: 1 addition & 2 deletions core/src/test/scala/org/apache/spark/AccumulatorSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -20,11 +20,10 @@ package org.apache.spark
import scala.collection.mutable
import scala.ref.WeakReference

import org.scalatest.FunSuite
import org.scalatest.Matchers


class AccumulatorSuite extends FunSuite with Matchers with LocalSparkContext {
class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContext {


implicit def setAccum[A]: AccumulableParam[mutable.Set[A], A] =
Expand Down
4 changes: 2 additions & 2 deletions core/src/test/scala/org/apache/spark/CacheManagerSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -18,15 +18,15 @@
package org.apache.spark

import org.mockito.Mockito._
import org.scalatest.{BeforeAndAfter, FunSuite}
import org.scalatest.BeforeAndAfter
import org.scalatest.mock.MockitoSugar

import org.apache.spark.executor.DataReadMethod
import org.apache.spark.rdd.RDD
import org.apache.spark.storage._

// TODO: Test the CacheManager's thread-safety aspects
class CacheManagerSuite extends FunSuite with LocalSparkContext with BeforeAndAfter
class CacheManagerSuite extends SparkFunSuite with LocalSparkContext with BeforeAndAfter
with MockitoSugar {

var blockManager: BlockManager = _
Expand Down
4 changes: 1 addition & 3 deletions core/src/test/scala/org/apache/spark/CheckpointSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -21,13 +21,11 @@ import java.io.File

import scala.reflect.ClassTag

import org.scalatest.FunSuite

import org.apache.spark.rdd._
import org.apache.spark.storage.{BlockId, StorageLevel, TestBlockId}
import org.apache.spark.util.Utils

class CheckpointSuite extends FunSuite with LocalSparkContext with Logging {
class CheckpointSuite extends SparkFunSuite with LocalSparkContext with Logging {
var checkpointDir: File = _
val partitioner = new HashPartitioner(2)

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,7 @@ import scala.collection.mutable.{HashSet, SynchronizedSet}
import scala.language.existentials
import scala.util.Random

import org.scalatest.{BeforeAndAfter, FunSuite}
import org.scalatest.BeforeAndAfter
import org.scalatest.concurrent.{PatienceConfiguration, Eventually}
import org.scalatest.concurrent.Eventually._
import org.scalatest.time.SpanSugar._
Expand All @@ -44,7 +44,7 @@ import org.apache.spark.storage.ShuffleIndexBlockId
* config options, in particular, a different shuffle manager class
*/
abstract class ContextCleanerSuiteBase(val shuffleManager: Class[_] = classOf[HashShuffleManager])
extends FunSuite with BeforeAndAfter with LocalSparkContext
extends SparkFunSuite with BeforeAndAfter with LocalSparkContext
{
implicit val defaultTimeout = timeout(10000 millis)
val conf = new SparkConf()
Expand Down
3 changes: 1 addition & 2 deletions core/src/test/scala/org/apache/spark/DistributedSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,6 @@

package org.apache.spark

import org.scalatest.FunSuite
import org.scalatest.concurrent.Timeouts._
import org.scalatest.Matchers
import org.scalatest.time.{Millis, Span}
Expand All @@ -28,7 +27,7 @@ class NotSerializableClass
class NotSerializableExn(val notSer: NotSerializableClass) extends Throwable() {}


class DistributedSuite extends FunSuite with Matchers with LocalSparkContext {
class DistributedSuite extends SparkFunSuite with Matchers with LocalSparkContext {

val clusterUrl = "local-cluster[2,1,512]"

Expand Down
3 changes: 1 addition & 2 deletions core/src/test/scala/org/apache/spark/DriverSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -19,14 +19,13 @@ package org.apache.spark

import java.io.File

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

import org.apache.spark.util.Utils

class DriverSuite extends FunSuite with Timeouts {
class DriverSuite extends SparkFunSuite with Timeouts {

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
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,7 @@ package org.apache.spark

import scala.collection.mutable

import org.scalatest.{BeforeAndAfter, FunSuite, PrivateMethodTester}
import org.scalatest.{BeforeAndAfter, PrivateMethodTester}
import org.apache.spark.executor.TaskMetrics
import org.apache.spark.scheduler._
import org.apache.spark.scheduler.cluster.ExecutorInfo
Expand All @@ -28,7 +28,11 @@ import org.apache.spark.util.ManualClock
/**
* Test add and remove behavior of ExecutorAllocationManager.
*/
class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext with BeforeAndAfter {
class ExecutorAllocationManagerSuite
extends SparkFunSuite
with LocalSparkContext
with BeforeAndAfter {

import ExecutorAllocationManager._
import ExecutorAllocationManagerSuite._

Expand Down
4 changes: 1 addition & 3 deletions core/src/test/scala/org/apache/spark/FailureSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -17,8 +17,6 @@

package org.apache.spark

import org.scalatest.FunSuite

import org.apache.spark.util.NonSerializable

import java.io.NotSerializableException
Expand All @@ -38,7 +36,7 @@ object FailureSuiteState {
}
}

class FailureSuite extends FunSuite with LocalSparkContext {
class FailureSuite extends SparkFunSuite with LocalSparkContext {

// Run a 3-task map job in which task 1 deterministically fails once, and check
// whether the job completes successfully and we ran 4 tasks in total.
Expand Down
3 changes: 1 addition & 2 deletions core/src/test/scala/org/apache/spark/FileServerSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -24,13 +24,12 @@ import javax.net.ssl.SSLException

import com.google.common.io.{ByteStreams, Files}
import org.apache.commons.lang3.RandomUtils
import org.scalatest.FunSuite

import org.apache.spark.util.Utils

import SSLSampleConfigs._

class FileServerSuite extends FunSuite with LocalSparkContext {
class FileServerSuite extends SparkFunSuite with LocalSparkContext {

@transient var tmpDir: File = _
@transient var tmpFile: File = _
Expand Down
3 changes: 1 addition & 2 deletions core/src/test/scala/org/apache/spark/FileSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -30,12 +30,11 @@ import org.apache.hadoop.mapred.{JobConf, FileAlreadyExistsException, FileSplit,
import org.apache.hadoop.mapreduce.Job
import org.apache.hadoop.mapreduce.lib.input.{FileSplit => NewFileSplit, TextInputFormat => NewTextInputFormat}
import org.apache.hadoop.mapreduce.lib.output.{TextOutputFormat => NewTextOutputFormat}
import org.scalatest.FunSuite

import org.apache.spark.rdd.{NewHadoopRDD, HadoopRDD}
import org.apache.spark.util.Utils

class FileSuite extends FunSuite with LocalSparkContext {
class FileSuite extends SparkFunSuite with LocalSparkContext {
var tempDir: File = _

override def beforeEach() {
Expand Down
8 changes: 6 additions & 2 deletions core/src/test/scala/org/apache/spark/FutureActionSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -20,10 +20,14 @@ package org.apache.spark
import scala.concurrent.Await
import scala.concurrent.duration.Duration

import org.scalatest.{BeforeAndAfter, FunSuite, Matchers}
import org.scalatest.{BeforeAndAfter, Matchers}


class FutureActionSuite extends FunSuite with BeforeAndAfter with Matchers with LocalSparkContext {
class FutureActionSuite
extends SparkFunSuite
with BeforeAndAfter
with Matchers
with LocalSparkContext {

before {
sc = new SparkContext("local", "FutureActionSuite")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,6 @@ import scala.language.postfixOps

import org.apache.spark.executor.TaskMetrics
import org.apache.spark.storage.BlockManagerId
import org.scalatest.FunSuite
import org.mockito.Mockito.{mock, spy, verify, when}
import org.mockito.Matchers
import org.mockito.Matchers._
Expand All @@ -31,7 +30,7 @@ import org.apache.spark.scheduler.TaskScheduler
import org.apache.spark.util.RpcUtils
import org.scalatest.concurrent.Eventually._

class HeartbeatReceiverSuite extends FunSuite with LocalSparkContext {
class HeartbeatReceiverSuite extends SparkFunSuite with LocalSparkContext {

test("HeartbeatReceiver") {
sc = spy(new SparkContext("local[2]", "test"))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,11 +17,9 @@

package org.apache.spark

import org.scalatest.FunSuite

import org.apache.spark.rdd.RDD

class ImplicitOrderingSuite extends FunSuite with LocalSparkContext {
class ImplicitOrderingSuite extends SparkFunSuite with LocalSparkContext {
// Tests that PairRDDFunctions grabs an implicit Ordering in various cases where it should.
test("basic inference of Orderings"){
sc = new SparkContext("local", "test")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@ import scala.concurrent.ExecutionContext.Implicits.global
import scala.concurrent.duration._
import scala.concurrent.future

import org.scalatest.{BeforeAndAfter, FunSuite}
import org.scalatest.BeforeAndAfter
import org.scalatest.Matchers

import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskStart}
Expand All @@ -34,7 +34,7 @@ import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskStart}
* (e.g. count) as well as multi-job action (e.g. take). We test the local and cluster schedulers
* in both FIFO and fair scheduling modes.
*/
class JobCancellationSuite extends FunSuite with Matchers with BeforeAndAfter
class JobCancellationSuite extends SparkFunSuite with Matchers with BeforeAndAfter
with LocalSparkContext {

override def afterEach() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,14 +19,13 @@ package org.apache.spark

import org.mockito.Mockito._
import org.mockito.Matchers.{any, isA}
import org.scalatest.FunSuite

import org.apache.spark.rpc.{RpcAddress, RpcEndpointRef, RpcCallContext, RpcEnv}
import org.apache.spark.scheduler.{CompressedMapStatus, MapStatus}
import org.apache.spark.shuffle.FetchFailedException
import org.apache.spark.storage.BlockManagerId

class MapOutputTrackerSuite extends FunSuite {
class MapOutputTrackerSuite extends SparkFunSuite {
private val conf = new SparkConf

def createRpcEnv(name: String, host: String = "localhost", port: Int = 0,
Expand Down
4 changes: 2 additions & 2 deletions core/src/test/scala/org/apache/spark/PartitioningSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -20,12 +20,12 @@ package org.apache.spark
import scala.collection.mutable.ArrayBuffer
import scala.math.abs

import org.scalatest.{FunSuite, PrivateMethodTester}
import org.scalatest.PrivateMethodTester

import org.apache.spark.rdd.RDD
import org.apache.spark.util.StatCounter

class PartitioningSuite extends FunSuite with SharedSparkContext with PrivateMethodTester {
class PartitioningSuite extends SparkFunSuite with SharedSparkContext with PrivateMethodTester {

test("HashPartitioner equality") {
val p2 = new HashPartitioner(2)
Expand Down
4 changes: 2 additions & 2 deletions core/src/test/scala/org/apache/spark/SSLOptionsSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -21,9 +21,9 @@ import java.io.File

import com.google.common.io.Files
import org.apache.spark.util.Utils
import org.scalatest.{BeforeAndAfterAll, FunSuite}
import org.scalatest.BeforeAndAfterAll

class SSLOptionsSuite extends FunSuite with BeforeAndAfterAll {
class SSLOptionsSuite extends SparkFunSuite with BeforeAndAfterAll {

test("test resolving property file as spark conf ") {
val keyStorePath = new File(this.getClass.getResource("/keystore").toURI).getAbsolutePath
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,11 +19,9 @@ package org.apache.spark

import java.io.File

import org.scalatest.FunSuite

import org.apache.spark.util.Utils

class SecurityManagerSuite extends FunSuite {
class SecurityManagerSuite extends SparkFunSuite {

test("set security with conf") {
val conf = new SparkConf
Expand Down
3 changes: 1 addition & 2 deletions core/src/test/scala/org/apache/spark/ShuffleSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,6 @@

package org.apache.spark

import org.scalatest.FunSuite
import org.scalatest.Matchers

import org.apache.spark.ShuffleSuite.NonJavaSerializableClass
Expand All @@ -26,7 +25,7 @@ import org.apache.spark.serializer.KryoSerializer
import org.apache.spark.storage.{ShuffleDataBlockId, ShuffleBlockId}
import org.apache.spark.util.MutablePair

abstract class ShuffleSuite extends FunSuite with Matchers with LocalSparkContext {
abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkContext {

val conf = new SparkConf(loadDefaults = false)

Expand Down
3 changes: 1 addition & 2 deletions core/src/test/scala/org/apache/spark/SparkConfSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -23,13 +23,12 @@ import scala.concurrent.duration._
import scala.language.postfixOps
import scala.util.{Try, Random}

import org.scalatest.FunSuite
import org.apache.spark.network.util.ByteUnit
import org.apache.spark.serializer.{KryoRegistrator, KryoSerializer}
import org.apache.spark.util.{RpcUtils, ResetSystemProperties}
import com.esotericsoftware.kryo.Kryo

class SparkConfSuite extends FunSuite with LocalSparkContext with ResetSystemProperties {
class SparkConfSuite extends SparkFunSuite with LocalSparkContext with ResetSystemProperties {
test("Test byteString conversion") {
val conf = new SparkConf()
// Simply exercise the API, we don't need a complete conversion test since that's handled in
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,10 +17,10 @@

package org.apache.spark

import org.scalatest.{Assertions, FunSuite}
import org.scalatest.Assertions
import org.apache.spark.storage.StorageLevel

class SparkContextInfoSuite extends FunSuite with LocalSparkContext {
class SparkContextInfoSuite extends SparkFunSuite with LocalSparkContext {
test("getPersistentRDDs only returns RDDs that are marked as cached") {
sc = new SparkContext("local", "test")
assert(sc.getPersistentRDDs.isEmpty === true)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,15 +17,15 @@

package org.apache.spark

import org.scalatest.{FunSuite, PrivateMethodTester}
import org.scalatest.PrivateMethodTester

import org.apache.spark.scheduler.{SchedulerBackend, TaskScheduler, TaskSchedulerImpl}
import org.apache.spark.scheduler.cluster.{SimrSchedulerBackend, SparkDeploySchedulerBackend}
import org.apache.spark.scheduler.cluster.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend}
import org.apache.spark.scheduler.local.LocalBackend

class SparkContextSchedulerCreationSuite
extends FunSuite with LocalSparkContext with PrivateMethodTester with Logging {
extends SparkFunSuite with LocalSparkContext with PrivateMethodTester with Logging {

def createTaskScheduler(master: String): TaskSchedulerImpl =
createTaskScheduler(master, new SparkConf())
Expand Down
Loading