You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by va...@apache.org on 2018/01/16 19:41:23 UTC

spark git commit: [SPARK-16139][TEST] Add logging functionality for leaked threads in tests

Repository: spark
Updated Branches:
  refs/heads/master 75db14864 -> 12db365b4


[SPARK-16139][TEST] Add logging functionality for leaked threads in tests

## What changes were proposed in this pull request?

Lots of our tests don't properly shutdown everything they create, and end up leaking lots of threads. For example, `TaskSetManagerSuite` doesn't stop the extra `TaskScheduler` and `DAGScheduler` it creates. There are a couple more instances, eg. in `DAGSchedulerSuite`.

This PR adds the possibility to print out the not properly stopped thread list after a test suite executed. The format is the following:

```
===== FINISHED o.a.s.scheduler.DAGSchedulerSuite: 'task end event should have updated accumulators (SPARK-20342)' =====

...

===== Global thread whitelist loaded with name /thread_whitelist from classpath: rpc-client.*, rpc-server.*, shuffle-client.*, shuffle-server.*' =====

ScalaTest-run:

===== THREADS NOT STOPPED PROPERLY =====

ScalaTest-run: dag-scheduler-event-loop
ScalaTest-run: globalEventExecutor-2-5
ScalaTest-run:

===== END OF THREAD DUMP =====

ScalaTest-run:

===== EITHER PUT THREAD NAME INTO THE WHITELIST FILE OR SHUT IT DOWN PROPERLY =====
```

With the help of this leaking threads has been identified in TaskSetManagerSuite. My intention is to hunt down and fix such bugs in later PRs.

## How was this patch tested?

Manual: TaskSetManagerSuite test executed and found out where are the leaking threads.
Automated: Pass the Jenkins.

Author: Gabor Somogyi <ga...@gmail.com>

Closes #19893 from gaborgsomogyi/SPARK-16139.


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

Branch: refs/heads/master
Commit: 12db365b4faf7a185708648d246fc4a2aae0c2c0
Parents: 75db148
Author: Gabor Somogyi <ga...@gmail.com>
Authored: Tue Jan 16 11:41:08 2018 -0800
Committer: Marcelo Vanzin <va...@cloudera.com>
Committed: Tue Jan 16 11:41:08 2018 -0800

----------------------------------------------------------------------
 .../scala/org/apache/spark/SparkFunSuite.scala  | 34 +++++++
 .../scala/org/apache/spark/ThreadAudit.scala    | 99 ++++++++++++++++++++
 .../spark/scheduler/TaskSetManagerSuite.scala   |  7 +-
 .../apache/spark/sql/SessionStateSuite.scala    |  1 +
 .../sql/sources/DataSourceAnalysisSuite.scala   |  1 +
 .../spark/sql/test/SharedSQLContext.scala       | 23 ++++-
 .../hive/HiveContextCompatibilitySuite.scala    |  1 +
 .../spark/sql/hive/HiveSessionStateSuite.scala  |  1 +
 .../spark/sql/hive/HiveSparkSubmitSuite.scala   |  2 +
 .../spark/sql/hive/client/HiveClientSuite.scala |  1 +
 .../sql/hive/client/HiveVersionSuite.scala      |  1 +
 .../spark/sql/hive/client/VersionsSuite.scala   |  2 +
 .../sql/hive/execution/HiveComparisonTest.scala |  2 +
 .../sql/hive/orc/OrcHadoopFsRelationSuite.scala |  1 +
 .../spark/sql/hive/test/TestHiveSingleton.scala |  1 +
 15 files changed, 174 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/12db365b/core/src/test/scala/org/apache/spark/SparkFunSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/org/apache/spark/SparkFunSuite.scala b/core/src/test/scala/org/apache/spark/SparkFunSuite.scala
index 18077c0..3af9d82 100644
--- a/core/src/test/scala/org/apache/spark/SparkFunSuite.scala
+++ b/core/src/test/scala/org/apache/spark/SparkFunSuite.scala
@@ -27,19 +27,53 @@ import org.apache.spark.util.AccumulatorContext
 
 /**
  * Base abstract class for all unit tests in Spark for handling common functionality.
+ *
+ * Thread audit happens normally here automatically when a new test suite created.
+ * The only prerequisite for that is that the test class must extend [[SparkFunSuite]].
+ *
+ * It is possible to override the default thread audit behavior by setting enableAutoThreadAudit
+ * to false and manually calling the audit methods, if desired. For example:
+ *
+ * class MyTestSuite extends SparkFunSuite {
+ *
+ *   override val enableAutoThreadAudit = false
+ *
+ *   protected override def beforeAll(): Unit = {
+ *     doThreadPreAudit()
+ *     super.beforeAll()
+ *   }
+ *
+ *   protected override def afterAll(): Unit = {
+ *     super.afterAll()
+ *     doThreadPostAudit()
+ *   }
+ * }
  */
 abstract class SparkFunSuite
   extends FunSuite
   with BeforeAndAfterAll
+  with ThreadAudit
   with Logging {
 // scalastyle:on
 
+  protected val enableAutoThreadAudit = true
+
+  protected override def beforeAll(): Unit = {
+    if (enableAutoThreadAudit) {
+      doThreadPreAudit()
+    }
+    super.beforeAll()
+  }
+
   protected override def afterAll(): Unit = {
     try {
       // Avoid leaking map entries in tests that use accumulators without SparkContext
       AccumulatorContext.clear()
     } finally {
       super.afterAll()
+      if (enableAutoThreadAudit) {
+        doThreadPostAudit()
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/12db365b/core/src/test/scala/org/apache/spark/ThreadAudit.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/org/apache/spark/ThreadAudit.scala b/core/src/test/scala/org/apache/spark/ThreadAudit.scala
new file mode 100644
index 0000000..b3cea9d
--- /dev/null
+++ b/core/src/test/scala/org/apache/spark/ThreadAudit.scala
@@ -0,0 +1,99 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.internal.Logging
+
+/**
+ * Thread audit for test suites.
+ */
+trait ThreadAudit extends Logging {
+
+  val threadWhiteList = Set(
+    /**
+     * Netty related internal threads.
+     * These are excluded because their lifecycle is handled by the netty itself
+     * and spark has no explicit effect on them.
+     */
+    "netty.*",
+
+    /**
+     * Netty related internal threads.
+     * A Single-thread singleton EventExecutor inside netty which creates such threads.
+     * These are excluded because their lifecycle is handled by the netty itself
+     * and spark has no explicit effect on them.
+     */
+    "globalEventExecutor.*",
+
+    /**
+     * Netty related internal threads.
+     * Checks if a thread is alive periodically and runs a task when a thread dies.
+     * These are excluded because their lifecycle is handled by the netty itself
+     * and spark has no explicit effect on them.
+     */
+    "threadDeathWatcher.*",
+
+    /**
+     * During [[SparkContext]] creation [[org.apache.spark.rpc.netty.NettyRpcEnv]]
+     * creates event loops. One is wrapped inside
+     * [[org.apache.spark.network.server.TransportServer]]
+     * the other one is inside [[org.apache.spark.network.client.TransportClient]].
+     * The thread pools behind shut down asynchronously triggered by [[SparkContext#stop]].
+     * Manually checked and all of them stopped properly.
+     */
+    "rpc-client.*",
+    "rpc-server.*",
+
+    /**
+     * During [[SparkContext]] creation BlockManager creates event loops. One is wrapped inside
+     * [[org.apache.spark.network.server.TransportServer]]
+     * the other one is inside [[org.apache.spark.network.client.TransportClient]].
+     * The thread pools behind shut down asynchronously triggered by [[SparkContext#stop]].
+     * Manually checked and all of them stopped properly.
+     */
+    "shuffle-client.*",
+    "shuffle-server.*"
+  )
+  private var threadNamesSnapshot: Set[String] = Set.empty
+
+  protected def doThreadPreAudit(): Unit = {
+    threadNamesSnapshot = runningThreadNames()
+  }
+
+  protected def doThreadPostAudit(): Unit = {
+    val shortSuiteName = this.getClass.getName.replaceAll("org.apache.spark", "o.a.s")
+
+    if (threadNamesSnapshot.nonEmpty) {
+      val remainingThreadNames = runningThreadNames().diff(threadNamesSnapshot)
+        .filterNot { s => threadWhiteList.exists(s.matches(_)) }
+      if (remainingThreadNames.nonEmpty) {
+        logWarning(s"\n\n===== POSSIBLE THREAD LEAK IN SUITE $shortSuiteName, " +
+          s"thread names: ${remainingThreadNames.mkString(", ")} =====\n")
+      }
+    } else {
+      logWarning("\n\n===== THREAD AUDIT POST ACTION CALLED " +
+        s"WITHOUT PRE ACTION IN SUITE $shortSuiteName =====\n")
+    }
+  }
+
+  private def runningThreadNames(): Set[String] = {
+    Thread.getAllStackTraces.keySet().asScala.map(_.getName).toSet
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/12db365b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala
index 2ce81ae..ca6a7e5 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala
@@ -683,7 +683,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg
     val conf = new SparkConf().set("spark.speculation", "true")
     sc = new SparkContext("local", "test", conf)
 
-    val sched = new FakeTaskScheduler(sc, ("execA", "host1"), ("execB", "host2"))
+    sched = new FakeTaskScheduler(sc, ("execA", "host1"), ("execB", "host2"))
     sched.initialize(new FakeSchedulerBackend() {
       override def killTask(
         taskId: Long,
@@ -709,6 +709,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg
         }
       }
     }
+    sched.dagScheduler.stop()
     sched.setDAGScheduler(dagScheduler)
 
     val singleTask = new ShuffleMapTask(0, 0, null, new Partition {
@@ -754,7 +755,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg
     sc.conf.set("spark.speculation", "true")
 
     var killTaskCalled = false
-    val sched = new FakeTaskScheduler(sc, ("exec1", "host1"),
+    sched = new FakeTaskScheduler(sc, ("exec1", "host1"),
       ("exec2", "host2"), ("exec3", "host3"))
     sched.initialize(new FakeSchedulerBackend() {
       override def killTask(
@@ -789,6 +790,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg
         }
       }
     }
+    sched.dagScheduler.stop()
     sched.setDAGScheduler(dagScheduler)
 
     val taskSet = FakeTask.createShuffleMapTaskSet(4, 0, 0,
@@ -1183,6 +1185,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg
     sc = new SparkContext("local", "test")
     sched = new FakeTaskScheduler(sc, ("exec1", "host1"))
     val mockDAGScheduler = mock(classOf[DAGScheduler])
+    sched.dagScheduler.stop()
     sched.dagScheduler = mockDAGScheduler
     val taskSet = FakeTask.createTaskSet(numTasks = 1, stageId = 0, stageAttemptId = 0)
     val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = new ManualClock(1))

http://git-wip-us.apache.org/repos/asf/spark/blob/12db365b/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 c016667..5d75f58 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
@@ -39,6 +39,7 @@ class SessionStateSuite extends SparkFunSuite
   protected var activeSession: SparkSession = _
 
   override def beforeAll(): Unit = {
+    super.beforeAll()
     activeSession = SparkSession.builder().master("local").getOrCreate()
   }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/12db365b/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceAnalysisSuite.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceAnalysisSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceAnalysisSuite.scala
index 735e07c..e1022e3 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceAnalysisSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceAnalysisSuite.scala
@@ -33,6 +33,7 @@ class DataSourceAnalysisSuite extends SparkFunSuite with BeforeAndAfterAll {
   private var targetPartitionSchema: StructType = _
 
   override def beforeAll(): Unit = {
+    super.beforeAll()
     targetAttributes = Seq('a.int, 'd.int, 'b.int, 'c.int)
     targetPartitionSchema = new StructType()
       .add("b", IntegerType)

http://git-wip-us.apache.org/repos/asf/spark/blob/12db365b/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 4d578e2..e6c7648 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
@@ -17,4 +17,25 @@
 
 package org.apache.spark.sql.test
 
-trait SharedSQLContext extends SQLTestUtils with SharedSparkSession
+trait SharedSQLContext extends SQLTestUtils with SharedSparkSession {
+
+  /**
+   * Suites extending [[SharedSQLContext]] are sharing resources (eg. SparkSession) in their tests.
+   * That trait initializes the spark session in its [[beforeAll()]] implementation before the
+   * automatic thread snapshot is performed, so the audit code could fail to report threads leaked
+   * by that shared session.
+   *
+   * The behavior is overridden here to take the snapshot before the spark session is initialized.
+   */
+  override protected val enableAutoThreadAudit = false
+
+  protected override def beforeAll(): Unit = {
+    doThreadPreAudit()
+    super.beforeAll()
+  }
+
+  protected override def afterAll(): Unit = {
+    super.afterAll()
+    doThreadPostAudit()
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/12db365b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveContextCompatibilitySuite.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveContextCompatibilitySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveContextCompatibilitySuite.scala
index 8a74236..a80db76 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveContextCompatibilitySuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveContextCompatibilitySuite.scala
@@ -24,6 +24,7 @@ import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite}
 
 class HiveContextCompatibilitySuite extends SparkFunSuite with BeforeAndAfterEach {
 
+  override protected val enableAutoThreadAudit = false
   private var sc: SparkContext = null
   private var hc: HiveContext = null
 

http://git-wip-us.apache.org/repos/asf/spark/blob/12db365b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSessionStateSuite.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSessionStateSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSessionStateSuite.scala
index 958ad3e..f7da3c4 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSessionStateSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSessionStateSuite.scala
@@ -30,6 +30,7 @@ class HiveSessionStateSuite extends SessionStateSuite
 
   override def beforeAll(): Unit = {
     // Reuse the singleton session
+    super.beforeAll()
     activeSession = spark
   }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/12db365b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala
index 21b3e28..10204f4 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala
@@ -44,6 +44,8 @@ class HiveSparkSubmitSuite
   with BeforeAndAfterEach
   with ResetSystemProperties {
 
+  override protected val enableAutoThreadAudit = false
+
   // TODO: rewrite these or mark them as slow tests to be run sparingly
 
   override def beforeEach() {

http://git-wip-us.apache.org/repos/asf/spark/blob/12db365b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientSuite.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientSuite.scala
index ce53ace..a5dfd89 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientSuite.scala
@@ -67,6 +67,7 @@ class HiveClientSuite(version: String)
   }
 
   override def beforeAll() {
+    super.beforeAll()
     client = init(true)
   }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/12db365b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveVersionSuite.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveVersionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveVersionSuite.scala
index 951ebfa..bb8a469 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveVersionSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveVersionSuite.scala
@@ -25,6 +25,7 @@ import org.apache.spark.SparkFunSuite
 import org.apache.spark.sql.hive.HiveUtils
 
 private[client] abstract class HiveVersionSuite(version: String) extends SparkFunSuite {
+  override protected val enableAutoThreadAudit = false
   protected var client: HiveClient = null
 
   protected def buildClient(hadoopConf: Configuration): HiveClient = {

http://git-wip-us.apache.org/repos/asf/spark/blob/12db365b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala
index e64389e..72536b8 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala
@@ -50,6 +50,8 @@ import org.apache.spark.util.{MutableURLClassLoader, Utils}
 @ExtendedHiveTest
 class VersionsSuite extends SparkFunSuite with Logging {
 
+  override protected val enableAutoThreadAudit = false
+
   import HiveClientBuilder.buildClient
 
   /**

http://git-wip-us.apache.org/repos/asf/spark/blob/12db365b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala
index cee82cd..272e6f5 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala
@@ -48,6 +48,8 @@ import org.apache.spark.sql.hive.test.{TestHive, TestHiveQueryExecution}
 abstract class HiveComparisonTest
   extends SparkFunSuite with BeforeAndAfterAll with GivenWhenThen {
 
+  override protected val enableAutoThreadAudit = false
+
   /**
    * Path to the test datasets. We find this by looking up "hive-test-path-helper.txt" file.
    *

http://git-wip-us.apache.org/repos/asf/spark/blob/12db365b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcHadoopFsRelationSuite.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcHadoopFsRelationSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcHadoopFsRelationSuite.scala
index f87162f..a1f054b 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcHadoopFsRelationSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcHadoopFsRelationSuite.scala
@@ -30,6 +30,7 @@ import org.apache.spark.sql.types._
 class OrcHadoopFsRelationSuite extends HadoopFsRelationTest {
   import testImplicits._
 
+  override protected val enableAutoThreadAudit = false
   override val dataSourceName: String =
     classOf[org.apache.spark.sql.execution.datasources.orc.OrcFileFormat].getCanonicalName
 

http://git-wip-us.apache.org/repos/asf/spark/blob/12db365b/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHiveSingleton.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHiveSingleton.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHiveSingleton.scala
index df7988f..d3fff37 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHiveSingleton.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHiveSingleton.scala
@@ -26,6 +26,7 @@ import org.apache.spark.sql.hive.client.HiveClient
 
 
 trait TestHiveSingleton extends SparkFunSuite with BeforeAndAfterAll {
+  override protected val enableAutoThreadAudit = false
   protected val spark: SparkSession = TestHive.sparkSession
   protected val hiveContext: TestHiveContext = TestHive
   protected val hiveClient: HiveClient =


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