diff options
author | windpiger <songjun@outlook.com> | 2017-02-23 22:57:23 -0800 |
---|---|---|
committer | Wenchen Fan <wenchen@databricks.com> | 2017-02-23 22:57:23 -0800 |
commit | 8f33731e796750e6f60dc9e2fc33a94d29d198b4 (patch) | |
tree | b61ffee3dc3ee046898ba6fb34185be7883f28d1 /sql/core | |
parent | d7e43b613aeb6d85b8522d7c16c6e0807843c964 (diff) | |
download | spark-8f33731e796750e6f60dc9e2fc33a94d29d198b4.tar.gz spark-8f33731e796750e6f60dc9e2fc33a94d29d198b4.tar.bz2 spark-8f33731e796750e6f60dc9e2fc33a94d29d198b4.zip |
[SPARK-19664][SQL] put hive.metastore.warehouse.dir in hadoopconf to overwrite its original value
## What changes were proposed in this pull request?
In [SPARK-15959](https://issues.apache.org/jira/browse/SPARK-15959), we bring back the `hive.metastore.warehouse.dir` , while in the logic, when use the value of `spark.sql.warehouse.dir` to overwrite `hive.metastore.warehouse.dir` , it set it to `sparkContext.conf` which does not overwrite the value is hadoopConf, I think it should put in `sparkContext.hadoopConfiguration` and overwrite the original value of hadoopConf
https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala#L64
## How was this patch tested?
N/A
Author: windpiger <songjun@outlook.com>
Closes #16996 from windpiger/hivemetawarehouseConf.
Diffstat (limited to 'sql/core')
-rw-r--r-- | sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala | 8 |
1 files changed, 6 insertions, 2 deletions
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala index 7ce9938f0d..bce84de45c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala @@ -42,9 +42,12 @@ private[sql] class SharedState(val sparkContext: SparkContext) extends Logging { val warehousePath: String = { val configFile = Utils.getContextOrSparkClassLoader.getResource("hive-site.xml") if (configFile != null) { + logInfo(s"loading hive config file: $configFile") sparkContext.hadoopConfiguration.addResource(configFile) } + // hive.metastore.warehouse.dir only stay in hadoopConf + sparkContext.conf.remove("hive.metastore.warehouse.dir") // Set the Hive metastore warehouse path to the one we use val hiveWarehouseDir = sparkContext.hadoopConfiguration.get("hive.metastore.warehouse.dir") if (hiveWarehouseDir != null && !sparkContext.conf.contains(WAREHOUSE_PATH.key)) { @@ -61,10 +64,11 @@ private[sql] class SharedState(val sparkContext: SparkContext) extends Logging { // When neither spark.sql.warehouse.dir nor hive.metastore.warehouse.dir is set, // we will set hive.metastore.warehouse.dir to the default value of spark.sql.warehouse.dir. val sparkWarehouseDir = sparkContext.conf.get(WAREHOUSE_PATH) - sparkContext.conf.set("hive.metastore.warehouse.dir", sparkWarehouseDir) + logInfo(s"Setting hive.metastore.warehouse.dir ('$hiveWarehouseDir') to the value of " + + s"${WAREHOUSE_PATH.key} ('$sparkWarehouseDir').") + sparkContext.hadoopConfiguration.set("hive.metastore.warehouse.dir", sparkWarehouseDir) sparkWarehouseDir } - } logInfo(s"Warehouse path is '$warehousePath'.") |