diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala index 598ccdeee4ad2..063f9ac2d2158 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala @@ -19,10 +19,12 @@ package org.apache.spark.sql.hive.client import java.io.{File, PrintStream} import java.util.{Map => JMap} +import java.net.URL import scala.collection.JavaConverters._ import scala.language.reflectiveCalls +import org.apache.hadoop.fs.FsUrlStreamHandlerFactory import org.apache.hadoop.fs.Path import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.metastore.api.{Database, FieldSchema} @@ -569,6 +571,12 @@ private[hive] class ClientWrapper( } def addJar(path: String): Unit = { + try { + URL.setURLStreamHandlerFactory(new FsUrlStreamHandlerFactory); + } catch { + // Factory can only be set once per JVM + case e: java.lang.Error if e.getMessage contains "factory already defined" => logDebug("Factory already define", e) + } val uri = new Path(path).toUri val jarURL = if (uri.getScheme == null) { // `path` is a local file path without a URL scheme diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index 8a5acaf3e10bc..6f64707e03926 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -968,6 +968,16 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { sql("DROP TABLE t1") } + test("SPARK-12868 ADD JAR FROM HDFS") { + val testJar = "hdfs://nn1:8020/foo.jar" + // This should fail with unknown host, as its just testing the URL parsing + // before SPARK-12868 it was failing with Malformed URI + val e = intercept[RuntimeException] { + sql(s"ADD JAR $testJar") + } + assert(e.getMessage.contains("java.net.UnknownHostException: nn1")) + } + test("CREATE TEMPORARY FUNCTION") { val funcJar = TestHive.getHiveFile("TestUDTF.jar").getCanonicalPath val jarURL = s"file://$funcJar"