-
Notifications
You must be signed in to change notification settings - Fork 28.6k
[SPARK-31692][SQL] Pass hadoop confs specifed via Spark confs to URLStreamHandlerfactory #28516
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
Conversation
sql/core/src/test/scala/org/apache/spark/sql/internal/SharedStateSuite.scala
Outdated
Show resolved
Hide resolved
sql/core/src/test/scala/org/apache/spark/sql/internal/SharedStateSuite.scala
Outdated
Show resolved
Hide resolved
sql/core/src/test/scala/org/apache/spark/sql/internal/SharedStateSuite.scala
Outdated
Show resolved
Hide resolved
sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala
Outdated
Show resolved
Hide resolved
BTW, thank you for your first contribution, @karuppayya . |
sql/core/src/test/scala/org/apache/spark/sql/internal/SharedStateSuite.scala
Outdated
Show resolved
Hide resolved
ok to test |
Test build #122584 has finished for PR 28516 at commit
|
Thank you for updating, but the following is insufficient.
For example, people can try like the following in Apache Spark 3.0.0 RC1. Could you elaborate a little more in a reproducible way which the other people can follow?
|
Test build #122598 has finished for PR 28516 at commit
|
I have come up with an repro with Local filesystem, which will be easier for testing
After the change
The type of FileSystem object created(you can check the last statement in the above snippets) in the above two cases are different, which should not have been the case |
Thanks. Please put the description in the PR description. It will be perfect. |
I updated the PR description with yours, @karuppayya . |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
+1, LGTM. Thank you, @karuppayya and @HyukjinKwon .
Merged to master/3.0.
…treamHandlerfactory ### What changes were proposed in this pull request? Pass hadoop confs specifed via Spark confs to URLStreamHandlerfactory ### Why are the changes needed? **BEFORE** ``` ➜ spark git:(SPARK-31692) ✗ ./bin/spark-shell --conf spark.hadoop.fs.file.impl=org.apache.hadoop.fs.RawLocalFileSystem scala> spark.sharedState res0: org.apache.spark.sql.internal.SharedState = org.apache.spark.sql.internal.SharedState5793cd84 scala> new java.net.URL("file:///tmp/1.txt").openConnection.getInputStream res1: java.io.InputStream = org.apache.hadoop.fs.ChecksumFileSystem$FSDataBoundedInputStream22846025 scala> import org.apache.hadoop.fs._ import org.apache.hadoop.fs._ scala> FileSystem.get(new Path("file:///tmp/1.txt").toUri, spark.sparkContext.hadoopConfiguration) res2: org.apache.hadoop.fs.FileSystem = org.apache.hadoop.fs.LocalFileSystem5a930c03 ``` **AFTER** ``` ➜ spark git:(SPARK-31692) ✗ ./bin/spark-shell --conf spark.hadoop.fs.file.impl=org.apache.hadoop.fs.RawLocalFileSystem scala> spark.sharedState res0: org.apache.spark.sql.internal.SharedState = org.apache.spark.sql.internal.SharedState5c24a636 scala> new java.net.URL("file:///tmp/1.txt").openConnection.getInputStream res1: java.io.InputStream = org.apache.hadoop.fs.FSDataInputStream2ba8f528 scala> import org.apache.hadoop.fs._ import org.apache.hadoop.fs._ scala> FileSystem.get(new Path("file:///tmp/1.txt").toUri, spark.sparkContext.hadoopConfiguration) res2: org.apache.hadoop.fs.FileSystem = LocalFS scala> FileSystem.get(new Path("file:///tmp/1.txt").toUri, spark.sparkContext.hadoopConfiguration).getClass res3: Class[_ <: org.apache.hadoop.fs.FileSystem] = class org.apache.hadoop.fs.RawLocalFileSystem ``` The type of FileSystem object created(you can check the last statement in the above snippets) in the above two cases are different, which should not have been the case ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Tested locally. Added Unit test Closes #28516 from karuppayya/SPARK-31692. Authored-by: Karuppayya Rajendran <karuppayya1990@gmail.com> Signed-off-by: Dongjoon Hyun <dongjoon@apache.org> (cherry picked from commit 7260146) Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
Thanks @dongjoon-hyun for updating the PR description |
BTW, there is a conflict on branch-2.4. |
I added you to the Apache Spark contributor group and assigned you SPARK-31692. Thank you so much again. |
Thanks @dongjoon-hyun , I will create a PR against 2.4 as well |
…treamHandlerfactory Pass hadoop confs specifed via Spark confs to URLStreamHandlerfactory **BEFORE** ``` ➜ spark git:(SPARK-31692) ✗ ./bin/spark-shell --conf spark.hadoop.fs.file.impl=org.apache.hadoop.fs.RawLocalFileSystem scala> spark.sharedState res0: org.apache.spark.sql.internal.SharedState = org.apache.spark.sql.internal.SharedState5793cd84 scala> new java.net.URL("file:///tmp/1.txt").openConnection.getInputStream res1: java.io.InputStream = org.apache.hadoop.fs.ChecksumFileSystem$FSDataBoundedInputStream22846025 scala> import org.apache.hadoop.fs._ import org.apache.hadoop.fs._ scala> FileSystem.get(new Path("file:///tmp/1.txt").toUri, spark.sparkContext.hadoopConfiguration) res2: org.apache.hadoop.fs.FileSystem = org.apache.hadoop.fs.LocalFileSystem5a930c03 ``` **AFTER** ``` ➜ spark git:(SPARK-31692) ✗ ./bin/spark-shell --conf spark.hadoop.fs.file.impl=org.apache.hadoop.fs.RawLocalFileSystem scala> spark.sharedState res0: org.apache.spark.sql.internal.SharedState = org.apache.spark.sql.internal.SharedState5c24a636 scala> new java.net.URL("file:///tmp/1.txt").openConnection.getInputStream res1: java.io.InputStream = org.apache.hadoop.fs.FSDataInputStream2ba8f528 scala> import org.apache.hadoop.fs._ import org.apache.hadoop.fs._ scala> FileSystem.get(new Path("file:///tmp/1.txt").toUri, spark.sparkContext.hadoopConfiguration) res2: org.apache.hadoop.fs.FileSystem = LocalFS scala> FileSystem.get(new Path("file:///tmp/1.txt").toUri, spark.sparkContext.hadoopConfiguration).getClass res3: Class[_ <: org.apache.hadoop.fs.FileSystem] = class org.apache.hadoop.fs.RawLocalFileSystem ``` The type of FileSystem object created(you can check the last statement in the above snippets) in the above two cases are different, which should not have been the case No Tested locally. Added Unit test Closes apache#28516 from karuppayya/SPARK-31692. Authored-by: Karuppayya Rajendran <karuppayya1990@gmail.com> Signed-off-by: Dongjoon Hyun <dongjoon@apache.org> (cherry picked from commit 7260146)
…treamHandlerfactory Pass hadoop confs specifed via Spark confs to URLStreamHandlerfactory **BEFORE** ``` ➜ spark git:(SPARK-31692) ✗ ./bin/spark-shell --conf spark.hadoop.fs.file.impl=org.apache.hadoop.fs.RawLocalFileSystem scala> spark.sharedState res0: org.apache.spark.sql.internal.SharedState = org.apache.spark.sql.internal.SharedState5793cd84 scala> new java.net.URL("file:///tmp/1.txt").openConnection.getInputStream res1: java.io.InputStream = org.apache.hadoop.fs.ChecksumFileSystem$FSDataBoundedInputStream22846025 scala> import org.apache.hadoop.fs._ import org.apache.hadoop.fs._ scala> FileSystem.get(new Path("file:///tmp/1.txt").toUri, spark.sparkContext.hadoopConfiguration) res2: org.apache.hadoop.fs.FileSystem = org.apache.hadoop.fs.LocalFileSystem5a930c03 ``` **AFTER** ``` ➜ spark git:(SPARK-31692) ✗ ./bin/spark-shell --conf spark.hadoop.fs.file.impl=org.apache.hadoop.fs.RawLocalFileSystem scala> spark.sharedState res0: org.apache.spark.sql.internal.SharedState = org.apache.spark.sql.internal.SharedState5c24a636 scala> new java.net.URL("file:///tmp/1.txt").openConnection.getInputStream res1: java.io.InputStream = org.apache.hadoop.fs.FSDataInputStream2ba8f528 scala> import org.apache.hadoop.fs._ import org.apache.hadoop.fs._ scala> FileSystem.get(new Path("file:///tmp/1.txt").toUri, spark.sparkContext.hadoopConfiguration) res2: org.apache.hadoop.fs.FileSystem = LocalFS scala> FileSystem.get(new Path("file:///tmp/1.txt").toUri, spark.sparkContext.hadoopConfiguration).getClass res3: Class[_ <: org.apache.hadoop.fs.FileSystem] = class org.apache.hadoop.fs.RawLocalFileSystem ``` The type of FileSystem object created(you can check the last statement in the above snippets) in the above two cases are different, which should not have been the case No Tested locally. Added Unit test Closes #28516 from karuppayya/SPARK-31692. Authored-by: Karuppayya Rajendran <karuppayya1990@gmail.com> Signed-off-by: Dongjoon Hyun <dongjoon@apache.org> (cherry picked from commit 7260146) Signed-off-by: Dongjoon Hyun <dongjoon@apache.org> (cherry picked from commit d639a12) Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
What changes were proposed in this pull request?
Pass hadoop confs specifed via Spark confs to URLStreamHandlerfactory
Why are the changes needed?
BEFORE
AFTER
The type of FileSystem object created(you can check the last statement in the above snippets) in the above two cases are different, which should not have been the case
Does this PR introduce any user-facing change?
No
How was this patch tested?
Tested locally.
Added Unit test