Skip to content

[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

Open
wants to merge 1 commit into
base: master
Choose a base branch
from

Conversation

pawel-big-lebowski
Copy link

Implement ResultTypeQueryable interface for AvroSerializationSchema and AvroRowDataSerializationSchema to make it exposed via existing lineage interfaces.

Does this pull request potentially affect one of the following parts:

  • Dependencies (does it add or upgrade a dependency): No
  • The public API, i.e., is any changed class annotated with @Public(Evolving): no
  • The serializers: (yes / no / don't know) no
  • The runtime per-record code paths (performance sensitive): (yes / no / don't know) no
  • Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Kubernetes/Yarn, ZooKeeper: (yes / no / don't know) no
  • The S3 file system connector: (yes / no / don't know) no

Documentation

  • Does this pull request introduce a new feature? (no) no

@pawel-big-lebowski pawel-big-lebowski marked this pull request as ready for review April 25, 2025 10:09
@flinkbot
Copy link
Collaborator

flinkbot commented Apr 25, 2025

CI report:

Bot commands The @flinkbot bot supports the following commands:
  • @flinkbot run azure re-run the last Azure build

Copy link
Contributor

@AHeise AHeise left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Looks like the right approach to me.

* @return TypeInformation of the produced type
*/
@Override
public TypeInformation getProducedType() {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This should be TypeInformation<T>. You probably need to cast the returns though but I think it makes things more explicit.

Copy link
Author

@pawel-big-lebowski pawel-big-lebowski Apr 25, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The common type for both cases is ResultTypeQueryable<GenericContainer> as both SpecificRecord and GenericRecord extend GenericContainer. However, I could not make it work for GenericRecordAvroTypeInfo to be casted into TypeInformation<GenericContainer>

Comment on lines +149 to +155
if (schema == null) {
throw new IllegalStateException(
"The produced type is not available before the schema is initialized.");
} else {
return new GenericRecordAvroTypeInfo(schema);
}
}
Copy link
Contributor

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>

Copy link
Author

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?

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

3 participants