You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by do...@apache.org on 2018/09/13 18:34:44 UTC

spark git commit: [SPARK-25338][TEST] Ensure to call super.beforeAll() and super.afterAll() in test cases

Repository: spark
Updated Branches:
  refs/heads/master a7e5aa6cd -> f60cd7cc3


[SPARK-25338][TEST] Ensure to call super.beforeAll() and super.afterAll() in test cases

## What changes were proposed in this pull request?

This PR ensures to call `super.afterAll()` in `override afterAll()` method for test suites.

* Some suites did not call `super.afterAll()`
* Some suites may call `super.afterAll()` only under certain condition
* Others never call `super.afterAll()`.

This PR also ensures to call `super.beforeAll()` in `override beforeAll()` for test suites.

## How was this patch tested?

Existing UTs

Closes #22337 from kiszk/SPARK-25338.

Authored-by: Kazuaki Ishizaki <is...@jp.ibm.com>
Signed-off-by: Dongjoon Hyun <do...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/f60cd7cc
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/f60cd7cc
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/f60cd7cc

Branch: refs/heads/master
Commit: f60cd7cc3ce663bb1517e059f5fd79c0098ebbcd
Parents: a7e5aa6
Author: Kazuaki Ishizaki <is...@jp.ibm.com>
Authored: Thu Sep 13 11:34:22 2018 -0700
Committer: Dongjoon Hyun <do...@apache.org>
Committed: Thu Sep 13 11:34:22 2018 -0700

----------------------------------------------------------------------
 .../deploy/master/ui/MasterWebUISuite.scala     |  7 ++++--
 .../flume/FlumePollingStreamSuite.scala         | 11 ++++++---
 .../spark/sql/kafka010/KafkaRelationSuite.scala |  9 ++++---
 .../spark/sql/kafka010/KafkaSinkSuite.scala     |  9 ++++---
 .../kafka010/DirectKafkaStreamSuite.scala       | 11 ++++++---
 .../streaming/kafka010/KafkaRDDSuite.scala      | 23 +++++++++++------
 .../kafka/DirectKafkaStreamSuite.scala          | 11 ++++++---
 .../streaming/kafka/KafkaClusterSuite.scala     | 11 ++++++---
 .../spark/streaming/kafka/KafkaRDDSuite.scala   | 23 +++++++++++------
 .../streaming/kafka/KafkaStreamSuite.scala      | 23 +++++++++++------
 .../kafka/ReliableKafkaStreamSuite.scala        | 13 +++++++---
 .../KinesisInputDStreamBuilderSuite.scala       |  6 ++++-
 .../streaming/kinesis/KinesisStreamSuite.scala  | 26 +++++++++++---------
 .../k8s/integrationtest/KubernetesSuite.scala   |  7 +++++-
 .../apache/spark/sql/SessionStateSuite.scala    | 15 ++++++-----
 .../execution/ExchangeCoordinatorSuite.scala    | 11 ++++++---
 .../ExternalAppendOnlyUnsafeRowArraySuite.scala |  6 ++++-
 .../SortBasedAggregationStoreSuite.scala        |  6 ++++-
 .../benchmark/WideSchemaBenchmark.scala         |  7 ++++--
 .../BasicWriteTaskStatsTrackerSuite.scala       |  6 ++++-
 .../execution/joins/BroadcastJoinSuite.scala    |  8 ++++--
 .../python/BatchEvalPythonExecSuite.scala       |  7 ++++--
 .../streaming/state/StateStoreRDDSuite.scala    |  7 ++++--
 .../sql/internal/ExecutorSideSQLConfSuite.scala |  8 ++++--
 .../streaming/FlatMapGroupsWithStateSuite.scala |  8 +-----
 .../apache/spark/sql/streaming/StreamTest.scala |  7 ++++--
 .../streaming/StreamingAggregationSuite.scala   |  8 +-----
 .../streaming/StreamingDeduplicationSuite.scala |  7 +-----
 .../spark/sql/test/SharedSQLContext.scala       |  7 ++++--
 .../sql/hive/thriftserver/UISeleniumSuite.scala |  9 ++++---
 .../hive/HiveExternalCatalogVersionsSuite.scala | 11 ++++++---
 .../sql/hive/execution/HiveUDAFSuite.scala      |  1 +
 .../execution/ObjectHashAggregateSuite.scala    |  1 +
 33 files changed, 216 insertions(+), 114 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/f60cd7cc/core/src/test/scala/org/apache/spark/deploy/master/ui/MasterWebUISuite.scala
----------------------------------------------------------------------
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 69a460f..f4558aa 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") {

http://git-wip-us.apache.org/repos/asf/spark/blob/f60cd7cc/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala
----------------------------------------------------------------------
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 4324cc6..9241b13 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()
     }
   }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/f60cd7cc/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala
----------------------------------------------------------------------
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 eb18697..8cfca56 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()
     }
   }

http://git-wip-us.apache.org/repos/asf/spark/blob/f60cd7cc/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala
----------------------------------------------------------------------
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 a2213e0..81832fb 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()
     }
   }

http://git-wip-us.apache.org/repos/asf/spark/blob/f60cd7cc/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/DirectKafkaStreamSuite.scala
----------------------------------------------------------------------
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 661b67a..1974bb1 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()
     }
   }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/f60cd7cc/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/KafkaRDDSuite.scala
----------------------------------------------------------------------
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 3ac6509..561bca5 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()
     }
   }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/f60cd7cc/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala
----------------------------------------------------------------------
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 ecca387..3fd37f4 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()
     }
   }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/f60cd7cc/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/KafkaClusterSuite.scala
----------------------------------------------------------------------
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 d66830c..73d5285 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()
     }
   }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/f60cd7cc/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala
----------------------------------------------------------------------
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 809699a..72f9541 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()
     }
   }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/f60cd7cc/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala
----------------------------------------------------------------------
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 426cd83..ed130f5 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()
     }
   }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/f60cd7cc/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala
----------------------------------------------------------------------
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 57f89cc..5da5ea4 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()
     }
   }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/f60cd7cc/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisInputDStreamBuilderSuite.scala
----------------------------------------------------------------------
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 e0e2684..361520e 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") {

http://git-wip-us.apache.org/repos/asf/spark/blob/f60cd7cc/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisStreamSuite.scala
----------------------------------------------------------------------
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 a7a68eb..6d27445 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()
     }
   }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/f60cd7cc/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala
----------------------------------------------------------------------
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 82e6efa..18541ba 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 {

http://git-wip-us.apache.org/repos/asf/spark/blob/f60cd7cc/sql/core/src/test/scala/org/apache/spark/sql/SessionStateSuite.scala
----------------------------------------------------------------------
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 7d13660..e1b5eba 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") {

http://git-wip-us.apache.org/repos/asf/spark/blob/f60cd7cc/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala
----------------------------------------------------------------------
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 41de731..c627c51 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(

http://git-wip-us.apache.org/repos/asf/spark/blob/f60cd7cc/sql/core/src/test/scala/org/apache/spark/sql/execution/ExternalAppendOnlyUnsafeRowArraySuite.scala
----------------------------------------------------------------------
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 ecc7264..b29de9c 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 = {

http://git-wip-us.apache.org/repos/asf/spark/blob/f60cd7cc/sql/core/src/test/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregationStoreSuite.scala
----------------------------------------------------------------------
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 3fad7df..dc67446 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()
 

http://git-wip-us.apache.org/repos/asf/spark/blob/f60cd7cc/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/WideSchemaBenchmark.scala
----------------------------------------------------------------------
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 a42891e..c368f17 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() {

http://git-wip-us.apache.org/repos/asf/spark/blob/f60cd7cc/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/BasicWriteTaskStatsTrackerSuite.scala
----------------------------------------------------------------------
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 bf3c8ed..32941d8 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()
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/spark/blob/f60cd7cc/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala
----------------------------------------------------------------------
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 bcdee79..b4ad1db 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()
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/spark/blob/f60cd7cc/sql/core/src/test/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExecSuite.scala
----------------------------------------------------------------------
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 2cc55ff..289cc66 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") {

http://git-wip-us.apache.org/repos/asf/spark/blob/f60cd7cc/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreRDDSuite.scala
----------------------------------------------------------------------
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 579a364..015415a 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") {

http://git-wip-us.apache.org/repos/asf/spark/blob/f60cd7cc/sql/core/src/test/scala/org/apache/spark/sql/internal/ExecutorSideSQLConfSuite.scala
----------------------------------------------------------------------
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 5b4736e..d885348 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 = {

http://git-wip-us.apache.org/repos/asf/spark/blob/f60cd7cc/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateSuite.scala
----------------------------------------------------------------------
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 e77ba1e..43463a8 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
 

http://git-wip-us.apache.org/repos/asf/spark/blob/f60cd7cc/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala
----------------------------------------------------------------------
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 491dc34..d878c34 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)

http://git-wip-us.apache.org/repos/asf/spark/blob/f60cd7cc/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala
----------------------------------------------------------------------
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 1ae6ff3..97dbb9b 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._
 

http://git-wip-us.apache.org/repos/asf/spark/blob/f60cd7cc/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala
----------------------------------------------------------------------
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 42ffd47..cfd7204 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()

http://git-wip-us.apache.org/repos/asf/spark/blob/f60cd7cc/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSQLContext.scala
----------------------------------------------------------------------
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 e6c7648..0dd24d2 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()
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/f60cd7cc/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/UISeleniumSuite.scala
----------------------------------------------------------------------
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 4c53dd8..fef18f1 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) = {

http://git-wip-us.apache.org/repos/asf/spark/blob/f60cd7cc/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala
----------------------------------------------------------------------
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 25df333..a7d6972 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 = {

http://git-wip-us.apache.org/repos/asf/spark/blob/f60cd7cc/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDAFSuite.scala
----------------------------------------------------------------------
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 7402c96..fe3dece 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}'")
 

http://git-wip-us.apache.org/repos/asf/spark/blob/f60cd7cc/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ObjectHashAggregateSuite.scala
----------------------------------------------------------------------
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 8dbcd24..0ef630b 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}'")
   }
 


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org