spark git commit: [SPARK-16901] Hive settings in hive-site.xml may be overridden by Hive's default values
Repository: spark Updated Branches: refs/heads/master 6cbde337a -> e679bc3c1 [SPARK-16901] Hive settings in hive-site.xml may be overridden by Hive's default values ## What changes were proposed in this pull request? When we create the HiveConf for metastore client, we use a Hadoop Conf as the base, which may contain Hive settings in hive-site.xml (https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala#L49). However, HiveConf's initialize function basically ignores the base Hadoop Conf and always its default values (i.e. settings with non-null default values) as the base (https://github.com/apache/hive/blob/release-1.2.1/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java#L2687). So, even a user put javax.jdo.option.ConnectionURL in hive-site.xml, it is not used and Hive will use its default, which is jdbc:derby:;databaseName=metastore_db;create=true. This issue only shows up when `spark.sql.hive.metastore.jars` is not set to builtin. ## How was this patch tested? New test in HiveSparkSubmitSuite. Author: Yin Huai Closes #14497 from yhuai/SPARK-16901. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/e679bc3c Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/e679bc3c Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/e679bc3c Branch: refs/heads/master Commit: e679bc3c1cd418ef0025d2ecbc547c9660cac433 Parents: 6cbde33 Author: Yin Huai Authored: Fri Aug 5 15:52:02 2016 -0700 Committer: Yin Huai Committed: Fri Aug 5 15:52:02 2016 -0700 -- .../spark/sql/hive/client/HiveClientImpl.scala | 24 +- .../spark/sql/hive/HiveSparkSubmitSuite.scala | 80 2 files changed, 101 insertions(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/e679bc3c/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala -- diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala index ef69ac7..3bf4ed5 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala @@ -141,14 +141,32 @@ private[hive] class HiveClientImpl( // so we should keep `conf` and reuse the existing instance of `CliSessionState`. originalState } else { -val hiveConf = new HiveConf(hadoopConf, classOf[SessionState]) +val hiveConf = new HiveConf(classOf[SessionState]) +// 1: we set all confs in the hadoopConf to this hiveConf. +// This hadoopConf contains user settings in Hadoop's core-site.xml file +// and Hive's hive-site.xml file. Note, we load hive-site.xml file manually in +// SharedState and put settings in this hadoopConf instead of relying on HiveConf +// to load user settings. Otherwise, HiveConf's initialize method will override +// settings in the hadoopConf. This issue only shows up when spark.sql.hive.metastore.jars +// is not set to builtin. When spark.sql.hive.metastore.jars is builtin, the classpath +// has hive-site.xml. So, HiveConf will use that to override its default values. +hadoopConf.iterator().asScala.foreach { entry => + val key = entry.getKey + val value = entry.getValue + if (key.toLowerCase.contains("password")) { +logDebug(s"Applying Hadoop and Hive config to Hive Conf: $key=xxx") + } else { +logDebug(s"Applying Hadoop and Hive config to Hive Conf: $key=$value") + } + hiveConf.set(key, value) +} // HiveConf is a Hadoop Configuration, which has a field of classLoader and // the initial value will be the current thread's context class loader // (i.e. initClassLoader at here). // We call initialConf.setClassLoader(initClassLoader) at here to make // this action explicit. hiveConf.setClassLoader(initClassLoader) -// First, we set all spark confs to this hiveConf. +// 2: we set all spark confs to this hiveConf. sparkConf.getAll.foreach { case (k, v) => if (k.toLowerCase.contains("password")) { logDebug(s"Applying Spark config to Hive Conf: $k=xxx") @@ -157,7 +175,7 @@ private[hive] class HiveClientImpl( } hiveConf.set(k, v) } -// Second, we set all entries in config to this hiveConf. +// 3: we set all entries in config to this hiveConf. extraConfig.foreach { case (k, v) => if (k.toLowerCase.contains("passwo
spark git commit: [SPARK-16901] Hive settings in hive-site.xml may be overridden by Hive's default values
Repository: spark Updated Branches: refs/heads/branch-2.0 d99d90982 -> b5d65b45d [SPARK-16901] Hive settings in hive-site.xml may be overridden by Hive's default values ## What changes were proposed in this pull request? When we create the HiveConf for metastore client, we use a Hadoop Conf as the base, which may contain Hive settings in hive-site.xml (https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala#L49). However, HiveConf's initialize function basically ignores the base Hadoop Conf and always its default values (i.e. settings with non-null default values) as the base (https://github.com/apache/hive/blob/release-1.2.1/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java#L2687). So, even a user put javax.jdo.option.ConnectionURL in hive-site.xml, it is not used and Hive will use its default, which is jdbc:derby:;databaseName=metastore_db;create=true. This issue only shows up when `spark.sql.hive.metastore.jars` is not set to builtin. ## How was this patch tested? New test in HiveSparkSubmitSuite. Author: Yin Huai Closes #14497 from yhuai/SPARK-16901. (cherry picked from commit e679bc3c1cd418ef0025d2ecbc547c9660cac433) Signed-off-by: Yin Huai Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/b5d65b45 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/b5d65b45 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/b5d65b45 Branch: refs/heads/branch-2.0 Commit: b5d65b45dfd34a7f451465ed3aac923077675166 Parents: d99d909 Author: Yin Huai Authored: Fri Aug 5 15:52:02 2016 -0700 Committer: Yin Huai Committed: Fri Aug 5 15:52:17 2016 -0700 -- .../spark/sql/hive/client/HiveClientImpl.scala | 24 +- .../spark/sql/hive/HiveSparkSubmitSuite.scala | 80 2 files changed, 101 insertions(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/b5d65b45/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala -- diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala index 6cdf3ef..1d40895 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala @@ -139,14 +139,32 @@ private[hive] class HiveClientImpl( // so we should keep `conf` and reuse the existing instance of `CliSessionState`. originalState } else { -val hiveConf = new HiveConf(hadoopConf, classOf[SessionState]) +val hiveConf = new HiveConf(classOf[SessionState]) +// 1: we set all confs in the hadoopConf to this hiveConf. +// This hadoopConf contains user settings in Hadoop's core-site.xml file +// and Hive's hive-site.xml file. Note, we load hive-site.xml file manually in +// SharedState and put settings in this hadoopConf instead of relying on HiveConf +// to load user settings. Otherwise, HiveConf's initialize method will override +// settings in the hadoopConf. This issue only shows up when spark.sql.hive.metastore.jars +// is not set to builtin. When spark.sql.hive.metastore.jars is builtin, the classpath +// has hive-site.xml. So, HiveConf will use that to override its default values. +hadoopConf.iterator().asScala.foreach { entry => + val key = entry.getKey + val value = entry.getValue + if (key.toLowerCase.contains("password")) { +logDebug(s"Applying Hadoop and Hive config to Hive Conf: $key=xxx") + } else { +logDebug(s"Applying Hadoop and Hive config to Hive Conf: $key=$value") + } + hiveConf.set(key, value) +} // HiveConf is a Hadoop Configuration, which has a field of classLoader and // the initial value will be the current thread's context class loader // (i.e. initClassLoader at here). // We call initialConf.setClassLoader(initClassLoader) at here to make // this action explicit. hiveConf.setClassLoader(initClassLoader) -// First, we set all spark confs to this hiveConf. +// 2: we set all spark confs to this hiveConf. sparkConf.getAll.foreach { case (k, v) => if (k.toLowerCase.contains("password")) { logDebug(s"Applying Spark config to Hive Conf: $k=xxx") @@ -155,7 +173,7 @@ private[hive] class HiveClientImpl( } hiveConf.set(k, v) } -// Second, we set all entries in config to this hiveConf. +// 3: we set all entries in config to thi