-
Notifications
You must be signed in to change notification settings - Fork 1.1k
/
AvroData.java
2642 lines (2416 loc) · 105 KB
/
AvroData.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
/*
* Copyright 2019 Confluent Inc.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package io.confluent.connect.avro;
import com.fasterxml.jackson.databind.JsonNode;
import com.fasterxml.jackson.databind.node.ArrayNode;
import com.fasterxml.jackson.databind.node.IntNode;
import com.fasterxml.jackson.databind.node.JsonNodeFactory;
import com.fasterxml.jackson.databind.node.NullNode;
import com.fasterxml.jackson.databind.node.ObjectNode;
import io.confluent.connect.schema.ConnectEnum;
import io.confluent.connect.schema.ConnectUnion;
import io.confluent.kafka.schemaregistry.utils.BoundedConcurrentHashMap;
import java.io.UnsupportedEncodingException;
import java.net.URLEncoder;
import java.util.regex.Pattern;
import org.apache.avro.AvroTypeException;
import org.apache.avro.JsonProperties;
import org.apache.avro.generic.GenericData;
import org.apache.avro.generic.GenericData.EnumSymbol;
import org.apache.avro.generic.GenericEnumSymbol;
import org.apache.avro.generic.GenericFixed;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.GenericRecordBuilder;
import org.apache.avro.generic.IndexedRecord;
import org.apache.avro.util.internal.JacksonUtils;
import org.apache.kafka.connect.data.ConnectSchema;
import org.apache.kafka.connect.data.Date;
import org.apache.kafka.connect.data.Decimal;
import org.apache.kafka.connect.data.Field;
import org.apache.kafka.connect.data.Schema;
import org.apache.kafka.connect.data.SchemaAndValue;
import org.apache.kafka.connect.data.SchemaBuilder;
import org.apache.kafka.connect.data.Struct;
import org.apache.kafka.connect.data.Time;
import org.apache.kafka.connect.data.Timestamp;
import org.apache.kafka.connect.errors.DataException;
import java.io.IOException;
import java.math.BigDecimal;
import java.nio.ByteBuffer;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.HashMap;
import java.util.HashSet;
import java.util.IdentityHashMap;
import java.util.Iterator;
import java.util.List;
import java.util.ListIterator;
import java.util.Map;
import java.util.Objects;
import java.util.Set;
import io.confluent.kafka.schemaregistry.avro.AvroSchema;
import io.confluent.kafka.serializers.NonRecordContainer;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Utilities for converting between our runtime data format and Avro, and (de)serializing that data.
*/
public class AvroData {
private static final Logger log = LoggerFactory.getLogger(AvroData.class);
public static final String NAMESPACE = "io.confluent.connect.avro";
// Avro does not permit empty schema names, which might be the ideal default since we also are
// not permitted to simply omit the name. Instead, make it very clear where the default is
// coming from.
public static final String DEFAULT_SCHEMA_NAME = "ConnectDefault";
public static final String DEFAULT_SCHEMA_FULL_NAME = NAMESPACE + "." + DEFAULT_SCHEMA_NAME;
public static final String MAP_ENTRY_TYPE_NAME = "MapEntry";
public static final String KEY_FIELD = "key";
public static final String VALUE_FIELD = "value";
public static final String CONNECT_NAME_PROP = "connect.name";
public static final String CONNECT_DOC_PROP = "connect.doc";
public static final String CONNECT_RECORD_DOC_PROP = "connect.record.doc";
public static final String CONNECT_ENUM_DOC_PROP = "connect.enum.doc";
public static final String CONNECT_VERSION_PROP = "connect.version";
public static final String CONNECT_DEFAULT_VALUE_PROP = "connect.default";
public static final String CONNECT_PARAMETERS_PROP = "connect.parameters";
public static final String CONNECT_INTERNAL_TYPE_NAME = "connect.internal.type";
public static final String AVRO_RECORD_DOC_PROP = NAMESPACE + ".record.doc";
public static final String AVRO_ENUM_DOC_PREFIX_PROP = NAMESPACE + ".enum.doc.";
public static final String AVRO_FIELD_DOC_PREFIX_PROP = NAMESPACE + ".field.doc.";
//This property is used to determine whether a default value in the Connect schema originated
//from an Avro field default
public static final String AVRO_FIELD_DEFAULT_FLAG_PROP = NAMESPACE + ".field.default";
public static final String AVRO_ENUM_DEFAULT_PREFIX_PROP = NAMESPACE + ".enum.default.";
public static final String CONNECT_TYPE_PROP = "connect.type";
public static final String CONNECT_TYPE_INT8 = "int8";
public static final String CONNECT_TYPE_INT16 = "int16";
public static final String AVRO_TYPE_UNION = NAMESPACE + ".Union";
public static final String AVRO_TYPE_ENUM = NAMESPACE + ".Enum";
public static final String AVRO_TYPE_ANYTHING = NAMESPACE + ".Anything";
public static final String GENERALIZED_TYPE_UNION = ConnectUnion.LOGICAL_PARAMETER;
public static final String GENERALIZED_TYPE_ENUM = ConnectEnum.LOGICAL_PARAMETER;
public static final String GENERALIZED_TYPE_UNION_PREFIX = "connect_union_";
public static final String GENERALIZED_TYPE_UNION_FIELD_PREFIX =
GENERALIZED_TYPE_UNION_PREFIX + "field_";
private static final Map<String, Schema.Type> NON_AVRO_TYPES_BY_TYPE_CODE = new HashMap<>();
private static Pattern NAME_START_CHAR = Pattern.compile("^[A-Za-z_]");
private static Pattern NAME_INVALID_CHARS = Pattern.compile("[^A-Za-z0-9_]");
static {
NON_AVRO_TYPES_BY_TYPE_CODE.put(CONNECT_TYPE_INT8, Schema.Type.INT8);
NON_AVRO_TYPES_BY_TYPE_CODE.put(CONNECT_TYPE_INT16, Schema.Type.INT16);
}
// Avro Java object types used by Connect schema types
private static final Map<Schema.Type, List<Class>> SIMPLE_AVRO_SCHEMA_TYPES = new HashMap<>();
static {
SIMPLE_AVRO_SCHEMA_TYPES.put(Schema.Type.INT32, Arrays.asList((Class) Integer.class));
SIMPLE_AVRO_SCHEMA_TYPES.put(Schema.Type.INT64, Arrays.asList((Class) Long.class));
SIMPLE_AVRO_SCHEMA_TYPES.put(Schema.Type.FLOAT32, Arrays.asList((Class) Float.class));
SIMPLE_AVRO_SCHEMA_TYPES.put(Schema.Type.FLOAT64, Arrays.asList((Class) Double.class));
SIMPLE_AVRO_SCHEMA_TYPES.put(Schema.Type.BOOLEAN, Arrays.asList((Class) Boolean.class));
SIMPLE_AVRO_SCHEMA_TYPES.put(Schema.Type.STRING, Arrays.asList((Class) CharSequence.class));
SIMPLE_AVRO_SCHEMA_TYPES.put(
Schema.Type.BYTES,
Arrays.asList((Class) ByteBuffer.class, (Class) byte[].class, (Class) GenericFixed.class));
SIMPLE_AVRO_SCHEMA_TYPES.put(Schema.Type.ARRAY, Arrays.asList((Class) Collection.class));
SIMPLE_AVRO_SCHEMA_TYPES.put(Schema.Type.MAP, Arrays.asList((Class) Map.class));
}
private static final Map<Schema.Type, org.apache.avro.Schema.Type> CONNECT_TYPES_TO_AVRO_TYPES
= new HashMap<>();
static {
CONNECT_TYPES_TO_AVRO_TYPES.put(Schema.Type.INT32, org.apache.avro.Schema.Type.INT);
CONNECT_TYPES_TO_AVRO_TYPES.put(Schema.Type.INT64, org.apache.avro.Schema.Type.LONG);
CONNECT_TYPES_TO_AVRO_TYPES.put(Schema.Type.FLOAT32, org.apache.avro.Schema.Type.FLOAT);
CONNECT_TYPES_TO_AVRO_TYPES.put(Schema.Type.FLOAT64, org.apache.avro.Schema.Type.DOUBLE);
CONNECT_TYPES_TO_AVRO_TYPES.put(Schema.Type.BOOLEAN, org.apache.avro.Schema.Type.BOOLEAN);
CONNECT_TYPES_TO_AVRO_TYPES.put(Schema.Type.STRING, org.apache.avro.Schema.Type.STRING);
CONNECT_TYPES_TO_AVRO_TYPES.put(Schema.Type.BYTES, org.apache.avro.Schema.Type.BYTES);
CONNECT_TYPES_TO_AVRO_TYPES.put(Schema.Type.ARRAY, org.apache.avro.Schema.Type.ARRAY);
CONNECT_TYPES_TO_AVRO_TYPES.put(Schema.Type.MAP, org.apache.avro.Schema.Type.MAP);
}
private static final String ANYTHING_SCHEMA_BOOLEAN_FIELD = "boolean";
private static final String ANYTHING_SCHEMA_BYTES_FIELD = "bytes";
private static final String ANYTHING_SCHEMA_DOUBLE_FIELD = "double";
private static final String ANYTHING_SCHEMA_FLOAT_FIELD = "float";
private static final String ANYTHING_SCHEMA_INT_FIELD = "int";
private static final String ANYTHING_SCHEMA_LONG_FIELD = "long";
private static final String ANYTHING_SCHEMA_STRING_FIELD = "string";
private static final String ANYTHING_SCHEMA_ARRAY_FIELD = "array";
private static final String ANYTHING_SCHEMA_MAP_FIELD = "map";
public static final org.apache.avro.Schema ANYTHING_SCHEMA_MAP_ELEMENT;
public static final org.apache.avro.Schema ANYTHING_SCHEMA;
private static final org.apache.avro.Schema
NULL_AVRO_SCHEMA =
org.apache.avro.Schema.create(org.apache.avro.Schema.Type.NULL);
static {
// Intuitively this should be a union schema. However, unions can't be named in Avro and this
// is a self-referencing type, so we need to use a format in which we can name the entire schema
ANYTHING_SCHEMA =
org.apache.avro.SchemaBuilder.record(AVRO_TYPE_ANYTHING).namespace(NAMESPACE).fields()
.optionalBoolean(ANYTHING_SCHEMA_BOOLEAN_FIELD)
.optionalBytes(ANYTHING_SCHEMA_BYTES_FIELD)
.optionalDouble(ANYTHING_SCHEMA_DOUBLE_FIELD)
.optionalFloat(ANYTHING_SCHEMA_FLOAT_FIELD)
.optionalInt(ANYTHING_SCHEMA_INT_FIELD)
.optionalLong(ANYTHING_SCHEMA_LONG_FIELD)
.optionalString(ANYTHING_SCHEMA_STRING_FIELD)
.name(ANYTHING_SCHEMA_ARRAY_FIELD).type().optional().array()
.items().type(AVRO_TYPE_ANYTHING)
.name(ANYTHING_SCHEMA_MAP_FIELD).type().optional().array()
.items().record(MAP_ENTRY_TYPE_NAME).namespace(NAMESPACE).fields()
.name(KEY_FIELD).type(AVRO_TYPE_ANYTHING).noDefault()
.name(VALUE_FIELD).type(AVRO_TYPE_ANYTHING).noDefault()
.endRecord()
.endRecord();
// This is convenient to have extracted; we can't define it before ANYTHING_SCHEMA because it
// uses ANYTHING_SCHEMA in its definition.
ANYTHING_SCHEMA_MAP_ELEMENT = ANYTHING_SCHEMA.getField("map").schema()
.getTypes().get(1) // The "map" field is optional, get the schema from the union type
.getElementType();
}
// Convert values in Connect form into their logical types. These logical converters are
// discovered by logical type names specified in the field
private static final HashMap<String, LogicalTypeConverter> TO_CONNECT_LOGICAL_CONVERTERS
= new HashMap<>();
static {
TO_CONNECT_LOGICAL_CONVERTERS.put(Decimal.LOGICAL_NAME, new LogicalTypeConverter() {
@Override
public Object convert(Schema schema, Object value) {
if (value instanceof byte[]) {
return Decimal.toLogical(schema, (byte[]) value);
} else if (value instanceof ByteBuffer) {
return Decimal.toLogical(schema, ((ByteBuffer) value).array());
}
throw new DataException(
"Invalid type for Decimal, underlying representation should be bytes but was "
+ value.getClass());
}
});
TO_CONNECT_LOGICAL_CONVERTERS.put(Date.LOGICAL_NAME, new LogicalTypeConverter() {
@Override
public Object convert(Schema schema, Object value) {
if (!(value instanceof Integer)) {
throw new DataException(
"Invalid type for Date, underlying representation should be int32 but was "
+ value.getClass());
}
return Date.toLogical(schema, (int) value);
}
});
TO_CONNECT_LOGICAL_CONVERTERS.put(Time.LOGICAL_NAME, new LogicalTypeConverter() {
@Override
public Object convert(Schema schema, Object value) {
if (!(value instanceof Integer)) {
throw new DataException(
"Invalid type for Time, underlying representation should be int32 but was "
+ value.getClass());
}
return Time.toLogical(schema, (int) value);
}
});
TO_CONNECT_LOGICAL_CONVERTERS.put(Timestamp.LOGICAL_NAME, new LogicalTypeConverter() {
@Override
public Object convert(Schema schema, Object value) {
if (!(value instanceof Long)) {
throw new DataException(
"Invalid type for Timestamp, underlying representation should be int64 but was "
+ value.getClass());
}
return Timestamp.toLogical(schema, (long) value);
}
});
}
static final String AVRO_PROP = "avro";
static final String AVRO_LOGICAL_TYPE_PROP = "logicalType";
static final String AVRO_LOGICAL_TIMESTAMP_MILLIS = "timestamp-millis";
static final String AVRO_LOGICAL_TIME_MILLIS = "time-millis";
static final String AVRO_LOGICAL_DATE = "date";
static final String AVRO_LOGICAL_DECIMAL = "decimal";
static final String AVRO_LOGICAL_DECIMAL_SCALE_PROP = "scale";
static final String AVRO_LOGICAL_DECIMAL_PRECISION_PROP = "precision";
static final String CONNECT_AVRO_FIXED_SIZE_PROP = "connect.fixed.size";
static final String CONNECT_AVRO_DECIMAL_PRECISION_PROP = "connect.decimal.precision";
static final Integer CONNECT_AVRO_DECIMAL_PRECISION_DEFAULT = 64;
private static final HashMap<String, LogicalTypeConverter> TO_AVRO_LOGICAL_CONVERTERS
= new HashMap<>();
static {
TO_AVRO_LOGICAL_CONVERTERS.put(Decimal.LOGICAL_NAME, new LogicalTypeConverter() {
@Override
public Object convert(Schema schema, Object value) {
if (!(value instanceof BigDecimal)) {
throw new DataException(
"Invalid type for Decimal, expected BigDecimal but was " + value.getClass());
}
return Decimal.fromLogical(schema, (BigDecimal) value);
}
});
TO_AVRO_LOGICAL_CONVERTERS.put(Date.LOGICAL_NAME, new LogicalTypeConverter() {
@Override
public Object convert(Schema schema, Object value) {
if (!(value instanceof java.util.Date)) {
throw new DataException(
"Invalid type for Date, expected Date but was " + value.getClass());
}
return Date.fromLogical(schema, (java.util.Date) value);
}
});
TO_AVRO_LOGICAL_CONVERTERS.put(Time.LOGICAL_NAME, new LogicalTypeConverter() {
@Override
public Object convert(Schema schema, Object value) {
if (!(value instanceof java.util.Date)) {
throw new DataException(
"Invalid type for Time, expected Date but was " + value.getClass());
}
return Time.fromLogical(schema, (java.util.Date) value);
}
});
TO_AVRO_LOGICAL_CONVERTERS.put(Timestamp.LOGICAL_NAME, new LogicalTypeConverter() {
@Override
public Object convert(Schema schema, Object value) {
if (!(value instanceof java.util.Date)) {
throw new DataException(
"Invalid type for Timestamp, expected Date but was " + value.getClass());
}
return Timestamp.fromLogical(schema, (java.util.Date) value);
}
});
}
private int unionIndex = 0;
private Map<Schema, org.apache.avro.Schema> fromConnectSchemaCache;
private Map<AvroSchema, Schema> toConnectSchemaCache;
private boolean connectMetaData;
private boolean generalizedSumTypeSupport;
private boolean ignoreDefaultForNullables;
private boolean enhancedSchemaSupport;
private boolean scrubInvalidNames;
private boolean discardTypeDocDefault;
private boolean allowOptionalMapKey;
public AvroData(int cacheSize) {
this(new AvroDataConfig.Builder()
.with(AvroDataConfig.SCHEMAS_CACHE_SIZE_CONFIG, cacheSize)
.build());
}
public AvroData(AvroDataConfig avroDataConfig) {
fromConnectSchemaCache = new BoundedConcurrentHashMap<>(avroDataConfig.schemaCacheSize());
toConnectSchemaCache = new BoundedConcurrentHashMap<>(avroDataConfig.schemaCacheSize());
this.connectMetaData = avroDataConfig.isConnectMetaData();
this.generalizedSumTypeSupport = avroDataConfig.isGeneralizedSumTypeSupport();
this.ignoreDefaultForNullables = avroDataConfig.ignoreDefaultForNullables();
this.enhancedSchemaSupport = avroDataConfig.isEnhancedAvroSchemaSupport();
this.scrubInvalidNames = avroDataConfig.isScrubInvalidNames();
this.discardTypeDocDefault = avroDataConfig.isDiscardTypeDocDefault();
this.allowOptionalMapKey = avroDataConfig.isAllowOptionalMapKeys();
}
/**
* Convert this object, in Connect data format, into an Avro object.
*/
public Object fromConnectData(Schema schema, Object value) {
org.apache.avro.Schema avroSchema = fromConnectSchema(schema);
return fromConnectData(schema, avroSchema, value);
}
protected Object fromConnectData(Schema schema, org.apache.avro.Schema avroSchema, Object value) {
return fromConnectData(schema, avroSchema, value, true, false);
}
/**
* Convert from Connect data format to Avro. This version assumes the Avro schema has already
* been converted and makes the use of NonRecordContainer optional
*
* @param schema the Connect schema
* @param avroSchema the corresponding
* @param logicalValue the Connect data to convert, which may be a value for
* a logical type
* @param requireContainer if true, wrap primitives, maps, and arrays in a
* NonRecordContainer before returning them
* @param requireSchemalessContainerNull if true, use a container representation of null because
* this is part of struct/array/map and we cannot represent
* nulls as true null because Anything cannot be a union
* type; otherwise, this is a top-level value and can return
* null
* @return the converted data
*/
private Object fromConnectData(
Schema schema,
org.apache.avro.Schema avroSchema,
Object logicalValue,
boolean requireContainer,
boolean requireSchemalessContainerNull
) {
Schema.Type schemaType = schema != null
? schema.type()
: schemaTypeForSchemalessJavaType(logicalValue);
if (schemaType == null) {
// Schemaless null data since schema is null and we got a null schema type from the value
if (requireSchemalessContainerNull) {
return new GenericRecordBuilder(ANYTHING_SCHEMA).build();
} else {
return null;
}
}
validateSchemaValue(schema, logicalValue);
if (logicalValue == null) {
// But if this is schemaless, we may not be able to return null directly
if (schema == null && requireSchemalessContainerNull) {
return new GenericRecordBuilder(ANYTHING_SCHEMA).build();
} else {
return null;
}
}
// If this is a logical type, convert it from the convenient Java type to the underlying
// serializeable format
Object value = logicalValue;
if (schema != null && schema.name() != null) {
LogicalTypeConverter logicalConverter = TO_AVRO_LOGICAL_CONVERTERS.get(schema.name());
if (logicalConverter != null) {
value = logicalConverter.convert(schema, logicalValue);
}
}
try {
switch (schemaType) {
case INT8: {
Byte byteValue = (Byte) value; // Check for correct type
Integer convertedByteValue = byteValue == null ? null : byteValue.intValue();
return maybeAddContainer(
avroSchema,
maybeWrapSchemaless(schema, convertedByteValue, ANYTHING_SCHEMA_INT_FIELD),
requireContainer);
}
case INT16: {
Short shortValue = (Short) value; // Check for correct type
Integer convertedShortValue = shortValue == null ? null : shortValue.intValue();
return maybeAddContainer(
avroSchema,
maybeWrapSchemaless(schema, convertedShortValue, ANYTHING_SCHEMA_INT_FIELD),
requireContainer);
}
case INT32:
Integer intValue = (Integer) value; // Check for correct type
return maybeAddContainer(
avroSchema,
maybeWrapSchemaless(schema, value, ANYTHING_SCHEMA_INT_FIELD),
requireContainer);
case INT64:
Long longValue = (Long) value; // Check for correct type
return maybeAddContainer(
avroSchema,
maybeWrapSchemaless(schema, value, ANYTHING_SCHEMA_LONG_FIELD),
requireContainer);
case FLOAT32:
Float floatValue = (Float) value; // Check for correct type
return maybeAddContainer(
avroSchema,
maybeWrapSchemaless(schema, value, ANYTHING_SCHEMA_FLOAT_FIELD),
requireContainer);
case FLOAT64:
Double doubleValue = (Double) value; // Check for correct type
return maybeAddContainer(
avroSchema,
maybeWrapSchemaless(schema, value, ANYTHING_SCHEMA_DOUBLE_FIELD),
requireContainer);
case BOOLEAN:
Boolean boolValue = (Boolean) value; // Check for correct type
return maybeAddContainer(
avroSchema,
maybeWrapSchemaless(schema, value, ANYTHING_SCHEMA_BOOLEAN_FIELD),
requireContainer);
case STRING:
if (generalizedSumTypeSupport && ConnectEnum.isEnum(schema)) {
String enumSchemaName = schema.parameters().get(GENERALIZED_TYPE_ENUM);
value = enumSymbol(avroSchema, value, enumSchemaName);
} else if (enhancedSchemaSupport && schema != null && schema.parameters() != null
&& schema.parameters().containsKey(AVRO_TYPE_ENUM)) {
String enumSchemaName = schema.parameters().get(AVRO_TYPE_ENUM);
value = enumSymbol(avroSchema, value, enumSchemaName);
} else {
String stringValue = (String) value; // Check for correct type
}
return maybeAddContainer(
avroSchema,
maybeWrapSchemaless(schema, value, ANYTHING_SCHEMA_STRING_FIELD),
requireContainer);
case BYTES: {
value = value instanceof byte[] ? ByteBuffer.wrap((byte[]) value) :
(ByteBuffer) value;
if (schema != null && isFixedSchema(schema)) {
int size = Integer.parseInt(schema.parameters().get(CONNECT_AVRO_FIXED_SIZE_PROP));
org.apache.avro.Schema fixedSchema = null;
if (avroSchema.getType() == org.apache.avro.Schema.Type.UNION) {
int index = 0;
for (org.apache.avro.Schema memberSchema : avroSchema.getTypes()) {
if (memberSchema.getType() == org.apache.avro.Schema.Type.FIXED
&& memberSchema.getFixedSize() == size
&& unionMemberFieldName(memberSchema, index)
.equals(unionMemberFieldName(schema, index))) {
fixedSchema = memberSchema;
}
index++;
}
if (fixedSchema == null) {
throw new DataException("Fixed size " + size + " not in union " + avroSchema);
}
} else {
fixedSchema = avroSchema;
}
value = new GenericData.Fixed(fixedSchema, ((ByteBuffer)value).array());
}
return maybeAddContainer(
avroSchema,
maybeWrapSchemaless(schema, value, ANYTHING_SCHEMA_BYTES_FIELD),
requireContainer);
}
case ARRAY: {
Collection<Object> list = (Collection<Object>) value;
// TODO most types don't need a new converted object since types pass through
List<Object> converted = new ArrayList<>(list.size());
Schema elementSchema = schema != null ? schema.valueSchema() : null;
org.apache.avro.Schema underlyingAvroSchema = avroSchemaForUnderlyingTypeIfOptional(
schema, avroSchema, scrubInvalidNames);
org.apache.avro.Schema elementAvroSchema =
schema != null ? underlyingAvroSchema.getElementType() : ANYTHING_SCHEMA;
for (Object val : list) {
converted.add(
fromConnectData(
elementSchema,
elementAvroSchema,
val,
false,
true
)
);
}
return maybeAddContainer(
avroSchema,
maybeWrapSchemaless(schema, converted, ANYTHING_SCHEMA_ARRAY_FIELD),
requireContainer);
}
case MAP: {
Map<Object, Object> map = (Map<Object, Object>) value;
org.apache.avro.Schema underlyingAvroSchema;
if (schema != null && schema.keySchema().type() == Schema.Type.STRING
&& (!schema.keySchema().isOptional() || allowOptionalMapKey)) {
// TODO most types don't need a new converted object since types pass through
underlyingAvroSchema = avroSchemaForUnderlyingTypeIfOptional(
schema, avroSchema, scrubInvalidNames);
Map<String, Object> converted = new HashMap<>();
for (Map.Entry<Object, Object> entry : map.entrySet()) {
// Key is a String, no conversion needed
Object convertedValue = fromConnectData(schema.valueSchema(),
underlyingAvroSchema.getValueType(),
entry.getValue(), false, true
);
converted.put((String) entry.getKey(), convertedValue);
}
return maybeAddContainer(avroSchema, converted, requireContainer);
} else {
List<GenericRecord> converted = new ArrayList<>(map.size());
underlyingAvroSchema = avroSchemaForUnderlyingMapEntryType(schema, avroSchema);
org.apache.avro.Schema elementSchema =
schema != null
? underlyingAvroSchema.getElementType()
: ANYTHING_SCHEMA_MAP_ELEMENT;
org.apache.avro.Schema avroKeySchema = elementSchema.getField(KEY_FIELD).schema();
org.apache.avro.Schema avroValueSchema = elementSchema.getField(VALUE_FIELD).schema();
for (Map.Entry<Object, Object> entry : map.entrySet()) {
Object keyConverted = fromConnectData(schema != null ? schema.keySchema() : null,
avroKeySchema, entry.getKey(), false, true);
Object valueConverted = fromConnectData(schema != null ? schema.valueSchema() : null,
avroValueSchema, entry.getValue(), false,
true);
converted.add(
new GenericRecordBuilder(elementSchema)
.set(KEY_FIELD, keyConverted)
.set(VALUE_FIELD, valueConverted)
.build()
);
}
return maybeAddContainer(
avroSchema, maybeWrapSchemaless(schema, converted, ANYTHING_SCHEMA_MAP_FIELD),
requireContainer);
}
}
case STRUCT: {
Struct struct = (Struct) value;
if (!struct.schema().equals(schema)) {
throw new DataException("Mismatching struct schema");
}
//This handles the inverting of a union which is held as a struct, where each field is
// one of the union types.
if (isUnionSchema(schema)) {
for (Field field : schema.fields()) {
Object object = ignoreDefaultForNullables
? struct.getWithoutDefault(field.name()) : struct.get(field);
if (object != null) {
return fromConnectData(
field.schema(),
avroSchema,
object,
false,
true
);
}
}
return fromConnectData(schema, avroSchema, null, false, true);
} else {
org.apache.avro.Schema underlyingAvroSchema = avroSchemaForUnderlyingTypeIfOptional(
schema, avroSchema, scrubInvalidNames);
GenericRecordBuilder convertedBuilder = new GenericRecordBuilder(underlyingAvroSchema);
for (Field field : schema.fields()) {
String fieldName = scrubName(field.name(), scrubInvalidNames);
org.apache.avro.Schema.Field theField = underlyingAvroSchema.getField(fieldName);
org.apache.avro.Schema fieldAvroSchema = theField.schema();
Object fieldValue = ignoreDefaultForNullables
? struct.getWithoutDefault(field.name()) : struct.get(field);
convertedBuilder.set(
fieldName,
fromConnectData(field.schema(), fieldAvroSchema, fieldValue, false, true)
);
}
return convertedBuilder.build();
}
}
default:
throw new DataException("Unknown schema type: " + schema.type());
}
} catch (ClassCastException e) {
throw new DataException("Invalid type for " + schema.type() + ": " + value.getClass());
}
}
private EnumSymbol enumSymbol(
org.apache.avro.Schema avroSchema, Object value, String enumSchemaName) {
org.apache.avro.Schema enumSchema;
if (avroSchema.getType() == org.apache.avro.Schema.Type.UNION) {
int enumIndex = avroSchema.getIndexNamed(enumSchemaName);
enumSchema = avroSchema.getTypes().get(enumIndex);
} else {
enumSchema = avroSchema;
}
return new GenericData.EnumSymbol(enumSchema, (String) value);
}
/**
* MapEntry types in connect Schemas are represented as Arrays of record.
* Return the array type from the union instead of the union itself.
*/
private static org.apache.avro.Schema avroSchemaForUnderlyingMapEntryType(
Schema schema,
org.apache.avro.Schema avroSchema) {
if (schema != null && schema.isOptional()) {
if (avroSchema.getType() == org.apache.avro.Schema.Type.UNION) {
for (org.apache.avro.Schema typeSchema : avroSchema.getTypes()) {
if (!typeSchema.getType().equals(org.apache.avro.Schema.Type.NULL)
&& Schema.Type.ARRAY.getName().equals(typeSchema.getType().getName())) {
return typeSchema;
}
}
} else {
throw new DataException(
"An optional schema should have an Avro Union type, not "
+ schema.type());
}
}
return avroSchema;
}
private static boolean crossReferenceSchemaNames(final Schema schema,
final org.apache.avro.Schema avroSchema,
final boolean scrubInvalidNames) {
String fullName = scrubFullName(schema.name(), scrubInvalidNames);
return Objects.equals(avroSchema.getFullName(), fullName)
|| Objects.equals(avroSchema.getType().getName(), schema.type().getName())
|| (schema.name() == null && avroSchema.getFullName().startsWith(DEFAULT_SCHEMA_FULL_NAME));
}
/**
* Connect optional fields are represented as a unions (null & type) in Avro
* Return the Avro schema of the actual type in the Union (instead of the union itself)
*/
private static org.apache.avro.Schema avroSchemaForUnderlyingTypeIfOptional(
Schema schema, org.apache.avro.Schema avroSchema, boolean scrubInvalidNames) {
if (schema != null && schema.isOptional()) {
if (avroSchema.getType() == org.apache.avro.Schema.Type.UNION) {
for (org.apache.avro.Schema typeSchema : avroSchema
.getTypes()) {
if (!typeSchema.getType().equals(org.apache.avro.Schema.Type.NULL)
&& crossReferenceSchemaNames(schema, typeSchema, scrubInvalidNames)) {
return typeSchema;
}
}
} else {
throw new DataException(
"An optional schema should have an Avro Union type, not "
+ schema.type());
}
}
return avroSchema;
}
private static Schema.Type schemaTypeForSchemalessJavaType(Object value) {
if (value == null) {
return null;
} else if (value instanceof Byte) {
return Schema.Type.INT8;
} else if (value instanceof Short) {
return Schema.Type.INT16;
} else if (value instanceof Integer) {
return Schema.Type.INT32;
} else if (value instanceof Long) {
return Schema.Type.INT64;
} else if (value instanceof Float) {
return Schema.Type.FLOAT32;
} else if (value instanceof Double) {
return Schema.Type.FLOAT64;
} else if (value instanceof Boolean) {
return Schema.Type.BOOLEAN;
} else if (value instanceof String) {
return Schema.Type.STRING;
} else if (value instanceof Collection) {
return Schema.Type.ARRAY;
} else if (value instanceof Map) {
return Schema.Type.MAP;
} else {
throw new DataException("Unknown Java type for schemaless data: " + value.getClass());
}
}
private static Object maybeAddContainer(org.apache.avro.Schema avroSchema, Object value,
boolean wrap) {
return wrap ? new NonRecordContainer(avroSchema, value) : value;
}
private static Object maybeWrapSchemaless(Schema schema, Object value, String typeField) {
if (schema != null) {
return value;
}
GenericRecordBuilder builder = new GenericRecordBuilder(ANYTHING_SCHEMA);
if (value != null) {
builder.set(typeField, value);
}
return builder.build();
}
public org.apache.avro.Schema fromConnectSchema(Schema schema) {
return fromConnectSchema(schema, new HashMap<Schema, org.apache.avro.Schema>());
}
public org.apache.avro.Schema fromConnectSchema(Schema schema,
Map<Schema, org.apache.avro.Schema> schemaMap) {
if (schema == null) {
return ANYTHING_SCHEMA;
}
org.apache.avro.Schema cached = fromConnectSchemaCache.get(schema);
if (cached != null) {
return cached;
}
FromConnectContext fromConnectContext = new FromConnectContext(schemaMap);
org.apache.avro.Schema finalSchema = fromConnectSchema(schema, fromConnectContext, false);
fromConnectSchemaCache.put(schema, finalSchema);
return finalSchema;
}
/**
* SchemaMap is a map of already resolved internal schemas, this avoids type re-declaration if a
* type is reused, this actually blows up if you don't do this and have a type used in multiple
* places.
*
* <p>Also it only holds reference the non-optional schemas as technically an optional is
* actually a union of null and the non-opitonal, which if used in multiple places some optional
* some non-optional will cause error as you redefine type.
*
* <p>This is different to the global schema cache which is used to hold/cache fully resolved
* schemas used to avoid re-resolving when presented with the same source schema.
*/
public org.apache.avro.Schema fromConnectSchema(Schema schema,
FromConnectContext fromConnectContext,
boolean ignoreOptional) {
if (schema == null) {
return ANYTHING_SCHEMA;
}
if (!isUnionSchema(schema) && !schema.isOptional()) {
org.apache.avro.Schema cached = fromConnectContext.schemaMap.get(schema);
if (cached != null) {
return cached;
}
}
// Extra type annotation information for otherwise lossy conversions
String connectType = null;
final org.apache.avro.Schema baseSchema;
switch (schema.type()) {
case INT8:
connectType = CONNECT_TYPE_INT8;
baseSchema = org.apache.avro.SchemaBuilder.builder().intType();
break;
case INT16:
connectType = CONNECT_TYPE_INT16;
baseSchema = org.apache.avro.SchemaBuilder.builder().intType();
break;
case INT32:
baseSchema = org.apache.avro.SchemaBuilder.builder().intType();
break;
case INT64:
baseSchema = org.apache.avro.SchemaBuilder.builder().longType();
break;
case FLOAT32:
baseSchema = org.apache.avro.SchemaBuilder.builder().floatType();
break;
case FLOAT64:
baseSchema = org.apache.avro.SchemaBuilder.builder().doubleType();
break;
case BOOLEAN:
baseSchema = org.apache.avro.SchemaBuilder.builder().booleanType();
break;
case STRING:
if ((generalizedSumTypeSupport || enhancedSchemaSupport)
&& schema.parameters() != null
&& (schema.parameters().containsKey(GENERALIZED_TYPE_ENUM)
|| schema.parameters().containsKey(AVRO_TYPE_ENUM))) {
String paramName = generalizedSumTypeSupport ? GENERALIZED_TYPE_ENUM : AVRO_TYPE_ENUM;
List<String> symbols = new ArrayList<>();
for (Map.Entry<String, String> entry : schema.parameters().entrySet()) {
if (entry.getKey().startsWith(paramName + ".")) {
String enumSymbol = entry.getKey().substring(paramName.length() + 1);
symbols.add(enumSymbol);
}
}
Pair<String, String> names = getNameOrDefault(fromConnectContext, schema.name());
String name = names.getValue();
String enumName = schema.parameters().get(paramName);
String enumDoc = schema.parameters().get(AVRO_ENUM_DOC_PREFIX_PROP + name);
String enumDefault = schema.parameters().get(AVRO_ENUM_DEFAULT_PREFIX_PROP + name);
baseSchema = discardTypeDocDefault
? org.apache.avro.SchemaBuilder.builder().enumeration(enumName)
.doc(schema.parameters().get(CONNECT_ENUM_DOC_PROP))
.symbols(symbols.toArray(new String[symbols.size()]))
: org.apache.avro.SchemaBuilder.builder().enumeration(enumName)
.doc(enumDoc)
.defaultSymbol(enumDefault)
.symbols(symbols.toArray(new String[symbols.size()]));
} else {
baseSchema = org.apache.avro.SchemaBuilder.builder().stringType();
}
break;
case BYTES:
if (isFixedSchema(schema)) {
Pair<String, String> names = getNameOrDefault(fromConnectContext, schema.name());
String namespace = names.getKey();
String name = names.getValue();
baseSchema = org.apache.avro.SchemaBuilder.builder()
.fixed(name)
.namespace(namespace)
.size(Integer.parseInt(schema.parameters().get(CONNECT_AVRO_FIXED_SIZE_PROP)));
} else {
baseSchema = org.apache.avro.SchemaBuilder.builder().bytesType();
}
if (Decimal.LOGICAL_NAME.equalsIgnoreCase(schema.name())) {
int scale = Integer.parseInt(schema.parameters().get(Decimal.SCALE_FIELD));
baseSchema.addProp(AVRO_LOGICAL_DECIMAL_SCALE_PROP, new IntNode(scale));
if (schema.parameters().containsKey(CONNECT_AVRO_DECIMAL_PRECISION_PROP)) {
String precisionValue = schema.parameters().get(CONNECT_AVRO_DECIMAL_PRECISION_PROP);
int precision = Integer.parseInt(precisionValue);
baseSchema.addProp(AVRO_LOGICAL_DECIMAL_PRECISION_PROP, new IntNode(precision));
} else {
baseSchema
.addProp(AVRO_LOGICAL_DECIMAL_PRECISION_PROP,
new IntNode(CONNECT_AVRO_DECIMAL_PRECISION_DEFAULT));
}
}
break;
case ARRAY:
baseSchema = org.apache.avro.SchemaBuilder.builder().array()
.items(fromConnectSchemaWithCycle(schema.valueSchema(), fromConnectContext, false));
break;
case MAP:
// Avro only supports string keys, so we match the representation when possible, but
// otherwise fall back on a record representation
if (schema.keySchema().type() == Schema.Type.STRING
&& (!schema.keySchema().isOptional() || allowOptionalMapKey)) {
baseSchema = org.apache.avro.SchemaBuilder.builder().map().values(
fromConnectSchemaWithCycle(schema.valueSchema(), fromConnectContext, false));
} else {
// Special record name indicates format
List<org.apache.avro.Schema.Field> fields = new ArrayList<>();
final org.apache.avro.Schema mapSchema;
if (schema.name() == null) {
mapSchema = org.apache.avro.Schema.createRecord(
MAP_ENTRY_TYPE_NAME,
null,
NAMESPACE,
false
);
} else {
Pair<String, String> names = getNameOrDefault(fromConnectContext, schema.name());
String namespace = names.getKey();
String name = names.getValue();
mapSchema = org.apache.avro.Schema.createRecord(name, null, namespace, false);
mapSchema.addProp(CONNECT_INTERNAL_TYPE_NAME, MAP_ENTRY_TYPE_NAME);
}
addAvroRecordField(
fields,
KEY_FIELD,
schema.keySchema(),
null,
fromConnectContext);
addAvroRecordField(
fields,
VALUE_FIELD,
schema.valueSchema(),
null,
fromConnectContext);
mapSchema.setFields(fields);
baseSchema = org.apache.avro.Schema.createArray(mapSchema);
}
break;
case STRUCT:
if (isUnionSchema(schema)) {
List<org.apache.avro.Schema> unionSchemas = new ArrayList<>();
if (schema.isOptional()) {
unionSchemas.add(org.apache.avro.SchemaBuilder.builder().nullType());
}
for (Field field : schema.fields()) {
unionSchemas.add(
fromConnectSchemaWithCycle(nonOptional(field.schema()), fromConnectContext, true));
}
baseSchema = org.apache.avro.Schema.createUnion(unionSchemas);
} else if (schema.isOptional()) {
List<org.apache.avro.Schema> unionSchemas = new ArrayList<>();
unionSchemas.add(org.apache.avro.SchemaBuilder.builder().nullType());
unionSchemas.add(
fromConnectSchemaWithCycle(nonOptional(schema), fromConnectContext, false));
baseSchema = org.apache.avro.Schema.createUnion(unionSchemas);
} else {
Pair<String, String> names = getNameOrDefault(fromConnectContext, schema.name());
String namespace = names.getKey();
String name = names.getValue();
String doc = schema.parameters() != null
? schema.parameters()
.get(discardTypeDocDefault ? CONNECT_RECORD_DOC_PROP : AVRO_RECORD_DOC_PROP)
: null;
baseSchema = org.apache.avro.Schema.createRecord(name, doc, namespace, false);
if (schema.name() != null) {
fromConnectContext.cycleReferences.put(schema.name(), baseSchema);
}
List<org.apache.avro.Schema.Field> fields = new ArrayList<>();
for (Field field : schema.fields()) {
String fieldName = scrubName(field.name());
String fieldDoc = null;
if (!discardTypeDocDefault && schema.parameters() != null) {
fieldDoc = schema.parameters().get(AVRO_FIELD_DOC_PREFIX_PROP + field.name());
}
addAvroRecordField(fields, fieldName, field.schema(), fieldDoc, fromConnectContext);
}
baseSchema.setFields(fields);
}
break;
default:
throw new DataException("Unknown schema type: " + schema.type());
}
org.apache.avro.Schema finalSchema = baseSchema;
if (!baseSchema.getType().equals(org.apache.avro.Schema.Type.UNION)) {
if (connectMetaData) {
if (schema.doc() != null) {
baseSchema.addProp(CONNECT_DOC_PROP, schema.doc());
}
if (schema.version() != null) {
baseSchema.addProp(CONNECT_VERSION_PROP,
JsonNodeFactory.instance.numberNode(schema.version()));
}
if (schema.parameters() != null) {
JsonNode params = parametersFromConnect(schema.parameters());
if (!params.isEmpty()) {
baseSchema.addProp(CONNECT_PARAMETERS_PROP, params);
}
}