27
27
import com .wepay .kafka .connect .bigquery .convert .logicaltype .LogicalTypeConverter ;
28
28
import com .wepay .kafka .connect .bigquery .exception .ConversionConnectException ;
29
29
30
- import org .apache .kafka .connect .data .Field ;
31
30
import org .apache .kafka .connect .data .Schema ;
32
31
33
- import java .util .ArrayList ;
34
32
import java .util .HashMap ;
35
- import java .util .LinkedList ;
36
33
import java .util .List ;
37
34
import java .util .Map ;
35
+ import java .util .Optional ;
36
+ import java .util .stream .Collectors ;
37
+ import java .util .stream .Stream ;
38
38
39
39
/**
40
40
* Class for converting from {@link Schema Kafka Connect Schemas} to
@@ -102,25 +102,26 @@ public com.google.cloud.bigquery.Schema convertSchema(Schema kafkaConnectSchema)
102
102
ConversionConnectException ("Top-level Kafka Connect schema must be of type 'struct'" );
103
103
}
104
104
105
- List <com .google .cloud .bigquery .Field > fields = new LinkedList <>();
106
-
107
- for (Field kafkaConnectField : kafkaConnectSchema .fields ()) {
108
- com .google .cloud .bigquery .Field bigQuerySchemaField =
109
- convertField (kafkaConnectField .schema (), kafkaConnectField .name ()).build ();
110
- fields .add (bigQuerySchemaField );
111
- }
105
+ List <com .google .cloud .bigquery .Field > fields = kafkaConnectSchema .fields ().stream ()
106
+ .flatMap (kafkaConnectField ->
107
+ convertField (kafkaConnectField .schema (), kafkaConnectField .name ())
108
+ .map (Stream ::of )
109
+ .orElse (Stream .empty ())
110
+ )
111
+ .map (com .google .cloud .bigquery .Field .Builder ::build )
112
+ .collect (Collectors .toList ());
112
113
113
114
return com .google .cloud .bigquery .Schema .of (fields );
114
115
}
115
116
116
- private com .google .cloud .bigquery .Field .Builder convertField (Schema kafkaConnectSchema ,
117
- String fieldName ) {
118
- com .google .cloud .bigquery .Field .Builder result ;
117
+ private Optional < com .google .cloud .bigquery .Field .Builder > convertField (Schema kafkaConnectSchema ,
118
+ String fieldName ) {
119
+ Optional < com .google .cloud .bigquery .Field .Builder > result ;
119
120
Schema .Type kafkaConnectSchemaType = kafkaConnectSchema .type ();
120
121
if (LogicalConverterRegistry .isRegisteredLogicalType (kafkaConnectSchema .name ())) {
121
- result = convertLogical (kafkaConnectSchema , fieldName );
122
+ result = Optional . of ( convertLogical (kafkaConnectSchema , fieldName ) );
122
123
} else if (PRIMITIVE_TYPE_MAP .containsKey (kafkaConnectSchemaType )) {
123
- result = convertPrimitive (kafkaConnectSchema , fieldName );
124
+ result = Optional . of ( convertPrimitive (kafkaConnectSchema , fieldName ) );
124
125
} else {
125
126
switch (kafkaConnectSchemaType ) {
126
127
case STRUCT :
@@ -138,11 +139,13 @@ private com.google.cloud.bigquery.Field.Builder convertField(Schema kafkaConnect
138
139
);
139
140
}
140
141
}
141
- setNullability (kafkaConnectSchema , result );
142
- if (kafkaConnectSchema .doc () != null ) {
143
- result .setDescription (kafkaConnectSchema .doc ());
144
- }
145
- return result ;
142
+ return result .map (res -> {
143
+ setNullability (kafkaConnectSchema , res );
144
+ if (kafkaConnectSchema .doc () != null ) {
145
+ res .setDescription (kafkaConnectSchema .doc ());
146
+ }
147
+ return res ;
148
+ });
146
149
}
147
150
148
151
private void setNullability (Schema kafkaConnectSchema ,
@@ -160,48 +163,54 @@ private void setNullability(Schema kafkaConnectSchema,
160
163
}
161
164
}
162
165
163
- private com .google .cloud .bigquery .Field .Builder convertStruct (Schema kafkaConnectSchema ,
164
- String fieldName ) {
165
- List <com .google .cloud .bigquery .Field > bigQueryRecordFields = new ArrayList <>();
166
-
167
- for (Field kafkaConnectField : kafkaConnectSchema .fields ()) {
168
- com .google .cloud .bigquery .Field .Builder bigQueryRecordFieldBuilder =
169
- convertField (kafkaConnectField .schema (), kafkaConnectField .name ());
170
- bigQueryRecordFields .add (bigQueryRecordFieldBuilder .build ());
166
+ private Optional <com .google .cloud .bigquery .Field .Builder > convertStruct (Schema kafkaConnectSchema ,
167
+ String fieldName ) {
168
+ List <com .google .cloud .bigquery .Field > bigQueryRecordFields = kafkaConnectSchema .fields ()
169
+ .stream ()
170
+ .flatMap (kafkaConnectField ->
171
+ convertField (kafkaConnectField .schema (), kafkaConnectField .name ())
172
+ .map (Stream ::of )
173
+ .orElse (Stream .empty ())
174
+ )
175
+ .map (com .google .cloud .bigquery .Field .Builder ::build )
176
+ .collect (Collectors .toList ());
177
+ if (bigQueryRecordFields .isEmpty ()) {
178
+ return Optional .empty ();
171
179
}
172
180
173
181
FieldList fieldList = FieldList .of (bigQueryRecordFields );
174
182
175
- return com .google .cloud .bigquery .Field .newBuilder (fieldName ,
176
- LegacySQLTypeName .RECORD ,
177
- fieldList );
183
+ return Optional . of ( com .google .cloud .bigquery .Field .newBuilder (fieldName ,
184
+ LegacySQLTypeName .RECORD ,
185
+ fieldList ) );
178
186
}
179
187
180
- private com .google .cloud .bigquery .Field .Builder convertArray (Schema kafkaConnectSchema ,
181
- String fieldName ) {
188
+ private Optional < com .google .cloud .bigquery .Field .Builder > convertArray (Schema kafkaConnectSchema ,
189
+ String fieldName ) {
182
190
Schema elementSchema = kafkaConnectSchema .valueSchema ();
183
- com .google .cloud .bigquery .Field .Builder elementFieldBuilder =
184
- convertField (elementSchema , fieldName );
185
- return elementFieldBuilder .setMode (com .google .cloud .bigquery .Field .Mode .REPEATED );
191
+ return convertField (elementSchema , fieldName )
192
+ .map (builder -> builder .setMode (com .google .cloud .bigquery .Field .Mode .REPEATED ));
186
193
}
187
194
188
- private com .google .cloud .bigquery .Field .Builder convertMap (Schema kafkaConnectSchema ,
189
- String fieldName ) {
195
+ private Optional < com .google .cloud .bigquery .Field .Builder > convertMap (Schema kafkaConnectSchema ,
196
+ String fieldName ) {
190
197
Schema keySchema = kafkaConnectSchema .keySchema ();
191
198
Schema valueSchema = kafkaConnectSchema .valueSchema ();
192
199
193
- com .google .cloud .bigquery .Field keyField =
194
- convertField (keySchema , MAP_KEY_FIELD_NAME ).build ();
195
- com .google .cloud .bigquery .Field valueField =
196
- convertField (valueSchema , MAP_VALUE_FIELD_NAME ).build ();
197
-
198
- com .google .cloud .bigquery .Field .Builder bigQueryRecordBuilder =
199
- com .google .cloud .bigquery .Field .newBuilder (fieldName ,
200
- LegacySQLTypeName .RECORD ,
201
- keyField ,
202
- valueField );
203
-
204
- return bigQueryRecordBuilder .setMode (com .google .cloud .bigquery .Field .Mode .REPEATED );
200
+ Optional <com .google .cloud .bigquery .Field > maybeKeyField = convertField (keySchema , MAP_KEY_FIELD_NAME )
201
+ .map (com .google .cloud .bigquery .Field .Builder ::build );
202
+ Optional <com .google .cloud .bigquery .Field > maybeValueField = convertField (valueSchema , MAP_VALUE_FIELD_NAME )
203
+ .map (com .google .cloud .bigquery .Field .Builder ::build );
204
+
205
+ return maybeKeyField .flatMap (keyField ->
206
+ maybeValueField .map (valueField ->
207
+ com .google .cloud .bigquery .Field .newBuilder (fieldName ,
208
+ LegacySQLTypeName .RECORD ,
209
+ keyField ,
210
+ valueField )
211
+ .setMode (com .google .cloud .bigquery .Field .Mode .REPEATED )
212
+ )
213
+ );
205
214
}
206
215
207
216
private com .google .cloud .bigquery .Field .Builder convertPrimitive (Schema kafkaConnectSchema ,
0 commit comments