-
Notifications
You must be signed in to change notification settings - Fork 28.3k
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
[SPARK-32832][SS] Use CaseInsensitiveMap for DataStreamReader/Writer options #29702
Conversation
cc @cloud-fan and @HeartSaVioR |
@@ -535,5 +536,5 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo | |||
|
|||
private var userSpecifiedSchema: Option[StructType] = None | |||
|
|||
private var extraOptions = new scala.collection.mutable.HashMap[String, String] | |||
private var extraOptions = CaseInsensitiveMap[String](Map.empty) |
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.
DataStreamReader
support reading DS v2, shall we get the original map at that place?
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.
Thanks. It's updated now.
Test build #128475 has finished for PR 29702 at commit
|
Test build #128477 has finished for PR 29702 at commit
|
Thank you again, @cloud-fan . Merged to master/3.0. |
…options This PR aims to fix indeterministic behavior on DataStreamReader/Writer options like the following. ```scala scala> spark.readStream.format("parquet").option("paTh", "1").option("PATH", "2").option("Path", "3").option("patH", "4").option("path", "5").load() org.apache.spark.sql.AnalysisException: Path does not exist: 1; ``` This will make the behavior deterministic. Yes, but the previous behavior is indeterministic. Pass the newly test cases. Closes #29702 from dongjoon-hyun/SPARK-32832. Authored-by: Dongjoon Hyun <dongjoon@apache.org> Signed-off-by: Dongjoon Hyun <dongjoon@apache.org> (cherry picked from commit 2f85f95) Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
Test build #128483 has finished for PR 29702 at commit
|
…iter options ### What changes were proposed in this pull request? This is a backport of #29702 . This PR aims to fix indeterministic behavior on DataStreamReader/Writer options like the following. ```scala scala> spark.readStream.format("parquet").option("paTh", "1").option("PATH", "2").option("Path", "3").option("patH", "4").option("path", "5").load() org.apache.spark.sql.AnalysisException: Path does not exist: 1; ``` ### Why are the changes needed? This will make the behavior deterministic. ### Does this PR introduce _any_ user-facing change? Yes, but the previous behavior is indeterministic. ### How was this patch tested? Pass the newly test cases. Closes #29707 from dongjoon-hyun/SPARK-32832-2. Authored-by: Dongjoon Hyun <dongjoon@apache.org> Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
…options This PR aims to fix indeterministic behavior on DataStreamReader/Writer options like the following. ```scala scala> spark.readStream.format("parquet").option("paTh", "1").option("PATH", "2").option("Path", "3").option("patH", "4").option("path", "5").load() org.apache.spark.sql.AnalysisException: Path does not exist: 1; ``` This will make the behavior deterministic. Yes, but the previous behavior is indeterministic. Pass the newly test cases. Closes apache#29702 from dongjoon-hyun/SPARK-32832. Authored-by: Dongjoon Hyun <dongjoon@apache.org> Signed-off-by: Dongjoon Hyun <dongjoon@apache.org> (cherry picked from commit 2f85f95) Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
What changes were proposed in this pull request?
This PR aims to fix indeterministic behavior on DataStreamReader/Writer options like the following.
Why are the changes needed?
This will make the behavior deterministic.
Does this PR introduce any user-facing change?
Yes, but the previous behavior is indeterministic.
How was this patch tested?
Pass the newly test cases.