Skip to content

Commit 979a670

Browse files
Tu-maimesTu-maimes
and
Tu-maimes
authored
[BUG] Fixed avro format support for storing null (#8424)
Co-authored-by: Tu-maimes <[email protected]>
1 parent 3ac977c commit 979a670

File tree

3 files changed

+89
-13
lines changed

3 files changed

+89
-13
lines changed

seatunnel-formats/seatunnel-format-avro/src/main/java/org/apache/seatunnel/format/avro/AvroToRowConverter.java

+4
Original file line numberDiff line numberDiff line change
@@ -38,6 +38,7 @@
3838
import java.util.HashMap;
3939
import java.util.List;
4040
import java.util.Map;
41+
import java.util.Objects;
4142

4243
public class AvroToRowConverter implements Serializable {
4344

@@ -82,6 +83,9 @@ public SeaTunnelRow converter(GenericRecord record, SeaTunnelRowType rowType) {
8283
}
8384

8485
private Object convertField(SeaTunnelDataType<?> dataType, Object val) {
86+
if (Objects.isNull(val)) {
87+
return null;
88+
}
8589
switch (dataType.getSqlType()) {
8690
case STRING:
8791
return val.toString();

seatunnel-formats/seatunnel-format-avro/src/main/java/org/apache/seatunnel/format/avro/SeaTunnelRowTypeToAvroSchemaConverter.java

+30-13
Original file line numberDiff line numberDiff line change
@@ -58,28 +58,39 @@ private static Schema seaTunnelDataType2AvroDataType(
5858

5959
switch (seaTunnelDataType.getSqlType()) {
6060
case STRING:
61-
return Schema.create(Schema.Type.STRING);
61+
return Schema.createUnion(
62+
Schema.create(Schema.Type.NULL), Schema.create(Schema.Type.STRING));
6263
case BYTES:
63-
return Schema.create(Schema.Type.BYTES);
64+
return Schema.createUnion(
65+
Schema.create(Schema.Type.NULL), Schema.create(Schema.Type.BYTES));
6466
case TINYINT:
6567
case SMALLINT:
6668
case INT:
67-
return Schema.create(Schema.Type.INT);
69+
return Schema.createUnion(
70+
Schema.create(Schema.Type.NULL), Schema.create(Schema.Type.INT));
6871
case BIGINT:
69-
return Schema.create(Schema.Type.LONG);
72+
return Schema.createUnion(
73+
Schema.create(Schema.Type.NULL), Schema.create(Schema.Type.LONG));
7074
case FLOAT:
71-
return Schema.create(Schema.Type.FLOAT);
75+
return Schema.createUnion(
76+
Schema.create(Schema.Type.NULL), Schema.create(Schema.Type.FLOAT));
7277
case DOUBLE:
73-
return Schema.create(Schema.Type.DOUBLE);
78+
return Schema.createUnion(
79+
Schema.create(Schema.Type.NULL), Schema.create(Schema.Type.DOUBLE));
7480
case BOOLEAN:
75-
return Schema.create(Schema.Type.BOOLEAN);
81+
return Schema.createUnion(
82+
Schema.create(Schema.Type.NULL), Schema.create(Schema.Type.BOOLEAN));
7683
case MAP:
7784
SeaTunnelDataType<?> valueType = ((MapType<?, ?>) seaTunnelDataType).getValueType();
78-
return Schema.createMap(seaTunnelDataType2AvroDataType(fieldName, valueType));
85+
return Schema.createUnion(
86+
Schema.create(Schema.Type.NULL),
87+
Schema.createMap(seaTunnelDataType2AvroDataType(fieldName, valueType)));
7988
case ARRAY:
8089
SeaTunnelDataType<?> elementType =
8190
((ArrayType<?, ?>) seaTunnelDataType).getElementType();
82-
return Schema.createArray(seaTunnelDataType2AvroDataType(fieldName, elementType));
91+
return Schema.createUnion(
92+
Schema.create(Schema.Type.NULL),
93+
Schema.createArray(seaTunnelDataType2AvroDataType(fieldName, elementType)));
8394
case ROW:
8495
SeaTunnelDataType<?>[] fieldTypes =
8596
((SeaTunnelRowType) seaTunnelDataType).getFieldTypes();
@@ -93,12 +104,18 @@ private static Schema seaTunnelDataType2AvroDataType(
93104
int precision = ((DecimalType) seaTunnelDataType).getPrecision();
94105
int scale = ((DecimalType) seaTunnelDataType).getScale();
95106
LogicalTypes.Decimal decimal = LogicalTypes.decimal(precision, scale);
96-
return decimal.addToSchema(Schema.create(Schema.Type.BYTES));
107+
return Schema.createUnion(
108+
Schema.create(Schema.Type.NULL),
109+
decimal.addToSchema(Schema.create(Schema.Type.BYTES)));
97110
case TIMESTAMP:
98-
return LogicalTypes.localTimestampMillis()
99-
.addToSchema(Schema.create(Schema.Type.LONG));
111+
return Schema.createUnion(
112+
Schema.create(Schema.Type.NULL),
113+
LogicalTypes.localTimestampMillis()
114+
.addToSchema(Schema.create(Schema.Type.LONG)));
100115
case DATE:
101-
return LogicalTypes.date().addToSchema(Schema.create(Schema.Type.INT));
116+
return Schema.createUnion(
117+
Schema.create(Schema.Type.NULL),
118+
LogicalTypes.date().addToSchema(Schema.create(Schema.Type.INT)));
102119
case NULL:
103120
return Schema.create(Schema.Type.NULL);
104121
default:

seatunnel-formats/seatunnel-format-avro/src/test/java/org/apache/seatunnel/format/avro/AvroSerializationSchemaTest.java

+55
Original file line numberDiff line numberDiff line change
@@ -177,4 +177,59 @@ public void testSerialization() throws IOException {
177177
LocalDateTime localDateTime1 = (LocalDateTime) subRow.getField(13);
178178
Assertions.assertEquals(localDateTime1.compareTo(localDateTime), 0);
179179
}
180+
181+
private SeaTunnelRow buildSeaTunnelRowValueNull() {
182+
SeaTunnelRow subSeaTunnelRow = new SeaTunnelRow(14);
183+
subSeaTunnelRow.setField(0, null);
184+
subSeaTunnelRow.setField(1, null);
185+
subSeaTunnelRow.setField(2, null);
186+
subSeaTunnelRow.setField(3, null);
187+
subSeaTunnelRow.setField(4, null);
188+
subSeaTunnelRow.setField(5, null);
189+
subSeaTunnelRow.setField(6, null);
190+
subSeaTunnelRow.setField(7, null);
191+
subSeaTunnelRow.setField(8, null);
192+
subSeaTunnelRow.setField(9, null);
193+
subSeaTunnelRow.setField(10, null);
194+
subSeaTunnelRow.setField(11, null);
195+
subSeaTunnelRow.setField(12, null);
196+
subSeaTunnelRow.setField(13, null);
197+
198+
SeaTunnelRow seaTunnelRow = new SeaTunnelRow(15);
199+
seaTunnelRow.setField(0, null);
200+
seaTunnelRow.setField(1, null);
201+
seaTunnelRow.setField(2, null);
202+
seaTunnelRow.setField(3, null);
203+
seaTunnelRow.setField(4, null);
204+
seaTunnelRow.setField(5, null);
205+
seaTunnelRow.setField(6, null);
206+
seaTunnelRow.setField(7, null);
207+
seaTunnelRow.setField(8, null);
208+
seaTunnelRow.setField(9, null);
209+
seaTunnelRow.setField(10, null);
210+
seaTunnelRow.setField(11, null);
211+
seaTunnelRow.setField(12, null);
212+
seaTunnelRow.setField(13, null);
213+
seaTunnelRow.setField(14, subSeaTunnelRow);
214+
return seaTunnelRow;
215+
}
216+
217+
@Test
218+
public void testSerializationValueNull() throws IOException {
219+
SeaTunnelRowType rowType = buildSeaTunnelRowType();
220+
CatalogTable catalogTable = CatalogTableUtil.getCatalogTable("", "", "", "test", rowType);
221+
SeaTunnelRow seaTunnelRow = buildSeaTunnelRowValueNull();
222+
AvroSerializationSchema serializationSchema = new AvroSerializationSchema(rowType);
223+
byte[] bytes = serializationSchema.serialize(seaTunnelRow);
224+
AvroDeserializationSchema deserializationSchema =
225+
new AvroDeserializationSchema(catalogTable);
226+
SeaTunnelRow deserialize = deserializationSchema.deserialize(bytes);
227+
String[] strArray1 = (String[]) seaTunnelRow.getField(1);
228+
String[] strArray2 = (String[]) deserialize.getField(1);
229+
Assertions.assertArrayEquals(strArray1, strArray2);
230+
SeaTunnelRow subRow = (SeaTunnelRow) deserialize.getField(14);
231+
Assertions.assertEquals(subRow.getField(9), null);
232+
Assertions.assertEquals(subRow.getField(12), null);
233+
Assertions.assertEquals(subRow.getField(13), null);
234+
}
180235
}

0 commit comments

Comments
 (0)