Skip to content

Commit 2ba1eba

Browse files
weiqingyMarcelo Vanzin
authored andcommitted
[SPARK-12868][SQL] Allow adding jars from hdfs
## What changes were proposed in this pull request? Spark 2.2 is going to be cut, it'll be great if SPARK-12868 can be resolved before that. There have been several PRs for this like [PR#16324](#16324) , but all of them are inactivity for a long time or have been closed. This PR added a SparkUrlStreamHandlerFactory, which relies on 'protocol' to choose the appropriate UrlStreamHandlerFactory like FsUrlStreamHandlerFactory to create URLStreamHandler. ## How was this patch tested? 1. Add a new unit test. 2. Check manually. Before: throw an exception with " failed unknown protocol: hdfs" <img width="914" alt="screen shot 2017-03-17 at 9 07 36 pm" src="https://cloud.githubusercontent.com/assets/8546874/24075277/5abe0a7c-0bd5-11e7-900e-ec3d3105da0b.png"> After: <img width="1148" alt="screen shot 2017-03-18 at 11 42 18 am" src="https://cloud.githubusercontent.com/assets/8546874/24075283/69382a60-0bd5-11e7-8d30-d9405c3aaaba.png"> Author: Weiqing Yang <[email protected]> Closes #17342 from weiqingy/SPARK-18910.
1 parent a277ae8 commit 2ba1eba

File tree

2 files changed

+22
-1
lines changed

2 files changed

+22
-1
lines changed

sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala

Lines changed: 9 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -17,12 +17,14 @@
1717

1818
package org.apache.spark.sql.internal
1919

20+
import java.net.URL
2021
import java.util.Locale
2122

2223
import scala.reflect.ClassTag
2324
import scala.util.control.NonFatal
2425

2526
import org.apache.hadoop.conf.Configuration
27+
import org.apache.hadoop.fs.FsUrlStreamHandlerFactory
2628

2729
import org.apache.spark.{SparkConf, SparkContext, SparkException}
2830
import org.apache.spark.internal.Logging
@@ -154,7 +156,13 @@ private[sql] class SharedState(val sparkContext: SparkContext) extends Logging {
154156
}
155157
}
156158

157-
object SharedState {
159+
object SharedState extends Logging {
160+
try {
161+
URL.setURLStreamHandlerFactory(new FsUrlStreamHandlerFactory())
162+
} catch {
163+
case e: Error =>
164+
logWarning("URL.setURLStreamHandlerFactory failed to set FsUrlStreamHandlerFactory")
165+
}
158166

159167
private val HIVE_EXTERNAL_CATALOG_CLASS_NAME = "org.apache.spark.sql.hive.HiveExternalCatalog"
160168

sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala

Lines changed: 13 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -19,6 +19,7 @@ package org.apache.spark.sql
1919

2020
import java.io.File
2121
import java.math.MathContext
22+
import java.net.{MalformedURLException, URL}
2223
import java.sql.Timestamp
2324
import java.util.concurrent.atomic.AtomicBoolean
2425

@@ -2606,4 +2607,16 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext {
26062607
case ae: AnalysisException => assert(ae.plan == null && ae.getMessage == ae.getSimpleMessage)
26072608
}
26082609
}
2610+
2611+
test("SPARK-12868: Allow adding jars from hdfs ") {
2612+
val jarFromHdfs = "hdfs://doesnotmatter/test.jar"
2613+
val jarFromInvalidFs = "fffs://doesnotmatter/test.jar"
2614+
2615+
// if 'hdfs' is not supported, MalformedURLException will be thrown
2616+
new URL(jarFromHdfs)
2617+
2618+
intercept[MalformedURLException] {
2619+
new URL(jarFromInvalidFs)
2620+
}
2621+
}
26092622
}

0 commit comments

Comments
 (0)