After thinking about it, I think an even better solution is to provide an interface for the TypeExtractor where the user can register mappings from class to TypeInformation. So that the TypeExctractor is more extensible. This would also solve you problem. What do you think?

On 02.03.2016 15:00, Gyula Fóra wrote:
Hi!

Yes I think, that sounds good :) We just need to make sure that this works
with things like the TupleTypeInfo which is built-on but I can still mix in
new Types for the fields.

  Thanks,
Gyula

Timo Walther <twal...@apache.org> ezt írta (időpont: 2016. márc. 2., Sze,
14:02):

The TypeExtractor's input type validation was designed for the built-in
TypeInformation classes.

In your case of a new, unknown TypeInformation, the validation should
simply skipped, because we can assume that you user knows what he is doing.
I can open a PR for that.


On 02.03.2016 11:34, Aljoscha Krettek wrote:
I think you have a point. Another user also just ran into problems with
the TypeExtractor. (The “Java Maps and TypeInformation” email).
So let’s figure out what needs to be changed to make it work for all
people.
Cheers,
Aljoscha
On 02 Mar 2016, at 11:15, Gyula Fóra <gyf...@apache.org> wrote:

Hey,

I have brought up this issue a couple months back but I would like to
do it
again.

I think the current way of validating the input type of udfs against the
out type of the preceeding operators is too aggressive and breaks a lot
of
code that should otherwise work.

This issue appears all the time when I want to use my own
TypeInformations<> for operators such as creating my own Tuple typeinfos
with custom types for the different fields and so.

I have a more complex streaming job which would not run if I have the
input
type validation. Replacing the Exceptions with logging my Job runs
perfectly (making my point) but you can see the errors that would have
been
reported as exceptions in the logs:

2016-03-02 11:06:03,447 ERROR
org.apache.flink.api.java.typeutils.TypeExtractor - Input mismatch:
Generic
object type ‘mypackage.TestEvent' expected but was ‘mypackage.Event’.
2016-03-02 11:06:03,450 ERROR
org.apache.flink.api.java.typeutils.TypeExtractor - Input mismatch:
Unknown
Error. Type is null.
2016-03-02 11:06:03,466 ERROR
org.apache.flink.api.java.typeutils.TypeExtractor - Input mismatch:
Basic
type expected.
2016-03-02 11:06:03,470 ERROR
org.apache.flink.api.java.typeutils.TypeExtractor - Input mismatch:
Basic
type expected.

Clearly all these errors where not valid in my case as my job runs
perfectly.

Would it make sense to change the current behaviour or am I just abusing
the .returns(..) and ResultTypeQueryable interfaces in unintended ways.

Cheers,
Gyula


Reply via email to