You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@flink.apache.org by "Lijie Wang (Jira)" <ji...@apache.org> on 2022/03/21 06:04:00 UTC

[jira] [Created] (FLINK-26760) The new CSV source (file system source + CSV format) does not support reading files whose file encoding is not UTF-8

Lijie Wang created FLINK-26760:
----------------------------------

             Summary: The new CSV source (file system source + CSV format) does not support reading files whose file encoding is not UTF-8
                 Key: FLINK-26760
                 URL: https://issues.apache.org/jira/browse/FLINK-26760
             Project: Flink
          Issue Type: Bug
          Components: Connectors / FileSystem
            Reporter: Lijie Wang
         Attachments: example.csv

The new CSV source (file system source + CSV format) does not support reading files whose file encoding is not UTF-8, but the legacy {{CsvTableSource}} supports it.

We provide an {{*example.csv*}} whose file encoding is {{{}ISO-8599-1{}}}.

When reading it with the legacy {{{}CsvTableSource{}}}, it executes correctly:
{code:java}
    @Test
    public void testLegacyCsvSource() {
        EnvironmentSettings environmentSettings = EnvironmentSettings.inBatchMode();
        TableEnvironment tEnv = TableEnvironment.create(environmentSettings);

        CsvTableSource.Builder builder = CsvTableSource.builder();

        CsvTableSource source =
                builder.path("example.csv")
                        .emptyColumnAsNull()
                        .lineDelimiter("\n")
                        .fieldDelimiter("|")
                        .field("name", DataTypes.STRING())
                        .build();
        ConnectorCatalogTable catalogTable = ConnectorCatalogTable.source(source, true);
        tEnv.getCatalog(tEnv.getCurrentCatalog())
                .ifPresent(
                        catalog -> {
                            try {
                                catalog.createTable(
                                        new ObjectPath(tEnv.getCurrentDatabase(), "example"),
                                        catalogTable,
                                        false);
                            } catch (Exception e) {
                                throw new RuntimeException(e);
                            }
                        });

        tEnv.executeSql("select count(name) from example").print();
    }
{code}
When reading it with the new CSV source (file system source + CSV format), it throws the following error:
{code:java}
    @Test
    public void testNewCsvSource() {
        EnvironmentSettings environmentSettings = EnvironmentSettings.inBatchMode();
        TableEnvironment tEnv = TableEnvironment.create(environmentSettings);

        String ddl =
                "create table example ("
                        + "    name string"
                        + ") with ("
                        + "    'connector' = 'filesystem',"
                        + "    'path' = 'example.csv',"
                        + "    'format' = 'csv',"
                        + "    'csv.array-element-delimiter' = '\n',"
                        + "    'csv.field-delimiter' = '|',"
                        + "    'csv.null-literal' = ''"
                        + ")";

        tEnv.executeSql(ddl);
        tEnv.executeSql("select count(name) from example").print();
    }
{code}
{code:java}
Caused by: java.lang.RuntimeException: One or more fetchers have encountered exception
	at org.apache.flink.connector.base.source.reader.fetcher.SplitFetcherManager.checkErrors(SplitFetcherManager.java:225)
	at org.apache.flink.connector.base.source.reader.SourceReaderBase.getNextFetch(SourceReaderBase.java:169)
	at org.apache.flink.connector.base.source.reader.SourceReaderBase.pollNext(SourceReaderBase.java:130)
	at org.apache.flink.streaming.api.operators.SourceOperator.emitNext(SourceOperator.java:385)
	at org.apache.flink.streaming.runtime.io.StreamTaskSourceInput.emitNext(StreamTaskSourceInput.java:68)
	at org.apache.flink.streaming.runtime.io.StreamOneInputProcessor.processInput(StreamOneInputProcessor.java:65)
	at org.apache.flink.streaming.runtime.tasks.StreamTask.processInput(StreamTask.java:519)
	at org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.runMailboxLoop(MailboxProcessor.java:203)
	at org.apache.flink.streaming.runtime.tasks.StreamTask.runMailboxLoop(StreamTask.java:804)
	at org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:753)
	at org.apache.flink.runtime.taskmanager.Task.runWithSystemExitMonitoring(Task.java:948)
	at org.apache.flink.runtime.taskmanager.Task.restoreAndInvoke(Task.java:927)
	at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:741)
	at org.apache.flink.runtime.taskmanager.Task.run(Task.java:563)
	at java.lang.Thread.run(Thread.java:748)
Caused by: java.lang.RuntimeException: SplitFetcher thread 0 received unexpected exception while polling the records
	at org.apache.flink.connector.base.source.reader.fetcher.SplitFetcher.runOnce(SplitFetcher.java:150)
	at org.apache.flink.connector.base.source.reader.fetcher.SplitFetcher.run(SplitFetcher.java:105)
	at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
	at java.util.concurrent.FutureTask.run(FutureTask.java:266)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
	... 1 more
Caused by: java.lang.RuntimeException: Invalid UTF-8 middle byte 0x54 (at char #9, byte #8): check content encoding, does not look like UTF-8
	at org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.MappingIterator._handleIOException(MappingIterator.java:417)
	at org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.MappingIterator.next(MappingIterator.java:203)
	at org.apache.flink.formats.csv.CsvReaderFormat$Reader.read(CsvReaderFormat.java:196)
	at org.apache.flink.connector.file.src.impl.StreamFormatAdapter$Reader.readBatch(StreamFormatAdapter.java:214)
	at org.apache.flink.connector.file.src.impl.FileSourceSplitReader.fetch(FileSourceSplitReader.java:67)
	at org.apache.flink.connector.base.source.reader.fetcher.FetchTask.run(FetchTask.java:58)
	at org.apache.flink.connector.base.source.reader.fetcher.SplitFetcher.runOnce(SplitFetcher.java:142)
	... 6 more
Caused by: java.io.CharConversionException: Invalid UTF-8 middle byte 0x54 (at char #9, byte #8): check content encoding, does not look like UTF-8
	at org.apache.flink.shaded.jackson2.com.fasterxml.jackson.dataformat.csv.impl.UTF8Reader.reportInvalidOther(UTF8Reader.java:507)
	at org.apache.flink.shaded.jackson2.com.fasterxml.jackson.dataformat.csv.impl.UTF8Reader.reportDeferredInvalid(UTF8Reader.java:518)
	at org.apache.flink.shaded.jackson2.com.fasterxml.jackson.dataformat.csv.impl.UTF8Reader.read(UTF8Reader.java:172)
	at org.apache.flink.shaded.jackson2.com.fasterxml.jackson.dataformat.csv.impl.CsvDecoder.loadMore(CsvDecoder.java:443)
	at org.apache.flink.shaded.jackson2.com.fasterxml.jackson.dataformat.csv.impl.CsvDecoder.nextString(CsvDecoder.java:679)
	at org.apache.flink.shaded.jackson2.com.fasterxml.jackson.dataformat.csv.CsvParser._handleNextEntry(CsvParser.java:915)
	at org.apache.flink.shaded.jackson2.com.fasterxml.jackson.dataformat.csv.CsvParser.nextFieldName(CsvParser.java:727)
	at org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.deser.std.BaseNodeDeserializer.deserializeObject(JsonNodeDeserializer.java:268)
	at org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.deser.std.JsonNodeDeserializer.deserialize(JsonNodeDeserializer.java:69)
	at org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.deser.std.JsonNodeDeserializer.deserialize(JsonNodeDeserializer.java:16)
	at org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.MappingIterator.nextValue(MappingIterator.java:280)
	at org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.MappingIterator.next(MappingIterator.java:199)
	... 11 more
{code}
 



--
This message was sent by Atlassian Jira
(v8.20.1#820001)