1818package org .apache .beam .sdk .io .csv ;
1919
2020import com .google .auto .value .AutoValue ;
21+ import java .io .Serializable ;
2122import java .util .HashMap ;
2223import java .util .Map ;
2324import java .util .Optional ;
25+ import org .apache .beam .sdk .coders .Coder ;
2426import org .apache .beam .sdk .schemas .Schema ;
2527import org .apache .beam .sdk .transforms .DoFn ;
2628import org .apache .beam .sdk .transforms .PTransform ;
2729import org .apache .beam .sdk .transforms .ParDo ;
2830import org .apache .beam .sdk .transforms .SerializableFunction ;
2931import org .apache .beam .sdk .transforms .errorhandling .BadRecord ;
3032import org .apache .beam .sdk .values .PCollection ;
33+ import org .apache .beam .sdk .values .Row ;
3134import org .apache .commons .csv .CSVFormat ;
3235
3336/** Stores parameters needed for CSV record parsing. */
3437@ AutoValue
35- abstract class CsvIOParseConfiguration {
38+ abstract class CsvIOParseConfiguration < T > implements Serializable {
3639
3740 /** A Dead Letter Queue that returns potential errors with {@link BadRecord}. */
3841 final PTransform <PCollection <BadRecord >, PCollection <BadRecord >> errorHandlerTransform =
3942 new BadRecordOutput ();
4043
41- static Builder builder () {
42- return new AutoValue_CsvIOParseConfiguration .Builder ();
44+ static < T > Builder < T > builder () {
45+ return new AutoValue_CsvIOParseConfiguration .Builder <> ();
4346 }
4447
4548 /** The expected {@link CSVFormat} of the parsed CSV record. */
@@ -51,20 +54,30 @@ static Builder builder() {
5154 /** A map of the {@link Schema.Field#getName()} to the custom CSV processing lambda. */
5255 abstract Map <String , SerializableFunction <String , Object >> getCustomProcessingMap ();
5356
57+ /** The expected {@link Coder} of the target type. */
58+ abstract Coder <T > getCoder ();
59+
60+ /** A {@link SerializableFunction} that converts from Row to the target type. */
61+ abstract SerializableFunction <Row , T > getFromRowFn ();
62+
5463 @ AutoValue .Builder
55- abstract static class Builder {
56- abstract Builder setCsvFormat (CSVFormat csvFormat );
64+ abstract static class Builder < T > implements Serializable {
65+ abstract Builder < T > setCsvFormat (CSVFormat csvFormat );
5766
58- abstract Builder setSchema (Schema schema );
67+ abstract Builder < T > setSchema (Schema schema );
5968
60- abstract Builder setCustomProcessingMap (
69+ abstract Builder < T > setCustomProcessingMap (
6170 Map <String , SerializableFunction <String , Object >> customProcessingMap );
6271
72+ abstract Builder <T > setCoder (Coder <T > coder );
73+
74+ abstract Builder <T > setFromRowFn (SerializableFunction <Row , T > fromRowFn );
75+
6376 abstract Optional <Map <String , SerializableFunction <String , Object >>> getCustomProcessingMap ();
6477
65- abstract CsvIOParseConfiguration autoBuild ();
78+ abstract CsvIOParseConfiguration < T > autoBuild ();
6679
67- final CsvIOParseConfiguration build () {
80+ final CsvIOParseConfiguration < T > build () {
6881 if (!getCustomProcessingMap ().isPresent ()) {
6982 setCustomProcessingMap (new HashMap <>());
7083 }
0 commit comments