[ 
https://issues.apache.org/jira/browse/FLINK-39291?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

ASF GitHub Bot updated FLINK-39291:
-----------------------------------
    Labels: pull-request-available  (was: )

> FlinkScalaKryoInstantiator uses StdInstantiatorStrategy directly, bypassing 
> constructors and causing NPE for classes requiring initialization
> ---------------------------------------------------------------------------------------------------------------------------------------------
>
>                 Key: FLINK-39291
>                 URL: https://issues.apache.org/jira/browse/FLINK-39291
>             Project: Flink
>          Issue Type: Improvement
>          Components: API / Type Serialization System
>            Reporter: Liu
>            Priority: Major
>              Labels: pull-request-available
>
> h1. Problem
> FlinkScalaKryoInstantiator.newKryo() sets the InstantiatorStrategy to 
> StdInstantiatorStrategy directly, which uses Objenesis to create instances 
> without invoking any constructor. This is inconsistent with 
> KryoSerializer.getKryoInstance()'s fallback path, which uses 
> DefaultInstantiatorStrategy with StdInstantiatorStrategy as a fallback — 
> meaning it first attempts to invoke the no-arg constructor and only falls 
> back to Objenesis if that fails.
> When flink-table-api-scala is on the classpath (e.g., added to the Flink lib 
> directory), KryoSerializer will load FlinkScalaKryoInstantiator and use the 
> Kryo instance it creates. Any class that relies on its no-arg constructor to 
> initialize internal state will then fail during deserialization, because the 
> constructor is never invoked.
> h1. Reproduction
> A concrete example is Apache Iceberg's SerializableByteBufferMap, which 
> initializes its wrapped field (Map<Integer, ByteBuffer>) in the no-arg 
> constructor. When deserialized via Kryo's MapSerializer, 
> MapSerializer.create() calls kryo.newInstance(), which (under 
> StdInstantiatorStrategy) bypasses the constructor, leaving wrapped = null. 
> Subsequently, MapSerializer.read() calls map.put(key, value), which delegates 
> to wrapped.put() — resulting in a NullPointerException.
> h1. Root Cause
> In FlinkScalaKryoInstantiator.scala (line 63):
>  k.setInstantiatorStrategy(new org.objenesis.strategy.StdInstantiatorStrategy)
> This was inherited from the original Twitter Chill code and preserved during 
> the Kryo 5 upgrade (FLINK-3154).
> h1. Fix
> Align FlinkScalaKryoInstantiator with KryoSerializer's default strategy:
> {code:java}
> val initStrategy = new 
> com.esotericsoftware.kryo.util.DefaultInstantiatorStrategy()
> initStrategy.setFallbackInstantiatorStrategy(new 
> org.objenesis.strategy.StdInstantiatorStrategy)
> k.setInstantiatorStrategy(initStrategy) {code}
> This change:
>  * First attempts to use the no-arg constructor (via 
> DefaultInstantiatorStrategy)
>  * Falls back to Objenesis only if no constructor is available
>  * Is backward-compatible: all Scala types with registered custom serializers 
> are unaffected
>  * Eliminates the behavioral inconsistency between the two Kryo 
> initialization code paths in KryoSerializer



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to