Skip to content

Commit ee3bd6d

Browse files
jiangzhodongjoon-hyun
authored andcommitted
[SPARK-25694][SQL] Add a config for URL.setURLStreamHandlerFactory
### What changes were proposed in this pull request? Add a property `spark.fsUrlStreamHandlerFactory.enabled` to allow users turn off the default registration of `org.apache.hadoop.fs.FsUrlStreamHandlerFactory` ### Why are the changes needed? This [SPARK-25694](https://issues.apache.org/jira/browse/SPARK-25694) is a long-standing issue. Originally, [[SPARK-12868][SQL] Allow adding jars from hdfs](#17342 ) added this for better Hive support. However, this have a side-effect when the users use Apache Spark without `-Phive`. This causes exceptions when the users tries to use another custom factories or 3rd party library (trying to set this). This configuration will unblock those non-hive users. ### Does this PR introduce any user-facing change? Yes. This provides a new user-configurable property. By default, the behavior is unchanged. ### How was this patch tested? Manual testing. **BEFORE** ``` $ build/sbt package $ bin/spark-shell scala> sql("show tables").show +--------+---------+-----------+ |database|tableName|isTemporary| +--------+---------+-----------+ +--------+---------+-----------+ scala> java.net.URL.setURLStreamHandlerFactory(new org.apache.hadoop.fs.FsUrlStreamHandlerFactory()) java.lang.Error: factory already defined at java.net.URL.setURLStreamHandlerFactory(URL.java:1134) ... 47 elided ``` **AFTER** ``` $ build/sbt package $ bin/spark-shell --conf spark.sql.defaultUrlStreamHandlerFactory.enabled=false scala> sql("show tables").show +--------+---------+-----------+ |database|tableName|isTemporary| +--------+---------+-----------+ +--------+---------+-----------+ scala> java.net.URL.setURLStreamHandlerFactory(new org.apache.hadoop.fs.FsUrlStreamHandlerFactory()) ``` Closes #26530 from jiangzho/master. Lead-authored-by: Zhou Jiang <zhou_jiang@apple.com> Co-authored-by: Dongjoon Hyun <dhyun@apple.com> Co-authored-by: zhou-jiang <zhou_jiang@apple.com> Signed-off-by: DB Tsai <d_tsai@apple.com>
1 parent 42f8f79 commit ee3bd6d

File tree

2 files changed

+49
-5
lines changed

2 files changed

+49
-5
lines changed

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

Lines changed: 20 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -36,6 +36,7 @@ import org.apache.spark.sql.execution.CacheManager
3636
import org.apache.spark.sql.execution.streaming.StreamExecution
3737
import org.apache.spark.sql.execution.ui.{SQLAppStatusListener, SQLAppStatusStore, SQLTab}
3838
import org.apache.spark.sql.internal.StaticSQLConf._
39+
import org.apache.spark.sql.internal.config.DEFAULT_URL_STREAM_HANDLER_FACTORY_ENABLED
3940
import org.apache.spark.sql.streaming.StreamingQuery
4041
import org.apache.spark.status.ElementTrackingStore
4142
import org.apache.spark.util.Utils
@@ -52,6 +53,8 @@ private[sql] class SharedState(
5253
initialConfigs: scala.collection.Map[String, String])
5354
extends Logging {
5455

56+
SharedState.setFsUrlStreamHandlerFactory(sparkContext.conf)
57+
5558
// Load hive-site.xml into hadoopConf and determine the warehouse path we want to use, based on
5659
// the config from both hive and Spark SQL. Finally set the warehouse config value to sparkConf.
5760
val warehousePath: String = {
@@ -191,11 +194,23 @@ private[sql] class SharedState(
191194
}
192195

193196
object SharedState extends Logging {
194-
try {
195-
URL.setURLStreamHandlerFactory(new FsUrlStreamHandlerFactory())
196-
} catch {
197-
case e: Error =>
198-
logWarning("URL.setURLStreamHandlerFactory failed to set FsUrlStreamHandlerFactory")
197+
@volatile private var fsUrlStreamHandlerFactoryInitialized = false
198+
199+
private def setFsUrlStreamHandlerFactory(conf: SparkConf): Unit = {
200+
if (!fsUrlStreamHandlerFactoryInitialized &&
201+
conf.get(DEFAULT_URL_STREAM_HANDLER_FACTORY_ENABLED)) {
202+
synchronized {
203+
if (!fsUrlStreamHandlerFactoryInitialized) {
204+
try {
205+
URL.setURLStreamHandlerFactory(new FsUrlStreamHandlerFactory())
206+
fsUrlStreamHandlerFactoryInitialized = true
207+
} catch {
208+
case NonFatal(_) =>
209+
logWarning("URL.setURLStreamHandlerFactory failed to set FsUrlStreamHandlerFactory")
210+
}
211+
}
212+
}
213+
}
199214
}
200215

201216
private val HIVE_EXTERNAL_CATALOG_CLASS_NAME = "org.apache.spark.sql.hive.HiveExternalCatalog"
Lines changed: 29 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,29 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
8+
*
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
*
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
16+
*/
17+
18+
package org.apache.spark.sql.internal
19+
20+
import org.apache.spark.internal.config.ConfigBuilder
21+
22+
package object config {
23+
24+
private[spark] val DEFAULT_URL_STREAM_HANDLER_FACTORY_ENABLED =
25+
ConfigBuilder("spark.sql.defaultUrlStreamHandlerFactory.enabled")
26+
.doc("When true, set FsUrlStreamHandlerFactory to support ADD JAR against HDFS locations")
27+
.booleanConf
28+
.createWithDefault(true)
29+
}

0 commit comments

Comments
 (0)