[ https://issues.apache.org/jira/browse/FLINK-35887?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17868333#comment-17868333 ]
Jacob Jona Fahlenkamp commented on FLINK-35887: ----------------------------------------------- [~dylanhz] Thanks. Without the TypeInfo the following exception is thrown: {code:java} java.lang.UnsupportedOperationException: Generic types have been disabled in the ExecutionConfig and type DebugTest$Foo is treated as a generic type. at org.apache.flink.api.java.typeutils.GenericTypeInfo.createSerializer(GenericTypeInfo.java:88) at org.apache.flink.api.java.typeutils.PojoTypeInfo.createPojoSerializer(PojoTypeInfo.java:355) at org.apache.flink.api.java.typeutils.PojoTypeInfo.createSerializer(PojoTypeInfo.java:347) at org.apache.flink.connector.datagen.functions.FromElementsGeneratorFunction.<init>(FromElementsGeneratorFunction.java:85) at org.apache.flink.streaming.api.environment.StreamExecutionEnvironment.fromData(StreamExecutionEnvironment.java:1208) at org.apache.flink.streaming.api.environment.StreamExecutionEnvironment.fromData(StreamExecutionEnvironment.java:1163) at DebugTest.test(DebugTest.java:50) {code} for example for this code: {code:java} import lombok.AllArgsConstructor; import lombok.Data; import lombok.NoArgsConstructor; import org.apache.flink.api.common.typeinfo.TypeInfoFactory; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeinfo.Types; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.PipelineOptions; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; import java.lang.reflect.Type; import java.util.Map; public class DebugTest { //@TypeInfo(FooFactory.class) <- with this uncommented it works in version 1.17.2 public interface Foo{} @Data @NoArgsConstructor @AllArgsConstructor public static class FooImpl implements Foo{ int i; } @Data @NoArgsConstructor @AllArgsConstructor public static class Bar{ Foo f; } public static class FooFactory extends TypeInfoFactory<Foo> { @Override public TypeInformation<Foo> createTypeInfo(Type type, Map<String, TypeInformation<?>> map) { return Types.POJO(Foo.class, Map.of()); } } @Test void test() throws Exception { var env = StreamExecutionEnvironment.getExecutionEnvironment( new Configuration().set(PipelineOptions.GENERIC_TYPES, false) // if ^this^ is removed it will work but still write a warning: // "Field Bar#f will be processed as GenericType. Please read the Flink documentation on "Data Types & // Serialization" for details of the effect on performance and schema evolution." ); var iterator = env.fromData(new Bar(new FooImpl(7))).executeAndCollect(); Assertions.assertEquals(7, ((FooImpl) iterator.next().getF()).getI()); } } {code} > Null Pointer Exception in TypeExtractor.isRecord when trying to provide type > info for interface > ----------------------------------------------------------------------------------------------- > > Key: FLINK-35887 > URL: https://issues.apache.org/jira/browse/FLINK-35887 > Project: Flink > Issue Type: Bug > Components: API / Type Serialization System > Affects Versions: 1.19.1 > Reporter: Jacob Jona Fahlenkamp > Priority: Major > > The following code > {code:java} > import org.apache.flink.api.common.typeinfo.TypeInfo; > import org.apache.flink.api.common.typeinfo.TypeInfoFactory; > import org.apache.flink.api.common.typeinfo.TypeInformation; > import org.apache.flink.api.common.typeinfo.Types; > import org.apache.flink.types.PojoTestUtils; > import org.junit.jupiter.api.Test; > import java.lang.reflect.Type; > import java.util.Map; > public class DebugTest { > @TypeInfo(FooFactory.class) > public interface Foo{} > public static class FooFactory extends TypeInfoFactory<Foo> { > @Override > public TypeInformation<Foo> createTypeInfo(Type type, Map<String, > TypeInformation<?>> map) { > return Types.POJO(Foo.class, Map.of()); > } > } > @Test > void test() { > PojoTestUtils.assertSerializedAsPojo(Foo.class); > } > } {code} > throws this exception: > {code:java} > java.lang.NullPointerException: Cannot invoke "java.lang.Class.getName()" > because the return value of "java.lang.Class.getSuperclass()" is null > at > org.apache.flink.api.java.typeutils.TypeExtractor.isRecord(TypeExtractor.java:2227) > at > org.apache.flink.api.java.typeutils.runtime.PojoSerializer.<init>(PojoSerializer.java:125) > at > org.apache.flink.api.java.typeutils.PojoTypeInfo.createPojoSerializer(PojoTypeInfo.java:359) > at > org.apache.flink.api.java.typeutils.PojoTypeInfo.createSerializer(PojoTypeInfo.java:347) > at > org.apache.flink.types.PojoTestUtils.assertSerializedAsPojo(PojoTestUtils.java:48) > {code} -- This message was sent by Atlassian Jira (v8.20.10#820010)