[ 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 and PR [10766|https://github.com/apache/spark/pull/10766] . For the nullValue option, according to the features description in spark-csv readme file, it is 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 NULL strings in csv files can be parsed as columns of null values 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 would be convert to nullValue strings. But In Spark, we finally convert nothing 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. {noformat} *no* further _formatting_ is done here{noformat} For example, a column has empty strings, if emptyValueInWrite is set to "EMPTY" string. {code:scala} Seq(("Tesla", "")).toDF("make", "comment").write.option("emptyValue", "EMPTY").csv(path){code} The saved csv file is shown as: {noformat} Tesla,EMPTY {noformat} and if we read this csv file with emptyValue(emptyValueInRead) set to "EMPTY" string. {code:java} spark.read.option("emptyValue", "EMPTY").csv(path).show() {code} we actually get the DataFrame which data is shown as: ||make||comment|| |tesla|EMPTY| but the DataFrame which data should be shown as below as expected: ||make||comment|| |tesla| | I found that Spark keeps the same behavior with the depended component univocity. Since Spark 2.4, for empty strings, there are emptyValueInRead for reading and emptyValueInWrite for writing that can be set in CSVOptions: {code:scala} // For writing, convert: ""(dataframe) => emptyValueInWrite(csv) // For reading, convert: "" (csv) => emptyValueInRead(dataframe){code} I think the read handling is not suitable, we can not convert "" or `{color:#172b4d}emptyValueInWrite`{color} values as ""(real empty strings) but get {color:#172b4d}emptyValueInRead's setting value actually{color}, it supposed to be as flows: {code:scala} // For reading, convert: "" or emptyValueInRead (csv) => ""(dataframe){code} {color:#de350b}*We can not recovery it to the original DataFrame.*{color} was: The csv data format is imported from databricks [spark-csv|https://github.com/databricks/spark-csv] by issue SPARK-12833 and PR [10766|https://github.com/apache/spark/pull/10766] . For the nullValue option, according to the features description in spark-csv readme file, it is 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 NULL strings in csv files can be parsed as columns of null values 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 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 would be convert to nullValue strings. But In Spark, we finally convert nothing 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 Since Spark 2.4, for empty strings, there are emptyValueInRead for reading and emptyValueInWrite for writing that can be set in CSVOptions: {code:scala} // For writing, convert: ""(dataframe) => emptyValueInWrite(csv) // For reading, convert: "" (csv) => emptyValueInRead(dataframe){code} I think the read handling is not suitable, we can not convert "" or `{color:#172b4d}emptyValueInWrite`{color} values as ""(real empty strings) but get {color:#172b4d}emptyValueInRead's setting value actually{color}, it supposed to be as flows: {code:scala} // For reading, convert: "" or emptyValueInRead (csv) => ""(dataframe){code} For example, a column has empty strings, if emptyValueInWrite is set to "EMPTY" string. {code:scala} Seq(("Tesla", "")).toDF("make", "comment").write.option("emptyValue", "EMPTY").csv(path){code} The saved csv file is shown as: {noformat} Tesla,EMPTY {noformat} and if we read this csv file with emptyValue(emptyValueInRead) set to "EMPTY" string. {code:java} spark.read.option("emptyValue", "EMPTY").csv(path).show() {code} we actually get the DataFrame which data is shown as: ||make||comment|| |tesla|EMPTY| but the DataFrame which data should be shown as below as expected: ||make||comment|| |tesla| | {color:#de350b}*We can not recovery it to the original DataFrame.*{color} > 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 and > PR [10766|https://github.com/apache/spark/pull/10766] . > For the nullValue option, according to the features description in spark-csv > readme file, it is 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 NULL strings in csv files can be parsed as columns of null > values 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 would be convert to > nullValue strings. But In Spark, we finally convert nothing 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. > {noformat} > *no* further _formatting_ is done here{noformat} > For example, a column has empty strings, if emptyValueInWrite is set to > "EMPTY" string. > {code:scala} > Seq(("Tesla", "")).toDF("make", "comment").write.option("emptyValue", > "EMPTY").csv(path){code} > The saved csv file is shown as: > {noformat} > Tesla,EMPTY {noformat} > and if we read this csv file with emptyValue(emptyValueInRead) set to "EMPTY" > string. > {code:java} > spark.read.option("emptyValue", "EMPTY").csv(path).show() > {code} > we actually get the DataFrame which data is shown as: > ||make||comment|| > |tesla|EMPTY| > but the DataFrame which data should be shown as below as expected: > ||make||comment|| > |tesla| | > I found that Spark keeps the same behavior with the depended component > univocity. > Since Spark 2.4, for empty strings, there are emptyValueInRead for reading > and emptyValueInWrite for writing that can be set in CSVOptions: > {code:scala} > // For writing, convert: ""(dataframe) => emptyValueInWrite(csv) > // For reading, convert: "" (csv) => emptyValueInRead(dataframe){code} > I think the read handling is not suitable, we can not convert "" or > `{color:#172b4d}emptyValueInWrite`{color} values as ""(real empty strings) > but get {color:#172b4d}emptyValueInRead's setting value actually{color}, it > supposed to be as flows: > {code:scala} > // For reading, convert: "" or emptyValueInRead (csv) => ""(dataframe){code} > > {color:#de350b}*We can not recovery it to the original DataFrame.*{color} -- 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