diff options
author | Cheng Hao <hao.cheng@intel.com> | 2015-05-27 14:21:00 -0700 |
---|---|---|
committer | Yin Huai <yhuai@databricks.com> | 2015-05-27 14:21:00 -0700 |
commit | db3fd054f240c7e38aba0732e471df65cd14011a (patch) | |
tree | ec60d5841a97c3451c264d692ce5d69c8f87bc27 /sql/hive | |
parent | b97ddff000b99adca3dd8fe13d01054fd5014fa0 (diff) | |
download | spark-db3fd054f240c7e38aba0732e471df65cd14011a.tar.gz spark-db3fd054f240c7e38aba0732e471df65cd14011a.tar.bz2 spark-db3fd054f240c7e38aba0732e471df65cd14011a.zip |
[SPARK-7853] [SQL] Fixes a class loader issue in Spark SQL
This PR is based on PR #6396 authored by chenghao-intel. Essentially, Spark SQL should use context classloader to load SerDe classes.
yhuai helped updating the test case, and I fixed a bug in the original `CliSuite`: while testing the CLI tool with `runCliWithin`, we don't append `\n` to the last query, thus the last query is never executed.
Original PR description is pasted below.
----
```
bin/spark-sql --jars ./sql/hive/src/test/resources/hive-hcatalog-core-0.13.1.jar
CREATE TABLE t1(a string, b string) ROW FORMAT SERDE 'org.apache.hive.hcatalog.data.JsonSerDe';
```
Throws exception like
```
15/05/26 00:16:33 ERROR SparkSQLDriver: Failed in [CREATE TABLE t1(a string, b string) ROW FORMAT SERDE 'org.apache.hive.hcatalog.data.JsonSerDe']
org.apache.spark.sql.execution.QueryExecutionException: FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. Cannot validate serde: org.apache.hive.hcatalog.data.JsonSerDe
at org.apache.spark.sql.hive.client.ClientWrapper$$anonfun$runHive$1.apply(ClientWrapper.scala:333)
at org.apache.spark.sql.hive.client.ClientWrapper$$anonfun$runHive$1.apply(ClientWrapper.scala:310)
at org.apache.spark.sql.hive.client.ClientWrapper.withHiveState(ClientWrapper.scala:139)
at org.apache.spark.sql.hive.client.ClientWrapper.runHive(ClientWrapper.scala:310)
at org.apache.spark.sql.hive.client.ClientWrapper.runSqlHive(ClientWrapper.scala:300)
at org.apache.spark.sql.hive.HiveContext.runSqlHive(HiveContext.scala:457)
at org.apache.spark.sql.hive.execution.HiveNativeCommand.run(HiveNativeCommand.scala:33)
at org.apache.spark.sql.execution.ExecutedCommand.sideEffectResult$lzycompute(commands.scala:57)
at org.apache.spark.sql.execution.ExecutedCommand.sideEffectResult(commands.scala:57)
at org.apache.spark.sql.execution.ExecutedCommand.doExecute(commands.scala:68)
at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:88)
at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:88)
at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:148)
at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:87)
at org.apache.spark.sql.SQLContext$QueryExecution.toRdd$lzycompute(SQLContext.scala:922)
at org.apache.spark.sql.SQLContext$QueryExecution.toRdd(SQLContext.scala:922)
at org.apache.spark.sql.DataFrame.<init>(DataFrame.scala:147)
at org.apache.spark.sql.DataFrame.<init>(DataFrame.scala:131)
at org.apache.spark.sql.DataFrame$.apply(DataFrame.scala:51)
at org.apache.spark.sql.SQLContext.sql(SQLContext.scala:727)
at org.apache.spark.sql.hive.thriftserver.AbstractSparkSQLDriver.run(AbstractSparkSQLDriver.scala:57)
```
Author: Cheng Hao <hao.cheng@intel.com>
Author: Cheng Lian <lian@databricks.com>
Author: Yin Huai <yhuai@databricks.com>
Closes #6435 from liancheng/classLoader and squashes the following commits:
d4c4845 [Cheng Lian] Fixes CliSuite
75e80e2 [Yin Huai] Update the fix.
fd26533 [Cheng Hao] scalastyle
dd78775 [Cheng Hao] workaround for classloader of IsolatedClientLoader
Diffstat (limited to 'sql/hive')
-rw-r--r-- | sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala | 18 | ||||
-rw-r--r-- | sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala | 2 |
2 files changed, 16 insertions, 4 deletions
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 b64768abab..9ab98fdcce 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 @@ -18,6 +18,7 @@ package org.apache.spark.sql.hive import java.io.{BufferedReader, File, InputStreamReader, PrintStream} +import java.net.{URL, URLClassLoader} import java.sql.Timestamp import java.util.{ArrayList => JArrayList} @@ -25,7 +26,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.collection.mutable.{ArrayBuffer, HashMap} import scala.language.implicitConversions import org.apache.hadoop.fs.{FileSystem, Path} @@ -188,8 +189,19 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { "Specify a vaild path to the correct hive jars using $HIVE_METASTORE_JARS " + s"or change $HIVE_METASTORE_VERSION to $hiveExecutionVersion.") } - val jars = getClass.getClassLoader match { - case urlClassLoader: java.net.URLClassLoader => urlClassLoader.getURLs + // We recursively add all jars in the class loader chain, + // starting from the given urlClassLoader. + def addJars(urlClassLoader: URLClassLoader): Array[URL] = { + val jarsInParent = urlClassLoader.getParent match { + case parent: URLClassLoader => addJars(parent) + case other => Array.empty[URL] + } + + urlClassLoader.getURLs ++ jarsInParent + } + + val jars = Utils.getContextOrSparkClassLoader match { + case urlClassLoader: URLClassLoader => addJars(urlClassLoader) case other => throw new IllegalArgumentException( "Unable to locate hive jars to connect to metastore " + diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala index 0b6f7a334a..294fc3bd7d 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala @@ -79,7 +79,7 @@ class HadoopTableReader( makeRDDForTable( hiveTable, Class.forName( - relation.tableDesc.getSerdeClassName, true, Utils.getSparkClassLoader) + relation.tableDesc.getSerdeClassName, true, Utils.getContextOrSparkClassLoader) .asInstanceOf[Class[Deserializer]], filterOpt = None) |