spark-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From hvanhov...@apache.org
Subject spark git commit: [SPARK-20410][SQL] Make sparkConf a def in SharedSQLContext
Date Thu, 20 Apr 2017 20:37:09 GMT
Repository: spark
Updated Branches:
  refs/heads/master d95e4d9d6 -> 033206355


[SPARK-20410][SQL] Make sparkConf a def in SharedSQLContext

## What changes were proposed in this pull request?
It is kind of annoying that `SharedSQLContext.sparkConf` is a val when overriding test cases,
because you cannot call `super` on it. This PR makes it a function.

## How was this patch tested?
Existing tests.

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #17705 from hvanhovell/SPARK-20410.


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

Branch: refs/heads/master
Commit: 033206355339677812a250b2b64818a261871fd2
Parents: d95e4d9
Author: Herman van Hovell <hvanhovell@databricks.com>
Authored: Thu Apr 20 22:37:04 2017 +0200
Committer: Herman van Hovell <hvanhovell@databricks.com>
Committed: Thu Apr 20 22:37:04 2017 +0200

----------------------------------------------------------------------
 .../spark/sql/AggregateHashMapSuite.scala       | 35 +++++++++-----------
 .../sql/DatasetSerializerRegistratorSuite.scala | 12 +++----
 .../DataSourceScanExecRedactionSuite.scala      | 11 ++----
 .../datasources/FileSourceStrategySuite.scala   |  2 +-
 .../CompactibleFileStreamLogSuite.scala         |  4 +--
 .../streaming/HDFSMetadataLogSuite.scala        |  4 +--
 .../spark/sql/test/SharedSQLContext.scala       |  7 ++--
 7 files changed, 32 insertions(+), 43 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/03320635/sql/core/src/test/scala/org/apache/spark/sql/AggregateHashMapSuite.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/AggregateHashMapSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/AggregateHashMapSuite.scala
index 3e85d95..7e61a68 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/AggregateHashMapSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/AggregateHashMapSuite.scala
@@ -19,13 +19,12 @@ package org.apache.spark.sql
 
 import org.scalatest.BeforeAndAfter
 
-class SingleLevelAggregateHashMapSuite extends DataFrameAggregateSuite with BeforeAndAfter
{
+import org.apache.spark.SparkConf
 
-  protected override def beforeAll(): Unit = {
-    sparkConf.set("spark.sql.codegen.fallback", "false")
-    sparkConf.set("spark.sql.codegen.aggregate.map.twolevel.enable", "false")
-    super.beforeAll()
-  }
+class SingleLevelAggregateHashMapSuite extends DataFrameAggregateSuite with BeforeAndAfter
{
+  override protected def sparkConf: SparkConf = super.sparkConf
+    .set("spark.sql.codegen.fallback", "false")
+    .set("spark.sql.codegen.aggregate.map.twolevel.enable", "false")
 
   // adding some checking after each test is run, assuring that the configs are not changed
   // in test code
@@ -38,12 +37,9 @@ class SingleLevelAggregateHashMapSuite extends DataFrameAggregateSuite
with Befo
 }
 
 class TwoLevelAggregateHashMapSuite extends DataFrameAggregateSuite with BeforeAndAfter {
-
-  protected override def beforeAll(): Unit = {
-    sparkConf.set("spark.sql.codegen.fallback", "false")
-    sparkConf.set("spark.sql.codegen.aggregate.map.twolevel.enable", "true")
-    super.beforeAll()
-  }
+  override protected def sparkConf: SparkConf = super.sparkConf
+    .set("spark.sql.codegen.fallback", "false")
+    .set("spark.sql.codegen.aggregate.map.twolevel.enable", "true")
 
   // adding some checking after each test is run, assuring that the configs are not changed
   // in test code
@@ -55,15 +51,14 @@ class TwoLevelAggregateHashMapSuite extends DataFrameAggregateSuite with
BeforeA
   }
 }
 
-class TwoLevelAggregateHashMapWithVectorizedMapSuite extends DataFrameAggregateSuite with
-BeforeAndAfter {
+class TwoLevelAggregateHashMapWithVectorizedMapSuite
+  extends DataFrameAggregateSuite
+  with BeforeAndAfter {
 
-  protected override def beforeAll(): Unit = {
-    sparkConf.set("spark.sql.codegen.fallback", "false")
-    sparkConf.set("spark.sql.codegen.aggregate.map.twolevel.enable", "true")
-    sparkConf.set("spark.sql.codegen.aggregate.map.vectorized.enable", "true")
-    super.beforeAll()
-  }
+  override protected def sparkConf: SparkConf = super.sparkConf
+    .set("spark.sql.codegen.fallback", "false")
+    .set("spark.sql.codegen.aggregate.map.twolevel.enable", "true")
+    .set("spark.sql.codegen.aggregate.map.vectorized.enable", "true")
 
   // adding some checking after each test is run, assuring that the configs are not changed
   // in test code

http://git-wip-us.apache.org/repos/asf/spark/blob/03320635/sql/core/src/test/scala/org/apache/spark/sql/DatasetSerializerRegistratorSuite.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSerializerRegistratorSuite.scala
b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSerializerRegistratorSuite.scala
index 92c5656..68f7de0 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSerializerRegistratorSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSerializerRegistratorSuite.scala
@@ -20,9 +20,9 @@ package org.apache.spark.sql
 import com.esotericsoftware.kryo.{Kryo, Serializer}
 import com.esotericsoftware.kryo.io.{Input, Output}
 
+import org.apache.spark.SparkConf
 import org.apache.spark.serializer.KryoRegistrator
 import org.apache.spark.sql.test.SharedSQLContext
-import org.apache.spark.sql.test.TestSparkSession
 
 /**
  * Test suite to test Kryo custom registrators.
@@ -30,12 +30,10 @@ import org.apache.spark.sql.test.TestSparkSession
 class DatasetSerializerRegistratorSuite extends QueryTest with SharedSQLContext {
   import testImplicits._
 
-  /**
-   * Initialize the [[TestSparkSession]] with a [[KryoRegistrator]].
-   */
-  protected override def beforeAll(): Unit = {
-    sparkConf.set("spark.kryo.registrator", TestRegistrator().getClass.getCanonicalName)
-    super.beforeAll()
+
+  override protected def sparkConf: SparkConf = {
+    // Make sure we use the KryoRegistrator
+    super.sparkConf.set("spark.kryo.registrator", TestRegistrator().getClass.getCanonicalName)
   }
 
   test("Kryo registrator") {

http://git-wip-us.apache.org/repos/asf/spark/blob/03320635/sql/core/src/test/scala/org/apache/spark/sql/execution/DataSourceScanExecRedactionSuite.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/DataSourceScanExecRedactionSuite.scala
b/sql/core/src/test/scala/org/apache/spark/sql/execution/DataSourceScanExecRedactionSuite.scala
index 05a2b2c..f7f1cce 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/DataSourceScanExecRedactionSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/DataSourceScanExecRedactionSuite.scala
@@ -18,22 +18,17 @@ package org.apache.spark.sql.execution
 
 import org.apache.hadoop.fs.Path
 
+import org.apache.spark.SparkConf
 import org.apache.spark.sql.QueryTest
 import org.apache.spark.sql.test.SharedSQLContext
-import org.apache.spark.util.Utils
 
 /**
  * Suite that tests the redaction of DataSourceScanExec
  */
 class DataSourceScanExecRedactionSuite extends QueryTest with SharedSQLContext {
 
-  import Utils._
-
-  override def beforeAll(): Unit = {
-    sparkConf.set("spark.redaction.string.regex",
-      "file:/[\\w_]+")
-    super.beforeAll()
-  }
+  override protected def sparkConf: SparkConf = super.sparkConf
+    .set("spark.redaction.string.regex", "file:/[\\w_]+")
 
   test("treeString is redacted") {
     withTempDir { dir =>

http://git-wip-us.apache.org/repos/asf/spark/blob/03320635/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala
b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala
index f361628..8703fe9 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala
@@ -42,7 +42,7 @@ import org.apache.spark.util.Utils
 class FileSourceStrategySuite extends QueryTest with SharedSQLContext with PredicateHelper
{
   import testImplicits._
 
-  protected override val sparkConf = new SparkConf().set("spark.default.parallelism", "1")
+  protected override def sparkConf = super.sparkConf.set("spark.default.parallelism", "1")
 
   test("unpartitioned table, single partition") {
     val table =

http://git-wip-us.apache.org/repos/asf/spark/blob/03320635/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/CompactibleFileStreamLogSuite.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/CompactibleFileStreamLogSuite.scala
b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/CompactibleFileStreamLogSuite.scala
index 20ac06f..3d480b1 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/CompactibleFileStreamLogSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/CompactibleFileStreamLogSuite.scala
@@ -28,8 +28,8 @@ import org.apache.spark.sql.test.SharedSQLContext
 class CompactibleFileStreamLogSuite extends SparkFunSuite with SharedSQLContext {
 
   /** To avoid caching of FS objects */
-  override protected val sparkConf =
-    new SparkConf().set(s"spark.hadoop.fs.$scheme.impl.disable.cache", "true")
+  override protected def sparkConf =
+    super.sparkConf.set(s"spark.hadoop.fs.$scheme.impl.disable.cache", "true")
 
   import CompactibleFileStreamLog._
 

http://git-wip-us.apache.org/repos/asf/spark/blob/03320635/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLogSuite.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLogSuite.scala
b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLogSuite.scala
index 662c446..7689bc0 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLogSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLogSuite.scala
@@ -38,8 +38,8 @@ import org.apache.spark.util.UninterruptibleThread
 class HDFSMetadataLogSuite extends SparkFunSuite with SharedSQLContext {
 
   /** To avoid caching of FS objects */
-  override protected val sparkConf =
-    new SparkConf().set(s"spark.hadoop.fs.$scheme.impl.disable.cache", "true")
+  override protected def sparkConf =
+    super.sparkConf.set(s"spark.hadoop.fs.$scheme.impl.disable.cache", "true")
 
   private implicit def toOption[A](a: A): Option[A] = Option(a)
 

http://git-wip-us.apache.org/repos/asf/spark/blob/03320635/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 3d76e05..81c69a3 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
@@ -30,7 +30,9 @@ import org.apache.spark.sql.{SparkSession, SQLContext}
  */
 trait SharedSQLContext extends SQLTestUtils with BeforeAndAfterEach with Eventually {
 
-  protected val sparkConf = new SparkConf()
+  protected def sparkConf = {
+    new SparkConf().set("spark.hadoop.fs.file.impl", classOf[DebugFilesystem].getName)
+  }
 
   /**
    * The [[TestSparkSession]] to use for all tests in this suite.
@@ -51,8 +53,7 @@ trait SharedSQLContext extends SQLTestUtils with BeforeAndAfterEach with
Eventua
   protected implicit def sqlContext: SQLContext = _spark.sqlContext
 
   protected def createSparkSession: TestSparkSession = {
-    new TestSparkSession(
-      sparkConf.set("spark.hadoop.fs.file.impl", classOf[DebugFilesystem].getName))
+    new TestSparkSession(sparkConf)
   }
 
   /**


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


Mime
View raw message