18
18
package org .apache .beam .sdk .io .csv ;
19
19
20
20
import com .google .auto .value .AutoValue ;
21
+ import java .io .Serializable ;
21
22
import java .util .HashMap ;
22
23
import java .util .Map ;
23
24
import java .util .Optional ;
25
+ import org .apache .beam .sdk .coders .Coder ;
24
26
import org .apache .beam .sdk .schemas .Schema ;
25
27
import org .apache .beam .sdk .transforms .DoFn ;
26
28
import org .apache .beam .sdk .transforms .PTransform ;
27
29
import org .apache .beam .sdk .transforms .ParDo ;
28
30
import org .apache .beam .sdk .transforms .SerializableFunction ;
29
31
import org .apache .beam .sdk .transforms .errorhandling .BadRecord ;
30
32
import org .apache .beam .sdk .values .PCollection ;
33
+ import org .apache .beam .sdk .values .Row ;
31
34
import org .apache .commons .csv .CSVFormat ;
32
35
33
36
/** Stores parameters needed for CSV record parsing. */
34
37
@ AutoValue
35
- abstract class CsvIOParseConfiguration {
38
+ abstract class CsvIOParseConfiguration < T > implements Serializable {
36
39
37
40
/** A Dead Letter Queue that returns potential errors with {@link BadRecord}. */
38
41
final PTransform <PCollection <BadRecord >, PCollection <BadRecord >> errorHandlerTransform =
39
42
new BadRecordOutput ();
40
43
41
- static Builder builder () {
42
- return new AutoValue_CsvIOParseConfiguration .Builder ();
44
+ static < T > Builder < T > builder () {
45
+ return new AutoValue_CsvIOParseConfiguration .Builder <> ();
43
46
}
44
47
45
48
/** The expected {@link CSVFormat} of the parsed CSV record. */
@@ -51,20 +54,30 @@ static Builder builder() {
51
54
/** A map of the {@link Schema.Field#getName()} to the custom CSV processing lambda. */
52
55
abstract Map <String , SerializableFunction <String , Object >> getCustomProcessingMap ();
53
56
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
+
54
63
@ 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 );
57
66
58
- abstract Builder setSchema (Schema schema );
67
+ abstract Builder < T > setSchema (Schema schema );
59
68
60
- abstract Builder setCustomProcessingMap (
69
+ abstract Builder < T > setCustomProcessingMap (
61
70
Map <String , SerializableFunction <String , Object >> customProcessingMap );
62
71
72
+ abstract Builder <T > setCoder (Coder <T > coder );
73
+
74
+ abstract Builder <T > setFromRowFn (SerializableFunction <Row , T > fromRowFn );
75
+
63
76
abstract Optional <Map <String , SerializableFunction <String , Object >>> getCustomProcessingMap ();
64
77
65
- abstract CsvIOParseConfiguration autoBuild ();
78
+ abstract CsvIOParseConfiguration < T > autoBuild ();
66
79
67
- final CsvIOParseConfiguration build () {
80
+ final CsvIOParseConfiguration < T > build () {
68
81
if (!getCustomProcessingMap ().isPresent ()) {
69
82
setCustomProcessingMap (new HashMap <>());
70
83
}
0 commit comments