diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala index b810bedac471..81a9c76511d8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala @@ -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 @@ -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 = { @@ -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" diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/config/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/config/package.scala new file mode 100644 index 000000000000..e26c4aadaf13 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/config/package.scala @@ -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 = + ConfigBuilder("spark.sql.defaultUrlStreamHandlerFactory.enabled") + .doc("When true, set FsUrlStreamHandlerFactory to support ADD JAR against HDFS locations") + .booleanConf + .createWithDefault(true) +}