-
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-25241][SQL] Configurable empty values when reading/writing CSV files #22234
Conversation
Should the new option be taken into account there: spark/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVDataSource.scala Line 94 in b461acb
and here: spark/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchema.scala Line 82 in 5264164
|
python/pyspark/sql/readwriter.py
Outdated
maxCharsPerColumn=None, maxMalformedLogPerPartition=None, mode=None, | ||
columnNameOfCorruptRecord=None, multiLine=None, charToEscapeQuoteEscaping=None, | ||
samplingRatio=None, enforceSchema=None): | ||
ignoreTrailingWhiteSpace=None, nullValue=None, emptyValue=None, nanValue=None, |
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.
It should be put at the last; otherwise, it's going to break existing Python app when the arguments are given positionally.
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.
We should add new parameter at the end. +1
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.
Done!
ok to test |
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.
In light of discussion in the ticket https://issues.apache.org/jira/browse/SPARK-17916, could you write a test and check the case when empty values are written without quotes as it was in Spark 2.3 by default.
Test build #95259 has finished for PR 22234 at commit
|
@MaxGekk I added what you suggested as well. |
python/pyspark/sql/readwriter.py
Outdated
nanValue=nanValue, positiveInf=positiveInf, negativeInf=negativeInf, | ||
dateFormat=dateFormat, timestampFormat=timestampFormat, maxColumns=maxColumns, | ||
maxCharsPerColumn=maxCharsPerColumn, | ||
emptyValue=emptyValue, nanValue=nanValue, positiveInf=positiveInf, |
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.
I would put this at the end as well for readability.
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.
Done!
@@ -117,6 +117,9 @@ class CSVOptions( | |||
|
|||
val nullValue = parameters.getOrElse("nullValue", "") | |||
|
|||
val emptyValueInRead = parameters.getOrElse("emptyValue", "") |
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.
I would just call it emptyValue
for consistency with other options here.
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.
I though that as well. Just for the shake of providing backwards compatibility as we already have in ignoreLeadingWhiteSpaceInRead
and ignoreLeadingWhiteSpaceFlagInWrite
I implemented that in that way.
What do you say?
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.
I had to name them differently names because the default values are different. Ah, yea then it makes sense here. I rushed to read.
Test build #95270 has finished for PR 22234 at commit
|
Test build #95271 has finished for PR 22234 at commit
|
Test build #95274 has finished for PR 22234 at commit
|
Seems okay but I or someone else should take a closer look before getting this in. |
@@ -79,7 +79,8 @@ private[csv] object CSVInferSchema { | |||
* point checking if it is an Int, as the final type must be Double or higher. | |||
*/ | |||
def inferField(typeSoFar: DataType, field: String, options: CSVOptions): DataType = { | |||
if (field == null || field.isEmpty || field == options.nullValue) { | |||
if (field == null || field.isEmpty || field == options.nullValue || | |||
field == options.emptyValueInRead) { |
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.
I wouldn't do this for now. It needs another review iteration. Let's revert this back.
// When there are empty strings or the values set in `nullValue`, put the | ||
// index as the suffix. | ||
if (value == null || value.isEmpty || value == options.nullValue || | ||
value == options.emptyValueInRead) { |
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.
ditto for excluding.
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.
Do I revert these both changes @HyukjinKwon then?
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.
Looks good. Let me take another look before getting this in.
Did we introduce any behavior change in #21273? Does this PR resolve it? |
From my understanding, yea. The problem here is sounds like ambiguity in empty strings since they can be interpreted as empty strings and also This PR proposes an ability explicitly set the empty value to work around the behaviour change. |
Have we documented the behavior changes in the migration guide? If not, can we do it? |
This is rather a quite corner case (see the elaborated cases in the JIRA SPARK-17916) and there's ambiguity to treat this as a bug or a proper behaviour change; however, I don't object if this can be worth enough as something that should be mentioned. cc @MaxGekk for a followup |
@HyukjinKwon Do you mean to update migration guide in master and probably in Spark 2.4? I don't think this should be considered as a bug because current version and previous versions of Spark can read saved CSV files correctly. Yes, for now empty strings are saved as |
Oh no I mean we fixed a bug.. |
@MaxGekk Could you take this PR over? I think we need to merge this to Spark 2.4. Users can set the behaviors to the previous one by this new conf |
@gatorsmile @HyukjinKwon Please, take a look at #22367 |
…sed as null when nullValue is set. ## What changes were proposed in this pull request? In the PR, I propose new CSV option `emptyValue` and an update in the SQL Migration Guide which describes how to revert previous behavior when empty strings were not written at all. Since Spark 2.4, empty strings are saved as `""` to distinguish them from saved `null`s. Closes #22234 Closes #22367 ## How was this patch tested? It was tested by `CSVSuite` and new tests added in the PR #22234 Closes #22389 from MaxGekk/csv-empty-value-master. Lead-authored-by: Mario Molina <mmolimar@gmail.com> Co-authored-by: Maxim Gekk <maxim.gekk@databricks.com> Signed-off-by: hyukjinkwon <gurwls223@apache.org> (cherry picked from commit c9cb393) Signed-off-by: hyukjinkwon <gurwls223@apache.org>
…sed as null when nullValue is set. ## What changes were proposed in this pull request? In the PR, I propose new CSV option `emptyValue` and an update in the SQL Migration Guide which describes how to revert previous behavior when empty strings were not written at all. Since Spark 2.4, empty strings are saved as `""` to distinguish them from saved `null`s. Closes apache#22234 Closes apache#22367 ## How was this patch tested? It was tested by `CSVSuite` and new tests added in the PR apache#22234 Closes apache#22389 from MaxGekk/csv-empty-value-master. Lead-authored-by: Mario Molina <mmolimar@gmail.com> Co-authored-by: Maxim Gekk <maxim.gekk@databricks.com> Signed-off-by: hyukjinkwon <gurwls223@apache.org>
What changes were proposed in this pull request?
There is an option in the CSV parser to set values when we have empty values in the CSV files or in our dataframes.
Currently, this option cannot be configured and always sets a default value (empty string for reading and
""
for writing).This PR is about enabling a new CSV option in the reader/writer to set custom empty values when reading/writing CSV files.
How was this patch tested?
The changes were tested by CSVSuite adding two unit tests.