Ferenc Csaky created FLINK-37820:
------------------------------------

             Summary: AsyncScalarFunction UDFs cannot be loaded via CompiledPlan
                 Key: FLINK-37820
                 URL: https://issues.apache.org/jira/browse/FLINK-37820
             Project: Flink
          Issue Type: Bug
    Affects Versions: 1.20.1, 1.19.2, 2.0.0
            Reporter: Ferenc Csaky
            Assignee: Ferenc Csaky
             Fix For: 1.19.3, 1.20.2, 2.0.1


If you try to load a {{CompiledPlan}} that contains any UDF that extends 
{{AsyncScalarFunction}} and is not loaded already, the plan will fail to load, 
cause {{RexNodeJsonDeserializer}} does not know about the {{ASYNC_SCALAR}} 
function kind 
[here|https://github.com/apache/flink/blob/a48d09049b1e97362b80ffce5d5077ce0654fda1/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/RexNodeJsonDeserializer.java#L471],
 and the following exception is thrown:
{code}
Exception in thread "main" org.apache.flink.table.api.TableException: Cannot 
load Plan: {...}.
        at 
org.apache.flink.table.api.internal.TableEnvironmentImpl.loadPlan(TableEnvironmentImpl.java:760)
                ...
Caused by: 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonMappingException:
 Unsupported anonymous function kind 'ASYNC_SCALAR' for class 
'com.myudf.MyAsyncScalarFunction'. (through reference chain: 
org.apache.flink.table.planner.plan.nodes.exec.serde.JsonPlanGraph["nodes"]->java.util.ArrayList[1]->org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecAsyncCalc["projection"]->java.util.ArrayList[1])
        at 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonMappingException.wrapWithPath(JsonMappingException.java:402)
        at 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonMappingException.wrapWithPath(JsonMappingException.java:373)
        at 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.deser.std.CollectionDeserializer._deserializeFromArray(CollectionDeserializer.java:375)
        at 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.deser.std.CollectionDeserializer.deserialize(CollectionDeserializer.java:244)
        at 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.deser.std.CollectionDeserializer.deserialize(CollectionDeserializer.java:28)
        at 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.deser.SettableBeanProperty.deserialize(SettableBeanProperty.java:542)
        at 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.deser.BeanDeserializer._deserializeWithErrorWrapping(BeanDeserializer.java:564)
        at 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.deser.BeanDeserializer._deserializeUsingPropertyBased(BeanDeserializer.java:439)
        at 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.deser.BeanDeserializerBase.deserializeFromObjectUsingNonDefault(BeanDeserializerBase.java:1405)
        at 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.deser.BeanDeserializer.deserializeFromObject(BeanDeserializer.java:352)
        at 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.deser.BeanDeserializer._deserializeOther(BeanDeserializer.java:220)
        at 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.deser.BeanDeserializer.deserialize(BeanDeserializer.java:187)
        at 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.jsontype.impl.AsPropertyTypeDeserializer._deserializeTypedForId(AsPropertyTypeDeserializer.java:144)
        at 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.jsontype.impl.AsPropertyTypeDeserializer.deserializeTypedFromObject(AsPropertyTypeDeserializer.java:110)
        at 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.deser.AbstractDeserializer.deserializeWithType(AbstractDeserializer.java:263)
        at 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.deser.std.CollectionDeserializer._deserializeFromArray(CollectionDeserializer.java:361)
        at 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.deser.std.CollectionDeserializer.deserialize(CollectionDeserializer.java:244)
        at 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.deser.std.CollectionDeserializer.deserialize(CollectionDeserializer.java:28)
        at 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.deser.SettableBeanProperty.deserialize(SettableBeanProperty.java:542)
        at 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.deser.BeanDeserializer._deserializeWithErrorWrapping(BeanDeserializer.java:564)
        at 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.deser.BeanDeserializer._deserializeUsingPropertyBased(BeanDeserializer.java:439)
        at 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.deser.BeanDeserializerBase.deserializeFromObjectUsingNonDefault(BeanDeserializerBase.java:1405)
        at 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.deser.BeanDeserializer.deserializeFromObject(BeanDeserializer.java:352)
        at 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.deser.BeanDeserializer.deserialize(BeanDeserializer.java:185)
        at 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.DeserializationContext.readValue(DeserializationContext.java:971)
        at 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.DeserializationContext.readValue(DeserializationContext.java:958)
        at 
org.apache.flink.table.planner.plan.nodes.exec.serde.ExecNodeGraphJsonDeserializer.deserialize(ExecNodeGraphJsonDeserializer.java:50)
        at 
org.apache.flink.table.planner.plan.nodes.exec.serde.ExecNodeGraphJsonDeserializer.deserialize(ExecNodeGraphJsonDeserializer.java:37)
        at 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.deser.DefaultDeserializationContext.readRootValue(DefaultDeserializationContext.java:323)
        at 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectReader._bindAndClose(ObjectReader.java:2105)
        at 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectReader.readValue(ObjectReader.java:1546)
        at 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectReader.readValue(ObjectReader.java:1566)
        at 
org.apache.flink.table.planner.delegation.StreamPlanner.loadPlan(StreamPlanner.scala:179)
        at 
org.apache.flink.table.api.internal.TableEnvironmentImpl.loadPlan(TableEnvironmentImpl.java:758)
        ... 4 more
Caused by: org.apache.flink.table.api.TableException: Unsupported anonymous 
function kind 'ASYNC_SCALAR' for class 'com.myudf.MyAsyncScalarFunction'.
        at 
org.apache.flink.table.planner.plan.nodes.exec.serde.RexNodeJsonDeserializer.deserializeFunctionClass(RexNodeJsonDeserializer.java:457)
        at 
org.apache.flink.table.planner.plan.nodes.exec.serde.RexNodeJsonDeserializer.deserializeCatalogFunction(RexNodeJsonDeserializer.java:480)
        at 
org.apache.flink.table.planner.plan.nodes.exec.serde.RexNodeJsonDeserializer.deserializeSqlOperator(RexNodeJsonDeserializer.java:351)
        at 
org.apache.flink.table.planner.plan.nodes.exec.serde.RexNodeJsonDeserializer.deserializeCall(RexNodeJsonDeserializer.java:318)
        at 
org.apache.flink.table.planner.plan.nodes.exec.serde.RexNodeJsonDeserializer.deserialize(RexNodeJsonDeserializer.java:148)
        at 
org.apache.flink.table.planner.plan.nodes.exec.serde.RexNodeJsonDeserializer.deserialize(RexNodeJsonDeserializer.java:130)
        at 
org.apache.flink.table.planner.plan.nodes.exec.serde.RexNodeJsonDeserializer.deserialize(RexNodeJsonDeserializer.java:117)
        at 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.deser.std.CollectionDeserializer._deserializeFromArray(CollectionDeserializer.java:359)
        ... 35 more
{code}



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

Reply via email to