Class CsvReaderFormat<T>

java.lang.Object
org.apache.flink.connector.file.src.reader.SimpleStreamFormat<T>
org.apache.flink.formats.csv.CsvReaderFormat<T>
Type Parameters:
T - The type of the returned elements.
All Implemented Interfaces:
Serializable, org.apache.flink.api.java.typeutils.ResultTypeQueryable<T>, org.apache.flink.connector.file.src.reader.StreamFormat<T>

@PublicEvolving public class CsvReaderFormat<T> extends org.apache.flink.connector.file.src.reader.SimpleStreamFormat<T>
A StreamFormat for reading CSV files.

The following example shows how to create a CsvReaderFormat where the schema for CSV parsing is automatically derived based on the fields of a POJO class.


 CsvReaderFormat<SomePojo> csvFormat = CsvReaderFormat.forPojo(SomePojo.class);
 FileSource<SomePojo> source =
         FileSource.forRecordStreamFormat(csvFormat, Path.fromLocalFile(filesPath)).build();
 
Note: you might need to add @JsonPropertyOrder({field1, field2, ...}) annotation from the Jackson library to your class definition with the fields order exactly matching those of the CSV file columns).

If you need more fine-grained control over the CSV schema or the parsing options, use the more low-level forSchema static factory method based on the Jackson library utilities:


 Function<CsvMapper, CsvSchema> schemaGenerator =
          mapper -> mapper.schemaFor(SomePojo.class)
                          .withColumnSeparator('|');
 CsvReaderFormat<SomePojo> csvFormat =
          CsvReaderFormat.forSchema(() -> new CsvMapper(), schemaGenerator, TypeInformation.of(SomePojo.class));
 FileSource<SomePojo> source =
         FileSource.forRecordStreamFormat(csvFormat, Path.fromLocalFile(filesPath)).build();
 
See Also:
  • Field Summary

    Fields inherited from interface org.apache.flink.connector.file.src.reader.StreamFormat

    FETCH_IO_SIZE
  • Method Summary

    Modifier and Type
    Method
    Description
    org.apache.flink.connector.file.src.reader.StreamFormat.Reader<T>
    createReader(org.apache.flink.configuration.Configuration config, org.apache.flink.core.fs.FSDataInputStream stream)
     
    static <T> CsvReaderFormat<T>
    forPojo(Class<T> pojoType)
    Builds a new CsvReaderFormat for reading CSV files mapped to the provided POJO class definition.
    static <T> CsvReaderFormat<T>
    forSchema(org.apache.flink.shaded.jackson2.com.fasterxml.jackson.dataformat.csv.CsvSchema schema, org.apache.flink.api.common.typeinfo.TypeInformation<T> typeInformation)
    Builds a new CsvReaderFormat using a CsvSchema.
    static <T> CsvReaderFormat<T>
    forSchema(org.apache.flink.util.function.SerializableSupplier<org.apache.flink.shaded.jackson2.com.fasterxml.jackson.dataformat.csv.CsvMapper> mapperFactory, org.apache.flink.util.function.SerializableFunction<org.apache.flink.shaded.jackson2.com.fasterxml.jackson.dataformat.csv.CsvMapper,org.apache.flink.shaded.jackson2.com.fasterxml.jackson.dataformat.csv.CsvSchema> schemaGenerator, org.apache.flink.api.common.typeinfo.TypeInformation<T> typeInformation)
    Builds a new CsvReaderFormat using a CsvSchema generator and CsvMapper factory.
    org.apache.flink.api.common.typeinfo.TypeInformation<T>
     
    Returns a new CsvReaderFormat configured to ignore all parsing errors.

    Methods inherited from class org.apache.flink.connector.file.src.reader.SimpleStreamFormat

    createReader, isSplittable, restoreReader

    Methods inherited from class java.lang.Object

    clone, equals, finalize, getClass, hashCode, notify, notifyAll, toString, wait, wait, wait
  • Method Details

    • forSchema

      public static <T> CsvReaderFormat<T> forSchema(org.apache.flink.shaded.jackson2.com.fasterxml.jackson.dataformat.csv.CsvSchema schema, org.apache.flink.api.common.typeinfo.TypeInformation<T> typeInformation)
      Builds a new CsvReaderFormat using a CsvSchema.
      Type Parameters:
      T - The type of the returned elements.
      Parameters:
      schema - The Jackson CSV schema configured for parsing specific CSV files.
      typeInformation - The Flink type descriptor of the returned elements.
    • forSchema

      public static <T> CsvReaderFormat<T> forSchema(org.apache.flink.util.function.SerializableSupplier<org.apache.flink.shaded.jackson2.com.fasterxml.jackson.dataformat.csv.CsvMapper> mapperFactory, org.apache.flink.util.function.SerializableFunction<org.apache.flink.shaded.jackson2.com.fasterxml.jackson.dataformat.csv.CsvMapper,org.apache.flink.shaded.jackson2.com.fasterxml.jackson.dataformat.csv.CsvSchema> schemaGenerator, org.apache.flink.api.common.typeinfo.TypeInformation<T> typeInformation)
      Builds a new CsvReaderFormat using a CsvSchema generator and CsvMapper factory.
      Type Parameters:
      T - The type of the returned elements.
      Parameters:
      mapperFactory - The factory creating the CsvMapper.
      schemaGenerator - A generator that creates and configures the Jackson CSV schema for parsing specific CSV files, from a mapper created by the mapper factory.
      typeInformation - The Flink type descriptor of the returned elements.
    • forPojo

      public static <T> CsvReaderFormat<T> forPojo(Class<T> pojoType)
      Builds a new CsvReaderFormat for reading CSV files mapped to the provided POJO class definition. Produced reader uses default mapper and schema settings, use forSchema if you need customizations.
      Type Parameters:
      T - The type of the returned elements.
      Parameters:
      pojoType - The type class of the POJO.
    • withIgnoreParseErrors

      public CsvReaderFormat<T> withIgnoreParseErrors()
      Returns a new CsvReaderFormat configured to ignore all parsing errors. All the other options directly carried over from the subject of the method call.
    • createReader

      public org.apache.flink.connector.file.src.reader.StreamFormat.Reader<T> createReader(org.apache.flink.configuration.Configuration config, org.apache.flink.core.fs.FSDataInputStream stream) throws IOException
      Specified by:
      createReader in class org.apache.flink.connector.file.src.reader.SimpleStreamFormat<T>
      Throws:
      IOException
    • getProducedType

      public org.apache.flink.api.common.typeinfo.TypeInformation<T> getProducedType()
      Specified by:
      getProducedType in interface org.apache.flink.api.java.typeutils.ResultTypeQueryable<T>
      Specified by:
      getProducedType in interface org.apache.flink.connector.file.src.reader.StreamFormat<T>
      Specified by:
      getProducedType in class org.apache.flink.connector.file.src.reader.SimpleStreamFormat<T>