From f88af1980e93e1a02c199769984d136a48a21329 Mon Sep 17 00:00:00 2001 From: trystanleftwich Date: Sun, 17 Jan 2016 13:37:00 -0800 Subject: [PATCH 1/2] [SPARK-12868] - ADD JAR via sparkSQL JDBC will fail when using a HDFS URL --- .../org/apache/spark/sql/hive/client/HiveClientImpl.scala | 8 ++++++++ 1 file changed, 8 insertions(+) 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 cf1ff55c96fc..b494285d75a9 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 @@ -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.{TableType => HTableType} @@ -509,6 +511,12 @@ private[hive] class HiveClientImpl( } 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 From ecd0e17cd68f02c8cbcf87f1bf523296fed3e642 Mon Sep 17 00:00:00 2001 From: trystanleftwich Date: Wed, 3 Feb 2016 10:46:57 -0800 Subject: [PATCH 2/2] Added a test for testing add jar with a HDFS URI --- .../spark/sql/hive/execution/HiveQuerySuite.scala | 10 ++++++++++ 1 file changed, 10 insertions(+) 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 9632d27a2ffc..9e32bf720820 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 @@ -985,6 +985,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"