-
Notifications
You must be signed in to change notification settings - Fork 13.5k
[FLINK-37729][flink-formats] ResultTypeQueryable for Avro serialization schema #26507
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
base: master
Are you sure you want to change the base?
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -19,12 +19,17 @@ | |
package org.apache.flink.formats.avro; | ||
|
||
import org.apache.flink.api.common.serialization.SerializationSchema; | ||
import org.apache.flink.api.common.typeinfo.TypeInformation; | ||
import org.apache.flink.api.java.typeutils.ResultTypeQueryable; | ||
import org.apache.flink.formats.avro.AvroFormatOptions.AvroEncoding; | ||
import org.apache.flink.formats.avro.typeutils.AvroTypeInfo; | ||
import org.apache.flink.formats.avro.typeutils.GenericRecordAvroTypeInfo; | ||
import org.apache.flink.util.Preconditions; | ||
import org.apache.flink.util.WrappingRuntimeException; | ||
|
||
import org.apache.avro.Schema; | ||
import org.apache.avro.Schema.Parser; | ||
import org.apache.avro.generic.GenericContainer; | ||
import org.apache.avro.generic.GenericData; | ||
import org.apache.avro.generic.GenericDatumWriter; | ||
import org.apache.avro.generic.GenericRecord; | ||
|
@@ -45,7 +50,8 @@ | |
* | ||
* @param <T> the type to be serialized | ||
*/ | ||
public class AvroSerializationSchema<T> implements SerializationSchema<T> { | ||
public class AvroSerializationSchema<T> | ||
implements SerializationSchema<T>, ResultTypeQueryable<GenericContainer> { | ||
|
||
/** | ||
* Creates {@link AvroSerializationSchema} that serializes {@link SpecificRecord} using provided | ||
|
@@ -220,4 +226,22 @@ public boolean equals(Object o) { | |
public int hashCode() { | ||
return Objects.hash(recordClazz, schema); | ||
} | ||
|
||
/** | ||
* Returns the type information of the produced type. Depending on the type of the record, | ||
* whether it is a specific or generic record, the type information will be different. It can be | ||
* either a {@link AvroTypeInfo} or a {@link GenericRecordAvroTypeInfo}. | ||
* | ||
* @return TypeInformation of the produced type | ||
*/ | ||
@Override | ||
public TypeInformation getProducedType() { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This should be There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The common type for both cases is |
||
if (recordClazz != null && SpecificRecord.class.isAssignableFrom(recordClazz)) { | ||
// if the record class is a specific record, we can use the schema to get the type info | ||
return new AvroTypeInfo(recordClazz); | ||
} else { | ||
// generic record | ||
return new GenericRecordAvroTypeInfo(schema); | ||
} | ||
} | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Shouldn't the produced type be of row? Naively I'd expect to have the same type bound to
SerializationSchema<RowData>, ResultTypeQueryable<GenericRecord>
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Do you know any convenient helper to convert
RowData
into sth extending TypeInformation?There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I see that in the Avro deserialization it returns the type as
TypeInformation<RowData>
.I can see in the Parquet Avro and the Orc code it is returning a type.
It would be good if what we do here is consistent with the Parquet Avro case.
I also see that there is a comment at the top of this class saying to keep this class in step with AvroRowDataDeserializationSchema} and schema converter {@link AvroSchemaConverter}. AvroSchemaConverter does deal with Type info.
It would be useful for me to understand what how lineage is calling this and why we end up in this class. Maybe a lineage unit test bringing in an Avro schema and a Parquet Avro schema should be helpful.