Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -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}
Expand Down Expand Up @@ -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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down