[
https://issues.apache.org/jira/browse/FLINK-31182?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Jane Chan updated FLINK-31182:
------------------------------
Description:
This issue is reported from the [user mail
list|https://lists.apache.org/thread/dglf8zgtt1yx3vrdytn0dlv7b3pw1nq6].
The stacktrace is
{code:text}
Caused by: org.apache.flink.table.api.TableException: Could not resolve
internal system function '$UNNEST_ROWS$1'. This is a bug, please file an issue.
at
org.apache.flink.table.planner.plan.nodes.exec.serde.RexNodeJsonDeserializer.deserializeInternalFunction(RexNodeJsonDeserializer.java:392)
at
org.apache.flink.table.planner.plan.nodes.exec.serde.RexNodeJsonDeserializer.deserializeSqlOperator(RexNodeJsonDeserializer.java:337)
at
org.apache.flink.table.planner.plan.nodes.exec.serde.RexNodeJsonDeserializer.deserializeCall(RexNodeJsonDeserializer.java:307)
at
org.apache.flink.table.planner.plan.nodes.exec.serde.RexNodeJsonDeserializer.deserialize(RexNodeJsonDeserializer.java:146)
at
org.apache.flink.table.planner.plan.nodes.exec.serde.RexNodeJsonDeserializer.deserialize(RexNodeJsonDeserializer.java:128)
at
org.apache.flink.table.planner.plan.nodes.exec.serde.RexNodeJsonDeserializer.deserialize(RexNodeJsonDeserializer.java:115)
{code}
The root cause is that although ModuleManager can resolve '$UNNEST_ROWS$1',
FunctionCatalogOperatorTable#convertToBridgingSqlFunction cannot infer output
type strategy.
was:
This issue is reported from the [user mail
list|https://lists.apache.org/thread/dglf8zgtt1yx3vrdytn0dlv7b3pw1nq6].
The stacktrace is
{code:text}
Caused by: org.apache.flink.table.api.TableException: Could not resolve
internal system function '$UNNEST_ROWS$1'. This is a bug, please file an issue.
at
org.apache.flink.table.planner.plan.nodes.exec.serde.RexNodeJsonDeserializer.deserializeInternalFunction(RexNodeJsonDeserializer.java:392)
at
org.apache.flink.table.planner.plan.nodes.exec.serde.RexNodeJsonDeserializer.deserializeSqlOperator(RexNodeJsonDeserializer.java:337)
at
org.apache.flink.table.planner.plan.nodes.exec.serde.RexNodeJsonDeserializer.deserializeCall(RexNodeJsonDeserializer.java:307)
at
org.apache.flink.table.planner.plan.nodes.exec.serde.RexNodeJsonDeserializer.deserialize(RexNodeJsonDeserializer.java:146)
at
org.apache.flink.table.planner.plan.nodes.exec.serde.RexNodeJsonDeserializer.deserialize(RexNodeJsonDeserializer.java:128)
at
org.apache.flink.table.planner.plan.nodes.exec.serde.RexNodeJsonDeserializer.deserialize(RexNodeJsonDeserializer.java:115)
{code}
The root cause can be identified by enabling the DEBUG level for ModuleManager
{code:text}
12089 [main] DEBUG org.apache.flink.table.module.ModuleManager [] - Got
FunctionDefinition '$unnest_rows$1' from 'core' module.
12089 [main] DEBUG org.apache.flink.table.module.ModuleManager [] - Cannot find
FunctionDefinition 'unnest_rows' from any loaded modules.
{code}
> CompiledPlan cannot deserialize BridgingSqlFunction correctly
> -------------------------------------------------------------
>
> Key: FLINK-31182
> URL: https://issues.apache.org/jira/browse/FLINK-31182
> Project: Flink
> Issue Type: Bug
> Components: Table SQL / Planner
> Affects Versions: 1.17.0, 1.18.0, 1.17.1
> Reporter: Jane Chan
> Priority: Major
>
> This issue is reported from the [user mail
> list|https://lists.apache.org/thread/dglf8zgtt1yx3vrdytn0dlv7b3pw1nq6].
> The stacktrace is
> {code:text}
> Caused by: org.apache.flink.table.api.TableException: Could not resolve
> internal system function '$UNNEST_ROWS$1'. This is a bug, please file an
> issue.
> at
> org.apache.flink.table.planner.plan.nodes.exec.serde.RexNodeJsonDeserializer.deserializeInternalFunction(RexNodeJsonDeserializer.java:392)
> at
> org.apache.flink.table.planner.plan.nodes.exec.serde.RexNodeJsonDeserializer.deserializeSqlOperator(RexNodeJsonDeserializer.java:337)
> at
> org.apache.flink.table.planner.plan.nodes.exec.serde.RexNodeJsonDeserializer.deserializeCall(RexNodeJsonDeserializer.java:307)
> at
> org.apache.flink.table.planner.plan.nodes.exec.serde.RexNodeJsonDeserializer.deserialize(RexNodeJsonDeserializer.java:146)
> at
> org.apache.flink.table.planner.plan.nodes.exec.serde.RexNodeJsonDeserializer.deserialize(RexNodeJsonDeserializer.java:128)
> at
> org.apache.flink.table.planner.plan.nodes.exec.serde.RexNodeJsonDeserializer.deserialize(RexNodeJsonDeserializer.java:115)
> {code}
> The root cause is that although ModuleManager can resolve '$UNNEST_ROWS$1',
> FunctionCatalogOperatorTable#convertToBridgingSqlFunction cannot infer output
> type strategy.
--
This message was sent by Atlassian Jira
(v8.20.10#820010)