From f7208a4589e61ee1b48bd8e7d1086e50e6c26266 Mon Sep 17 00:00:00 2001 From: Karuppayya Rajendran Date: Tue, 12 May 2020 15:18:14 -0700 Subject: [PATCH 1/6] Pass hadoop conf to stream handlerfactory --- .../spark/sql/internal/SharedState.scala | 8 +-- .../spark/sql/internal/SharedStateSuite.scala | 57 +++++++++++++++++++ 2 files changed, 61 insertions(+), 4 deletions(-) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/internal/SharedStateSuite.scala 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 47119ab903da..ea20d61e756c 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) 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(ctx: SparkContext): Unit = { if (!fsUrlStreamHandlerFactoryInitialized && - conf.get(DEFAULT_URL_STREAM_HANDLER_FACTORY_ENABLED)) { + ctx.conf.get(DEFAULT_URL_STREAM_HANDLER_FACTORY_ENABLED)) { synchronized { if (!fsUrlStreamHandlerFactoryInitialized) { try { - URL.setURLStreamHandlerFactory(new FsUrlStreamHandlerFactory()) + URL.setURLStreamHandlerFactory(new FsUrlStreamHandlerFactory(ctx.hadoopConfiguration)) 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 000000000000..48b9ba51622c --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/SharedStateSuite.scala @@ -0,0 +1,57 @@ +/* + * 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 the user-facing [[org.apache.spark.sql.catalog.Catalog]]. + */ +class SharedStateSuite extends SharedSparkSession { + + override protected def sparkConf: SparkConf = { + super.sparkConf + .set("spark.hadoop.test", "value") + } + + test("Url handler factory should have the hadoop configs from Spark conf") { + // Accesing shared state to init the object sicne 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("test") == "value") + + + } +} \ No newline at end of file From b34802a3e3ce2a77abe104e29e709afc08be4621 Mon Sep 17 00:00:00 2001 From: Karuppayya Rajendran Date: Tue, 12 May 2020 15:23:17 -0700 Subject: [PATCH 2/6] Fixing typos --- .../scala/org/apache/spark/sql/internal/SharedStateSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 index 48b9ba51622c..980e4f562363 100644 --- 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 @@ -37,7 +37,7 @@ class SharedStateSuite extends SharedSparkSession { } test("Url handler factory should have the hadoop configs from Spark conf") { - // Accesing shared state to init the object sicne it is `lazy val` + // Accessing shared state to init the object since it is `lazy val` spark.sharedState val field = classOf[URL].getDeclaredField("factory") field.setAccessible(true) From 485fe6a403bbad786b8744314a8e2e2b51d182af Mon Sep 17 00:00:00 2001 From: Karuppayya Rajendran Date: Tue, 12 May 2020 15:51:20 -0700 Subject: [PATCH 3/6] Address review comments --- .../org/apache/spark/sql/internal/SharedStateSuite.scala | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) 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 index 980e4f562363..14b172ab3cb9 100644 --- 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 @@ -27,7 +27,7 @@ import org.apache.spark.sql.test.SharedSparkSession /** - * Tests for the user-facing [[org.apache.spark.sql.catalog.Catalog]]. + * Tests for [[org.apache.spark.sql.internal.SharedState]]. */ class SharedStateSuite extends SharedSparkSession { @@ -51,7 +51,5 @@ class SharedStateSuite extends SharedSparkSession { assert(conf.isInstanceOf[Configuration]) assert(conf.asInstanceOf[Configuration].get("test") == "value") - - } } \ No newline at end of file From 2798e1a2bc91ee473d4a12a2d2f496254526c31d Mon Sep 17 00:00:00 2001 From: Karuppayya Rajendran Date: Tue, 12 May 2020 16:03:18 -0700 Subject: [PATCH 4/6] Address review comments --- .../scala/org/apache/spark/sql/internal/SharedState.scala | 8 ++++---- .../org/apache/spark/sql/internal/SharedStateSuite.scala | 4 ++-- 2 files changed, 6 insertions(+), 6 deletions(-) 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 ea20d61e756c..ce4385d88f1e 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) + 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(ctx: SparkContext): Unit = { + private def setFsUrlStreamHandlerFactory(conf: SparkConf, hadoopConf: Configuration): Unit = { if (!fsUrlStreamHandlerFactoryInitialized && - ctx.conf.get(DEFAULT_URL_STREAM_HANDLER_FACTORY_ENABLED)) { + conf.get(DEFAULT_URL_STREAM_HANDLER_FACTORY_ENABLED)) { synchronized { if (!fsUrlStreamHandlerFactoryInitialized) { try { - URL.setURLStreamHandlerFactory(new FsUrlStreamHandlerFactory(ctx.hadoopConfiguration)) + 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 index 14b172ab3cb9..c52e925cfaec 100644 --- 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 @@ -33,7 +33,7 @@ class SharedStateSuite extends SharedSparkSession { override protected def sparkConf: SparkConf = { super.sparkConf - .set("spark.hadoop.test", "value") + .set("spark.hadoop.fs.defaultFS", "file:///") } test("Url handler factory should have the hadoop configs from Spark conf") { @@ -50,6 +50,6 @@ class SharedStateSuite extends SharedSparkSession { val conf = confField.get(streamFactory) assert(conf.isInstanceOf[Configuration]) - assert(conf.asInstanceOf[Configuration].get("test") == "value") + assert(conf.asInstanceOf[Configuration].get("fs.defaultFS") == "file:///") } } \ No newline at end of file From 86f121bb9a78d9c18540d73841585f9c94e08de0 Mon Sep 17 00:00:00 2001 From: Karuppayya Rajendran Date: Tue, 12 May 2020 21:33:12 -0700 Subject: [PATCH 5/6] Fix failure: Linter checks --- .../scala/org/apache/spark/sql/internal/SharedStateSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 index c52e925cfaec..7b67425acd4b 100644 --- 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 @@ -52,4 +52,4 @@ class SharedStateSuite extends SharedSparkSession { assert(conf.isInstanceOf[Configuration]) assert(conf.asInstanceOf[Configuration].get("fs.defaultFS") == "file:///") } -} \ No newline at end of file +} From 2e002540517b67efd245ef386c75eb4dc4e84e75 Mon Sep 17 00:00:00 2001 From: Karuppayya Rajendran Date: Wed, 13 May 2020 12:03:27 -0700 Subject: [PATCH 6/6] Handle review comments: Add JIRA id to test name --- .../scala/org/apache/spark/sql/internal/SharedStateSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 index 7b67425acd4b..81bf15342423 100644 --- 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 @@ -36,7 +36,7 @@ class SharedStateSuite extends SharedSparkSession { .set("spark.hadoop.fs.defaultFS", "file:///") } - test("Url handler factory should have the hadoop configs from Spark conf") { + 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")