Skip to content

Conversation

@sureshthalamati
Copy link
Contributor

@sureshthalamati sureshthalamati commented May 4, 2016

What changes were proposed in this pull request?

Currently empty quoted strings in the input CSV file are incorrectly recognized as null value. This patch adds fix to recognizes quoted empty strings(eg; 1, "") in the data are recognized by default as empty string values.

New CSV option emptyValue is added to allows users to specify the value to write to the output CSV file for empty string, and also the value in in an input file that should be interpreted as empty string in addition to the empty quoted string.

DATA :
col1,col2
1,"-"
2,""
3,
4,"A

Default Old behaviour:
scala> val df = spark.read.format("csv").option("nullValue", "-").option("header", true).load("/Users/suresht/sparktests/emptystring.csv")
df: org.apache.spark.sql.DataFrame = [col1: string, col2: string]

scala> df.show
+----+----+
|col1|col2|
+----+----+
| 1|null|
| 2|null|
| 3|null|
| 4| A|
+----+----+

Default New behavior :

scala> val df = spark.read.format("csv").option("nullValue", "-").option("header", true).load("/Users/suresht/sparktests/emptystring.csv")
df: org.apache.spark.sql.DataFrame = [col1: string, col2: string]

scala> df.show
+----+----+
|col1|col2|
+----+----+
| 1|null|
| 2| |
| 3|null|
| 4| A|
+----+----+

Example using the emptyValue option:

val df = spark.createDataFrame(Seq((1, "")))
df.write.format("csv").option("emptyValue", "EMPTY").save("/tmp/data1")

cat part-r-00000-8d867267-c291-4277-9951-a8b969c0a4d8.csv
1,EMPTY

scala> spark.read.format("csv").option("emptyValue", "EMPTY").load("/tmp/data1").show
+---+---+
|_c0|_c1|
+---+---+
| 1| |
+---+---+

How was this patch tested?

Added new unit tests to the CSVSuite.

@falaki @rxin

@rxin
Copy link
Contributor

rxin commented May 5, 2016

Shouldn't we always infer these as empty strings, and then users can do a simple project to turn them into nulls?

@rxin
Copy link
Contributor

rxin commented May 5, 2016

Basically I'm asking why is this config needed if we just treat them as empty strings?

@sureshthalamati
Copy link
Contributor Author

Thanks for reviewing the PR, Reynold. Only reason I added the option is to keep the current behavior by default. I agree with you we can treat them as empty strings always. I am also not aware of any scenario where user wants to treat empty strings as null values.

If there is no need to keep the current default behavior , I will update the PR. Please let me know

@rxin
Copy link
Contributor

rxin commented May 5, 2016

Yea let's update it.

cc @HyukjinKwon @falaki

@HyukjinKwon
Copy link
Member

HyukjinKwon commented May 5, 2016

+1 for treating them as empty strings without additional options.

@sureshthalamati sureshthalamati force-pushed the empstring_fix_spark-15125 branch from e6207e7 to 2629a7c Compare May 6, 2016 20:41
@sureshthalamati sureshthalamati changed the title [SPARK-15125][SQL] New option to the CSV data source to allows users to specify to how to interpret empty quoted strings. [SPARK-15125][SQL] Changing CSV data source mapping of empty quoted strings in the data to empty strings instead of null May 6, 2016
@sureshthalamati
Copy link
Contributor Author

Thank you for the feedback , Reynold , HyukjinKwon. Update the PR.

Copy link
Member

Choose a reason for hiding this comment

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

Could I ask what happen if we don't set nullValue?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

If nullValue is not set it will return empty string for null values by default. The reason I set it explicitly is to make sure my fix is working. Before my fix it was retruning null for the empty quoted string , and empty string for null values by default.

@HyukjinKwon
Copy link
Member

HyukjinKwon commented May 7, 2016

Here is what I think CSV datasource should handle "", empty string and nullValue for reading.

"","a",

should produce the records as below:

  1. With the option, nullValue set to "a", I think

    Row("", null, null)
    
  2. Without any options, I think

    Row("", "a", null)
    
  3. With the option, nullValue set to "", I think

    Row(null, "a", null)
    

Would this make sense? If so, we need to give a default value, null for nullValue. (Currently the default is "")

@HyukjinKwon
Copy link
Member

HyukjinKwon commented May 7, 2016

In case of writing, I think

Row("", "a", null)

should produce the CSV as below:

  1. With the option, nullValue set to "a", I think

    ,,
    
  2. Without any options, I think

    ,a,
    
  3. With the option, nullValue set to "", I think

    ,a,
    

+Sorry I just updated the writing examples above.

@sureshthalamati
Copy link
Contributor Author

I am not sure what was the history behind returning empty String for null value. In my opinion it should be null be default. current behavior is also inconsistent; for numerics it will return null and for strings it will return empty string by default.

Example:
See the Year (int), and comment (String in the following data).
year,make,model,comment,price
2017,Tesla,Mode 3,looks nice.,35000.99
,Chevy,Bolt,,29000.00
2015,Porsche,"",,
scala> val df= sqlContext.read.format("csv").option("header", "true").option("inferSchema", "true").load("/tmp/test1.csv")
df: org.apache.spark.sql.DataFrame = [year: int, make: string ... 3 more fields]

scala> df.show
+----+-------+------+-----------+--------+
|year| make| model| comment| price|
+----+-------+------+-----------+--------+
|2017| Tesla|Mode 3|looks nice.|35000.99|
|null| Chevy| Bolt| | 29000.0|
|2015|Porsche| null| | null|
+----+-------+------+-----------+--------+

I can update this PR to change the nullValue default if needed,

@sureshthalamati
Copy link
Contributor Author

@HyukjinKwon does your previous comment for meant for some other PR ? This PR does not have any change you mentioned above. Am I missing some thing ?

@andrewor14

@HyukjinKwon
Copy link
Member

HyukjinKwon commented May 13, 2016

@sureshthalamati oh, the comments are not related with this PR but moving the discussion to here was suggested. So, i did. Sorry for that if it was confusing. I will move this topic to dev-mailing list or JIRA.

(I removed/cleaned up my unrelated comments).

Copy link
Contributor

Choose a reason for hiding this comment

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

Hard coding this is not a good idea. Please add a new option in CSVOption and pass to the parser. The default value could be "".

Copy link
Contributor

Choose a reason for hiding this comment

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

Would you also add a regression unit-test to make sure this patch also fixes https://issues.apache.org/jira/browse/SPARK-17916?

@sureshthalamati
Copy link
Contributor Author

Thanks for the input @falaki . Sorry for the delated reply, sone how I missed the notifications. I will update the patch with option and also add test case for 17916.

… to specify to interpret empty quoted strings as null or an empty string.
@sureshthalamati sureshthalamati force-pushed the empstring_fix_spark-15125 branch from 2629a7c to b128fbb Compare October 20, 2016 08:22
val permissive = ParseModes.isPermissiveMode(parseMode)

val nullValue = parameters.getOrElse("nullValue", "")
val emptyValue = parameters.getOrElse("emptyValue", "")
Copy link
Member

Choose a reason for hiding this comment

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

When nullValue and emptyValue are both "" in default, don't they conflict?

Copy link
Member

@HyukjinKwon HyukjinKwon Oct 20, 2016

Choose a reason for hiding this comment

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

+1 for documenting the explicit precedence

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yes, null and empty can not be differentiated when they are set to same value. Currently null value check has higher precedence than the empty value.

input.csv
1,
2,””

Output will be:
1, null
2, null

I think this behavior is ok. By default Univocity CSV parser used in spark also returns null for empty strings.

I agree we should document this behavior.

@HyukjinKwon
Copy link
Member

FWIW, it seems read.csv() in R seems not differenciating it from "".

> bt <- "A,B,C,D
+ ,\"\",20"
>
> b<- read.csv(text=bt, na.strings=c(""))
> b
   A  B  C  D
1 NA NA 20 NA

@felixcheung
Copy link
Member

hmm, @HyukjinKwon the behavior with your R example is because of the na.strings=c("") parameter, which tells it to treat empty "" as NA (== JVM null in Spark)

@HyukjinKwon
Copy link
Member

HyukjinKwon commented Oct 20, 2016

Oh, sure it is true. I set na.strings as "" because the equivalent option, nullValue has the same default value in Spark's CSV :).

EDITED: Hm, it seems it produces the same output without na.strings=c("") actually..

> bt <- "A,B,C,D
+ ,\"\",20"
>
> b<- read.csv(text=bt)
> b
   A  B  C  D
1 NA NA 20 NA

@antoniobarbuzzi
Copy link

I agree with @HyukjinKwon and comment #12904 (comment), except for third use case, i.e. I'd rather do this:

With the option, nullValue set to ""
"","a", should be converted to Row("", "a", null)

In fact, just like the univocity parser does here:

if the parser does not read any character from the input, and the input is within quotes , the empty is used instead of an empty string

Regarding the PR, I'd deletegate the handling of this to the univocity parser, setting the settings.setEmptyValue(params.emptyValue) in the univocity's CSVSettings here and here too.

@SparkQA
Copy link

SparkQA commented Nov 3, 2016

Test build #3410 has finished for PR 12904 at commit c7aa4aa.

  • This patch fails Spark unit tests.
  • This patch does not merge cleanly.
  • This patch adds no public classes.

@sureshthalamati
Copy link
Contributor Author

I was testing the fix with different scenarios mentioned in the comments. I can not make
CSV writer write quoted empty string for empty strings in the data. One of the issue I filed got fixed , but still can not make it work.

uniVocity/univocity-parsers#123

@gatorsmile
Copy link
Member

What is the status of this PR?

@gatorsmile
Copy link
Member

@falaki Should we continue this PR?

@MaxGekk
Copy link
Member

MaxGekk commented Jul 8, 2018

The issue has been already solved by 7a2d489 . The PR can be closed.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

10 participants