aboutsummaryrefslogtreecommitdiff
path: root/sql/hive/src/main
diff options
context:
space:
mode:
authorYin Huai <yhuai@databricks.com>2016-06-16 17:06:24 -0700
committerShixiong Zhu <shixiong@databricks.com>2016-06-16 17:06:24 -0700
commitd9c6628c47de547dc537310e3c775c7f3e0e4a12 (patch)
tree44d897bb65296bb52bdee0a3f306bbf224740c53 /sql/hive/src/main
parent62d2fa5e996d428caaea005041b17ec115473762 (diff)
downloadspark-d9c6628c47de547dc537310e3c775c7f3e0e4a12.tar.gz
spark-d9c6628c47de547dc537310e3c775c7f3e0e4a12.tar.bz2
spark-d9c6628c47de547dc537310e3c775c7f3e0e4a12.zip
[SPARK-15991] SparkContext.hadoopConfiguration should be always the base of hadoop conf created by SessionState
## What changes were proposed in this pull request? Before this patch, after a SparkSession has been created, hadoop conf set directly to SparkContext.hadoopConfiguration will not affect the hadoop conf created by SessionState. This patch makes the change to always use SparkContext.hadoopConfiguration as the base. This patch also changes the behavior of hive-site.xml support added in https://github.com/apache/spark/pull/12689/. With this patch, we will load hive-site.xml to SparkContext.hadoopConfiguration. ## How was this patch tested? New test in SparkSessionBuilderSuite. Author: Yin Huai <yhuai@databricks.com> Closes #13711 from yhuai/SPARK-15991.
Diffstat (limited to 'sql/hive/src/main')
-rw-r--r--sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSharedState.scala5
1 files changed, 3 insertions, 2 deletions
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSharedState.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSharedState.scala
index 6b7a333f2d..12b4962fba 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSharedState.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSharedState.scala
@@ -36,11 +36,12 @@ private[hive] class HiveSharedState(override val sparkContext: SparkContext)
*/
// This needs to be a lazy val at here because TestHiveSharedState is overriding it.
lazy val metadataHive: HiveClient = {
- HiveUtils.newClientForMetadata(sparkContext.conf, hadoopConf)
+ HiveUtils.newClientForMetadata(sparkContext.conf, sparkContext.hadoopConfiguration)
}
/**
* A catalog that interacts with the Hive metastore.
*/
- override lazy val externalCatalog = new HiveExternalCatalog(metadataHive, hadoopConf)
+ override lazy val externalCatalog =
+ new HiveExternalCatalog(metadataHive, sparkContext.hadoopConfiguration)
}