7
7
import com .google .protobuf .Descriptors .OneofDescriptor ;
8
8
import com .google .protobuf .GeneratedMessageV3 ;
9
9
import com .google .protobuf .InvalidProtocolBufferException ;
10
+ import com .google .protobuf .MapEntry ;
10
11
import com .google .protobuf .Message ;
11
12
import org .apache .kafka .connect .data .Date ;
12
13
import org .apache .kafka .connect .data .Decimal ;
28
29
import java .util .HashMap ;
29
30
import java .util .List ;
30
31
import java .util .Map ;
32
+ import java .util .stream .Collectors ;
31
33
import com .google .protobuf .util .Timestamps ;
32
34
33
35
import static com .google .protobuf .Descriptors .FieldDescriptor .Type .BOOL ;
@@ -47,6 +49,7 @@ class ProtobufData {
47
49
private final Method newBuilder ;
48
50
private final Schema schema ;
49
51
private final String legacyName ;
52
+ private final boolean useConnectSchemaMap ;
50
53
public static final Descriptors .FieldDescriptor .Type [] PROTO_TYPES_WITH_DEFAULTS = new Descriptors .FieldDescriptor .Type [] { INT32 , INT64 , SINT32 , SINT64 , FLOAT , DOUBLE , BOOL , STRING , BYTES , ENUM };
51
54
private HashMap <String , String > connectProtoNameMap = new HashMap <String , String >();
52
55
@@ -87,7 +90,12 @@ private String getProtoFieldName(String descriptorForTypeName, String connectFie
87
90
}
88
91
89
92
ProtobufData (Class <? extends com .google .protobuf .GeneratedMessageV3 > clazz , String legacyName ) {
93
+ this (clazz , legacyName , false );
94
+ }
95
+
96
+ ProtobufData (Class <? extends com .google .protobuf .GeneratedMessageV3 > clazz , String legacyName , boolean useConnectSchemaMap ) {
90
97
this .legacyName = legacyName ;
98
+ this .useConnectSchemaMap = useConnectSchemaMap ;
91
99
92
100
try {
93
101
this .newBuilder = clazz .getDeclaredMethod ("newBuilder" );
@@ -194,6 +202,14 @@ private Schema toConnectSchema(Descriptors.FieldDescriptor descriptor) {
194
202
builder = Date .builder ();
195
203
break ;
196
204
}
205
+
206
+ if (shouldConvertToConnectSchemaMap (descriptor )) {
207
+ FieldDescriptor keyFieldDescriptor = descriptor .getMessageType ().findFieldByName ("key" );
208
+ FieldDescriptor valueFieldDescriptor = descriptor .getMessageType ().findFieldByName ("value" );
209
+ builder = SchemaBuilder .map (toConnectSchema (keyFieldDescriptor ), toConnectSchema (valueFieldDescriptor ));
210
+ break ;
211
+ }
212
+
197
213
String jsonName = descriptor .getJsonName ();
198
214
builder = SchemaBuilder .struct ().name (jsonName .substring (0 , 1 ).toUpperCase () + jsonName .substring (1 ));
199
215
for (Descriptors .FieldDescriptor fieldDescriptor : descriptor .getMessageType ().getFields ()) {
@@ -210,7 +226,7 @@ private Schema toConnectSchema(Descriptors.FieldDescriptor descriptor) {
210
226
builder .optional ();
211
227
Schema schema = builder .build ();
212
228
213
- if (descriptor .isRepeated ()) {
229
+ if (descriptor .isRepeated () && ! shouldConvertToConnectSchemaMap ( descriptor ) ) {
214
230
final SchemaBuilder arrayBuilder = SchemaBuilder .array (schema );
215
231
arrayBuilder .optional ();
216
232
schema = arrayBuilder .build ();
@@ -219,6 +235,10 @@ private Schema toConnectSchema(Descriptors.FieldDescriptor descriptor) {
219
235
return schema ;
220
236
}
221
237
238
+ private boolean shouldConvertToConnectSchemaMap (FieldDescriptor descriptor ) {
239
+ return useConnectSchemaMap && descriptor .isMapField ();
240
+ }
241
+
222
242
private boolean isProtobufTimestamp (Schema schema ) {
223
243
return Timestamp .SCHEMA .name ().equals (schema .name ());
224
244
}
@@ -340,6 +360,15 @@ Object toConnectData(Schema schema, Object value) {
340
360
break ;
341
361
}
342
362
363
+ case MAP : {
364
+ Collection <MapEntry > original = (Collection <MapEntry >) value ;
365
+ converted = original .stream ().collect (Collectors .toMap (
366
+ entry -> toConnectData (schema .keySchema (), entry .getKey ()),
367
+ entry -> toConnectData (schema .valueSchema (), entry .getValue ())
368
+ ));
369
+ break ;
370
+ }
371
+
343
372
case STRUCT : {
344
373
final Message message = (Message ) value ; // Validate type
345
374
0 commit comments