Skip to content
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-32364][SQL] Use CaseInsensitiveMap for DataFrameReader/Writer options #29160

Closed
wants to merge 5 commits into from
Closed

[SPARK-32364][SQL] Use CaseInsensitiveMap for DataFrameReader/Writer options #29160

wants to merge 5 commits into from

Conversation

dongjoon-hyun
Copy link
Member

@dongjoon-hyun dongjoon-hyun commented Jul 20, 2020

What changes were proposed in this pull request?

When a user have multiple options like path, paTH, and PATH for the same key path, option/options is non-deterministic because extraOptions is HashMap. This PR aims to use CaseInsensitiveMap instead of HashMap to fix this bug fundamentally.

Why are the changes needed?

Like the following, DataFrame's option/options have been non-deterministic in terms of case-insensitivity because it stores the options at extraOptions which is using HashMap class.

spark.read
  .option("paTh", "1")
  .option("PATH", "2")
  .option("Path", "3")
  .option("patH", "4")
  .load("5")
...
org.apache.spark.sql.AnalysisException:
Path does not exist: file:/.../1;

Does this PR introduce any user-facing change?

Yes. However, this is a bug fix for the indeterministic cases.

How was this patch tested?

Pass the Jenkins or GitHub Action with newly added test cases.

@dongjoon-hyun
Copy link
Member Author

Could you review this please, @cloud-fan ?

@SparkQA

This comment has been minimized.

@SparkQA

This comment has been minimized.

@dongjoon-hyun dongjoon-hyun changed the title [SPARK-32364][SQL] path argument of DataFrame.load/save should override the existing options [SPARK-32364][SQL] Use CaseInsensitiveMap for DataFrameReader/Writer options Jul 22, 2020
@dongjoon-hyun
Copy link
Member Author

Hi, @cloud-fan and @HyukjinKwon .
This PR aims to fix this problem completely. Please review the approach.

@SparkQA

This comment has been minimized.

@dongjoon-hyun
Copy link
Member Author

I updated the PR according to your comment, @cloud-fan . It looks much better indeed. Thank you so much.

@SparkQA

This comment has been minimized.

Copy link
Contributor

@cloud-fan cloud-fan left a comment

Choose a reason for hiding this comment

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

LGTM except one comment

@SparkQA

This comment has been minimized.

@dongjoon-hyun
Copy link
Member Author

Thank you, @cloud-fan . The PR is updated to use Seq instead of Map.

@SparkQA

This comment has been minimized.

@dongjoon-hyun
Copy link
Member Author

Retest this please.

@dongjoon-hyun
Copy link
Member Author

Thank you, @HyukjinKwon .

@SparkQA

This comment has been minimized.

dongjoon-hyun added a commit that referenced this pull request Jul 22, 2020
…options

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

When a user have multiple options like `path`, `paTH`, and `PATH` for the same key `path`, `option/options` is non-deterministic because `extraOptions` is `HashMap`. This PR aims to use `CaseInsensitiveMap` instead of `HashMap` to fix this bug fundamentally.

### Why are the changes needed?

Like the following, DataFrame's `option/options` have been non-deterministic in terms of case-insensitivity because it stores the options at `extraOptions` which is using `HashMap` class.

```scala
spark.read
  .option("paTh", "1")
  .option("PATH", "2")
  .option("Path", "3")
  .option("patH", "4")
  .load("5")
...
org.apache.spark.sql.AnalysisException:
Path does not exist: file:/.../1;
```

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

Yes. However, this is a bug fix for the indeterministic cases.

### How was this patch tested?

Pass the Jenkins or GitHub Action with newly added test cases.

Closes #29160 from dongjoon-hyun/SPARK-32364.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
(cherry picked from commit cd16a10)
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
@dongjoon-hyun
Copy link
Member Author

Merged to master/3.0 since GitHub Action passed. Thank you all. I'll make a backporting PR to branch-2.4.

@SparkQA
Copy link

SparkQA commented Jul 22, 2020

Test build #126332 has finished for PR 29160 at commit 7ca5da6.

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

dongjoon-hyun added a commit that referenced this pull request Jul 23, 2020
…mentation

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

This is a follow-up of #29160. We already removed the indeterministicity. This PR aims the following for the existing code base.
1. Add an explicit document to `DataFrameReader/DataFrameWriter`.

2. Add `toMap` to `CaseInsensitiveMap` in order to return `originalMap: Map[String, T]` because it's more consistent with the existing `case-sensitive key names` behavior for the existing code pattern like `AppendData.byName(..., extraOptions.toMap)`. Previously, it was `HashMap.toMap`.

3. During (2), we need to change the following to keep the original logic using `CaseInsensitiveMap.++`.
```scala
- val params = extraOptions.toMap ++ connectionProperties.asScala.toMap
+ val params = extraOptions ++ connectionProperties.asScala
```

4. Additionally, use `.toMap` in the following because `dsOptions.asCaseSensitiveMap()` is used later.
```scala
- val options = sessionOptions ++ extraOptions
+ val options = sessionOptions.filterKeys(!extraOptions.contains(_)) ++ extraOptions.toMap
  val dsOptions = new CaseInsensitiveStringMap(options.asJava)
```

### Why are the changes needed?

`extraOptions.toMap` is used in several places (e.g. `DataFrameReader`) to hand over `Map[String, T]`. In this case, `CaseInsensitiveMap[T] private (val originalMap: Map[String, T])` had better return `originalMap`.

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

No.

### How was this patch tested?

Pass the Jenkins or GitHub Action with the existing tests and newly add test case at `JDBCSuite`.

Closes #29191 from dongjoon-hyun/SPARK-32364-3.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
dongjoon-hyun added a commit that referenced this pull request Jul 23, 2020
…mentation

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

This is a follow-up of #29160. We already removed the indeterministicity. This PR aims the following for the existing code base.
1. Add an explicit document to `DataFrameReader/DataFrameWriter`.

2. Add `toMap` to `CaseInsensitiveMap` in order to return `originalMap: Map[String, T]` because it's more consistent with the existing `case-sensitive key names` behavior for the existing code pattern like `AppendData.byName(..., extraOptions.toMap)`. Previously, it was `HashMap.toMap`.

3. During (2), we need to change the following to keep the original logic using `CaseInsensitiveMap.++`.
```scala
- val params = extraOptions.toMap ++ connectionProperties.asScala.toMap
+ val params = extraOptions ++ connectionProperties.asScala
```

4. Additionally, use `.toMap` in the following because `dsOptions.asCaseSensitiveMap()` is used later.
```scala
- val options = sessionOptions ++ extraOptions
+ val options = sessionOptions.filterKeys(!extraOptions.contains(_)) ++ extraOptions.toMap
  val dsOptions = new CaseInsensitiveStringMap(options.asJava)
```

### Why are the changes needed?

`extraOptions.toMap` is used in several places (e.g. `DataFrameReader`) to hand over `Map[String, T]`. In this case, `CaseInsensitiveMap[T] private (val originalMap: Map[String, T])` had better return `originalMap`.

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

No.

### How was this patch tested?

Pass the Jenkins or GitHub Action with the existing tests and newly add test case at `JDBCSuite`.

Closes #29191 from dongjoon-hyun/SPARK-32364-3.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
(cherry picked from commit aed8dba)
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
dongjoon-hyun added a commit that referenced this pull request Jul 23, 2020
…iter options

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

This PR is a backport of SPARK-32364 (#29160, #29191).

When a user have multiple options like `path`, `paTH`, and `PATH` for the same key `path`, `option/options` is indeterministic because `extraOptions` is `HashMap`. This PR aims to use `CaseInsensitiveMap` instead of `HashMap` to fix this bug fundamentally.

Like the following, DataFrame's `option/options` have been non-deterministic in terms of case-insensitivity because it stores the options at `extraOptions` which is using `HashMap` class.

```scala
spark.read
  .option("paTh", "1")
  .option("PATH", "2")
  .option("Path", "3")
  .option("patH", "4")
  .load("5")
...
org.apache.spark.sql.AnalysisException:
Path does not exist: file:/.../1;
```

Also, this PR adds the following.

1. Add an explicit document to `DataFrameReader/DataFrameWriter`.

2. Add `toMap` to `CaseInsensitiveMap` in order to return `originalMap: Map[String, T]` because it's more consistent with the existing `case-sensitive key names` behavior for the existing code pattern like `AppendData.byName(..., extraOptions.toMap)`. Previously, it was `HashMap.toMap`.

3. During (2), we need to change the following to keep the original logic using `CaseInsensitiveMap.++`.
```scala
- val params = extraOptions.toMap ++ connectionProperties.asScala.toMap
+ val params = extraOptions ++ connectionProperties.asScala
```

4. Additionally, use `.toMap` in the following because `dsOptions.asCaseSensitiveMap()` is used later.
```scala
- val options = sessionOptions ++ extraOptions
+ val options = sessionOptions.filterKeys(!extraOptions.contains(_)) ++ extraOptions.toMap
  val dsOptions = new CaseInsensitiveStringMap(options.asJava)
```

`extraOptions.toMap` is used in several places (e.g. `DataFrameReader`) to hand over `Map[String, T]`. In this case, `CaseInsensitiveMap[T] private (val originalMap: Map[String, T])` had better return `originalMap`.

### Why are the changes needed?

This will fix indeterministic behavior.

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

Yes.

### How was this patch tested?

Pass the Jenkins with the existing tests and newly add test cases.

Closes #29209 from dongjoon-hyun/SPARK-32364-2.4.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
srowen pushed a commit that referenced this pull request Sep 2, 2020
### What changes were proposed in this pull request?

This is a follow-up of #29160. This allows Spark SQL project to compile for Scala 2.13.

### Why are the changes needed?

It's needed for #28545

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

No

### How was this patch tested?

I compiled with Scala 2.13. It fails in `Spark REPL` project, which will be fixed by #28545

Closes #29584 from karolchmist/SPARK-32364-scala-2.13.

Authored-by: Karol Chmist <info+github@chmist.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
@cloud-fan
Copy link
Contributor

@dongjoon-hyun shall we fix the issue in DataStreamReader/Writer as well? cc @HeartSaVioR

@dongjoon-hyun
Copy link
Member Author

Sure, I'll make a PR for that tomorrow, @cloud-fan .

@dongjoon-hyun
Copy link
Member Author

BTW, @cloud-fan . Is @HeartSaVioR working on that? I'm wondering the reason why you ping him in that task.

@HeartSaVioR
Copy link
Contributor

I guess he pinged me to ask for reviewing on your next PR as it would be SS-related change, probably.

@cloud-fan
Copy link
Contributor

Yea, I was wanting @HeartSaVioR to review your PR :)

@dongjoon-hyun
Copy link
Member Author

Got it. If the PR is ready, I'll ping both of you. :)

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
Projects
None yet
Development

Successfully merging this pull request may close these issues.

5 participants