Description
Consider the following code:
DataSet<FooBarPojo> d1 = env.fromElements(new FooBarPojo());
DataSet<Tuple2<FooBarPojo, FooBarPojo>> d2 = d1.map(new MapFunction<FooBarPojo, Tuple2<FooBarPojo, FooBarPojo>>() {
@Override
public Tuple2<FooBarPojo, FooBarPojo> map(FooBarPojo value) throws Exception
});
where FooBarPojo is the following type:
public class FooBarPojo {
public int foo, bar;
public FooBarPojo() {}
}
This should print a tuple type with two identical fields:
Java Tuple2<PojoType<FooBarPojo, fields = [bar: Integer, foo: Integer]>, PojoType<FooBarPojo, fields = [bar: Integer, foo: Integer]>>
But it prints the following instead:
Java Tuple2<PojoType<FooBarPojo, fields = [bar: Integer, foo: Integer]>, GenericType<FooBarPojo>>
Note, that this problem causes some co-groups in Gelly to crash with "org.apache.flink.api.common.InvalidProgramException: The pair of co-group keys are not compatible with each other" when the vertex ID type is a POJO, because the second field of the Edge type gets to be a generic type, but the POJO gets recognized in the Vertex type, and getNumberOfKeyFields returns different numbers for the POJO and the generic type.
The source of the problem is the mechanism in TypeExtractor that would detect recursive types (see the "alreadySeen" field in TypeExtractor), as it mistakes the second appearance of FooBarPojo with a recursive field.
Specifically the following happens: createTypeInfoWithTypeHierarchy starts to process the Tuple2<FooBarPojo, FooBarPojo> type, and in line 434 it calls itself for the first field, which proceeds into the privateGetForClass case which correctly detects that it is a POJO, and correctly returns a PojoTypeInfo; but in the meantime in line 1191, privateGetForClass adds PojoTypeInfo to "alreadySeen". Then the outer createTypeInfoWithTypeHierarchy approaches the second field, goes into privateGetForClass, which mistakenly returns a GenericTypeInfo, as it thinks in line 1187, that a recursive type is being processed.
(Note, that if we comment out the recursive type detection (the lines that do their thing with the alreadySeen field), then the output is correct.)