Skip to content

[SPARK-25694][SQL] Add a config for URL.setURLStreamHandlerFactory #26530

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Closed
wants to merge 10 commits into from
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 @@ -36,6 +36,7 @@ import org.apache.spark.sql.execution.CacheManager
import org.apache.spark.sql.execution.streaming.StreamExecution
import org.apache.spark.sql.execution.ui.{SQLAppStatusListener, SQLAppStatusStore, SQLTab}
import org.apache.spark.sql.internal.StaticSQLConf._
import org.apache.spark.sql.internal.config.DEFAULT_URL_STREAM_HANDLER_FACTORY_ENABLED
import org.apache.spark.sql.streaming.StreamingQuery
import org.apache.spark.status.ElementTrackingStore
import org.apache.spark.util.Utils
Expand All @@ -52,6 +53,8 @@ private[sql] class SharedState(
initialConfigs: scala.collection.Map[String, String])
extends Logging {

SharedState.setFsUrlStreamHandlerFactory(sparkContext.conf)

// Load hive-site.xml into hadoopConf and determine the warehouse path we want to use, based on
// the config from both hive and Spark SQL. Finally set the warehouse config value to sparkConf.
val warehousePath: String = {
Expand Down Expand Up @@ -191,11 +194,23 @@ private[sql] class SharedState(
}

object SharedState extends Logging {
try {
URL.setURLStreamHandlerFactory(new FsUrlStreamHandlerFactory())
} catch {
case e: Error =>
logWarning("URL.setURLStreamHandlerFactory failed to set FsUrlStreamHandlerFactory")
@volatile private var fsUrlStreamHandlerFactoryInitialized = false

private def setFsUrlStreamHandlerFactory(conf: SparkConf): Unit = {
if (!fsUrlStreamHandlerFactoryInitialized &&
conf.get(DEFAULT_URL_STREAM_HANDLER_FACTORY_ENABLED)) {
synchronized {
if (!fsUrlStreamHandlerFactoryInitialized) {
try {
URL.setURLStreamHandlerFactory(new FsUrlStreamHandlerFactory())
fsUrlStreamHandlerFactoryInitialized = true
} catch {
case NonFatal(_) =>
logWarning("URL.setURLStreamHandlerFactory failed to set FsUrlStreamHandlerFactory")
}
}
}
}
}

private val HIVE_EXTERNAL_CATALOG_CLASS_NAME = "org.apache.spark.sql.hive.HiveExternalCatalog"
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,29 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.spark.sql.internal

import org.apache.spark.internal.config.ConfigBuilder

package object config {

private[spark] val DEFAULT_URL_STREAM_HANDLER_FACTORY_ENABLED =
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why is it in a separate file not in StaticSQLConf or SQLConf?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Made a followup - #26570

ConfigBuilder("spark.sql.defaultUrlStreamHandlerFactory.enabled")
.doc("When true, set FsUrlStreamHandlerFactory to support ADD JAR against HDFS locations")
.booleanConf
.createWithDefault(true)
}