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

Conversation

jiangzho
Copy link
Contributor

@jiangzho jiangzho commented Nov 14, 2019

What changes were proposed in this pull request?

Add a property spark.fsUrlStreamHandlerFactory.enabled to allow users turn off the default registration of org.apache.hadoop.fs.FsUrlStreamHandlerFactory

Why are the changes needed?

This SPARK-25694 is a long-standing issue. Originally, [SPARK-12868][SQL] Allow adding jars from hdfs added this for better Hive support. However, this have a side-effect when the users use Apache Spark without -Phive. This causes exceptions when the users tries to use another custom factories or 3rd party library (trying to set this). This configuration will unblock those non-hive users.

Does this PR introduce any user-facing change?

Yes. This provides a new user-configurable property.
By default, the behavior is unchanged.

How was this patch tested?

Manual testing.

BEFORE

$ build/sbt package
$ bin/spark-shell
scala> sql("show tables").show
+--------+---------+-----------+
|database|tableName|isTemporary|
+--------+---------+-----------+
+--------+---------+-----------+


scala> java.net.URL.setURLStreamHandlerFactory(new org.apache.hadoop.fs.FsUrlStreamHandlerFactory())
java.lang.Error: factory already defined
  at java.net.URL.setURLStreamHandlerFactory(URL.java:1134)
  ... 47 elided

AFTER

$ build/sbt package
$ bin/spark-shell --conf spark.sql.defaultUrlStreamHandlerFactory.enabled=false
scala> sql("show tables").show
+--------+---------+-----------+
|database|tableName|isTemporary|
+--------+---------+-----------+
+--------+---------+-----------+

scala> java.net.URL.setURLStreamHandlerFactory(new org.apache.hadoop.fs.FsUrlStreamHandlerFactory())

@dongjoon-hyun
Copy link
Member

ok to test

@dongjoon-hyun
Copy link
Member

Thank you for making your first PR, @jiangzho .

@dongjoon-hyun dongjoon-hyun changed the title [SPARK-25694][SQL]URL.setURLStreamHandlerFactory causing incompatible HttpURLConnection issue [SPARK-25694][SQL] Add a config for URL.setURLStreamHandlerFactory Nov 14, 2019
@SparkQA

This comment has been minimized.

@dongjoon-hyun
Copy link
Member

Retest this please.

@SparkQA

This comment has been minimized.

@dongjoon-hyun
Copy link
Member

Retest this please

@SparkQA

This comment has been minimized.

@dbtsai
Copy link
Member

dbtsai commented Nov 17, 2019

Add @steveloughran for more feedback.

@dongjoon-hyun
Copy link
Member

Hi, @srowen , @dbtsai .
This PR is rebased and updated according to the review comments.
Now we have a renamed spark.sql.defaultUrlStreamHandlerFactory.enabled config.
Could you review this once more?

@SparkQA

This comment has been minimized.

@SparkQA
Copy link

SparkQA commented Nov 18, 2019

Test build #113970 has finished for PR 26530 at commit 4367b15.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Nov 18, 2019

Test build #113971 has finished for PR 26530 at commit 558062b.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@dongjoon-hyun
Copy link
Member

Now, AtomicBoolean is reverted~

Copy link
Member

@dbtsai dbtsai left a comment

Choose a reason for hiding this comment

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

LGTM.

Copy link
Member

@srowen srowen left a comment

Choose a reason for hiding this comment

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

OK by me. I think the checking outside the synchronized block isn't that important here as there won't be contention, but this is OK.

@dongjoon-hyun
Copy link
Member

Thank you for review and approval, @dbtsai and @srowen . I really want to put this into Apache Spark master. :)

@SparkQA
Copy link

SparkQA commented Nov 18, 2019

Test build #113974 has finished for PR 26530 at commit d9e0ec7.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Nov 18, 2019

Test build #113976 has finished for PR 26530 at commit c4ec2d5.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@dbtsai dbtsai closed this in ee3bd6d Nov 18, 2019
@dbtsai
Copy link
Member

dbtsai commented Nov 18, 2019

Merged into master. Thanks all!

@SparkQA
Copy link

SparkQA commented Nov 18, 2019

Test build #113979 has finished for PR 26530 at commit cd9ed48.

  • This patch fails PySpark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.


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

@dongjoon-hyun
Copy link
Member

Thank you for merging, @dbtsai . Thanks for the follow-up, @HyukjinKwon .

@jiangzho . You are added to the Apache Spark Contributor group.
https://issues.apache.org/jira/browse/SPARK-25694 is assigned to you.
Thank you for making this PR.

@SparkQA
Copy link

SparkQA commented Nov 18, 2019

Test build #113985 has finished for PR 26530 at commit 755c9c0.

  • This patch fails due to an unknown error code, -9.
  • This patch merges cleanly.
  • This patch adds no public classes.

HyukjinKwon added a commit that referenced this pull request Nov 19, 2019
…Factory.enabled' into StaticSQLConf.scala

### What changes were proposed in this pull request?

This PR is a followup of #26530 and proposes to move the configuration `spark.sql.defaultUrlStreamHandlerFactory.enabled` to `StaticSQLConf.scala` for consistency.

### Why are the changes needed?

To put the similar configurations together and for readability.

### Does this PR introduce any user-facing change?

No.

### How was this patch tested?

Manually tested as described in #26530.

Closes #26570 from HyukjinKwon/SPARK-25694.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
@steveloughran
Copy link
Contributor

Sorry, missed this. I don't think the Hadoop HTTP FS should be registering its classes as the handlers of HTTP/HTTPS requests at all, as it breaks all the code which expects to cast it to well known types.

HADOOP-14598 fixes it for the https connections -really people should be using a version of the hadoop libraries with spark. These are old, fixed, bugs you are trying to work around here.

@dongjoon-hyun
Copy link
Member

Thank you, @steveloughran . Sure.
Since HADOOP-14598 is Hadoop 2.9.0+, we will remove the legacy setURLStreamHanderFactory after we upgrade from Hadoop 2.7.x to Hadoop 2.9+ someday.

@dongjoon-hyun
Copy link
Member

Hi, @jiangzho , @dbtsai , @holdenk and all.
There exists an on-going bug report and fix for setURLStreamHandlerFactory.

This PR is also helpful and safe. I'll backport this to branch-2.4.

dongjoon-hyun added a commit that referenced this pull request May 18, 2020
Add a property `spark.fsUrlStreamHandlerFactory.enabled` to allow users turn off the default registration of `org.apache.hadoop.fs.FsUrlStreamHandlerFactory`

This [SPARK-25694](https://issues.apache.org/jira/browse/SPARK-25694) is a long-standing issue. Originally, [[SPARK-12868][SQL] Allow adding jars from hdfs](#17342 ) added this for better Hive support. However, this have a side-effect when the users use Apache Spark without `-Phive`. This causes exceptions when the users tries to use another custom factories or 3rd party library (trying to set this). This configuration will unblock those non-hive users.

Yes. This provides a new user-configurable property.
By default, the behavior is unchanged.

Manual testing.

**BEFORE**
```
$ build/sbt package
$ bin/spark-shell
scala> sql("show tables").show
+--------+---------+-----------+
|database|tableName|isTemporary|
+--------+---------+-----------+
+--------+---------+-----------+

scala> java.net.URL.setURLStreamHandlerFactory(new org.apache.hadoop.fs.FsUrlStreamHandlerFactory())
java.lang.Error: factory already defined
  at java.net.URL.setURLStreamHandlerFactory(URL.java:1134)
  ... 47 elided
```

**AFTER**
```
$ build/sbt package
$ bin/spark-shell --conf spark.sql.defaultUrlStreamHandlerFactory.enabled=false
scala> sql("show tables").show
+--------+---------+-----------+
|database|tableName|isTemporary|
+--------+---------+-----------+
+--------+---------+-----------+

scala> java.net.URL.setURLStreamHandlerFactory(new org.apache.hadoop.fs.FsUrlStreamHandlerFactory())
```

Closes #26530 from jiangzho/master.

Lead-authored-by: Zhou Jiang <zhou_jiang@apple.com>
Co-authored-by: Dongjoon Hyun <dhyun@apple.com>
Co-authored-by: zhou-jiang <zhou_jiang@apple.com>
Signed-off-by: DB Tsai <d_tsai@apple.com>
(cherry picked from commit ee3bd6d)
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
dongjoon-hyun pushed a commit that referenced this pull request May 18, 2020
…Factory.enabled' into StaticSQLConf.scala

This PR is a followup of #26530 and proposes to move the configuration `spark.sql.defaultUrlStreamHandlerFactory.enabled` to `StaticSQLConf.scala` for consistency.

To put the similar configurations together and for readability.

No.

Manually tested as described in #26530.

Closes #26570 from HyukjinKwon/SPARK-25694.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
(cherry picked from commit 8469614)
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
otterc pushed a commit to linkedin/spark that referenced this pull request Mar 22, 2023
Add a property `spark.fsUrlStreamHandlerFactory.enabled` to allow users turn off the default registration of `org.apache.hadoop.fs.FsUrlStreamHandlerFactory`

This [SPARK-25694](https://issues.apache.org/jira/browse/SPARK-25694) is a long-standing issue. Originally, [[SPARK-12868][SQL] Allow adding jars from hdfs](apache#17342 ) added this for better Hive support. However, this have a side-effect when the users use Apache Spark without `-Phive`. This causes exceptions when the users tries to use another custom factories or 3rd party library (trying to set this). This configuration will unblock those non-hive users.

Yes. This provides a new user-configurable property.
By default, the behavior is unchanged.

Manual testing.

**BEFORE**
```
$ build/sbt package
$ bin/spark-shell
scala> sql("show tables").show
+--------+---------+-----------+
|database|tableName|isTemporary|
+--------+---------+-----------+
+--------+---------+-----------+

scala> java.net.URL.setURLStreamHandlerFactory(new org.apache.hadoop.fs.FsUrlStreamHandlerFactory())
java.lang.Error: factory already defined
  at java.net.URL.setURLStreamHandlerFactory(URL.java:1134)
  ... 47 elided
```

**AFTER**
```
$ build/sbt package
$ bin/spark-shell --conf spark.sql.defaultUrlStreamHandlerFactory.enabled=false
scala> sql("show tables").show
+--------+---------+-----------+
|database|tableName|isTemporary|
+--------+---------+-----------+
+--------+---------+-----------+

scala> java.net.URL.setURLStreamHandlerFactory(new org.apache.hadoop.fs.FsUrlStreamHandlerFactory())
```

Closes apache#26530 from jiangzho/master.

Lead-authored-by: Zhou Jiang <zhou_jiang@apple.com>
Co-authored-by: Dongjoon Hyun <dhyun@apple.com>
Co-authored-by: zhou-jiang <zhou_jiang@apple.com>
Signed-off-by: DB Tsai <d_tsai@apple.com>
(cherry picked from commit ee3bd6d)

[SPARK-25694][SQL][FOLLOW-UP] Move 'spark.sql.defaultUrlStreamHandlerFactory.enabled' into StaticSQLConf.scala

This PR is a followup of apache#26530 and proposes to move the configuration `spark.sql.defaultUrlStreamHandlerFactory.enabled` to `StaticSQLConf.scala` for consistency.

To put the similar configurations together and for readability.

No.

Manually tested as described in apache#26530.

Closes apache#26570 from HyukjinKwon/SPARK-25694.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
(cherry picked from commit 8469614)

RB=1906279
BUG=LIHADOOP-50577
G=superfriends-reviewers
R=zolin,yezhou,chsingh,mshen,fli,latang
A=chsingh
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

7 participants