Repository: spark
Updated Branches:
  refs/heads/master 9db73ec12 -> 78a430ea4


http://git-wip-us.apache.org/repos/asf/spark/blob/78a430ea/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala
 
b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala
index 178bd1f..301aa5a 100644
--- 
a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala
+++ 
b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala
@@ -113,8 +113,8 @@ class HiveThriftBinaryServerSuite extends 
HiveThriftJdbcTest {
     withJdbcStatement { statement =>
       val resultSet = statement.executeQuery("SET spark.sql.hive.version")
       resultSet.next()
-      assert(resultSet.getString(1) ===
-        s"spark.sql.hive.version=${HiveContext.hiveExecutionVersion}")
+      assert(resultSet.getString(1) === "spark.sql.hive.version")
+      assert(resultSet.getString(2) === HiveContext.hiveExecutionVersion)
     }
   }
 
@@ -238,7 +238,7 @@ class HiveThriftBinaryServerSuite extends 
HiveThriftJdbcTest {
       // first session, we get the default value of the session status
       { statement =>
 
-        val rs1 = statement.executeQuery(s"SET ${SQLConf.SHUFFLE_PARTITIONS}")
+        val rs1 = statement.executeQuery(s"SET 
${SQLConf.SHUFFLE_PARTITIONS.key}")
         rs1.next()
         defaultV1 = rs1.getString(1)
         assert(defaultV1 != "200")
@@ -256,19 +256,21 @@ class HiveThriftBinaryServerSuite extends 
HiveThriftJdbcTest {
       { statement =>
 
         val queries = Seq(
-            s"SET ${SQLConf.SHUFFLE_PARTITIONS}=291",
+            s"SET ${SQLConf.SHUFFLE_PARTITIONS.key}=291",
             "SET hive.cli.print.header=true"
             )
 
         queries.map(statement.execute)
-        val rs1 = statement.executeQuery(s"SET ${SQLConf.SHUFFLE_PARTITIONS}")
+        val rs1 = statement.executeQuery(s"SET 
${SQLConf.SHUFFLE_PARTITIONS.key}")
         rs1.next()
-        assert("spark.sql.shuffle.partitions=291" === rs1.getString(1))
+        assert("spark.sql.shuffle.partitions" === rs1.getString(1))
+        assert("291" === rs1.getString(2))
         rs1.close()
 
         val rs2 = statement.executeQuery("SET hive.cli.print.header")
         rs2.next()
-        assert("hive.cli.print.header=true" === rs2.getString(1))
+        assert("hive.cli.print.header" === rs2.getString(1))
+        assert("true" === rs2.getString(2))
         rs2.close()
       },
 
@@ -276,7 +278,7 @@ class HiveThriftBinaryServerSuite extends 
HiveThriftJdbcTest {
       // default value
       { statement =>
 
-        val rs1 = statement.executeQuery(s"SET ${SQLConf.SHUFFLE_PARTITIONS}")
+        val rs1 = statement.executeQuery(s"SET 
${SQLConf.SHUFFLE_PARTITIONS.key}")
         rs1.next()
         assert(defaultV1 === rs1.getString(1))
         rs1.close()
@@ -404,8 +406,8 @@ class HiveThriftHttpServerSuite extends HiveThriftJdbcTest {
     withJdbcStatement { statement =>
       val resultSet = statement.executeQuery("SET spark.sql.hive.version")
       resultSet.next()
-      assert(resultSet.getString(1) ===
-        s"spark.sql.hive.version=${HiveContext.hiveExecutionVersion}")
+      assert(resultSet.getString(1) === "spark.sql.hive.version")
+      assert(resultSet.getString(2) === HiveContext.hiveExecutionVersion)
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/78a430ea/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala
 
b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala
index 82c0b49..432de25 100644
--- 
a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala
+++ 
b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala
@@ -47,17 +47,17 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with 
BeforeAndAfter {
     // Add Locale setting
     Locale.setDefault(Locale.US)
     // Set a relatively small column batch size for testing purposes
-    TestHive.setConf(SQLConf.COLUMN_BATCH_SIZE, "5")
+    TestHive.setConf(SQLConf.COLUMN_BATCH_SIZE, 5)
     // Enable in-memory partition pruning for testing purposes
-    TestHive.setConf(SQLConf.IN_MEMORY_PARTITION_PRUNING, "true")
+    TestHive.setConf(SQLConf.IN_MEMORY_PARTITION_PRUNING, true)
   }
 
   override def afterAll() {
     TestHive.cacheTables = false
     TimeZone.setDefault(originalTimeZone)
     Locale.setDefault(originalLocale)
-    TestHive.setConf(SQLConf.COLUMN_BATCH_SIZE, 
originalColumnBatchSize.toString)
-    TestHive.setConf(SQLConf.IN_MEMORY_PARTITION_PRUNING, 
originalInMemoryPartitionPruning.toString)
+    TestHive.setConf(SQLConf.COLUMN_BATCH_SIZE, originalColumnBatchSize)
+    TestHive.setConf(SQLConf.IN_MEMORY_PARTITION_PRUNING, 
originalInMemoryPartitionPruning)
   }
 
   /** A list of tests deemed out of scope currently and thus completely 
disregarded. */

http://git-wip-us.apache.org/repos/asf/spark/blob/78a430ea/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/SortMergeCompatibilitySuite.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/SortMergeCompatibilitySuite.scala
 
b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/SortMergeCompatibilitySuite.scala
index 65d070b..f458567 100644
--- 
a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/SortMergeCompatibilitySuite.scala
+++ 
b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/SortMergeCompatibilitySuite.scala
@@ -26,11 +26,11 @@ import org.apache.spark.sql.hive.test.TestHive
 class SortMergeCompatibilitySuite extends HiveCompatibilitySuite {
   override def beforeAll() {
     super.beforeAll()
-    TestHive.setConf(SQLConf.SORTMERGE_JOIN, "true")
+    TestHive.setConf(SQLConf.SORTMERGE_JOIN, true)
   }
 
   override def afterAll() {
-    TestHive.setConf(SQLConf.SORTMERGE_JOIN, "false")
+    TestHive.setConf(SQLConf.SORTMERGE_JOIN, false)
     super.afterAll()
   }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/78a430ea/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
index c50835d..4a66d65 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
@@ -21,15 +21,13 @@ import java.io.File
 import java.net.{URL, URLClassLoader}
 import java.sql.Timestamp
 
-import org.apache.hadoop.hive.common.StatsSetupConst
-import org.apache.hadoop.hive.common.`type`.HiveDecimal
-import org.apache.spark.sql.catalyst.ParserDialect
-
 import scala.collection.JavaConversions._
 import scala.collection.mutable.HashMap
 import scala.language.implicitConversions
 
 import org.apache.hadoop.fs.{FileSystem, Path}
+import org.apache.hadoop.hive.common.StatsSetupConst
+import org.apache.hadoop.hive.common.`type`.HiveDecimal
 import org.apache.hadoop.hive.conf.HiveConf
 import org.apache.hadoop.hive.ql.metadata.Table
 import org.apache.hadoop.hive.ql.parse.VariableSubstitution
@@ -39,6 +37,9 @@ import org.apache.hadoop.hive.serde2.io.{DateWritable, 
TimestampWritable}
 import org.apache.spark.SparkContext
 import org.apache.spark.annotation.Experimental
 import org.apache.spark.sql._
+import org.apache.spark.sql.SQLConf.SQLConfEntry
+import org.apache.spark.sql.SQLConf.SQLConfEntry._
+import org.apache.spark.sql.catalyst.ParserDialect
 import org.apache.spark.sql.catalyst.analysis._
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.execution.{ExecutedCommand, ExtractPythonUdfs, 
SetCommand}
@@ -69,13 +70,14 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) {
 
   import HiveContext._
 
+  println("create HiveContext")
+
   /**
    * When true, enables an experimental feature where metastore tables that 
use the parquet SerDe
    * are automatically converted to use the Spark SQL parquet table scan, 
instead of the Hive
    * SerDe.
    */
-  protected[sql] def convertMetastoreParquet: Boolean =
-    getConf("spark.sql.hive.convertMetastoreParquet", "true") == "true"
+  protected[sql] def convertMetastoreParquet: Boolean = 
getConf(CONVERT_METASTORE_PARQUET)
 
   /**
    * When true, also tries to merge possibly different but compatible Parquet 
schemas in different
@@ -84,7 +86,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) {
    * This configuration is only effective when 
"spark.sql.hive.convertMetastoreParquet" is true.
    */
   protected[sql] def convertMetastoreParquetWithSchemaMerging: Boolean =
-    getConf("spark.sql.hive.convertMetastoreParquet.mergeSchema", "false") == 
"true"
+    getConf(CONVERT_METASTORE_PARQUET_WITH_SCHEMA_MERGING)
 
   /**
    * When true, a table created by a Hive CTAS statement (no USING clause) 
will be
@@ -98,8 +100,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) {
    *   - The CTAS statement specifies SequenceFile (STORED AS SEQUENCEFILE) as 
the file format
    *     and no SerDe is specified (no ROW FORMAT SERDE clause).
    */
-  protected[sql] def convertCTAS: Boolean =
-    getConf("spark.sql.hive.convertCTAS", "false").toBoolean
+  protected[sql] def convertCTAS: Boolean = getConf(CONVERT_CTAS)
 
   /**
    * The version of the hive client that will be used to communicate with the 
metastore.  Note that
@@ -117,8 +118,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) {
    *              option is only valid when using the execution version of 
Hive.
    *  - maven - download the correct version of hive on demand from maven.
    */
-  protected[hive] def hiveMetastoreJars: String =
-    getConf(HIVE_METASTORE_JARS, "builtin")
+  protected[hive] def hiveMetastoreJars: String = getConf(HIVE_METASTORE_JARS)
 
   /**
    * A comma separated list of class prefixes that should be loaded using the 
classloader that
@@ -128,11 +128,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) 
{
    * custom appenders that are used by log4j.
    */
   protected[hive] def hiveMetastoreSharedPrefixes: Seq[String] =
-    getConf("spark.sql.hive.metastore.sharedPrefixes", jdbcPrefixes)
-      .split(",").filterNot(_ == "")
-
-  private def jdbcPrefixes = Seq(
-    "com.mysql.jdbc", "org.postgresql", "com.microsoft.sqlserver", 
"oracle.jdbc").mkString(",")
+    getConf(HIVE_METASTORE_SHARED_PREFIXES).filterNot(_ == "")
 
   /**
    * A comma separated list of class prefixes that should explicitly be 
reloaded for each version
@@ -140,14 +136,12 @@ class HiveContext(sc: SparkContext) extends 
SQLContext(sc) {
    * prefix that typically would be shared (i.e. org.apache.spark.*)
    */
   protected[hive] def hiveMetastoreBarrierPrefixes: Seq[String] =
-    getConf("spark.sql.hive.metastore.barrierPrefixes", "")
-      .split(",").filterNot(_ == "")
+    getConf(HIVE_METASTORE_BARRIER_PREFIXES).filterNot(_ == "")
 
   /*
    * hive thrift server use background spark sql thread pool to execute sql 
queries
    */
-  protected[hive] def hiveThriftServerAsync: Boolean =
-    getConf("spark.sql.hive.thriftServer.async", "true").toBoolean
+  protected[hive] def hiveThriftServerAsync: Boolean = 
getConf(HIVE_THRIFT_SERVER_ASYNC)
 
   @transient
   protected[sql] lazy val substitutor = new VariableSubstitution()
@@ -364,7 +358,11 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) 
{
     hiveconf.set(key, value)
   }
 
-  /* A catalyst metadata catalog that points to the Hive Metastore. */
+  private[sql] override def setConf[T](entry: SQLConfEntry[T], value: T): Unit 
= {
+    setConf(entry.key, entry.stringConverter(value))
+  }
+
+    /* A catalyst metadata catalog that points to the Hive Metastore. */
   @transient
   override protected[sql] lazy val catalog =
     new HiveMetastoreCatalog(metadataHive, this) with OverrideCatalog
@@ -402,8 +400,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) {
   protected[hive] class SQLSession extends super.SQLSession {
     protected[sql] override lazy val conf: SQLConf = new SQLConf {
       override def dialect: String = getConf(SQLConf.DIALECT, "hiveql")
-      override def caseSensitiveAnalysis: Boolean =
-        getConf(SQLConf.CASE_SENSITIVE, "false").toBoolean
+      override def caseSensitiveAnalysis: Boolean = 
getConf(SQLConf.CASE_SENSITIVE, false)
     }
 
     /**
@@ -519,7 +516,50 @@ private[hive] object HiveContext {
   val hiveExecutionVersion: String = "0.13.1"
 
   val HIVE_METASTORE_VERSION: String = "spark.sql.hive.metastore.version"
-  val HIVE_METASTORE_JARS: String = "spark.sql.hive.metastore.jars"
+  val HIVE_METASTORE_JARS = stringConf("spark.sql.hive.metastore.jars",
+    defaultValue = Some("builtin"),
+    doc = "Location of the jars that should be used to instantiate the 
HiveMetastoreClient. This" +
+      " property can be one of three options: " +
+      "1. \"builtin\" Use Hive 0.13.1, which is bundled with the Spark 
assembly jar when " +
+      "<code>-Phive</code> is enabled. When this option is chosen, " +
+      "spark.sql.hive.metastore.version must be either <code>0.13.1</code> or 
not defined. " +
+      "2. \"maven\" Use Hive jars of specified version downloaded from Maven 
repositories." +
+      "3. A classpath in the standard format for both Hive and Hadoop.")
+
+  val CONVERT_METASTORE_PARQUET = 
booleanConf("spark.sql.hive.convertMetastoreParquet",
+    defaultValue = Some(true),
+    doc = "When set to false, Spark SQL will use the Hive SerDe for parquet 
tables instead of " +
+      "the built in support.")
+
+  val CONVERT_METASTORE_PARQUET_WITH_SCHEMA_MERGING = booleanConf(
+    "spark.sql.hive.convertMetastoreParquet.mergeSchema",
+    defaultValue = Some(false),
+    doc = "TODO")
+
+  val CONVERT_CTAS = booleanConf("spark.sql.hive.convertCTAS",
+    defaultValue = Some(false),
+    doc = "TODO")
+
+  val HIVE_METASTORE_SHARED_PREFIXES = 
stringSeqConf("spark.sql.hive.metastore.sharedPrefixes",
+    defaultValue = Some(jdbcPrefixes),
+    doc = "A comma separated list of class prefixes that should be loaded 
using the classloader " +
+      "that is shared between Spark SQL and a specific version of Hive. An 
example of classes " +
+      "that should be shared is JDBC drivers that are needed to talk to the 
metastore. Other " +
+      "classes that need to be shared are those that interact with classes 
that are already " +
+      "shared. For example, custom appenders that are used by log4j.")
+
+  private def jdbcPrefixes = Seq(
+    "com.mysql.jdbc", "org.postgresql", "com.microsoft.sqlserver", 
"oracle.jdbc")
+
+  val HIVE_METASTORE_BARRIER_PREFIXES = 
stringSeqConf("spark.sql.hive.metastore.barrierPrefixes",
+    defaultValue = Some(Seq()),
+    doc = "A comma separated list of class prefixes that should explicitly be 
reloaded for each " +
+      "version of Hive that Spark SQL is communicating with. For example, Hive 
UDFs that are " +
+      "declared in a prefix that typically would be shared (i.e. 
<code>org.apache.spark.*</code>).")
+
+  val HIVE_THRIFT_SERVER_ASYNC = 
booleanConf("spark.sql.hive.thriftServer.async",
+    defaultValue = Some(true),
+    doc = "TODO")
 
   /** Constructs a configuration for hive, where the metastore is located in a 
temp directory. */
   def newTemporaryConfiguration(): Map[String, String] = {

http://git-wip-us.apache.org/repos/asf/spark/blob/78a430ea/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala
index 9215509..f901bd8 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala
@@ -112,12 +112,11 @@ class TestHiveContext(sc: SparkContext) extends 
HiveContext(sc) {
   protected[hive] class SQLSession extends super.SQLSession {
     /** Fewer partitions to speed up testing. */
     protected[sql] override lazy val conf: SQLConf = new SQLConf {
-      override def numShufflePartitions: Int = 
getConf(SQLConf.SHUFFLE_PARTITIONS, "5").toInt
+      override def numShufflePartitions: Int = 
getConf(SQLConf.SHUFFLE_PARTITIONS, 5)
       // TODO as in unit test, conf.clear() probably be called, all of the 
value will be cleared.
       // The super.getConf(SQLConf.DIALECT) is "sql" by default, we need to 
set it as "hiveql"
       override def dialect: String = super.getConf(SQLConf.DIALECT, "hiveql")
-      override def caseSensitiveAnalysis: Boolean =
-        getConf(SQLConf.CASE_SENSITIVE, "false").toBoolean
+      override def caseSensitiveAnalysis: Boolean = 
getConf(SQLConf.CASE_SENSITIVE, false)
     }
   }
 

http://git-wip-us.apache.org/repos/asf/spark/blob/78a430ea/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala
index a0d80dc..af68615 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala
@@ -81,11 +81,11 @@ class HiveParquetSuite extends QueryTest with ParquetTest {
     }
   }
 
-  withSQLConf(SQLConf.PARQUET_USE_DATA_SOURCE_API -> "true") {
+  withSQLConf(SQLConf.PARQUET_USE_DATA_SOURCE_API.key -> "true") {
     run("Parquet data source enabled")
   }
 
-  withSQLConf(SQLConf.PARQUET_USE_DATA_SOURCE_API -> "false") {
+  withSQLConf(SQLConf.PARQUET_USE_DATA_SOURCE_API.key -> "false") {
     run("Parquet data source disabled")
   }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/78a430ea/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala
 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala
index 79a85b2..cc294bc 100644
--- 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala
@@ -456,7 +456,7 @@ class MetastoreDataSourcesSuite extends QueryTest with 
SQLTestUtils with BeforeA
       withTable("savedJsonTable") {
         val df = (1 to 10).map(i => i -> s"str$i").toDF("a", "b")
 
-        withSQLConf(SQLConf.DEFAULT_DATA_SOURCE_NAME -> "json") {
+        withSQLConf(SQLConf.DEFAULT_DATA_SOURCE_NAME.key -> "json") {
           // Save the df as a managed table (by not specifying the path).
           df.write.saveAsTable("savedJsonTable")
 
@@ -484,7 +484,7 @@ class MetastoreDataSourcesSuite extends QueryTest with 
SQLTestUtils with BeforeA
         }
 
         // Create an external table by specifying the path.
-        withSQLConf(SQLConf.DEFAULT_DATA_SOURCE_NAME -> "not a source name") {
+        withSQLConf(SQLConf.DEFAULT_DATA_SOURCE_NAME.key -> "not a source 
name") {
           df.write
             .format("org.apache.spark.sql.json")
             .mode(SaveMode.Append)
@@ -508,7 +508,7 @@ class MetastoreDataSourcesSuite extends QueryTest with 
SQLTestUtils with BeforeA
           s"""{ "a": $i, "b": "str$i" }"""
         }))
 
-        withSQLConf(SQLConf.DEFAULT_DATA_SOURCE_NAME -> "not a source name") {
+        withSQLConf(SQLConf.DEFAULT_DATA_SOURCE_NAME.key -> "not a source 
name") {
           df.write
             .format("json")
             .mode(SaveMode.Append)
@@ -516,7 +516,7 @@ class MetastoreDataSourcesSuite extends QueryTest with 
SQLTestUtils with BeforeA
             .saveAsTable("savedJsonTable")
         }
 
-        withSQLConf(SQLConf.DEFAULT_DATA_SOURCE_NAME -> "json") {
+        withSQLConf(SQLConf.DEFAULT_DATA_SOURCE_NAME.key -> "json") {
           createExternalTable("createdJsonTable", tempPath.toString)
           assert(table("createdJsonTable").schema === df.schema)
           checkAnswer(sql("SELECT * FROM createdJsonTable"), df)
@@ -533,7 +533,7 @@ class MetastoreDataSourcesSuite extends QueryTest with 
SQLTestUtils with BeforeA
         checkAnswer(read.json(tempPath.toString), df)
 
         // Try to specify the schema.
-        withSQLConf(SQLConf.DEFAULT_DATA_SOURCE_NAME -> "not a source name") {
+        withSQLConf(SQLConf.DEFAULT_DATA_SOURCE_NAME.key -> "not a source 
name") {
           val schema = StructType(StructField("b", StringType, true) :: Nil)
           createExternalTable(
             "createdJsonTable",
@@ -563,8 +563,8 @@ class MetastoreDataSourcesSuite extends QueryTest with 
SQLTestUtils with BeforeA
 
   test("scan a parquet table created through a CTAS statement") {
     withSQLConf(
-      "spark.sql.hive.convertMetastoreParquet" -> "true",
-      SQLConf.PARQUET_USE_DATA_SOURCE_API -> "true") {
+      HiveContext.CONVERT_METASTORE_PARQUET.key -> "true",
+      SQLConf.PARQUET_USE_DATA_SOURCE_API.key -> "true") {
 
       withTempTable("jt") {
         (1 to 10).map(i => i -> s"str$i").toDF("a", 
"b").registerTempTable("jt")
@@ -706,7 +706,7 @@ class MetastoreDataSourcesSuite extends QueryTest with 
SQLTestUtils with BeforeA
   }
 
   test("SPARK-6024 wide schema support") {
-    withSQLConf(SQLConf.SCHEMA_STRING_LENGTH_THRESHOLD -> "4000") {
+    withSQLConf(SQLConf.SCHEMA_STRING_LENGTH_THRESHOLD.key -> "4000") {
       withTable("wide_schema") {
         // We will need 80 splits for this schema if the threshold is 4000.
         val schema = StructType((1 to 5000).map(i => StructField(s"c_$i", 
StringType, true)))

http://git-wip-us.apache.org/repos/asf/spark/blob/78a430ea/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala
index 78c94e6..f067ea0 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala
@@ -167,7 +167,7 @@ class StatisticsSuite extends QueryTest with 
BeforeAndAfterAll {
       ctx.conf.settings.synchronized {
         val tmp = ctx.conf.autoBroadcastJoinThreshold
 
-        sql(s"""SET ${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD}=-1""")
+        sql(s"""SET ${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key}=-1""")
         df = sql(query)
         bhj = df.queryExecution.sparkPlan.collect { case j: BroadcastHashJoin 
=> j }
         assert(bhj.isEmpty, "BroadcastHashJoin still planned even though it is 
switched off")
@@ -176,7 +176,7 @@ class StatisticsSuite extends QueryTest with 
BeforeAndAfterAll {
         assert(shj.size === 1,
           "ShuffledHashJoin should be planned when BroadcastHashJoin is turned 
off")
 
-        sql(s"""SET ${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD}=$tmp""")
+        sql(s"""SET ${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key}=$tmp""")
       }
 
       after()
@@ -225,7 +225,7 @@ class StatisticsSuite extends QueryTest with 
BeforeAndAfterAll {
     ctx.conf.settings.synchronized {
       val tmp = ctx.conf.autoBroadcastJoinThreshold
 
-      sql(s"SET ${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD}=-1")
+      sql(s"SET ${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key}=-1")
       df = sql(leftSemiJoinQuery)
       bhj = df.queryExecution.sparkPlan.collect {
         case j: BroadcastLeftSemiJoinHash => j
@@ -238,7 +238,7 @@ class StatisticsSuite extends QueryTest with 
BeforeAndAfterAll {
       assert(shj.size === 1,
         "LeftSemiJoinHash should be planned when BroadcastHashJoin is turned 
off")
 
-      sql(s"SET ${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD}=$tmp")
+      sql(s"SET ${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key}=$tmp")
     }
 
   }

http://git-wip-us.apache.org/repos/asf/spark/blob/78a430ea/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala
 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala
index 6d8d99e..51dabc6 100644
--- 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala
@@ -1084,14 +1084,16 @@ class HiveQuerySuite extends HiveComparisonTest with 
BeforeAndAfter {
     val testKey = "spark.sql.key.usedfortestonly"
     val testVal = "test.val.0"
     val nonexistentKey = "nonexistent"
-    val KV = "([^=]+)=([^=]*)".r
-    def collectResults(df: DataFrame): Set[(String, String)] =
+    def collectResults(df: DataFrame): Set[Any] =
       df.collect().map {
         case Row(key: String, value: String) => key -> value
-        case Row(KV(key, value)) => key -> value
+        case Row(key: String, defaultValue: String, doc: String) => (key, 
defaultValue, doc)
       }.toSet
     conf.clear()
 
+    val expectedConfs = conf.getAllDefinedConfs.toSet
+    assertResult(expectedConfs)(collectResults(sql("SET -v")))
+
     // "SET" itself returns all config variables currently specified in 
SQLConf.
     // TODO: Should we be listing the default here always? probably...
     assert(sql("SET").collect().size == 0)
@@ -1102,16 +1104,12 @@ class HiveQuerySuite extends HiveComparisonTest with 
BeforeAndAfter {
 
     assert(hiveconf.get(testKey, "") == testVal)
     assertResult(Set(testKey -> testVal))(collectResults(sql("SET")))
-    assertResult(Set(testKey -> testVal))(collectResults(sql("SET -v")))
 
     sql(s"SET ${testKey + testKey}=${testVal + testVal}")
     assert(hiveconf.get(testKey + testKey, "") == testVal + testVal)
     assertResult(Set(testKey -> testVal, (testKey + testKey) -> (testVal + 
testVal))) {
       collectResults(sql("SET"))
     }
-    assertResult(Set(testKey -> testVal, (testKey + testKey) -> (testVal + 
testVal))) {
-      collectResults(sql("SET -v"))
-    }
 
     // "SET key"
     assertResult(Set(testKey -> testVal)) {

http://git-wip-us.apache.org/repos/asf/spark/blob/78a430ea/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala
 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala
index 984d97d..e1c9926 100644
--- 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala
@@ -24,7 +24,7 @@ import org.apache.spark.sql._
 import org.apache.spark.sql.hive.test.TestHive
 import org.apache.spark.sql.hive.test.TestHive._
 import org.apache.spark.sql.hive.test.TestHive.implicits._
-import org.apache.spark.sql.hive.{HiveQLDialect, MetastoreRelation}
+import org.apache.spark.sql.hive.{HiveContext, HiveQLDialect, 
MetastoreRelation}
 import org.apache.spark.sql.parquet.ParquetRelation2
 import org.apache.spark.sql.sources.LogicalRelation
 import org.apache.spark.sql.types._
@@ -191,9 +191,9 @@ class SQLQuerySuite extends QueryTest {
       }
     }
 
-    val originalConf = getConf("spark.sql.hive.convertCTAS", "false")
+    val originalConf = convertCTAS
 
-    setConf("spark.sql.hive.convertCTAS", "true")
+    setConf(HiveContext.CONVERT_CTAS, true)
 
     sql("CREATE TABLE ctas1 AS SELECT key k, value FROM src ORDER BY k, value")
     sql("CREATE TABLE IF NOT EXISTS ctas1 AS SELECT key k, value FROM src 
ORDER BY k, value")
@@ -235,7 +235,7 @@ class SQLQuerySuite extends QueryTest {
     checkRelation("ctas1", false)
     sql("DROP TABLE ctas1")
 
-    setConf("spark.sql.hive.convertCTAS", originalConf)
+    setConf(HiveContext.CONVERT_CTAS, originalConf)
   }
 
   test("SQL Dialect Switching") {
@@ -332,7 +332,7 @@ class SQLQuerySuite extends QueryTest {
 
     val origUseParquetDataSource = conf.parquetUseDataSourceApi
     try {
-      setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, "false")
+      setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, false)
       sql(
         """CREATE TABLE ctas5
           | STORED AS parquet AS
@@ -348,7 +348,7 @@ class SQLQuerySuite extends QueryTest {
         "MANAGED_TABLE"
       )
 
-      val default = getConf("spark.sql.hive.convertMetastoreParquet", "true")
+      val default = convertMetastoreParquet
       // use the Hive SerDe for parquet tables
       sql("set spark.sql.hive.convertMetastoreParquet = false")
       checkAnswer(
@@ -356,7 +356,7 @@ class SQLQuerySuite extends QueryTest {
         sql("SELECT key, value FROM src ORDER BY key, value").collect().toSeq)
       sql(s"set spark.sql.hive.convertMetastoreParquet = $default")
     } finally {
-      setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, 
origUseParquetDataSource.toString)
+      setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, origUseParquetDataSource)
     }
   }
 
@@ -603,8 +603,8 @@ class SQLQuerySuite extends QueryTest {
     // generates an invalid query plan.
     val rdd = sparkContext.makeRDD((1 to 5).map(i => s"""{"a":[$i, ${i + 
1}]}"""))
     read.json(rdd).registerTempTable("data")
-    val originalConf = getConf("spark.sql.hive.convertCTAS", "false")
-    setConf("spark.sql.hive.convertCTAS", "false")
+    val originalConf = convertCTAS
+    setConf(HiveContext.CONVERT_CTAS, false)
 
     sql("CREATE TABLE explodeTest (key bigInt)")
     table("explodeTest").queryExecution.analyzed match {
@@ -621,7 +621,7 @@ class SQLQuerySuite extends QueryTest {
 
     sql("DROP TABLE explodeTest")
     dropTempTable("data")
-    setConf("spark.sql.hive.convertCTAS", originalConf)
+    setConf(HiveContext.CONVERT_CTAS, originalConf)
   }
 
   test("sanity test for SPARK-6618") {

http://git-wip-us.apache.org/repos/asf/spark/blob/78a430ea/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala
index 3864349..c2e0980 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala
@@ -153,7 +153,7 @@ class ParquetMetastoreSuiteBase extends 
ParquetPartitioningTest {
     val rdd2 = sparkContext.parallelize((1 to 10).map(i => s"""{"a":[$i, 
null]}"""))
     read.json(rdd2).registerTempTable("jt_array")
 
-    setConf("spark.sql.hive.convertMetastoreParquet", "true")
+    setConf(HiveContext.CONVERT_METASTORE_PARQUET, true)
   }
 
   override def afterAll(): Unit = {
@@ -164,7 +164,7 @@ class ParquetMetastoreSuiteBase extends 
ParquetPartitioningTest {
     sql("DROP TABLE normal_parquet")
     sql("DROP TABLE IF EXISTS jt")
     sql("DROP TABLE IF EXISTS jt_array")
-    setConf("spark.sql.hive.convertMetastoreParquet", "false")
+    setConf(HiveContext.CONVERT_METASTORE_PARQUET, false)
   }
 
   test(s"conversion is working") {
@@ -199,14 +199,14 @@ class ParquetDataSourceOnMetastoreSuite extends 
ParquetMetastoreSuiteBase {
         |  OUTPUTFORMAT 
'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat'
       """.stripMargin)
 
-    conf.setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, "true")
+    conf.setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, true)
   }
 
   override def afterAll(): Unit = {
     super.afterAll()
     sql("DROP TABLE IF EXISTS test_parquet")
 
-    setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, originalConf.toString)
+    setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, originalConf)
   }
 
   test("scan an empty parquet table") {
@@ -546,12 +546,12 @@ class ParquetDataSourceOffMetastoreSuite extends 
ParquetMetastoreSuiteBase {
 
   override def beforeAll(): Unit = {
     super.beforeAll()
-    conf.setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, "false")
+    conf.setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, false)
   }
 
   override def afterAll(): Unit = {
     super.afterAll()
-    setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, originalConf.toString)
+    setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, originalConf)
   }
 
   test("MetastoreRelation in InsertIntoTable will not be converted") {
@@ -692,12 +692,12 @@ class ParquetDataSourceOnSourceSuite extends 
ParquetSourceSuiteBase {
 
   override def beforeAll(): Unit = {
     super.beforeAll()
-    conf.setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, "true")
+    conf.setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, true)
   }
 
   override def afterAll(): Unit = {
     super.afterAll()
-    setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, originalConf.toString)
+    setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, originalConf)
   }
 
   test("values in arrays and maps stored in parquet are always nullable") {
@@ -750,12 +750,12 @@ class ParquetDataSourceOffSourceSuite extends 
ParquetSourceSuiteBase {
 
   override def beforeAll(): Unit = {
     super.beforeAll()
-    conf.setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, "false")
+    conf.setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, false)
   }
 
   override def afterAll(): Unit = {
     super.afterAll()
-    setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, originalConf.toString)
+    setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, originalConf)
   }
 }
 


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to