Skip to content

Commit 6a6de97

Browse files
SPARK-12868: Allow Add jar to add jars from hdfs/s3n urls.
1 parent a133057 commit 6a6de97

File tree

2 files changed

+36
-2
lines changed

2 files changed

+36
-2
lines changed

sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala

Lines changed: 26 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -18,13 +18,14 @@
1818
package org.apache.spark.sql.hive.client
1919

2020
import java.io.{File, PrintStream}
21+
import java.net.{MalformedURLException, URL}
2122

2223
import scala.collection.JavaConverters._
2324
import scala.collection.mutable.ArrayBuffer
2425
import scala.language.reflectiveCalls
2526

2627
import org.apache.hadoop.conf.Configuration
27-
import org.apache.hadoop.fs.Path
28+
import org.apache.hadoop.fs.{FileSystem, FsUrlStreamHandlerFactory, Path}
2829
import org.apache.hadoop.hive.conf.HiveConf
2930
import org.apache.hadoop.hive.metastore.{TableType => HiveTableType}
3031
import org.apache.hadoop.hive.metastore.api.{Database => HiveDatabase, FieldSchema}
@@ -86,6 +87,9 @@ private[hive] class HiveClientImpl(
8687
// Circular buffer to hold what hive prints to STDOUT and ERR. Only printed when failures occur.
8788
private val outputBuffer = new CircularBuffer()
8889

90+
// An object lock to ensure URL factory is registered exactly once.
91+
object URLFactoryRegistrationLock{}
92+
8993
private val shim = version match {
9094
case hive.v12 => new Shim_v0_12()
9195
case hive.v13 => new Shim_v0_13()
@@ -690,7 +694,27 @@ private[hive] class HiveClientImpl(
690694
new File(path).toURI.toURL
691695
} else {
692696
// `path` is a URL with a scheme
693-
uri.toURL
697+
try {
698+
uri.toURL
699+
} catch {
700+
case e: MalformedURLException =>
701+
Option(FileSystem.get(uri, hadoopConf)) match {
702+
case Some(fs) =>
703+
URLFactoryRegistrationLock.synchronized {
704+
try {
705+
// check one more time, in case another thread set the factory.
706+
uri.toURL
707+
} catch {
708+
case e: MalformedURLException =>
709+
// Register the URLStreamHanlerFactory so hdfs urls work.
710+
URL.setURLStreamHandlerFactory(new FsUrlStreamHandlerFactory(hadoopConf))
711+
uri.toURL
712+
}
713+
}
714+
case None =>
715+
throw e
716+
}
717+
}
694718
}
695719
clientLoader.addJar(jarURL)
696720
runSqlHive(s"ADD JAR $path")

sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala

Lines changed: 10 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -856,6 +856,16 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter {
856856
sql("DROP TABLE alter1")
857857
}
858858

859+
test("SPARK-12868 ADD JAR FROM HDFS") {
860+
val testJar = "hdfs://nn:8020/foo.jar"
861+
// This should fail with unknown host, as its just testing the URL parsing
862+
// before SPARK-12868 it was failing with Malformed URI
863+
val e = intercept[RuntimeException] {
864+
sql(s"ADD JAR $testJar")
865+
}
866+
assert(e.getMessage.contains("java.net.UnknownHostException: nn1"))
867+
}
868+
859869
test("ADD JAR command 2") {
860870
// this is a test case from mapjoin_addjar.q
861871
val testJar = TestHive.getHiveFile("hive-hcatalog-core-0.13.1.jar").getCanonicalPath

0 commit comments

Comments
 (0)