You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by ma...@apache.org on 2015/05/22 23:44:22 UTC

spark git commit: [SPARK-7758] [SQL] Override more configs to avoid failure when connect to a postgre sql

Repository: spark
Updated Branches:
  refs/heads/master eac00691d -> 31d5d463e


[SPARK-7758] [SQL] Override more configs to avoid failure when connect to a postgre sql

https://issues.apache.org/jira/browse/SPARK-7758

When initializing `executionHive`, we only masks
`javax.jdo.option.ConnectionURL` to override metastore location.  However,
other properties that relates to the actual Hive metastore data source are not
masked.  For example, when using Spark SQL with a PostgreSQL backed Hive
metastore, `executionHive` actually tries to use settings read from
`hive-site.xml`, which talks about PostgreSQL, to connect to the temporary
Derby metastore, thus causes error.

To fix this, we need to mask all metastore data source properties.
Specifically, according to the code of [Hive `ObjectStore.getDataSourceProps()`
method] [1], all properties whose name mentions "jdo" and "datanucleus" must be
included.

[1]: https://github.com/apache/hive/blob/release-0.13.1/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java#L288

Have tested using postgre sql as metastore, it worked fine.

Author: WangTaoTheTonic <wa...@huawei.com>

Closes #6314 from WangTaoTheTonic/SPARK-7758 and squashes the following commits:

ca7ae7c [WangTaoTheTonic] add comments
86caf2c [WangTaoTheTonic] delete unused import
e4f0feb [WangTaoTheTonic] block more data source related property
92a81fa [WangTaoTheTonic] fix style check
e3e683d [WangTaoTheTonic] override more configs to avoid failuer connecting to postgre sql


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/31d5d463
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/31d5d463
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/31d5d463

Branch: refs/heads/master
Commit: 31d5d463e76b6611c854c6cf27059fec8198adc9
Parents: eac0069
Author: WangTaoTheTonic <wa...@huawei.com>
Authored: Fri May 22 14:43:16 2015 -0700
Committer: Michael Armbrust <mi...@databricks.com>
Committed: Fri May 22 14:43:16 2015 -0700

----------------------------------------------------------------------
 .../scala/org/apache/spark/SparkContext.scala     |  2 +-
 .../org/apache/spark/sql/hive/HiveContext.scala   | 18 +++++++++++++++---
 2 files changed, 16 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/31d5d463/core/src/main/scala/org/apache/spark/SparkContext.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala
index ad78bdf..ea6c0de 100644
--- a/core/src/main/scala/org/apache/spark/SparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/SparkContext.scala
@@ -1884,7 +1884,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
    *
    * @param f the closure to clean
    * @param checkSerializable whether or not to immediately check <tt>f</tt> for serializability
-   * @throws <tt>SparkException<tt> if <tt>checkSerializable</tt> is set but <tt>f</tt> is not
+   * @throws SparkException if <tt>checkSerializable</tt> is set but <tt>f</tt> is not
    *   serializable
    */
   private[spark] def clean[F <: AnyRef](f: F, checkSerializable: Boolean = true): F = {

http://git-wip-us.apache.org/repos/asf/spark/blob/31d5d463/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
----------------------------------------------------------------------
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 863a5db..a8e8e70 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
@@ -25,6 +25,7 @@ import org.apache.hadoop.hive.ql.parse.VariableSubstitution
 import org.apache.spark.sql.catalyst.ParserDialect
 
 import scala.collection.JavaConversions._
+import scala.collection.mutable.HashMap
 import scala.language.implicitConversions
 
 import org.apache.hadoop.fs.{FileSystem, Path}
@@ -153,7 +154,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) {
    * Hive 13 as this is the version of Hive that is packaged with Spark SQL.  This copy of the
    * client is used for execution related tasks like registering temporary functions or ensuring
    * that the ThreadLocal SessionState is correctly populated.  This copy of Hive is *not* used
-   * for storing peristent metadata, and only point to a dummy metastore in a temporary directory.
+   * for storing persistent metadata, and only point to a dummy metastore in a temporary directory.
    */
   @transient
   protected[hive] lazy val executionHive: ClientWrapper = {
@@ -507,8 +508,19 @@ private[hive] object HiveContext {
   def newTemporaryConfiguration(): Map[String, String] = {
     val tempDir = Utils.createTempDir()
     val localMetastore = new File(tempDir, "metastore").getAbsolutePath
-    Map(
-      "javax.jdo.option.ConnectionURL" -> s"jdbc:derby:;databaseName=$localMetastore;create=true")
+    val propMap: HashMap[String, String] = HashMap()
+    // We have to mask all properties in hive-site.xml that relates to metastore data source
+    // as we used a local metastore here.
+    HiveConf.ConfVars.values().foreach { confvar  =>
+      if (confvar.varname.contains("datanucleus") || confvar.varname.contains("jdo")) {
+        propMap.put(confvar.varname, confvar.defaultVal)
+      }
+    }
+    propMap.put("javax.jdo.option.ConnectionURL",
+      s"jdbc:derby:;databaseName=$localMetastore;create=true")
+    propMap.put("datanucleus.rdbms.datastoreAdapterClassName",
+      "org.datanucleus.store.rdbms.adapter.DerbyAdapter")
+    propMap.toMap
   }
 
   protected val primitiveTypes =


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org