[
https://issues.apache.org/jira/browse/FLINK-31182?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17692450#comment-17692450
]
Jane Chan edited comment on FLINK-31182 at 2/23/23 6:08 AM:
------------------------------------------------------------
[~Sergey Nuyanzin] Yes, the check on the output type inference strategy was
introduced in FunctionCatalogTableOperator in FLINK-15487 for FLIP-65 a long
time ago.
I guess it worked well before FLIP-190 was introduced
because`LogicalUnnestRule` will infer the output type and convert UNNEST to
EXPLODE. (see [LogicalUnnestRule
L#99|https://github.com/apache/flink/blob/master/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/LogicalUnnestRule.scala#L99]).
As a result, although the BuiltinFunctionDefinition for UNNEST always has a
"MISSING" type inference, it does not affect the execution.
However, after CompiledPlan is introduced, RexNodeJsonDeserializer relies on
the OperatorTable to lookup functions, which always get a static placeholder
(i.e. MISSING) for UNNEST, and due to this check, the deserialization failed.
Could you help to take a look, cc [~godfreyhe] and [~twalthr]
was (Author: qingyue):
[~Sergey Nuyanzin] Yes, the check on the output type inference strategy was
introduced in FunctionCatalogTableOperator in FLINK-15487 for FLIP-65 a long
time ago.
I guess it worked well before FLIP-190 was introduced
because`LogicalUnnestRule` will infer the output type and convert UNNEST to
EXPLODE. (see [LogicalUnnestRule
L#99|https://github.com/apache/flink/blob/master/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/LogicalUnnestRule.scala#L99]).
As a result, although the BuiltinFunctionDefinition for UNNEST always has a
"MISSING" type inference, it does not affect the execution.
However, after CompiledPlan is introduced, RexNodeJsonDeserializer relies on
the OperatorTable to lookup functions, which always get a static placeholder
(i.e. MISSING) for UNNEST, and due to this check, the deserialization failed.
> CompiledPlan cannot deserialize BridgingSqlFunction with MissingTypeStrategy
> ----------------------------------------------------------------------------
>
> 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
> Attachments: screenshot-1.png
>
>
> This issue is reported from the [user mail
> list|https://lists.apache.org/thread/y6fgzyx330omhkr40376knw8k4oczz3s].
> The stacktrace is
> {code:java}
> Unable to find source-code formatter for language: text. Available languages
> are: actionscript, ada, applescript, bash, c, c#, c++, cpp, css, erlang, go,
> groovy, haskell, html, java, javascript, js, json, lua, none, nyan, objc,
> perl, php, python, r, rainbow, ruby, scala, sh, sql, swift, visualbasic, xml,
> yamlCaused 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',
> the output type strategy is "Missing"; as a result,
> FunctionCatalogOperatorTable#convertToBridgingSqlFunction returns empty.
> !screenshot-1.png|width=675,height=295!
--
This message was sent by Atlassian Jira
(v8.20.10#820010)