[ https://issues.apache.org/jira/browse/SPARK-37604?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Wei Guo updated SPARK-37604: ---------------------------- Description: The csv data format is imported from databricks [spark-csv|https://github.com/databricks/spark-csv] by issue SPARK-12833 with PR [10766|https://github.com/apache/spark/pull/10766] . {*}For the nullValue option{*}, according to features described in spark-csv readme file, it's designed as: {noformat} When reading files: nullValue: specifies a string that indicates a null value, any fields matching this string will be set as nulls in the DataFrame When writing files: nullValue: specifies a string that indicates a null value, nulls in the DataFrame will be written as this string. {noformat} For example, when writing: {code:scala} Seq(("Tesla", null:String)).toDF("make", "comment").write.option("nullValue", "NULL").csv(path){code} The saved csv file is shown as: {noformat} Tesla,NULL {noformat} When reading: {code:java} spark.read.option("nullValue", "NULL").csv(path).show() {code} The parsed dataframe is shown as: ||make||comment|| |Tesla|null| We can find that null columns in dataframe can be saved as "NULL" strings in csv files and {color:#00875a}*"NULL" strings in csv files can be parsed as null columns*{color} in dataframe. That is: {noformat} When writing, convert null(in dataframe) to nullValue(in csv) When reading, convert nullValue or nothing(in csv) to null(in dataframe) {noformat} But actually, the option nullValue in depended component univocity's {*}_CommonSettings_{*}, is designed as that: {noformat} when reading, if the parser does not read any character from the input, the nullValue is used instead of an empty string. when writing, if the writer has a null object to write to the output, the nullValue is used instead of an empty string.{noformat} {*}There is a difference when reading{*}. In univocity, nothing content would be convert to nullValue strings. But In Spark, we finally convert nothing content or nullValue strings to null in *_UnivocityParser_ _nullSafeDatum_* method: {code:java} private def nullSafeDatum( datum: String, name: String, nullable: Boolean, options: CSVOptions)(converter: ValueConverter): Any = { if (datum == options.nullValue || datum == null) { if (!nullable) { throw QueryExecutionErrors.foundNullValueForNotNullableFieldError(name) } null } else { converter.apply(datum) } } {code} ------------ {*}For the emptyValue option{*}, we add a emptyValueInRead option for reading and a emptyValueInWrite option for writing. I found that both Spark keeps the same behaviors for emptyValue with univocity. {noformat} When reading, 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. When writing, if the writer has an empty String to write to the output, the emptyValue is used instead of an empty string.{noformat} For example, when writing: {code:scala} Seq(("Tesla", {code} {color:#910091}""{color} {code:scala} )).toDF("make", "comment").write.option("emptyValue", "EMPTY").csv(path){code} The saved csv file is shown as: {noformat} Tesla,EMPTY {noformat} When reading: {code:java} spark.read.option("emptyValue", "EMPTY").csv(path).show() {code} The parsed dataframe is shown as: ||make||comment|| |Tesla|EMPTY| We can find that empty columns in dataframe can be saved as "EMPTY" strings in csv files, *{color:#de350b}but "EMPTY" strings in csv files can not be parsed as empty columns{color}* in dataframe. That is: {noformat} When writing, convert "" empty(in dataframe) to emptyValue(in csv) When reading, convert "\"\"" quoted empty strings to emptyValue(in dataframe) {noformat} There is obvious difference between nullValue and emptyValue in read handling. For nullValue, we try to convert nothing or nullValue strings to null in dataframe, but for emptyValue, we just try to convert "\"\""(quoted empty strings) to emptyValue rather than to convert both "\"\""(quoted empty strings) and emptyValue strings to ""(empty) in dataframe. I think it's better that we keep the similar behavior(try to recover emptyValue to "") for emptyValue as nullValue when reading, so I suggest that the emptyValueInRead(in CSVOptions) should be designed as that any fields matching this string will be set as empty values "" when reading. was: The csv data format is imported from databricks [spark-csv|https://github.com/databricks/spark-csv] by issue SPARK-12833 with PR [10766|https://github.com/apache/spark/pull/10766] . {*}For the nullValue option{*}, according to features described in spark-csv readme file, it's designed as: {noformat} When reading files: nullValue: specifies a string that indicates a null value, any fields matching this string will be set as nulls in the DataFrame When writing files: nullValue: specifies a string that indicates a null value, nulls in the DataFrame will be written as this string. {noformat} For example, when writing: {code:scala} Seq(("Tesla", null:String)).toDF("make", "comment").write.option("nullValue", "NULL").csv(path){code} The saved csv file is shown as: {noformat} Tesla,NULL {noformat} When reading: {code:java} spark.read.option("nullValue", "NULL").csv(path).show() {code} The parsed dataframe is shown as: ||make||comment|| |Tesla|null| We can find that null columns in dataframe can be saved as "NULL" strings in csv files and {color:#00875a}*"NULL" strings in csv files can be parsed as null columns*{color} in dataframe. That is: {noformat} When writing, convert null(in dataframe) to nullValue(in csv) When reading, convert nullValue or nothing(in csv) to null(in dataframe) {noformat} But actually, the option nullValue in depended component univocity's {*}_CommonSettings_{*}, is designed as that: {noformat} when reading, if the parser does not read any character from the input, the nullValue is used instead of an empty string. when writing, if the writer has a null object to write to the output, the nullValue is used instead of an empty string.{noformat} There is a difference when reading. In univocity, nothing content would be convert to nullValue strings. But In Spark, we finally convert nothing content or nullValue strings to null in *_UnivocityParser_ _nullSafeDatum_* method: {code:java} private def nullSafeDatum( datum: String, name: String, nullable: Boolean, options: CSVOptions)(converter: ValueConverter): Any = { if (datum == options.nullValue || datum == null) { if (!nullable) { throw QueryExecutionErrors.foundNullValueForNotNullableFieldError(name) } null } else { converter.apply(datum) } } {code} ------------ {*}For the emptyValue option{*}, we add a emptyValueInRead option for reading and a emptyValueInWrite option for writing. I found that both Spark keeps the same behaviors for emptyValue with univocity. {noformat} When reading, 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. When writing, if the writer has an empty String to write to the output, the emptyValue is used instead of an empty string.{noformat} For example, when writing: {code:scala} Seq(("Tesla", {code} {color:#910091}""{color} {code:scala} )).toDF("make", "comment").write.option("emptyValue", "EMPTY").csv(path){code} The saved csv file is shown as: {noformat} Tesla,EMPTY {noformat} When reading: {code:java} spark.read.option("emptyValue", "EMPTY").csv(path).show() {code} The parsed dataframe is shown as: ||make||comment|| |Tesla|EMPTY| We can find that empty columns in dataframe can be saved as "EMPTY" strings in csv files, *{color:#de350b}but "EMPTY" strings in csv files can not be parsed as empty columns{color}* in dataframe. That is: {noformat} When writing, convert "" empty(in dataframe) to emptyValue(in csv) When reading, convert "\"\"" quoted empty strings to emptyValue(in dataframe) {noformat} There is obvious difference between nullValue and emptyValue in read handling. For nullValue, we try to convert nothing or nullValue strings to null in dataframe, but for emptyValue, we just try to convert "\"\""(quoted empty strings) to emptyValue rather than to convert both "\"\""(quoted empty strings) and emptyValue strings to ""(empty) in dataframe. I think it's better that we keep the similar behavior(try to recover emptyValue to "") for emptyValue as nullValue when reading, so I suggest that the emptyValueInRead(in CSVOptions) should be designed as that any fields matching this string will be set as empty values "" when reading. > The option emptyValueInRead(in CSVOptions) is suggested to be designed as > that any fields matching this string will be set as empty values "" when > reading > ---------------------------------------------------------------------------------------------------------------------------------------------------------- > > Key: SPARK-37604 > URL: https://issues.apache.org/jira/browse/SPARK-37604 > Project: Spark > Issue Type: Bug > Components: SQL > Affects Versions: 2.4.0, 3.2.0 > Reporter: Wei Guo > Priority: Major > > The csv data format is imported from databricks > [spark-csv|https://github.com/databricks/spark-csv] by issue SPARK-12833 with > PR [10766|https://github.com/apache/spark/pull/10766] . > {*}For the nullValue option{*}, according to features described in spark-csv > readme file, it's designed as: > {noformat} > When reading files: > nullValue: specifies a string that indicates a null value, any fields > matching this string will be set as nulls in the DataFrame > When writing files: > nullValue: specifies a string that indicates a null value, nulls in the > DataFrame will be written as this string. > {noformat} > For example, when writing: > {code:scala} > Seq(("Tesla", null:String)).toDF("make", "comment").write.option("nullValue", > "NULL").csv(path){code} > The saved csv file is shown as: > {noformat} > Tesla,NULL > {noformat} > When reading: > {code:java} > spark.read.option("nullValue", "NULL").csv(path).show() > {code} > The parsed dataframe is shown as: > ||make||comment|| > |Tesla|null| > We can find that null columns in dataframe can be saved as "NULL" strings in > csv files and {color:#00875a}*"NULL" strings in csv files can be parsed as > null columns*{color} in dataframe. That is: > {noformat} > When writing, convert null(in dataframe) to nullValue(in csv) > When reading, convert nullValue or nothing(in csv) to null(in dataframe) > {noformat} > But actually, the option nullValue in depended component univocity's > {*}_CommonSettings_{*}, is designed as that: > {noformat} > when reading, if the parser does not read any character from the input, the > nullValue is used instead of an empty string. > when writing, if the writer has a null object to write to the output, the > nullValue is used instead of an empty string.{noformat} > {*}There is a difference when reading{*}. In univocity, nothing content would > be convert to nullValue strings. But In Spark, we finally convert nothing > content or nullValue strings to null in *_UnivocityParser_ _nullSafeDatum_* > method: > {code:java} > private def nullSafeDatum( > datum: String, > name: String, > nullable: Boolean, > options: CSVOptions)(converter: ValueConverter): Any = { > if (datum == options.nullValue || datum == null) { > if (!nullable) { > throw QueryExecutionErrors.foundNullValueForNotNullableFieldError(name) > } > null > } else { > converter.apply(datum) > } > } {code} > > ------------ > > {*}For the emptyValue option{*}, we add a emptyValueInRead option for > reading and a emptyValueInWrite option for writing. I found that both Spark > keeps the same behaviors for emptyValue with univocity. > {noformat} > When reading, 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. > When writing, if the writer has an empty String to write to the output, the > emptyValue is used instead of an empty string.{noformat} > For example, when writing: > {code:scala} > Seq(("Tesla", > {code} > {color:#910091}""{color} > {code:scala} > )).toDF("make", "comment").write.option("emptyValue", "EMPTY").csv(path){code} > The saved csv file is shown as: > {noformat} > Tesla,EMPTY {noformat} > When reading: > {code:java} > spark.read.option("emptyValue", "EMPTY").csv(path).show() > {code} > The parsed dataframe is shown as: > ||make||comment|| > |Tesla|EMPTY| > We can find that empty columns in dataframe can be saved as "EMPTY" strings > in csv files, *{color:#de350b}but "EMPTY" strings in csv files can not be > parsed as empty columns{color}* in dataframe. That is: > {noformat} > When writing, convert "" empty(in dataframe) to emptyValue(in csv) > When reading, convert "\"\"" quoted empty strings to emptyValue(in dataframe) > {noformat} > > There is obvious difference between nullValue and emptyValue in read > handling. For nullValue, we try to convert nothing or nullValue strings to > null in dataframe, but for emptyValue, we just try to convert "\"\""(quoted > empty strings) to emptyValue rather than to convert both "\"\""(quoted empty > strings) and emptyValue strings to ""(empty) in dataframe. > I think it's better that we keep the similar behavior(try to recover > emptyValue to "") for emptyValue as nullValue when reading, so I suggest that > the emptyValueInRead(in CSVOptions) should be designed as that any fields > matching this string will be set as empty values "" when reading. -- This message was sent by Atlassian Jira (v8.20.1#820001) --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org