[
https://issues.apache.org/jira/browse/FLINK-26722?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17508641#comment-17508641
]
zl edited comment on FLINK-26722 at 3/18/22, 8:20 AM:
------------------------------------------------------
I think it has something to do with field parsing.
when use CsvTableSource, we use
[RowCsvInputFormat|https://github.com/apache/flink/blob/master/flink-java/src/main/java/org/apache/flink/api/java/io/RowCsvInputFormat.java]
for reading data and
[StringParser#parseField|https://github.com/apache/flink/blob/master/flink-core/src/main/java/org/apache/flink/types/parser/StringParser.java#L47]
for parsing string field. when a string field is empty(""), if
emptyColumnAsNull is enabled,
[RowCsvInputFormat#L221|https://github.com/apache/flink/blob/master/flink-java/src/main/java/org/apache/flink/api/java/io/RowCsvInputFormat.java#L221]will
set the field value to null.
when use the new file connector with csv format, we use
[CsvReaderFormat|https://github.com/apache/flink/blob/master/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvReaderFormat.java]
for reading data and
[CsvToRowDataConverters#convertToString|https://github.com/apache/flink/blob/master/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvToRowDataConverters.java#L271]
for parsing string field. when a string field is empty(""),
[CsvToRowDataConverters.java#L109|https://github.com/apache/flink/blob/master/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvToRowDataConverters.java#L109]will
set the field value to empty string ("").
I think the way that CsvTableSource treats empty string may be more reasonable,
the new file source with csv format should be consistent with it. It means that
when *_csv.ignore-parse-errors_* is enabled,
[CsvToRowDataConverters#convertToString|https://github.com/apache/flink/blob/master/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvToRowDataConverters.java#L271]
covert empty string ("") to null, otherwise convert ("") to ("").
was (Author: leo zhou):
I think it has something to do with field parsing.
when use CsvTableSource, we use
[RowCsvInputFormat|https://github.com/apache/flink/blob/master/flink-java/src/main/java/org/apache/flink/api/java/io/RowCsvInputFormat.java]
for reading data and
[StringParser#parseField|https://github.com/apache/flink/blob/master/flink-core/src/main/java/org/apache/flink/types/parser/StringParser.java#L47]
for parsing string field. when a string field is empty(""), if
emptyColumnAsNull is enabled,
[RowCsvInputFormat#L221|https://github.com/apache/flink/blob/master/flink-java/src/main/java/org/apache/flink/api/java/io/RowCsvInputFormat.java#L221]will
set the field value to null.
when use the new file connector with csv format, we use
[CsvReaderFormat|https://github.com/apache/flink/blob/master/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvReaderFormat.java]
for reading data and
[CsvToRowDataConverters#convertToString|https://github.com/apache/flink/blob/master/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvToRowDataConverters.java#L271]
for parsing string field. when a string field is empty(""),
[CsvToRowDataConverters.java#L109|https://github.com/apache/flink/blob/master/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvToRowDataConverters.java#L109]will
set the field value to empty string ("").
I think the way that CsvTableSource treats empty string may be more reasonable,
the new file source with csv format should be consistent with it
> the result is wrong when using file connector with csv format
> -------------------------------------------------------------
>
> Key: FLINK-26722
> URL: https://issues.apache.org/jira/browse/FLINK-26722
> Project: Flink
> Issue Type: Bug
> Components: Formats (JSON, Avro, Parquet, ORC, SequenceFile)
> Reporter: zl
> Priority: Major
> Attachments: CsvTest1.java, example.csv,
> image-2022-03-18-15-32-28-914.png
>
>
> CsvTest1.java execute a same query on a same dataset (Attachment example.csv)
> with CsvTableSource and the new file connector respectively, but the result
> is different. The results are as follows:
> !image-2022-03-18-15-32-28-914.png!
--
This message was sent by Atlassian Jira
(v8.20.1#820001)