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 4d2be13c4841..ec82c06fd323 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 @@ -28,7 +28,7 @@ import org.apache.hadoop.fs.FsUrlStreamHandlerFactory import org.apache.spark.{SparkConf, SparkContext, SparkException} import org.apache.spark.internal.Logging -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.execution.CacheManager import org.apache.spark.sql.execution.ui.{SQLAppStatusListener, SQLAppStatusStore, SQLTab} @@ -157,7 +157,15 @@ private[sql] class SharedState(val sparkContext: SparkContext) extends Logging { object SharedState extends Logging { try { - URL.setURLStreamHandlerFactory(new FsUrlStreamHandlerFactory()) + SparkSession.getActiveSession match { + case Some(spark) => + // scalastyle:off hadoopconfiguration + URL.setURLStreamHandlerFactory( + new FsUrlStreamHandlerFactory(spark.sparkContext.hadoopConfiguration)) + // scalastyle:on hadoopconfiguration + case _ => + URL.setURLStreamHandlerFactory(new FsUrlStreamHandlerFactory()) + } } catch { case e: Error => logWarning("URL.setURLStreamHandlerFactory failed to set FsUrlStreamHandlerFactory") 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 000000000000..e4ccbf824968 --- /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, SparkFunSuite} +import org.apache.spark.sql.test.SharedSparkSession + + +/** + * Tests for [[org.apache.spark.sql.internal.SharedState]]. + */ +class SharedStateSuite extends SparkFunSuite with 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' external catalog to init the code in Object SharedState + spark.sharedState.externalCatalog + 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:///") + } +}