diff --git a/core/src/test/scala/org/apache/spark/deploy/master/ui/MasterWebUISuite.scala b/core/src/test/scala/org/apache/spark/deploy/master/ui/MasterWebUISuite.scala index 69a460fbc7dba..f4558aa3eb893 100644 --- a/core/src/test/scala/org/apache/spark/deploy/master/ui/MasterWebUISuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/master/ui/MasterWebUISuite.scala @@ -53,8 +53,11 @@ class MasterWebUISuite extends SparkFunSuite with BeforeAndAfterAll { } override def afterAll() { - masterWebUI.stop() - super.afterAll() + try { + masterWebUI.stop() + } finally { + super.afterAll() + } } test("kill application") { diff --git a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala index 4324cc6d0f804..9241b13c100f1 100644 --- a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala +++ b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala @@ -50,13 +50,18 @@ class FlumePollingStreamSuite extends SparkFunSuite with BeforeAndAfterAll with val utils = new PollingFlumeTestUtils override def beforeAll(): Unit = { + super.beforeAll() _sc = new SparkContext(conf) } override def afterAll(): Unit = { - if (_sc != null) { - _sc.stop() - _sc = null + try { + if (_sc != null) { + _sc.stop() + _sc = null + } + } finally { + super.afterAll() } } diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala index eb186970fc25d..8cfca56433f5d 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala @@ -48,9 +48,12 @@ class KafkaRelationSuite extends QueryTest with SharedSQLContext with KafkaTest } override def afterAll(): Unit = { - if (testUtils != null) { - testUtils.teardown() - testUtils = null + try { + if (testUtils != null) { + testUtils.teardown() + testUtils = null + } + } finally { super.afterAll() } } diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala index a2213e024bd98..81832fbdcd7ec 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala @@ -48,9 +48,12 @@ class KafkaSinkSuite extends StreamTest with SharedSQLContext with KafkaTest { } override def afterAll(): Unit = { - if (testUtils != null) { - testUtils.teardown() - testUtils = null + try { + if (testUtils != null) { + testUtils.teardown() + testUtils = null + } + } finally { super.afterAll() } } diff --git a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/DirectKafkaStreamSuite.scala b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/DirectKafkaStreamSuite.scala index 661b67a8ab68a..1974bb1e12e15 100644 --- a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/DirectKafkaStreamSuite.scala +++ b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/DirectKafkaStreamSuite.scala @@ -59,14 +59,19 @@ class DirectKafkaStreamSuite private var kafkaTestUtils: KafkaTestUtils = _ override def beforeAll { + super.beforeAll() kafkaTestUtils = new KafkaTestUtils kafkaTestUtils.setup() } override def afterAll { - if (kafkaTestUtils != null) { - kafkaTestUtils.teardown() - kafkaTestUtils = null + try { + if (kafkaTestUtils != null) { + kafkaTestUtils.teardown() + kafkaTestUtils = null + } + } finally { + super.afterAll() } } diff --git a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/KafkaRDDSuite.scala b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/KafkaRDDSuite.scala index 3ac6509b04707..561bca5f55370 100644 --- a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/KafkaRDDSuite.scala +++ b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/KafkaRDDSuite.scala @@ -44,20 +44,27 @@ class KafkaRDDSuite extends SparkFunSuite with BeforeAndAfterAll { private var sc: SparkContext = _ override def beforeAll { + super.beforeAll() sc = new SparkContext(sparkConf) kafkaTestUtils = new KafkaTestUtils kafkaTestUtils.setup() } override def afterAll { - if (sc != null) { - sc.stop - sc = null - } - - if (kafkaTestUtils != null) { - kafkaTestUtils.teardown() - kafkaTestUtils = null + try { + try { + if (sc != null) { + sc.stop + sc = null + } + } finally { + if (kafkaTestUtils != null) { + kafkaTestUtils.teardown() + kafkaTestUtils = null + } + } + } finally { + super.afterAll() } } diff --git a/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala b/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala index ecca38784e777..3fd37f4c8ac90 100644 --- a/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala +++ b/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala @@ -57,14 +57,19 @@ class DirectKafkaStreamSuite private var kafkaTestUtils: KafkaTestUtils = _ override def beforeAll { + super.beforeAll() kafkaTestUtils = new KafkaTestUtils kafkaTestUtils.setup() } override def afterAll { - if (kafkaTestUtils != null) { - kafkaTestUtils.teardown() - kafkaTestUtils = null + try { + if (kafkaTestUtils != null) { + kafkaTestUtils.teardown() + kafkaTestUtils = null + } + } finally { + super.afterAll() } } diff --git a/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/KafkaClusterSuite.scala b/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/KafkaClusterSuite.scala index d66830cbacdee..73d528518d486 100644 --- a/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/KafkaClusterSuite.scala +++ b/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/KafkaClusterSuite.scala @@ -32,6 +32,7 @@ class KafkaClusterSuite extends SparkFunSuite with BeforeAndAfterAll { private var kafkaTestUtils: KafkaTestUtils = _ override def beforeAll() { + super.beforeAll() kafkaTestUtils = new KafkaTestUtils kafkaTestUtils.setup() @@ -41,9 +42,13 @@ class KafkaClusterSuite extends SparkFunSuite with BeforeAndAfterAll { } override def afterAll() { - if (kafkaTestUtils != null) { - kafkaTestUtils.teardown() - kafkaTestUtils = null + try { + if (kafkaTestUtils != null) { + kafkaTestUtils.teardown() + kafkaTestUtils = null + } + } finally { + super.afterAll() } } diff --git a/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala b/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala index 809699a739962..72f954149fefe 100644 --- a/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala +++ b/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala @@ -35,20 +35,27 @@ class KafkaRDDSuite extends SparkFunSuite with BeforeAndAfterAll { private var sc: SparkContext = _ override def beforeAll { + super.beforeAll() sc = new SparkContext(sparkConf) kafkaTestUtils = new KafkaTestUtils kafkaTestUtils.setup() } override def afterAll { - if (sc != null) { - sc.stop - sc = null - } - - if (kafkaTestUtils != null) { - kafkaTestUtils.teardown() - kafkaTestUtils = null + try { + try { + if (sc != null) { + sc.stop + sc = null + } + } finally { + if (kafkaTestUtils != null) { + kafkaTestUtils.teardown() + kafkaTestUtils = null + } + } + } finally { + super.afterAll() } } diff --git a/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala b/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala index 426cd83b4ddf8..ed130f5990955 100644 --- a/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala +++ b/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala @@ -35,19 +35,26 @@ class KafkaStreamSuite extends SparkFunSuite with Eventually with BeforeAndAfter private var kafkaTestUtils: KafkaTestUtils = _ override def beforeAll(): Unit = { + super.beforeAll() kafkaTestUtils = new KafkaTestUtils kafkaTestUtils.setup() } override def afterAll(): Unit = { - if (ssc != null) { - ssc.stop() - ssc = null - } - - if (kafkaTestUtils != null) { - kafkaTestUtils.teardown() - kafkaTestUtils = null + try { + try { + if (ssc != null) { + ssc.stop() + ssc = null + } + } finally { + if (kafkaTestUtils != null) { + kafkaTestUtils.teardown() + kafkaTestUtils = null + } + } + } finally { + super.afterAll() } } diff --git a/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala b/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala index 57f89cc7dbc65..5da5ea49d77ed 100644 --- a/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala +++ b/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala @@ -51,6 +51,7 @@ class ReliableKafkaStreamSuite extends SparkFunSuite private var tempDirectory: File = null override def beforeAll(): Unit = { + super.beforeAll() kafkaTestUtils = new KafkaTestUtils kafkaTestUtils.setup() @@ -65,11 +66,15 @@ class ReliableKafkaStreamSuite extends SparkFunSuite } override def afterAll(): Unit = { - Utils.deleteRecursively(tempDirectory) + try { + Utils.deleteRecursively(tempDirectory) - if (kafkaTestUtils != null) { - kafkaTestUtils.teardown() - kafkaTestUtils = null + if (kafkaTestUtils != null) { + kafkaTestUtils.teardown() + kafkaTestUtils = null + } + } finally { + super.afterAll() } } diff --git a/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisInputDStreamBuilderSuite.scala b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisInputDStreamBuilderSuite.scala index e0e26847aa0ec..361520e292266 100644 --- a/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisInputDStreamBuilderSuite.scala +++ b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisInputDStreamBuilderSuite.scala @@ -40,7 +40,11 @@ class KinesisInputDStreamBuilderSuite extends TestSuiteBase with BeforeAndAfterE .checkpointAppName(checkpointAppName) override def afterAll(): Unit = { - ssc.stop() + try { + ssc.stop() + } finally { + super.afterAll() + } } test("should raise an exception if the StreamingContext is missing") { diff --git a/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisStreamSuite.scala b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisStreamSuite.scala index a7a68eba910bf..6d27445c5b606 100644 --- a/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisStreamSuite.scala +++ b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisStreamSuite.scala @@ -71,17 +71,21 @@ abstract class KinesisStreamTests(aggregateTestData: Boolean) extends KinesisFun } override def afterAll(): Unit = { - if (ssc != null) { - ssc.stop() - } - if (sc != null) { - sc.stop() - } - if (testUtils != null) { - // Delete the Kinesis stream as well as the DynamoDB table generated by - // Kinesis Client Library when consuming the stream - testUtils.deleteStream() - testUtils.deleteDynamoDBTable(appName) + try { + if (ssc != null) { + ssc.stop() + } + if (sc != null) { + sc.stop() + } + if (testUtils != null) { + // Delete the Kinesis stream as well as the DynamoDB table generated by + // Kinesis Client Library when consuming the stream + testUtils.deleteStream() + testUtils.deleteDynamoDBTable(appName) + } + } finally { + super.afterAll() } } diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala index 82e6efa2707d9..18541baf05813 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala @@ -62,6 +62,7 @@ private[spark] class KubernetesSuite extends SparkFunSuite s"${(1024 + memOverheadConstant*1024 + additionalMemory).toInt}Mi" override def beforeAll(): Unit = { + super.beforeAll() // The scalatest-maven-plugin gives system properties that are referenced but not set null // values. We need to remove the null-value properties before initializing the test backend. val nullValueProperties = System.getProperties.asScala @@ -93,7 +94,11 @@ private[spark] class KubernetesSuite extends SparkFunSuite } override def afterAll(): Unit = { - testBackend.cleanUp() + try { + testBackend.cleanUp() + } finally { + super.afterAll() + } } before { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SessionStateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SessionStateSuite.scala index 7d1366092d1e6..e1b5eba53f06a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SessionStateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SessionStateSuite.scala @@ -41,13 +41,16 @@ class SessionStateSuite extends SparkFunSuite { } override def afterAll(): Unit = { - if (activeSession != null) { - activeSession.stop() - activeSession = null - SparkSession.clearActiveSession() - SparkSession.clearDefaultSession() + try { + if (activeSession != null) { + activeSession.stop() + activeSession = null + SparkSession.clearActiveSession() + SparkSession.clearDefaultSession() + } + } finally { + super.afterAll() } - super.afterAll() } test("fork new session and inherit RuntimeConfig options") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala index 41de731d41f82..c627c51655c8d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala @@ -31,6 +31,7 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { private var originalInstantiatedSparkSession: Option[SparkSession] = _ override protected def beforeAll(): Unit = { + super.beforeAll() originalActiveSparkSession = SparkSession.getActiveSession originalInstantiatedSparkSession = SparkSession.getDefaultSession @@ -39,9 +40,13 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { } override protected def afterAll(): Unit = { - // Set these states back. - originalActiveSparkSession.foreach(ctx => SparkSession.setActiveSession(ctx)) - originalInstantiatedSparkSession.foreach(ctx => SparkSession.setDefaultSession(ctx)) + try { + // Set these states back. + originalActiveSparkSession.foreach(ctx => SparkSession.setActiveSession(ctx)) + originalInstantiatedSparkSession.foreach(ctx => SparkSession.setDefaultSession(ctx)) + } finally { + super.afterAll() + } } private def checkEstimation( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExternalAppendOnlyUnsafeRowArraySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ExternalAppendOnlyUnsafeRowArraySuite.scala index ecc7264d79442..b29de9c4adbaa 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExternalAppendOnlyUnsafeRowArraySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ExternalAppendOnlyUnsafeRowArraySuite.scala @@ -29,7 +29,11 @@ class ExternalAppendOnlyUnsafeRowArraySuite extends SparkFunSuite with LocalSpar private val random = new java.util.Random() private var taskContext: TaskContext = _ - override def afterAll(): Unit = TaskContext.unset() + override def afterAll(): Unit = try { + TaskContext.unset() + } finally { + super.afterAll() + } private def withExternalArray(inMemoryThreshold: Int, spillThreshold: Int) (f: ExternalAppendOnlyUnsafeRowArray => Unit): Unit = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregationStoreSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregationStoreSuite.scala index 3fad7dfddadcc..dc67446460877 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregationStoreSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregationStoreSuite.scala @@ -39,7 +39,11 @@ class SortBasedAggregationStoreSuite extends SparkFunSuite with LocalSparkConte new TaskContextImpl(0, 0, 0, 0, 0, taskManager, new Properties, null)) } - override def afterAll(): Unit = TaskContext.unset() + override def afterAll(): Unit = try { + TaskContext.unset() + } finally { + super.afterAll() + } private val rand = new java.util.Random() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/WideSchemaBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/WideSchemaBenchmark.scala index a42891e55a18a..c368f17a84364 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/WideSchemaBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/WideSchemaBenchmark.scala @@ -54,8 +54,11 @@ class WideSchemaBenchmark extends SparkFunSuite with BeforeAndAfterEach { } override def afterAll() { - super.afterAll() - out.close() + try { + out.close() + } finally { + super.afterAll() + } } override def afterEach() { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/BasicWriteTaskStatsTrackerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/BasicWriteTaskStatsTrackerSuite.scala index bf3c8ede9a980..32941d8d2cd11 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/BasicWriteTaskStatsTrackerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/BasicWriteTaskStatsTrackerSuite.scala @@ -49,7 +49,11 @@ class BasicWriteTaskStatsTrackerSuite extends SparkFunSuite { * In teardown delete the temp dir. */ protected override def afterAll(): Unit = { - Utils.deleteRecursively(tempDir) + try { + Utils.deleteRecursively(tempDir) + } finally { + super.afterAll() + } } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala index bcdee792f4c70..b4ad1db20a9ea 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala @@ -54,8 +54,12 @@ class BroadcastJoinSuite extends QueryTest with SQLTestUtils { } override def afterAll(): Unit = { - spark.stop() - spark = null + try { + spark.stop() + spark = null + } finally { + super.afterAll() + } } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExecSuite.scala index 2cc55ff88b983..289cc667a1c66 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExecSuite.scala @@ -37,8 +37,11 @@ class BatchEvalPythonExecSuite extends SparkPlanTest with SharedSQLContext { } override def afterAll(): Unit = { - spark.sessionState.functionRegistry.dropFunction(FunctionIdentifier("dummyPythonUDF")) - super.afterAll() + try { + spark.sessionState.functionRegistry.dropFunction(FunctionIdentifier("dummyPythonUDF")) + } finally { + super.afterAll() + } } test("Python UDF: push down deterministic FilterExec predicates") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreRDDSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreRDDSuite.scala index 579a364ebc3e5..015415a534ff5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreRDDSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreRDDSuite.scala @@ -49,8 +49,11 @@ class StateStoreRDDSuite extends SparkFunSuite with BeforeAndAfter with BeforeAn } override def afterAll(): Unit = { - super.afterAll() - Utils.deleteRecursively(new File(tempDir)) + try { + super.afterAll() + } finally { + Utils.deleteRecursively(new File(tempDir)) + } } test("versioning and immutability") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/ExecutorSideSQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/ExecutorSideSQLConfSuite.scala index 5b4736ef4f7f3..d885348f3774a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/ExecutorSideSQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/ExecutorSideSQLConfSuite.scala @@ -38,8 +38,12 @@ class ExecutorSideSQLConfSuite extends SparkFunSuite with SQLTestUtils { } override def afterAll(): Unit = { - spark.stop() - spark = null + try { + spark.stop() + spark = null + } finally { + super.afterAll() + } } override def withSQLConf(pairs: (String, String)*)(f: => Unit): Unit = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateSuite.scala index e77ba1ec9f1eb..43463a84093ce 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateSuite.scala @@ -45,19 +45,13 @@ case class RunningCount(count: Long) case class Result(key: Long, count: Int) -class FlatMapGroupsWithStateSuite extends StateStoreMetricsTest - with BeforeAndAfterAll { +class FlatMapGroupsWithStateSuite extends StateStoreMetricsTest { import testImplicits._ import GroupStateImpl._ import GroupStateTimeout._ import FlatMapGroupsWithStateSuite._ - override def afterAll(): Unit = { - super.afterAll() - StateStore.stop() - } - test("GroupState - get, exists, update, remove") { var state: GroupStateImpl[String] = null diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala index 491dc34afa143..d878c345c2988 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala @@ -79,8 +79,11 @@ trait StreamTest extends QueryTest with SharedSQLContext with TimeLimits with Be implicit val defaultSignaler: Signaler = ThreadSignaler override def afterAll(): Unit = { - super.afterAll() - StateStore.stop() // stop the state store maintenance thread and unload store providers + try { + super.afterAll() + } finally { + StateStore.stop() // stop the state store maintenance thread and unload store providers + } } protected val defaultTrigger = Trigger.ProcessingTime(0) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala index 1ae6ff3a90989..97dbb9b0360ec 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala @@ -46,13 +46,7 @@ object FailureSingleton { var firstTime = true } -class StreamingAggregationSuite extends StateStoreMetricsTest - with BeforeAndAfterAll with Assertions { - - override def afterAll(): Unit = { - super.afterAll() - StateStore.stop() - } +class StreamingAggregationSuite extends StateStoreMetricsTest with Assertions { import testImplicits._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala index 42ffd472eb843..cfd7204ea2931 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala @@ -26,15 +26,10 @@ import org.apache.spark.sql.execution.streaming.state.StateStore import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf -class StreamingDeduplicationSuite extends StateStoreMetricsTest with BeforeAndAfterAll { +class StreamingDeduplicationSuite extends StateStoreMetricsTest { import testImplicits._ - override def afterAll(): Unit = { - super.afterAll() - StateStore.stop() - } - test("deduplicate with all columns") { val inputData = MemoryStream[String] val result = inputData.toDS().dropDuplicates() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSQLContext.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSQLContext.scala index e6c7648c986ae..0dd24d2d56b82 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSQLContext.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSQLContext.scala @@ -35,7 +35,10 @@ trait SharedSQLContext extends SQLTestUtils with SharedSparkSession { } protected override def afterAll(): Unit = { - super.afterAll() - doThreadPostAudit() + try { + super.afterAll() + } finally { + doThreadPostAudit() + } } } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/UISeleniumSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/UISeleniumSuite.scala index 4c53dd8f4616c..fef18f147b057 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/UISeleniumSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/UISeleniumSuite.scala @@ -46,10 +46,13 @@ class UISeleniumSuite } override def afterAll(): Unit = { - if (webDriver != null) { - webDriver.quit() + try { + if (webDriver != null) { + webDriver.quit() + } + } finally { + super.afterAll() } - super.afterAll() } override protected def serverStartCommand(port: Int) = { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala index 25df3339e62f3..a7d6972fa71f7 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala @@ -49,10 +49,13 @@ class HiveExternalCatalogVersionsSuite extends SparkSubmitTestUtils { private val unusedJar = TestUtils.createJarWithClasses(Seq.empty) override def afterAll(): Unit = { - Utils.deleteRecursively(wareHousePath) - Utils.deleteRecursively(tmpDataDir) - Utils.deleteRecursively(sparkTestingDir) - super.afterAll() + try { + Utils.deleteRecursively(wareHousePath) + Utils.deleteRecursively(tmpDataDir) + Utils.deleteRecursively(sparkTestingDir) + } finally { + super.afterAll() + } } private def tryDownloadSpark(version: String, path: String): Unit = { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDAFSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDAFSuite.scala index 7402c9626873c..fe3deceb08067 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDAFSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDAFSuite.scala @@ -37,6 +37,7 @@ class HiveUDAFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { import testImplicits._ protected override def beforeAll(): Unit = { + super.beforeAll() sql(s"CREATE TEMPORARY FUNCTION mock AS '${classOf[MockUDAF].getName}'") sql(s"CREATE TEMPORARY FUNCTION hive_max AS '${classOf[GenericUDAFMax].getName}'") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ObjectHashAggregateSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ObjectHashAggregateSuite.scala index 8dbcd24cd78de..0ef630bbd3670 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ObjectHashAggregateSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ObjectHashAggregateSuite.scala @@ -43,6 +43,7 @@ class ObjectHashAggregateSuite import testImplicits._ protected override def beforeAll(): Unit = { + super.beforeAll() sql(s"CREATE TEMPORARY FUNCTION hive_max AS '${classOf[GenericUDAFMax].getName}'") }