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 47119ab903da7..ce4385d88f1e9 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 @@ -53,7 +53,7 @@ private[sql] class SharedState( initialConfigs: scala.collection.Map[String, String]) extends Logging { - SharedState.setFsUrlStreamHandlerFactory(sparkContext.conf) + SharedState.setFsUrlStreamHandlerFactory(sparkContext.conf, sparkContext.hadoopConfiguration) private val (conf, hadoopConf) = { // Load hive-site.xml into hadoopConf and determine the warehouse path which will be set into @@ -174,13 +174,13 @@ private[sql] class SharedState( object SharedState extends Logging { @volatile private var fsUrlStreamHandlerFactoryInitialized = false - private def setFsUrlStreamHandlerFactory(conf: SparkConf): Unit = { + private def setFsUrlStreamHandlerFactory(conf: SparkConf, hadoopConf: Configuration): Unit = { if (!fsUrlStreamHandlerFactoryInitialized && conf.get(DEFAULT_URL_STREAM_HANDLER_FACTORY_ENABLED)) { synchronized { if (!fsUrlStreamHandlerFactoryInitialized) { try { - URL.setURLStreamHandlerFactory(new FsUrlStreamHandlerFactory()) + URL.setURLStreamHandlerFactory(new FsUrlStreamHandlerFactory(hadoopConf)) fsUrlStreamHandlerFactoryInitialized = true } catch { case NonFatal(_) => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/SharedStateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/SharedStateSuite.scala new file mode 100644 index 0000000000000..81bf15342423c --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/SharedStateSuite.scala @@ -0,0 +1,55 @@ +/* + * 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 java.net.URL + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.FsUrlStreamHandlerFactory + +import org.apache.spark.SparkConf +import org.apache.spark.sql.test.SharedSparkSession + + +/** + * Tests for [[org.apache.spark.sql.internal.SharedState]]. + */ +class SharedStateSuite extends SharedSparkSession { + + override protected def sparkConf: SparkConf = { + super.sparkConf + .set("spark.hadoop.fs.defaultFS", "file:///") + } + + test("SPARK-31692: Url handler factory should have the hadoop configs from Spark conf") { + // Accessing shared state to init the object since it is `lazy val` + spark.sharedState + val field = classOf[URL].getDeclaredField("factory") + field.setAccessible(true) + val value = field.get(null) + assert(value.isInstanceOf[FsUrlStreamHandlerFactory]) + val streamFactory = value.asInstanceOf[FsUrlStreamHandlerFactory] + + val confField = classOf[FsUrlStreamHandlerFactory].getDeclaredField("conf") + confField.setAccessible(true) + val conf = confField.get(streamFactory) + + assert(conf.isInstanceOf[Configuration]) + assert(conf.asInstanceOf[Configuration].get("fs.defaultFS") == "file:///") + } +}