aboutsummaryrefslogtreecommitdiff
path: root/sql/hive
diff options
context:
space:
mode:
authorMarcelo Vanzin <vanzin@cloudera.com>2016-04-05 15:19:51 -0700
committerMarcelo Vanzin <vanzin@cloudera.com>2016-04-05 15:19:51 -0700
commitd5ee9d5c240fca5c15b21efc4a760b06a1f39fd6 (patch)
treeb0c0d55466cdd5678849cc32d914408f9dd84472 /sql/hive
parent7329fe272d3ead7db9bc3e1e32adb7329dabc607 (diff)
downloadspark-d5ee9d5c240fca5c15b21efc4a760b06a1f39fd6.tar.gz
spark-d5ee9d5c240fca5c15b21efc4a760b06a1f39fd6.tar.bz2
spark-d5ee9d5c240fca5c15b21efc4a760b06a1f39fd6.zip
[SPARK-529][SQL] Modify SQLConf to use new config API from core.
Because SQL keeps track of all known configs, some customization was needed in SQLConf to allow that, since the core API does not have that feature. Tested via existing (and slightly updated) unit tests. Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #11570 from vanzin/SPARK-529-sql.
Diffstat (limited to 'sql/hive')
-rw-r--r--sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala96
1 files changed, 54 insertions, 42 deletions
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 073b954a5f..505e5c0bb6 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
@@ -42,6 +42,7 @@ import org.apache.hadoop.util.VersionInfo
import org.apache.spark.{SparkConf, SparkContext}
import org.apache.spark.api.java.JavaSparkContext
import org.apache.spark.internal.Logging
+import org.apache.spark.internal.config.ConfigEntry
import org.apache.spark.sql._
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.analysis._
@@ -54,8 +55,7 @@ import org.apache.spark.sql.execution.ui.SQLListener
import org.apache.spark.sql.hive.client._
import org.apache.spark.sql.hive.execution.{DescribeHiveTableCommand, HiveNativeCommand}
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.internal.SQLConf.SQLConfEntry
-import org.apache.spark.sql.internal.SQLConf.SQLConfEntry._
+import org.apache.spark.sql.internal.SQLConf._
import org.apache.spark.sql.types._
import org.apache.spark.unsafe.types.UTF8String
import org.apache.spark.util.Utils
@@ -318,7 +318,7 @@ class HiveContext private[hive](
hiveconf.set(key, value)
}
- override private[sql] def setConf[T](entry: SQLConfEntry[T], value: T): Unit = {
+ override private[sql] def setConf[T](entry: ConfigEntry[T], value: T): Unit = {
setConf(entry.key, entry.stringConverter(value))
}
@@ -413,19 +413,19 @@ private[hive] object HiveContext extends Logging {
/** The version of hive used internally by Spark SQL. */
val hiveExecutionVersion: String = "1.2.1"
- val HIVE_METASTORE_VERSION = stringConf("spark.sql.hive.metastore.version",
- defaultValue = Some(hiveExecutionVersion),
- doc = "Version of the Hive metastore. Available options are " +
+ val HIVE_METASTORE_VERSION = SQLConfigBuilder("spark.sql.hive.metastore.version")
+ .doc("Version of the Hive metastore. Available options are " +
s"<code>0.12.0</code> through <code>$hiveExecutionVersion</code>.")
+ .stringConf
+ .createWithDefault(hiveExecutionVersion)
- val HIVE_EXECUTION_VERSION = stringConf(
- key = "spark.sql.hive.version",
- defaultValue = Some(hiveExecutionVersion),
- doc = "Version of Hive used internally by Spark SQL.")
+ val HIVE_EXECUTION_VERSION = SQLConfigBuilder("spark.sql.hive.version")
+ .doc("Version of Hive used internally by Spark SQL.")
+ .stringConf
+ .createWithDefault(hiveExecutionVersion)
- val HIVE_METASTORE_JARS = stringConf("spark.sql.hive.metastore.jars",
- defaultValue = Some("builtin"),
- doc = s"""
+ val HIVE_METASTORE_JARS = SQLConfigBuilder("spark.sql.hive.metastore.jars")
+ .doc(s"""
| Location of the jars that should be used to instantiate the HiveMetastoreClient.
| This property can be one of three options: "
| 1. "builtin"
@@ -436,49 +436,61 @@ private[hive] object HiveContext extends Logging {
| 2. "maven"
| Use Hive jars of specified version downloaded from Maven repositories.
| 3. A classpath in the standard format for both Hive and Hadoop.
- """.stripMargin)
- 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 = "When true, also tries to merge possibly different but compatible Parquet schemas in " +
- "different Parquet data files. This configuration is only effective " +
- "when \"spark.sql.hive.convertMetastoreParquet\" is true.")
+ """.stripMargin)
+ .stringConf
+ .createWithDefault("builtin")
- val CONVERT_METASTORE_ORC = booleanConf("spark.sql.hive.convertMetastoreOrc",
- defaultValue = Some(true),
- doc = "When set to false, Spark SQL will use the Hive SerDe for ORC tables instead of " +
+ val CONVERT_METASTORE_PARQUET = SQLConfigBuilder("spark.sql.hive.convertMetastoreParquet")
+ .doc("When set to false, Spark SQL will use the Hive SerDe for parquet tables instead of " +
"the built in support.")
-
- val CONVERT_CTAS = booleanConf("spark.sql.hive.convertCTAS",
- defaultValue = Some(false),
- doc = "When true, a table created by a Hive CTAS statement (no USING clause) will be " +
+ .booleanConf
+ .createWithDefault(true)
+
+ val CONVERT_METASTORE_PARQUET_WITH_SCHEMA_MERGING =
+ SQLConfigBuilder("spark.sql.hive.convertMetastoreParquet.mergeSchema")
+ .doc("When true, also tries to merge possibly different but compatible Parquet schemas in " +
+ "different Parquet data files. This configuration is only effective " +
+ "when \"spark.sql.hive.convertMetastoreParquet\" is true.")
+ .booleanConf
+ .createWithDefault(false)
+
+ val CONVERT_CTAS = SQLConfigBuilder("spark.sql.hive.convertCTAS")
+ .doc("When true, a table created by a Hive CTAS statement (no USING clause) will be " +
"converted to a data source table, using the data source set by spark.sql.sources.default.")
+ .booleanConf
+ .createWithDefault(false)
- 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 " +
+ val CONVERT_METASTORE_ORC = SQLConfigBuilder("spark.sql.hive.convertMetastoreOrc")
+ .doc("When set to false, Spark SQL will use the Hive SerDe for ORC tables instead of " +
+ "the built in support.")
+ .booleanConf
+ .createWithDefault(true)
+
+ val HIVE_METASTORE_SHARED_PREFIXES = SQLConfigBuilder("spark.sql.hive.metastore.sharedPrefixes")
+ .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.")
+ .stringConf
+ .toSequence
+ .createWithDefault(jdbcPrefixes)
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 " +
+ val HIVE_METASTORE_BARRIER_PREFIXES = SQLConfigBuilder("spark.sql.hive.metastore.barrierPrefixes")
+ .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 = "When set to true, Hive Thrift server executes SQL queries in an asynchronous way.")
+ .stringConf
+ .toSequence
+ .createWithDefault(Nil)
+
+ val HIVE_THRIFT_SERVER_ASYNC = SQLConfigBuilder("spark.sql.hive.thriftServer.async")
+ .doc("When set to true, Hive Thrift server executes SQL queries in an asynchronous way.")
+ .booleanConf
+ .createWithDefault(true)
/**
* The version of the hive client that will be used to communicate with the metastore. Note that