Daniel Lopes created FLINK-37297: ------------------------------------ Summary: Either type Pojo deserialization fails with NPE Key: FLINK-37297 URL: https://issues.apache.org/jira/browse/FLINK-37297 Project: Flink Issue Type: Bug Components: API / Core Affects Versions: 1.20.0 Reporter: Daniel Lopes
To replicate: When running a local test with StreamExecutionEnvironment with generic types turned off and using a lombok POJO with four different Either fields. When these fields are null or containing a Left instance, the POJO fails deserialization at this [line|https://github.com/apache/flink/blob/9b411b3f1f7f2caae7e653e7630386376e13bcc3/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/EitherSerializer.java#L88-L88] of the code where EitherSerializer "always creates a Right value instance", thus throwing NullPointerException in this circumstance. This behavior is not observed when composing/wrapping Either types inside Java Records. Stack trace: {code:java} Caused by: java.lang.RuntimeException: Cannot instantiate class. at org.apache.flink.api.java.typeutils.runtime.PojoSerializer.createInstance(PojoSerializer.java:214) at org.apache.flink.api.java.typeutils.runtime.PojoSerializer.deserialize(PojoSerializer.java:446) at org.apache.flink.streaming.runtime.streamrecord.StreamElementSerializer.deserialize(StreamElementSerializer.java:204) at org.apache.flink.streaming.runtime.streamrecord.StreamElementSerializer.deserialize(StreamElementSerializer.java:44) at org.apache.flink.runtime.plugable.NonReusingDeserializationDelegate.read(NonReusingDeserializationDelegate.java:53) at org.apache.flink.runtime.io.network.api.serialization.NonSpanningWrapper.readInto(NonSpanningWrapper.java:337) at org.apache.flink.runtime.io.network.api.serialization.SpillingAdaptiveSpanningRecordDeserializer.readNonSpanningRecord(SpillingAdaptiveSpanningRecordDeserializer.java:128) at org.apache.flink.runtime.io.network.api.serialization.SpillingAdaptiveSpanningRecordDeserializer.readNextRecord(SpillingAdaptiveSpanningRecordDeserializer.java:103) at org.apache.flink.runtime.io.network.api.serialization.SpillingAdaptiveSpanningRecordDeserializer.getNextRecord(SpillingAdaptiveSpanningRecordDeserializer.java:93) at org.apache.flink.streaming.runtime.io.AbstractStreamTaskNetworkInput.emitNext(AbstractStreamTaskNetworkInput.java:103) at org.apache.flink.streaming.runtime.io.StreamOneInputProcessor.processInput(StreamOneInputProcessor.java:65) at org.apache.flink.streaming.runtime.tasks.StreamTask.processInput(StreamTask.java:638) at org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.runMailboxLoop(MailboxProcessor.java:231) at org.apache.flink.streaming.runtime.tasks.StreamTask.runMailboxLoop(StreamTask.java:973) at org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:917) at org.apache.flink.runtime.taskmanager.Task.runWithSystemExitMonitoring(Task.java:970) at org.apache.flink.runtime.taskmanager.Task.restoreAndInvoke(Task.java:949) at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:763) at org.apache.flink.runtime.taskmanager.Task.run(Task.java:575) at java.base/java.lang.Thread.run(Thread.java:840) Caused by: java.lang.NullPointerException at java.base/java.util.Objects.requireNonNull(Objects.java:209) at org.apache.flink.types.Either$Right.<init>(Either.java:146) at org.apache.flink.types.Either.Right(Either.java:46) at org.apache.flink.api.java.typeutils.runtime.EitherSerializer.createInstance(EitherSerializer.java:89) at org.apache.flink.api.java.typeutils.runtime.EitherSerializer.createInstance(EitherSerializer.java:38) at org.apache.flink.api.java.typeutils.runtime.PojoSerializer.initializeFields(PojoSerializer.java:222) at org.apache.flink.api.java.typeutils.runtime.PojoSerializer.createInstance(PojoSerializer.java:211) ... 19 more {code} -- This message was sent by Atlassian Jira (v8.20.10#820010)