[ 
https://issues.apache.org/jira/browse/FLINK-34898?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17834685#comment-17834685
 ] 

Feng Jin commented on FLINK-34898:
----------------------------------

[~chloehe]   Thank you for the update, sorry for the late reply. Based on my 
testing, this seems to be an issue with the CAST function implementation. The 
specific reason may be related to this Jira ticket.     
https://issues.apache.org/jira/browse/FLINK-18673 

 

FLINK-18673 added support for ROW() as a parameter for UDFs. During the 
validation of SqlCall, the Validate function's Operand Node will be skipped.

+org.apache.flink.table.planner.calcite.FlinkCalciteSqlValidator#validateColumnListParams+

 
{code:java}
//代码占位符

@Override
public void validateColumnListParams(
        SqlFunction function, List<RelDataType> argTypes, List<SqlNode> 
operands) {
    // we don't support column lists and translate them into the unknown type 
in the type
    // factory,
    // this makes it possible to ignore them in the validator and fall back to 
regular row types
    // see also SqlFunction#deriveType
}
{code}
 

But SqlFunction::deriveType will indirectly call 
org.apache.calcite.sql.validate.SqlValidator#deriveType to infer the type of 
Row().

 

 

!截屏2024-04-07 22.05.40.png!

 

In the SqlCastFunction, it will directly obtaining ValidatedNode type. 

 
{code:java}
//代码占位符

    @Override
    public boolean checkOperandTypes(SqlCallBinding callBinding, boolean 
throwOnFailure) {
        final SqlNode left = callBinding.operand(0);
        final SqlNode right = callBinding.operand(1);
        if (SqlUtil.isNullLiteral(left, false) || left instanceof 
SqlDynamicParam) {
            return true;
        }
        RelDataType validatedNodeType = 
callBinding.getValidator().getValidatedNodeType(left);
//        RelDataType validatedNodeType = SqlTypeUtil.deriveType(callBinding, 
left);
        RelDataType returnType = SqlTypeUtil.deriveType(callBinding, right);
        if (!canCastFrom(returnType, validatedNodeType)) {
            if (throwOnFailure) {
                throw callBinding.newError(
                        RESOURCE.cannotCastValue(
                                validatedNodeType.toString(), 
returnType.toString()));
            }
            return false;
        } {code}
 

 

And the corresponding error message is as follows:

 
{code:java}
//代码占位符

.SqlBasicCall: ROW(1, 2)
        at 
org.apache.flink.table.planner.calcite.FlinkPlannerImpl.org$apache$flink$table$planner$calcite$FlinkPlannerImpl$$validate(FlinkPlannerImpl.scala:200)
        at 
org.apache.flink.table.planner.calcite.FlinkPlannerImpl.validate(FlinkPlannerImpl.scala:117)
        at 
org.apache.flink.table.planner.operations.SqlNodeToOperationConversion.convert(SqlNodeToOperationConversion.java:259)
        at 
org.apache.flink.table.planner.delegation.ParserImpl.parse(ParserImpl.java:106)
        at 
org.apache.flink.table.api.internal.TableEnvironmentImpl.executeSql(TableEnvironmentImpl.java:728)
        at 
org.apache.flink.table.examples.java.basics.WordCountSQLExample.main(WordCountSQLExample.java:49)
Caused by: java.lang.UnsupportedOperationException: class 
org.apache.calcite.sql.SqlBasicCall: ROW(1, 2)
        at org.apache.calcite.util.Util.needToImplement(Util.java:1101)
        at 
org.apache.calcite.sql.validate.SqlValidatorImpl.getValidatedNodeType(SqlValidatorImpl.java:1777)
        at 
org.apache.calcite.sql.fun.SqlCastFunction.checkOperandTypes(SqlCastFunction.java:138)
 {code}
 

 

If we use SqlTypeUtil.deriveType, we can get the correct type. Therefore, I 
think we can modify this section to fix the issue.

 
{code:java}
//代码占位符

RelDataType validatedNodeType = SqlTypeUtil.deriveType(callBinding, left); 
{code}
 

 

 

> Cannot create ARRAY of named STRUCTs
> ------------------------------------
>
>                 Key: FLINK-34898
>                 URL: https://issues.apache.org/jira/browse/FLINK-34898
>             Project: Flink
>          Issue Type: Bug
>    Affects Versions: 1.19.0
>            Reporter: Chloe He
>            Priority: Major
>         Attachments: image-2024-03-21-12-00-00-183.png, 截屏2024-04-07 
> 22.05.40.png
>
>
> I want to construct data that consists of arrays of named STRUCT. For 
> example, one field may look like `[\{"a": 1}]`. I am able to construct this 
> named STRUCT as
> {code:java}
> SELECT CAST(ROW(1) as ROW<a INT>) AS row1;  {code}
> but when I try to wrap this in an ARRAY, it fails:
> {code:java}
> SELECT ARRAY[CAST(ROW(1) as ROW<a INT>)] AS row1;  
> // error
> Caused by: java.lang.UnsupportedOperationException: class 
> org.apache.calcite.sql.SqlBasicCall: ROW(1)
> {code}
> These are the workarounds that I found:
> {code:java}
> SELECT ROW(ROW(CAST(ROW(1) as ROW<a INT>))) AS row1; 
> // or
> SELECT cast(ARRAY[ROW(1)] as ARRAY<ROW<a INT>>); {code}
> but I think this is a bug that we need to follow up and fix.



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

Reply via email to