You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by ma...@apache.org on 2014/09/23 21:27:16 UTC
git commit: [SPARK-3646][SQL] Copy SQL configuration from SparkConf
when a SQLContext is created.
Repository: spark
Updated Branches:
refs/heads/master 1c62f97e9 -> a08153f8a
[SPARK-3646][SQL] Copy SQL configuration from SparkConf when a SQLContext is created.
This will allow us to take advantage of things like the spark.defaults file.
Author: Michael Armbrust <mi...@databricks.com>
Closes #2493 from marmbrus/copySparkConf and squashes the following commits:
0bd1377 [Michael Armbrust] Copy SQL configuration from SparkConf when a SQLContext is created.
Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/a08153f8
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/a08153f8
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/a08153f8
Branch: refs/heads/master
Commit: a08153f8a3e7bad81bae330ec4152651da5e7804
Parents: 1c62f97
Author: Michael Armbrust <mi...@databricks.com>
Authored: Tue Sep 23 12:27:12 2014 -0700
Committer: Michael Armbrust <mi...@databricks.com>
Committed: Tue Sep 23 12:27:12 2014 -0700
----------------------------------------------------------------------
.../src/main/scala/org/apache/spark/sql/SQLContext.scala | 5 +++++
.../scala/org/apache/spark/sql/test/TestSQLContext.scala | 6 +++++-
.../test/scala/org/apache/spark/sql/SQLConfSuite.scala | 11 ++++++++++-
3 files changed, 20 insertions(+), 2 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/spark/blob/a08153f8/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
index b245e1a..a42bedb 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
@@ -75,6 +75,11 @@ class SQLContext(@transient val sparkContext: SparkContext)
protected[sql] def executePlan(plan: LogicalPlan): this.QueryExecution =
new this.QueryExecution { val logical = plan }
+ sparkContext.getConf.getAll.foreach {
+ case (key, value) if key.startsWith("spark.sql") => setConf(key, value)
+ case _ =>
+ }
+
/**
* :: DeveloperApi ::
* Allows catalyst LogicalPlans to be executed as a SchemaRDD. Note that the LogicalPlan
http://git-wip-us.apache.org/repos/asf/spark/blob/a08153f8/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala
index 265b677..6bb81c7 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala
@@ -22,7 +22,11 @@ import org.apache.spark.sql.{SQLConf, SQLContext}
/** A SQLContext that can be used for local testing. */
object TestSQLContext
- extends SQLContext(new SparkContext("local[2]", "TestSQLContext", new SparkConf())) {
+ extends SQLContext(
+ new SparkContext(
+ "local[2]",
+ "TestSQLContext",
+ new SparkConf().set("spark.sql.testkey", "true"))) {
/** Fewer partitions to speed up testing. */
override private[spark] def numShufflePartitions: Int =
http://git-wip-us.apache.org/repos/asf/spark/blob/a08153f8/sql/core/src/test/scala/org/apache/spark/sql/SQLConfSuite.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLConfSuite.scala
index 584f71b..60701f0 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/SQLConfSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLConfSuite.scala
@@ -17,16 +17,25 @@
package org.apache.spark.sql
+import org.scalatest.FunSuiteLike
+
import org.apache.spark.sql.test._
/* Implicits */
import TestSQLContext._
-class SQLConfSuite extends QueryTest {
+class SQLConfSuite extends QueryTest with FunSuiteLike {
val testKey = "test.key.0"
val testVal = "test.val.0"
+ test("propagate from spark conf") {
+ // We create a new context here to avoid order dependence with other tests that might call
+ // clear().
+ val newContext = new SQLContext(TestSQLContext.sparkContext)
+ assert(newContext.getConf("spark.sql.testkey", "false") == "true")
+ }
+
test("programmatic ways of basic setting and getting") {
clear()
assert(getAllConfs.size === 0)
---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org