wuchong commented on a change in pull request #12133:
URL: https://github.com/apache/flink/pull/12133#discussion_r426121118
##########
File path:
flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/typeutils/AvroSchemaConverter.java
##########
@@ -157,4 +166,118 @@ private AvroSchemaConverter() {
}
throw new IllegalArgumentException("Unsupported Avro type '" +
schema.getType() + "'.");
}
+
+ /**
+ * Converts Flink SQL {@link LogicalType} (can be nested) into an Avro
schema.
+ *
+ * @param logicalType logical type
+ * @return Avro's {@link Schema} matching this logical type.
+ */
+ public static Schema convertToSchema(LogicalType logicalType) {
+ return convertToSchema(logicalType, 0);
+ }
+
+ private static Schema convertToSchema(LogicalType logicalType, int
rowTypeCounter) {
+ switch (logicalType.getTypeRoot()) {
+ case NULL:
+ return SchemaBuilder.builder().nullType();
+ case BOOLEAN:
+ return
getNullableBuilder(logicalType).booleanType();
+ case INTEGER:
+ return
getNullableBuilder(logicalType).intType();
+ case BIGINT:
+ return
getNullableBuilder(logicalType).longType();
+ case FLOAT:
+ return
getNullableBuilder(logicalType).floatType();
+ case DOUBLE:
+ return
getNullableBuilder(logicalType).doubleType();
+ case CHAR:
+ case VARCHAR:
+ return
getNullableBuilder(logicalType).stringType();
+ case BINARY:
+ case VARBINARY:
+ return
getNullableBuilder(logicalType).bytesType();
+ case TIMESTAMP_WITHOUT_TIME_ZONE:
+ // use long to represents Timestamp
+ final TimestampType timestampType =
(TimestampType) logicalType;
+ int precision = timestampType.getPrecision();
+ org.apache.avro.LogicalType avroLogicalType;
+ if (precision == 3) {
Review comment:
`precision <= 3`?
##########
File path:
flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/typeutils/AvroSchemaConverter.java
##########
@@ -157,4 +166,118 @@ private AvroSchemaConverter() {
}
throw new IllegalArgumentException("Unsupported Avro type '" +
schema.getType() + "'.");
}
+
+ /**
+ * Converts Flink SQL {@link LogicalType} (can be nested) into an Avro
schema.
+ *
+ * @param logicalType logical type
+ * @return Avro's {@link Schema} matching this logical type.
+ */
+ public static Schema convertToSchema(LogicalType logicalType) {
+ return convertToSchema(logicalType, 0);
+ }
+
+ private static Schema convertToSchema(LogicalType logicalType, int
rowTypeCounter) {
+ switch (logicalType.getTypeRoot()) {
+ case NULL:
+ return SchemaBuilder.builder().nullType();
+ case BOOLEAN:
+ return
getNullableBuilder(logicalType).booleanType();
+ case INTEGER:
+ return
getNullableBuilder(logicalType).intType();
+ case BIGINT:
+ return
getNullableBuilder(logicalType).longType();
+ case FLOAT:
+ return
getNullableBuilder(logicalType).floatType();
+ case DOUBLE:
+ return
getNullableBuilder(logicalType).doubleType();
+ case CHAR:
+ case VARCHAR:
+ return
getNullableBuilder(logicalType).stringType();
+ case BINARY:
+ case VARBINARY:
+ return
getNullableBuilder(logicalType).bytesType();
+ case TIMESTAMP_WITHOUT_TIME_ZONE:
+ // use long to represents Timestamp
+ final TimestampType timestampType =
(TimestampType) logicalType;
+ int precision = timestampType.getPrecision();
+ org.apache.avro.LogicalType avroLogicalType;
+ if (precision == 3) {
+ avroLogicalType =
LogicalTypes.timestampMillis();
+ } else if (precision == 9) {
Review comment:
`precision <= 6`?
##########
File path:
flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/typeutils/AvroSchemaConverter.java
##########
@@ -157,4 +166,118 @@ private AvroSchemaConverter() {
}
throw new IllegalArgumentException("Unsupported Avro type '" +
schema.getType() + "'.");
}
+
+ /**
+ * Converts Flink SQL {@link LogicalType} (can be nested) into an Avro
schema.
+ *
+ * @param logicalType logical type
+ * @return Avro's {@link Schema} matching this logical type.
+ */
+ public static Schema convertToSchema(LogicalType logicalType) {
+ return convertToSchema(logicalType, 0);
+ }
+
+ private static Schema convertToSchema(LogicalType logicalType, int
rowTypeCounter) {
+ switch (logicalType.getTypeRoot()) {
+ case NULL:
+ return SchemaBuilder.builder().nullType();
+ case BOOLEAN:
+ return
getNullableBuilder(logicalType).booleanType();
+ case INTEGER:
+ return
getNullableBuilder(logicalType).intType();
+ case BIGINT:
+ return
getNullableBuilder(logicalType).longType();
+ case FLOAT:
+ return
getNullableBuilder(logicalType).floatType();
+ case DOUBLE:
+ return
getNullableBuilder(logicalType).doubleType();
+ case CHAR:
+ case VARCHAR:
+ return
getNullableBuilder(logicalType).stringType();
+ case BINARY:
+ case VARBINARY:
+ return
getNullableBuilder(logicalType).bytesType();
+ case TIMESTAMP_WITHOUT_TIME_ZONE:
+ // use long to represents Timestamp
+ final TimestampType timestampType =
(TimestampType) logicalType;
+ int precision = timestampType.getPrecision();
+ org.apache.avro.LogicalType avroLogicalType;
+ if (precision == 3) {
+ avroLogicalType =
LogicalTypes.timestampMillis();
+ } else if (precision == 9) {
+ avroLogicalType =
LogicalTypes.timestampMicros();
+ } else {
+ throw new
IllegalArgumentException("Avro Timestamp does not support Timestamp with
precision: " +
+ precision +
+ ", it only supports precision
of 3 or 9.");
+ }
+ return
avroLogicalType.addToSchema(SchemaBuilder.builder().longType());
+ case DATE:
+ // use int to represents Date
+ return
LogicalTypes.date().addToSchema(SchemaBuilder.builder().intType());
+ case TIME_WITHOUT_TIME_ZONE:
+ // use int to represents Time, we only support
millisecond when deserialization
+ return
LogicalTypes.timeMillis().addToSchema(SchemaBuilder.builder().intType());
+ case DECIMAL:
+ DecimalType decimalType = (DecimalType)
logicalType;
+ // store BigDecimal as byte[]
+ return LogicalTypes
+ .decimal(decimalType.getPrecision(),
decimalType.getScale())
+
.addToSchema(SchemaBuilder.builder().bytesType());
+ case ROW:
+ RowType rowType = (RowType) logicalType;
+ List<String> fieldNames =
rowType.getFieldNames();
+ // we have to make sure the record name is
different in a Schema
+ SchemaBuilder.FieldAssembler<Schema> builder =
SchemaBuilder
+ .builder()
+ .record("row_" + rowTypeCounter)
+ .fields();
+ rowTypeCounter++;
+ for (int i = 0; i < rowType.getFieldCount();
i++) {
+ builder = builder
+ .name(fieldNames.get(i))
+
.type(convertToSchema(rowType.getTypeAt(i), rowTypeCounter))
+ .noDefault();
+ }
+ return builder.endRecord();
+ case MAP:
+ MapType mapType = (MapType) logicalType;
+ if (!hasFamily(mapType.getKeyType(),
LogicalTypeFamily.CHARACTER_STRING)) {
+ throw new IllegalArgumentException(
+ "Avro assumes map keys are
strings, " + mapType.getKeyType() +
+ " type as key type is
not supported.");
+ }
+ return SchemaBuilder
+ .builder()
+ .nullable() // will be UNION of Array
and null
Review comment:
`will be UNION of map and null`?
##########
File path:
flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/AvroRowDataDeserializationSchema.java
##########
@@ -0,0 +1,371 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You 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 org.apache.flink.formats.avro;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.formats.avro.typeutils.AvroSchemaConverter;
+import org.apache.flink.formats.avro.utils.MutableByteArrayInputStream;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.DecimalType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.LogicalTypeFamily;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.logical.utils.LogicalTypeChecks;
+import org.apache.flink.table.types.logical.utils.LogicalTypeUtils;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericFixed;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.avro.io.DatumReader;
+import org.apache.avro.io.Decoder;
+import org.apache.avro.io.DecoderFactory;
+import org.apache.avro.specific.SpecificDatumReader;
+import org.joda.time.DateTime;
+import org.joda.time.DateTimeFieldType;
+import org.joda.time.LocalDate;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.lang.reflect.Array;
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+
+/**
+ * Deserialization schema from Avro bytes to {@link RowData}.
+ *
+ * <p>Deserializes the <code>byte[]</code> messages into (nested) Flink
RowData. It converts Avro types
+ * into types that are compatible with Flink's Table & SQL API.
+ *
+ * <p>Projects with Avro records containing logical date/time types need to
add a JodaTime
+ * dependency.
+ *
+ * <p>Note: Changes in this class need to be kept in sync with the
corresponding runtime
+ * class {@link AvroRowDataSerializationSchema} and schema converter {@link
AvroSchemaConverter}.
+ */
+@PublicEvolving
+public class AvroRowDataDeserializationSchema implements
DeserializationSchema<RowData> {
+
+ private static final long serialVersionUID = 1L;
+
+ /**
+ * Used for converting Date type.
+ */
+ private static final int MILLIS_PER_DAY = 86400_000;
+
+ /**
+ * Logical type describing the result type.
+ */
+ private final RowType rowType;
+
+ /**
+ * Type information describing the result type.
+ */
+ private final TypeInformation<RowData> typeInfo;
+
+ /**
+ * Runtime instance that performs the actual work.
+ */
+ private final DeserializationRuntimeConverter runtimeConverter;
+
+ /**
+ * Record to deserialize byte array.
+ */
+ private transient IndexedRecord record;
+
+ /**
+ * Reader that deserializes byte array into a record.
+ */
+ private transient DatumReader<IndexedRecord> datumReader;
+
+ /**
+ * Input stream to read message from.
+ */
+ private transient MutableByteArrayInputStream inputStream;
+
+ /**
+ * Avro decoder that decodes binary data.
+ */
+ private transient Decoder decoder;
+
+ /**
+ * Creates a Avro deserialization schema for the given logical type.
+ *
+ * @param rowType The logical type used to deserialize the data.
+ * @param typeInfo The TypeInformation to be used by {@link
AvroRowDataDeserializationSchema#getProducedType()}.
+ */
+ public AvroRowDataDeserializationSchema(
+ RowType rowType,
+ TypeInformation<RowData> typeInfo) {
+ this.rowType = rowType;
+ this.typeInfo = typeInfo;
+ this.runtimeConverter = createRowConverter(rowType);
+ }
+
+ @Override
+ public void open(InitializationContext context) throws Exception {
+ final Schema schema =
AvroSchemaConverter.convertToSchema(rowType);
+ this.record = new GenericData.Record(schema);
+ this.datumReader = new SpecificDatumReader<>(schema);
+ this.inputStream = new MutableByteArrayInputStream();
+ this.decoder =
DecoderFactory.get().binaryDecoder(this.inputStream, null);
+ }
+
+ @Override
+ public RowData deserialize(byte[] message) throws IOException {
+ try {
+ inputStream.setBuffer(message);
+ record = datumReader.read(record, decoder);
+ return (RowData) runtimeConverter.convert(record);
+ } catch (Exception e) {
+ throw new IOException("Failed to deserialize Avro
record.", e);
+ }
+ }
+
+ @Override
+ public boolean isEndOfStream(RowData nextElement) {
+ return false;
+ }
+
+ @Override
+ public TypeInformation<RowData> getProducedType() {
+ return typeInfo;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ final AvroRowDataDeserializationSchema that =
(AvroRowDataDeserializationSchema) o;
+ return Objects.equals(rowType, that.rowType) &&
+ Objects.equals(typeInfo, that.typeInfo);
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(rowType, typeInfo);
+ }
+
+ //
-------------------------------------------------------------------------------------
+ // Runtime Converters
+ //
-------------------------------------------------------------------------------------
+
+ /**
+ * Runtime converter that converts Avro data structures into objects of
Flink Table & SQL
+ * internal data structures.
+ */
+ @FunctionalInterface
+ private interface DeserializationRuntimeConverter extends Serializable {
+ Object convert(Object object);
+ }
+
+ private DeserializationRuntimeConverter createRowConverter(RowType
rowType) {
+ final DeserializationRuntimeConverter[] fieldConverters =
rowType.getFields().stream()
+ .map(RowType.RowField::getType)
+ .map(this::createNullableConverter)
+ .toArray(DeserializationRuntimeConverter[]::new);
+ final int arity = rowType.getFieldCount();
+
+ return avroObject -> {
+ IndexedRecord record = (IndexedRecord) avroObject;
+ GenericRowData row = new GenericRowData(arity);
+ for (int i = 0; i < arity; ++i) {
+ row.setField(i,
fieldConverters[i].convert(record.get(i)));
+ }
+ return row;
+ };
+ }
+
+ /**
+ * Creates a runtime converter which is null safe.
+ */
+ private DeserializationRuntimeConverter
createNullableConverter(LogicalType type) {
+ final DeserializationRuntimeConverter converter =
createConverter(type);
+ return avroObject -> {
+ if (avroObject == null) {
+ return null;
+ }
+ return converter.convert(avroObject);
+ };
+ }
+
+ /**
+ * Creates a runtime converter which assuming input object is not null.
+ */
+ private DeserializationRuntimeConverter createConverter(LogicalType
type) {
+ switch (type.getTypeRoot()) {
+ case NULL:
+ return avroObject -> null;
+ case BOOLEAN: // boolean
+ case TINYINT: // short
+ case SMALLINT: // int
+ case INTEGER: // int
+ case INTERVAL_YEAR_MONTH: // long
+ case BIGINT: // long
+ case INTERVAL_DAY_TIME: // long
+ case FLOAT: // float
+ case DOUBLE: // double
+ return avroObject -> avroObject;
+ case DATE:
+ return this::convertToDate;
+ case TIME_WITHOUT_TIME_ZONE:
+ return this::convertToTime;
+ case TIMESTAMP_WITH_TIME_ZONE:
+ case TIMESTAMP_WITHOUT_TIME_ZONE:
+ return this::convertToTimestamp;
+ case CHAR:
+ case VARCHAR:
+ return avroObject ->
StringData.fromString(avroObject.toString());
+ case BINARY:
+ case VARBINARY:
+ return this::convertToBytes;
+ case DECIMAL:
+ return createDecimalConverter((DecimalType)
type);
+ case ARRAY:
+ return createArrayConverter((ArrayType) type);
+ case ROW:
+ return createRowConverter((RowType) type);
+ case MAP:
+ case MULTISET:
+ return createMapConverter((MapType) type);
+ case RAW:
+ default:
+ throw new
UnsupportedOperationException("Unsupported type: " + type);
+ }
+ }
+
+ private DeserializationRuntimeConverter
createDecimalConverter(DecimalType decimalType) {
+ return avroObject -> {
+ final byte[] bytes;
+ if (avroObject instanceof GenericFixed) {
+ bytes = ((GenericFixed) avroObject).bytes();
+ } else if (avroObject instanceof ByteBuffer) {
+ ByteBuffer byteBuffer = (ByteBuffer) avroObject;
+ bytes = new byte[byteBuffer.remaining()];
+ byteBuffer.get(bytes);
+ } else {
+ bytes = (byte[]) avroObject;
+ }
+ return DecimalData.fromUnscaledBytes(bytes,
decimalType.getPrecision(), decimalType.getScale());
+ };
+ }
+
+ private DeserializationRuntimeConverter createArrayConverter(ArrayType
arrayType) {
+ final DeserializationRuntimeConverter elementConverter =
createNullableConverter(arrayType.getElementType());
+ final Class<?> elementClass =
LogicalTypeUtils.toInternalConversionClass(arrayType.getElementType());
+
+ return avroObject -> {
+ final List<?> list = (List<?>) avroObject;
+ final int length = list.size();
+ final Object[] array = (Object[])
Array.newInstance(elementClass, length);
+ for (int i = 0; i < length; ++i) {
+ array[i] =
elementConverter.convert(list.get(i));
+ }
+ return new GenericArrayData(array);
+ };
+ }
+
+ private DeserializationRuntimeConverter createMapConverter(MapType
mapType) {
+ LogicalType keyType = mapType.getKeyType();
+ if (!LogicalTypeChecks.hasFamily(keyType,
LogicalTypeFamily.CHARACTER_STRING)) {
+ throw new UnsupportedOperationException(
+ "Avro format doesn't support non-string as key
type of map. " +
+ "The map type is: " +
mapType.asSummaryString());
+ }
+ final DeserializationRuntimeConverter keyConverter =
createConverter(keyType);
+ final DeserializationRuntimeConverter valueConverter =
createConverter(mapType.getValueType());
+
+ return avroObject -> {
+ final Map<?, ?> map = (Map<?, ?>) avroObject;
+ Map<Object, Object> result = new HashMap<>();
+ for (Map.Entry<?, ?> entry : map.entrySet()) {
+ Object key =
keyConverter.convert(entry.getKey());
+ Object value =
valueConverter.convert(entry.getValue());
+ result.put(key, value);
+ }
+ return new GenericMapData(result);
+ };
+ }
+
+ private TimestampData convertToTimestamp(Object object) {
+ final long millis;
+ if (object instanceof Long) {
+ millis = (Long) object;
+ } else {
+ // use 'provided' Joda time
+ final DateTime value = (DateTime) object;
+ millis = value.toDate().getTime();
+ }
+ return TimestampData.fromEpochMillis(millis);
+ }
+
+ private int convertToDate(Object object) {
+ final long millis;
+ if (object instanceof Integer) {
+ final Integer value = (Integer) object;
+ millis = (long) value * MILLIS_PER_DAY;
Review comment:
Just return the day value. Don't need to recomute MILLIS_PER_DAY.
##########
File path:
flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/AvroRowDataDeSerializationSchemaTest.java
##########
@@ -0,0 +1,155 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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 org.apache.flink.formats.avro;
+
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.formats.avro.typeutils.AvroSchemaConverter;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.RowType;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.avro.io.DatumWriter;
+import org.apache.avro.io.Encoder;
+import org.apache.avro.io.EncoderFactory;
+import org.apache.avro.specific.SpecificDatumWriter;
+import org.junit.Test;
+
+import java.io.ByteArrayOutputStream;
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.flink.table.api.DataTypes.ARRAY;
+import static org.apache.flink.table.api.DataTypes.BIGINT;
+import static org.apache.flink.table.api.DataTypes.BOOLEAN;
+import static org.apache.flink.table.api.DataTypes.BYTES;
+import static org.apache.flink.table.api.DataTypes.DATE;
+import static org.apache.flink.table.api.DataTypes.DECIMAL;
+import static org.apache.flink.table.api.DataTypes.DOUBLE;
+import static org.apache.flink.table.api.DataTypes.FIELD;
+import static org.apache.flink.table.api.DataTypes.FLOAT;
+import static org.apache.flink.table.api.DataTypes.INT;
+import static org.apache.flink.table.api.DataTypes.MAP;
+import static org.apache.flink.table.api.DataTypes.ROW;
+import static org.apache.flink.table.api.DataTypes.STRING;
+import static org.apache.flink.table.api.DataTypes.TIME;
+import static org.apache.flink.table.api.DataTypes.TIMESTAMP;
+import static org.junit.Assert.assertArrayEquals;
+
+/**
+ * Test for the Avro serialization and deserialization schema.
+ */
+public class AvroRowDataDeSerializationSchemaTest {
+
+ @Test
+ public void testSerializeDeserialize() throws Exception {
+ final DataType dataType = ROW(
+ FIELD("bool", BOOLEAN()),
+ FIELD("int", INT()),
+ FIELD("bigint", BIGINT()),
+ FIELD("float", FLOAT()),
+ FIELD("double",DOUBLE()),
+ FIELD("name", STRING()),
+ FIELD("bytes", BYTES()),
+ FIELD("decimal", DECIMAL(9, 6)),
+ FIELD("doubles", ARRAY(DOUBLE())),
+ FIELD("time", TIME(0)),
+ FIELD("date", DATE()),
+ FIELD("timestamp3", TIMESTAMP(3)),
+ FIELD("timestamp9", TIMESTAMP(9)),
+ FIELD("map", MAP(STRING(), BIGINT())),
+ FIELD("map2map", MAP(STRING(), MAP(STRING(), INT()))),
+ FIELD("map2array", MAP(STRING(), ARRAY(INT()))));
+ final RowType rowType = (RowType) dataType.getLogicalType();
+ final TypeInformation<RowData> typeInfo = new
RowDataTypeInfo(rowType);
+
+ final Schema schema =
AvroSchemaConverter.convertToSchema(rowType);
+ final GenericRecord record = new GenericData.Record(schema);
+ record.put(0, true);
+ record.put(1, 33);
+ record.put(2, 44L);
+ record.put(3, 12.34F);
+ record.put(4, 23.45);
+ record.put(5, "hello avro");
+ record.put(6, ByteBuffer.wrap(new byte[]{1, 2, 4, 5, 6, 7, 8,
12}));
+
+ ByteBuffer byteBuffer =
ByteBuffer.wrap(DecimalData.fromBigDecimal(
+ BigDecimal.valueOf(123456789, 6), 9, 6)
+ .toUnscaledBytes());
+ record.put(7, byteBuffer);
+
+ List<Double> doubles = new ArrayList<>();
+ doubles.add(1.2);
+ doubles.add(3.4);
+ doubles.add(567.8901);
+ record.put(8, doubles);
+
+ record.put(9, 18397);
+ record.put(10, 10087);
+ record.put(11, 1589530213123L);
+ record.put(12, 1589530213123000000L);
Review comment:
Add some more tests for data from Joda time.
##########
File path:
flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/AvroRowDataDeserializationSchema.java
##########
@@ -0,0 +1,371 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You 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 org.apache.flink.formats.avro;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.formats.avro.typeutils.AvroSchemaConverter;
+import org.apache.flink.formats.avro.utils.MutableByteArrayInputStream;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.DecimalType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.LogicalTypeFamily;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.logical.utils.LogicalTypeChecks;
+import org.apache.flink.table.types.logical.utils.LogicalTypeUtils;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericFixed;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.avro.io.DatumReader;
+import org.apache.avro.io.Decoder;
+import org.apache.avro.io.DecoderFactory;
+import org.apache.avro.specific.SpecificDatumReader;
+import org.joda.time.DateTime;
+import org.joda.time.DateTimeFieldType;
+import org.joda.time.LocalDate;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.lang.reflect.Array;
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+
+/**
+ * Deserialization schema from Avro bytes to {@link RowData}.
+ *
+ * <p>Deserializes the <code>byte[]</code> messages into (nested) Flink
RowData. It converts Avro types
+ * into types that are compatible with Flink's Table & SQL API.
+ *
+ * <p>Projects with Avro records containing logical date/time types need to
add a JodaTime
+ * dependency.
+ *
+ * <p>Note: Changes in this class need to be kept in sync with the
corresponding runtime
+ * class {@link AvroRowDataSerializationSchema} and schema converter {@link
AvroSchemaConverter}.
+ */
+@PublicEvolving
+public class AvroRowDataDeserializationSchema implements
DeserializationSchema<RowData> {
+
+ private static final long serialVersionUID = 1L;
+
+ /**
+ * Used for converting Date type.
+ */
+ private static final int MILLIS_PER_DAY = 86400_000;
+
+ /**
+ * Logical type describing the result type.
+ */
+ private final RowType rowType;
+
+ /**
+ * Type information describing the result type.
+ */
+ private final TypeInformation<RowData> typeInfo;
+
+ /**
+ * Runtime instance that performs the actual work.
+ */
+ private final DeserializationRuntimeConverter runtimeConverter;
+
+ /**
+ * Record to deserialize byte array.
+ */
+ private transient IndexedRecord record;
+
+ /**
+ * Reader that deserializes byte array into a record.
+ */
+ private transient DatumReader<IndexedRecord> datumReader;
+
+ /**
+ * Input stream to read message from.
+ */
+ private transient MutableByteArrayInputStream inputStream;
+
+ /**
+ * Avro decoder that decodes binary data.
+ */
+ private transient Decoder decoder;
+
+ /**
+ * Creates a Avro deserialization schema for the given logical type.
+ *
+ * @param rowType The logical type used to deserialize the data.
+ * @param typeInfo The TypeInformation to be used by {@link
AvroRowDataDeserializationSchema#getProducedType()}.
+ */
+ public AvroRowDataDeserializationSchema(
+ RowType rowType,
+ TypeInformation<RowData> typeInfo) {
+ this.rowType = rowType;
+ this.typeInfo = typeInfo;
+ this.runtimeConverter = createRowConverter(rowType);
+ }
+
+ @Override
+ public void open(InitializationContext context) throws Exception {
+ final Schema schema =
AvroSchemaConverter.convertToSchema(rowType);
+ this.record = new GenericData.Record(schema);
+ this.datumReader = new SpecificDatumReader<>(schema);
+ this.inputStream = new MutableByteArrayInputStream();
+ this.decoder =
DecoderFactory.get().binaryDecoder(this.inputStream, null);
+ }
+
+ @Override
+ public RowData deserialize(byte[] message) throws IOException {
+ try {
+ inputStream.setBuffer(message);
+ record = datumReader.read(record, decoder);
+ return (RowData) runtimeConverter.convert(record);
+ } catch (Exception e) {
+ throw new IOException("Failed to deserialize Avro
record.", e);
+ }
+ }
+
+ @Override
+ public boolean isEndOfStream(RowData nextElement) {
+ return false;
+ }
+
+ @Override
+ public TypeInformation<RowData> getProducedType() {
+ return typeInfo;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ final AvroRowDataDeserializationSchema that =
(AvroRowDataDeserializationSchema) o;
+ return Objects.equals(rowType, that.rowType) &&
+ Objects.equals(typeInfo, that.typeInfo);
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(rowType, typeInfo);
+ }
+
+ //
-------------------------------------------------------------------------------------
+ // Runtime Converters
+ //
-------------------------------------------------------------------------------------
+
+ /**
+ * Runtime converter that converts Avro data structures into objects of
Flink Table & SQL
+ * internal data structures.
+ */
+ @FunctionalInterface
+ private interface DeserializationRuntimeConverter extends Serializable {
+ Object convert(Object object);
+ }
+
+ private DeserializationRuntimeConverter createRowConverter(RowType
rowType) {
+ final DeserializationRuntimeConverter[] fieldConverters =
rowType.getFields().stream()
+ .map(RowType.RowField::getType)
+ .map(this::createNullableConverter)
+ .toArray(DeserializationRuntimeConverter[]::new);
+ final int arity = rowType.getFieldCount();
+
+ return avroObject -> {
+ IndexedRecord record = (IndexedRecord) avroObject;
+ GenericRowData row = new GenericRowData(arity);
+ for (int i = 0; i < arity; ++i) {
+ row.setField(i,
fieldConverters[i].convert(record.get(i)));
+ }
+ return row;
+ };
+ }
+
+ /**
+ * Creates a runtime converter which is null safe.
+ */
+ private DeserializationRuntimeConverter
createNullableConverter(LogicalType type) {
+ final DeserializationRuntimeConverter converter =
createConverter(type);
+ return avroObject -> {
+ if (avroObject == null) {
+ return null;
+ }
+ return converter.convert(avroObject);
+ };
+ }
+
+ /**
+ * Creates a runtime converter which assuming input object is not null.
+ */
+ private DeserializationRuntimeConverter createConverter(LogicalType
type) {
+ switch (type.getTypeRoot()) {
+ case NULL:
+ return avroObject -> null;
+ case BOOLEAN: // boolean
+ case TINYINT: // short
+ case SMALLINT: // int
+ case INTEGER: // int
+ case INTERVAL_YEAR_MONTH: // long
+ case BIGINT: // long
+ case INTERVAL_DAY_TIME: // long
+ case FLOAT: // float
+ case DOUBLE: // double
+ return avroObject -> avroObject;
+ case DATE:
+ return this::convertToDate;
+ case TIME_WITHOUT_TIME_ZONE:
+ return this::convertToTime;
+ case TIMESTAMP_WITH_TIME_ZONE:
+ case TIMESTAMP_WITHOUT_TIME_ZONE:
+ return this::convertToTimestamp;
+ case CHAR:
+ case VARCHAR:
+ return avroObject ->
StringData.fromString(avroObject.toString());
+ case BINARY:
+ case VARBINARY:
+ return this::convertToBytes;
+ case DECIMAL:
+ return createDecimalConverter((DecimalType)
type);
+ case ARRAY:
+ return createArrayConverter((ArrayType) type);
+ case ROW:
+ return createRowConverter((RowType) type);
+ case MAP:
+ case MULTISET:
+ return createMapConverter((MapType) type);
+ case RAW:
+ default:
+ throw new
UnsupportedOperationException("Unsupported type: " + type);
+ }
+ }
+
+ private DeserializationRuntimeConverter
createDecimalConverter(DecimalType decimalType) {
+ return avroObject -> {
+ final byte[] bytes;
+ if (avroObject instanceof GenericFixed) {
+ bytes = ((GenericFixed) avroObject).bytes();
+ } else if (avroObject instanceof ByteBuffer) {
+ ByteBuffer byteBuffer = (ByteBuffer) avroObject;
+ bytes = new byte[byteBuffer.remaining()];
+ byteBuffer.get(bytes);
+ } else {
+ bytes = (byte[]) avroObject;
+ }
+ return DecimalData.fromUnscaledBytes(bytes,
decimalType.getPrecision(), decimalType.getScale());
+ };
+ }
+
+ private DeserializationRuntimeConverter createArrayConverter(ArrayType
arrayType) {
+ final DeserializationRuntimeConverter elementConverter =
createNullableConverter(arrayType.getElementType());
+ final Class<?> elementClass =
LogicalTypeUtils.toInternalConversionClass(arrayType.getElementType());
+
+ return avroObject -> {
+ final List<?> list = (List<?>) avroObject;
+ final int length = list.size();
+ final Object[] array = (Object[])
Array.newInstance(elementClass, length);
+ for (int i = 0; i < length; ++i) {
+ array[i] =
elementConverter.convert(list.get(i));
+ }
+ return new GenericArrayData(array);
+ };
+ }
+
+ private DeserializationRuntimeConverter createMapConverter(MapType
mapType) {
+ LogicalType keyType = mapType.getKeyType();
+ if (!LogicalTypeChecks.hasFamily(keyType,
LogicalTypeFamily.CHARACTER_STRING)) {
+ throw new UnsupportedOperationException(
+ "Avro format doesn't support non-string as key
type of map. " +
+ "The map type is: " +
mapType.asSummaryString());
+ }
+ final DeserializationRuntimeConverter keyConverter =
createConverter(keyType);
+ final DeserializationRuntimeConverter valueConverter =
createConverter(mapType.getValueType());
+
+ return avroObject -> {
+ final Map<?, ?> map = (Map<?, ?>) avroObject;
+ Map<Object, Object> result = new HashMap<>();
+ for (Map.Entry<?, ?> entry : map.entrySet()) {
+ Object key =
keyConverter.convert(entry.getKey());
+ Object value =
valueConverter.convert(entry.getValue());
+ result.put(key, value);
+ }
+ return new GenericMapData(result);
+ };
+ }
+
+ private TimestampData convertToTimestamp(Object object) {
+ final long millis;
+ if (object instanceof Long) {
+ millis = (Long) object;
+ } else {
+ // use 'provided' Joda time
+ final DateTime value = (DateTime) object;
+ millis = value.toDate().getTime();
+ }
+ return TimestampData.fromEpochMillis(millis);
+ }
+
+ private int convertToDate(Object object) {
+ final long millis;
+ if (object instanceof Integer) {
+ final Integer value = (Integer) object;
+ millis = (long) value * MILLIS_PER_DAY;
+ } else {
+ // use 'provided' Joda time
+ final LocalDate value = (LocalDate) object;
+ millis = value.toDate().getTime();
Review comment:
ditto
```
new java.sql.Date(value.toDate().getTime()).toLocalDate().toEpochDay();
```
##########
File path:
flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/typeutils/AvroSchemaConverter.java
##########
@@ -157,4 +166,118 @@ private AvroSchemaConverter() {
}
throw new IllegalArgumentException("Unsupported Avro type '" +
schema.getType() + "'.");
}
+
+ /**
+ * Converts Flink SQL {@link LogicalType} (can be nested) into an Avro
schema.
+ *
+ * @param logicalType logical type
+ * @return Avro's {@link Schema} matching this logical type.
+ */
+ public static Schema convertToSchema(LogicalType logicalType) {
+ return convertToSchema(logicalType, 0);
+ }
+
+ private static Schema convertToSchema(LogicalType logicalType, int
rowTypeCounter) {
+ switch (logicalType.getTypeRoot()) {
+ case NULL:
+ return SchemaBuilder.builder().nullType();
+ case BOOLEAN:
+ return
getNullableBuilder(logicalType).booleanType();
+ case INTEGER:
+ return
getNullableBuilder(logicalType).intType();
+ case BIGINT:
+ return
getNullableBuilder(logicalType).longType();
+ case FLOAT:
+ return
getNullableBuilder(logicalType).floatType();
+ case DOUBLE:
+ return
getNullableBuilder(logicalType).doubleType();
+ case CHAR:
+ case VARCHAR:
+ return
getNullableBuilder(logicalType).stringType();
+ case BINARY:
+ case VARBINARY:
+ return
getNullableBuilder(logicalType).bytesType();
+ case TIMESTAMP_WITHOUT_TIME_ZONE:
Review comment:
We can also support `TIMESTAMP_WITH_LOCAL_TIME_ZONE`. You can add a
`case TIMESTAMP_WITH_LOCAL_TIME_ZONE` along with `TIMESTAMP_WITHOUT_TIME_ZONE`,
then use `LogicalTypeChecks#getPrecision(logiclType)` to get precision.
##########
File path:
flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/typeutils/AvroSchemaConverter.java
##########
@@ -157,4 +166,118 @@ private AvroSchemaConverter() {
}
throw new IllegalArgumentException("Unsupported Avro type '" +
schema.getType() + "'.");
}
+
+ /**
+ * Converts Flink SQL {@link LogicalType} (can be nested) into an Avro
schema.
+ *
+ * @param logicalType logical type
+ * @return Avro's {@link Schema} matching this logical type.
+ */
+ public static Schema convertToSchema(LogicalType logicalType) {
+ return convertToSchema(logicalType, 0);
+ }
+
+ private static Schema convertToSchema(LogicalType logicalType, int
rowTypeCounter) {
+ switch (logicalType.getTypeRoot()) {
+ case NULL:
+ return SchemaBuilder.builder().nullType();
+ case BOOLEAN:
+ return
getNullableBuilder(logicalType).booleanType();
+ case INTEGER:
+ return
getNullableBuilder(logicalType).intType();
+ case BIGINT:
+ return
getNullableBuilder(logicalType).longType();
+ case FLOAT:
+ return
getNullableBuilder(logicalType).floatType();
+ case DOUBLE:
+ return
getNullableBuilder(logicalType).doubleType();
+ case CHAR:
+ case VARCHAR:
+ return
getNullableBuilder(logicalType).stringType();
+ case BINARY:
+ case VARBINARY:
+ return
getNullableBuilder(logicalType).bytesType();
+ case TIMESTAMP_WITHOUT_TIME_ZONE:
+ // use long to represents Timestamp
+ final TimestampType timestampType =
(TimestampType) logicalType;
+ int precision = timestampType.getPrecision();
+ org.apache.avro.LogicalType avroLogicalType;
+ if (precision == 3) {
+ avroLogicalType =
LogicalTypes.timestampMillis();
+ } else if (precision == 9) {
+ avroLogicalType =
LogicalTypes.timestampMicros();
+ } else {
+ throw new
IllegalArgumentException("Avro Timestamp does not support Timestamp with
precision: " +
+ precision +
+ ", it only supports precision
of 3 or 9.");
+ }
+ return
avroLogicalType.addToSchema(SchemaBuilder.builder().longType());
+ case DATE:
+ // use int to represents Date
+ return
LogicalTypes.date().addToSchema(SchemaBuilder.builder().intType());
+ case TIME_WITHOUT_TIME_ZONE:
+ // use int to represents Time, we only support
millisecond when deserialization
+ return
LogicalTypes.timeMillis().addToSchema(SchemaBuilder.builder().intType());
+ case DECIMAL:
+ DecimalType decimalType = (DecimalType)
logicalType;
+ // store BigDecimal as byte[]
+ return LogicalTypes
+ .decimal(decimalType.getPrecision(),
decimalType.getScale())
+
.addToSchema(SchemaBuilder.builder().bytesType());
+ case ROW:
+ RowType rowType = (RowType) logicalType;
+ List<String> fieldNames =
rowType.getFieldNames();
+ // we have to make sure the record name is
different in a Schema
+ SchemaBuilder.FieldAssembler<Schema> builder =
SchemaBuilder
+ .builder()
+ .record("row_" + rowTypeCounter)
+ .fields();
+ rowTypeCounter++;
+ for (int i = 0; i < rowType.getFieldCount();
i++) {
+ builder = builder
+ .name(fieldNames.get(i))
+
.type(convertToSchema(rowType.getTypeAt(i), rowTypeCounter))
+ .noDefault();
+ }
+ return builder.endRecord();
+ case MAP:
Review comment:
Add `MULTIMAP` support?
##########
File path:
flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/AvroRowDataDeserializationSchema.java
##########
@@ -0,0 +1,371 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You 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 org.apache.flink.formats.avro;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.formats.avro.typeutils.AvroSchemaConverter;
+import org.apache.flink.formats.avro.utils.MutableByteArrayInputStream;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.DecimalType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.LogicalTypeFamily;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.logical.utils.LogicalTypeChecks;
+import org.apache.flink.table.types.logical.utils.LogicalTypeUtils;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericFixed;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.avro.io.DatumReader;
+import org.apache.avro.io.Decoder;
+import org.apache.avro.io.DecoderFactory;
+import org.apache.avro.specific.SpecificDatumReader;
+import org.joda.time.DateTime;
+import org.joda.time.DateTimeFieldType;
+import org.joda.time.LocalDate;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.lang.reflect.Array;
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+
+/**
+ * Deserialization schema from Avro bytes to {@link RowData}.
+ *
+ * <p>Deserializes the <code>byte[]</code> messages into (nested) Flink
RowData. It converts Avro types
+ * into types that are compatible with Flink's Table & SQL API.
+ *
+ * <p>Projects with Avro records containing logical date/time types need to
add a JodaTime
+ * dependency.
+ *
+ * <p>Note: Changes in this class need to be kept in sync with the
corresponding runtime
+ * class {@link AvroRowDataSerializationSchema} and schema converter {@link
AvroSchemaConverter}.
+ */
+@PublicEvolving
+public class AvroRowDataDeserializationSchema implements
DeserializationSchema<RowData> {
+
+ private static final long serialVersionUID = 1L;
+
+ /**
+ * Used for converting Date type.
+ */
+ private static final int MILLIS_PER_DAY = 86400_000;
+
+ /**
+ * Logical type describing the result type.
+ */
+ private final RowType rowType;
+
+ /**
+ * Type information describing the result type.
+ */
+ private final TypeInformation<RowData> typeInfo;
+
+ /**
+ * Runtime instance that performs the actual work.
+ */
+ private final DeserializationRuntimeConverter runtimeConverter;
+
+ /**
+ * Record to deserialize byte array.
+ */
+ private transient IndexedRecord record;
+
+ /**
+ * Reader that deserializes byte array into a record.
+ */
+ private transient DatumReader<IndexedRecord> datumReader;
+
+ /**
+ * Input stream to read message from.
+ */
+ private transient MutableByteArrayInputStream inputStream;
+
+ /**
+ * Avro decoder that decodes binary data.
+ */
+ private transient Decoder decoder;
+
+ /**
+ * Creates a Avro deserialization schema for the given logical type.
+ *
+ * @param rowType The logical type used to deserialize the data.
+ * @param typeInfo The TypeInformation to be used by {@link
AvroRowDataDeserializationSchema#getProducedType()}.
+ */
+ public AvroRowDataDeserializationSchema(
+ RowType rowType,
+ TypeInformation<RowData> typeInfo) {
+ this.rowType = rowType;
+ this.typeInfo = typeInfo;
+ this.runtimeConverter = createRowConverter(rowType);
+ }
+
+ @Override
+ public void open(InitializationContext context) throws Exception {
+ final Schema schema =
AvroSchemaConverter.convertToSchema(rowType);
+ this.record = new GenericData.Record(schema);
+ this.datumReader = new SpecificDatumReader<>(schema);
+ this.inputStream = new MutableByteArrayInputStream();
+ this.decoder =
DecoderFactory.get().binaryDecoder(this.inputStream, null);
+ }
+
+ @Override
+ public RowData deserialize(byte[] message) throws IOException {
+ try {
+ inputStream.setBuffer(message);
+ record = datumReader.read(record, decoder);
+ return (RowData) runtimeConverter.convert(record);
+ } catch (Exception e) {
+ throw new IOException("Failed to deserialize Avro
record.", e);
+ }
+ }
+
+ @Override
+ public boolean isEndOfStream(RowData nextElement) {
+ return false;
+ }
+
+ @Override
+ public TypeInformation<RowData> getProducedType() {
+ return typeInfo;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ final AvroRowDataDeserializationSchema that =
(AvroRowDataDeserializationSchema) o;
+ return Objects.equals(rowType, that.rowType) &&
+ Objects.equals(typeInfo, that.typeInfo);
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(rowType, typeInfo);
+ }
+
+ //
-------------------------------------------------------------------------------------
+ // Runtime Converters
+ //
-------------------------------------------------------------------------------------
+
+ /**
+ * Runtime converter that converts Avro data structures into objects of
Flink Table & SQL
+ * internal data structures.
+ */
+ @FunctionalInterface
+ private interface DeserializationRuntimeConverter extends Serializable {
+ Object convert(Object object);
+ }
+
+ private DeserializationRuntimeConverter createRowConverter(RowType
rowType) {
+ final DeserializationRuntimeConverter[] fieldConverters =
rowType.getFields().stream()
+ .map(RowType.RowField::getType)
+ .map(this::createNullableConverter)
+ .toArray(DeserializationRuntimeConverter[]::new);
+ final int arity = rowType.getFieldCount();
+
+ return avroObject -> {
+ IndexedRecord record = (IndexedRecord) avroObject;
+ GenericRowData row = new GenericRowData(arity);
+ for (int i = 0; i < arity; ++i) {
+ row.setField(i,
fieldConverters[i].convert(record.get(i)));
+ }
+ return row;
+ };
+ }
+
+ /**
+ * Creates a runtime converter which is null safe.
+ */
+ private DeserializationRuntimeConverter
createNullableConverter(LogicalType type) {
+ final DeserializationRuntimeConverter converter =
createConverter(type);
+ return avroObject -> {
+ if (avroObject == null) {
+ return null;
+ }
+ return converter.convert(avroObject);
+ };
+ }
+
+ /**
+ * Creates a runtime converter which assuming input object is not null.
+ */
+ private DeserializationRuntimeConverter createConverter(LogicalType
type) {
+ switch (type.getTypeRoot()) {
+ case NULL:
+ return avroObject -> null;
+ case BOOLEAN: // boolean
+ case TINYINT: // short
+ case SMALLINT: // int
+ case INTEGER: // int
+ case INTERVAL_YEAR_MONTH: // long
+ case BIGINT: // long
+ case INTERVAL_DAY_TIME: // long
+ case FLOAT: // float
+ case DOUBLE: // double
+ return avroObject -> avroObject;
+ case DATE:
+ return this::convertToDate;
+ case TIME_WITHOUT_TIME_ZONE:
+ return this::convertToTime;
+ case TIMESTAMP_WITH_TIME_ZONE:
+ case TIMESTAMP_WITHOUT_TIME_ZONE:
+ return this::convertToTimestamp;
+ case CHAR:
+ case VARCHAR:
+ return avroObject ->
StringData.fromString(avroObject.toString());
+ case BINARY:
+ case VARBINARY:
+ return this::convertToBytes;
+ case DECIMAL:
+ return createDecimalConverter((DecimalType)
type);
+ case ARRAY:
+ return createArrayConverter((ArrayType) type);
+ case ROW:
+ return createRowConverter((RowType) type);
+ case MAP:
+ case MULTISET:
+ return createMapConverter((MapType) type);
+ case RAW:
+ default:
+ throw new
UnsupportedOperationException("Unsupported type: " + type);
+ }
+ }
+
+ private DeserializationRuntimeConverter
createDecimalConverter(DecimalType decimalType) {
+ return avroObject -> {
+ final byte[] bytes;
+ if (avroObject instanceof GenericFixed) {
+ bytes = ((GenericFixed) avroObject).bytes();
+ } else if (avroObject instanceof ByteBuffer) {
+ ByteBuffer byteBuffer = (ByteBuffer) avroObject;
+ bytes = new byte[byteBuffer.remaining()];
+ byteBuffer.get(bytes);
+ } else {
+ bytes = (byte[]) avroObject;
+ }
+ return DecimalData.fromUnscaledBytes(bytes,
decimalType.getPrecision(), decimalType.getScale());
Review comment:
We should avoid depending LogicalType in the hot path (runtime code).
You can get the precision and scale value out of the lambda with `final`
modifier.
##########
File path:
flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/AvroRowDataDeserializationSchema.java
##########
@@ -0,0 +1,371 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You 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 org.apache.flink.formats.avro;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.serialization.DeserializationSchema;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.formats.avro.typeutils.AvroSchemaConverter;
+import org.apache.flink.formats.avro.utils.MutableByteArrayInputStream;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.DecimalType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.LogicalTypeFamily;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.logical.utils.LogicalTypeChecks;
+import org.apache.flink.table.types.logical.utils.LogicalTypeUtils;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericFixed;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.avro.io.DatumReader;
+import org.apache.avro.io.Decoder;
+import org.apache.avro.io.DecoderFactory;
+import org.apache.avro.specific.SpecificDatumReader;
+import org.joda.time.DateTime;
+import org.joda.time.DateTimeFieldType;
+import org.joda.time.LocalDate;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.lang.reflect.Array;
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+
+/**
+ * Deserialization schema from Avro bytes to {@link RowData}.
+ *
+ * <p>Deserializes the <code>byte[]</code> messages into (nested) Flink
RowData. It converts Avro types
+ * into types that are compatible with Flink's Table & SQL API.
+ *
+ * <p>Projects with Avro records containing logical date/time types need to
add a JodaTime
+ * dependency.
+ *
+ * <p>Note: Changes in this class need to be kept in sync with the
corresponding runtime
+ * class {@link AvroRowDataSerializationSchema} and schema converter {@link
AvroSchemaConverter}.
+ */
+@PublicEvolving
+public class AvroRowDataDeserializationSchema implements
DeserializationSchema<RowData> {
+
+ private static final long serialVersionUID = 1L;
+
+ /**
+ * Used for converting Date type.
+ */
+ private static final int MILLIS_PER_DAY = 86400_000;
+
+ /**
+ * Logical type describing the result type.
+ */
+ private final RowType rowType;
+
+ /**
+ * Type information describing the result type.
+ */
+ private final TypeInformation<RowData> typeInfo;
+
+ /**
+ * Runtime instance that performs the actual work.
+ */
+ private final DeserializationRuntimeConverter runtimeConverter;
+
+ /**
+ * Record to deserialize byte array.
+ */
+ private transient IndexedRecord record;
+
+ /**
+ * Reader that deserializes byte array into a record.
+ */
+ private transient DatumReader<IndexedRecord> datumReader;
+
+ /**
+ * Input stream to read message from.
+ */
+ private transient MutableByteArrayInputStream inputStream;
+
+ /**
+ * Avro decoder that decodes binary data.
+ */
+ private transient Decoder decoder;
+
+ /**
+ * Creates a Avro deserialization schema for the given logical type.
+ *
+ * @param rowType The logical type used to deserialize the data.
+ * @param typeInfo The TypeInformation to be used by {@link
AvroRowDataDeserializationSchema#getProducedType()}.
+ */
+ public AvroRowDataDeserializationSchema(
+ RowType rowType,
+ TypeInformation<RowData> typeInfo) {
+ this.rowType = rowType;
+ this.typeInfo = typeInfo;
+ this.runtimeConverter = createRowConverter(rowType);
+ }
+
+ @Override
+ public void open(InitializationContext context) throws Exception {
+ final Schema schema =
AvroSchemaConverter.convertToSchema(rowType);
+ this.record = new GenericData.Record(schema);
+ this.datumReader = new SpecificDatumReader<>(schema);
+ this.inputStream = new MutableByteArrayInputStream();
+ this.decoder =
DecoderFactory.get().binaryDecoder(this.inputStream, null);
+ }
+
+ @Override
+ public RowData deserialize(byte[] message) throws IOException {
+ try {
+ inputStream.setBuffer(message);
+ record = datumReader.read(record, decoder);
+ return (RowData) runtimeConverter.convert(record);
+ } catch (Exception e) {
+ throw new IOException("Failed to deserialize Avro
record.", e);
+ }
+ }
+
+ @Override
+ public boolean isEndOfStream(RowData nextElement) {
+ return false;
+ }
+
+ @Override
+ public TypeInformation<RowData> getProducedType() {
+ return typeInfo;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ final AvroRowDataDeserializationSchema that =
(AvroRowDataDeserializationSchema) o;
+ return Objects.equals(rowType, that.rowType) &&
+ Objects.equals(typeInfo, that.typeInfo);
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(rowType, typeInfo);
+ }
+
+ //
-------------------------------------------------------------------------------------
+ // Runtime Converters
+ //
-------------------------------------------------------------------------------------
+
+ /**
+ * Runtime converter that converts Avro data structures into objects of
Flink Table & SQL
+ * internal data structures.
+ */
+ @FunctionalInterface
+ private interface DeserializationRuntimeConverter extends Serializable {
+ Object convert(Object object);
+ }
+
+ private DeserializationRuntimeConverter createRowConverter(RowType
rowType) {
+ final DeserializationRuntimeConverter[] fieldConverters =
rowType.getFields().stream()
+ .map(RowType.RowField::getType)
+ .map(this::createNullableConverter)
+ .toArray(DeserializationRuntimeConverter[]::new);
+ final int arity = rowType.getFieldCount();
+
+ return avroObject -> {
+ IndexedRecord record = (IndexedRecord) avroObject;
+ GenericRowData row = new GenericRowData(arity);
+ for (int i = 0; i < arity; ++i) {
+ row.setField(i,
fieldConverters[i].convert(record.get(i)));
+ }
+ return row;
+ };
+ }
+
+ /**
+ * Creates a runtime converter which is null safe.
+ */
+ private DeserializationRuntimeConverter
createNullableConverter(LogicalType type) {
+ final DeserializationRuntimeConverter converter =
createConverter(type);
+ return avroObject -> {
+ if (avroObject == null) {
+ return null;
+ }
+ return converter.convert(avroObject);
+ };
+ }
+
+ /**
+ * Creates a runtime converter which assuming input object is not null.
+ */
+ private DeserializationRuntimeConverter createConverter(LogicalType
type) {
+ switch (type.getTypeRoot()) {
+ case NULL:
+ return avroObject -> null;
+ case BOOLEAN: // boolean
+ case TINYINT: // short
+ case SMALLINT: // int
+ case INTEGER: // int
+ case INTERVAL_YEAR_MONTH: // long
+ case BIGINT: // long
+ case INTERVAL_DAY_TIME: // long
+ case FLOAT: // float
+ case DOUBLE: // double
+ return avroObject -> avroObject;
+ case DATE:
+ return this::convertToDate;
+ case TIME_WITHOUT_TIME_ZONE:
+ return this::convertToTime;
+ case TIMESTAMP_WITH_TIME_ZONE:
+ case TIMESTAMP_WITHOUT_TIME_ZONE:
+ return this::convertToTimestamp;
+ case CHAR:
+ case VARCHAR:
+ return avroObject ->
StringData.fromString(avroObject.toString());
+ case BINARY:
+ case VARBINARY:
+ return this::convertToBytes;
+ case DECIMAL:
+ return createDecimalConverter((DecimalType)
type);
+ case ARRAY:
+ return createArrayConverter((ArrayType) type);
+ case ROW:
+ return createRowConverter((RowType) type);
+ case MAP:
+ case MULTISET:
+ return createMapConverter((MapType) type);
+ case RAW:
+ default:
+ throw new
UnsupportedOperationException("Unsupported type: " + type);
+ }
+ }
+
+ private DeserializationRuntimeConverter
createDecimalConverter(DecimalType decimalType) {
+ return avroObject -> {
+ final byte[] bytes;
+ if (avroObject instanceof GenericFixed) {
+ bytes = ((GenericFixed) avroObject).bytes();
+ } else if (avroObject instanceof ByteBuffer) {
+ ByteBuffer byteBuffer = (ByteBuffer) avroObject;
+ bytes = new byte[byteBuffer.remaining()];
+ byteBuffer.get(bytes);
+ } else {
+ bytes = (byte[]) avroObject;
+ }
+ return DecimalData.fromUnscaledBytes(bytes,
decimalType.getPrecision(), decimalType.getScale());
+ };
+ }
+
+ private DeserializationRuntimeConverter createArrayConverter(ArrayType
arrayType) {
+ final DeserializationRuntimeConverter elementConverter =
createNullableConverter(arrayType.getElementType());
+ final Class<?> elementClass =
LogicalTypeUtils.toInternalConversionClass(arrayType.getElementType());
+
+ return avroObject -> {
+ final List<?> list = (List<?>) avroObject;
+ final int length = list.size();
+ final Object[] array = (Object[])
Array.newInstance(elementClass, length);
+ for (int i = 0; i < length; ++i) {
+ array[i] =
elementConverter.convert(list.get(i));
+ }
+ return new GenericArrayData(array);
+ };
+ }
+
+ private DeserializationRuntimeConverter createMapConverter(MapType
mapType) {
+ LogicalType keyType = mapType.getKeyType();
+ if (!LogicalTypeChecks.hasFamily(keyType,
LogicalTypeFamily.CHARACTER_STRING)) {
+ throw new UnsupportedOperationException(
+ "Avro format doesn't support non-string as key
type of map. " +
+ "The map type is: " +
mapType.asSummaryString());
+ }
+ final DeserializationRuntimeConverter keyConverter =
createConverter(keyType);
+ final DeserializationRuntimeConverter valueConverter =
createConverter(mapType.getValueType());
+
+ return avroObject -> {
+ final Map<?, ?> map = (Map<?, ?>) avroObject;
+ Map<Object, Object> result = new HashMap<>();
+ for (Map.Entry<?, ?> entry : map.entrySet()) {
+ Object key =
keyConverter.convert(entry.getKey());
+ Object value =
valueConverter.convert(entry.getValue());
+ result.put(key, value);
+ }
+ return new GenericMapData(result);
+ };
+ }
+
+ private TimestampData convertToTimestamp(Object object) {
+ final long millis;
+ if (object instanceof Long) {
+ millis = (Long) object;
+ } else {
+ // use 'provided' Joda time
+ final DateTime value = (DateTime) object;
+ millis = value.toDate().getTime();
Review comment:
This might be not correct, because the `java.util.Date` will use local
time zone.
A simpler way to to this is convert it into sql Timestamp first:
```java
TimestampData.fromTimestamp(new java.sql.Timestamp(value.getMillis());
```
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]