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 8de95fe64e66..5827b28c6d50 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 = { 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'.")