[ https://issues.apache.org/jira/browse/FLINK-31755?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17713864#comment-17713864 ]
Aitozi commented on FLINK-31755: -------------------------------- CC [~lincoln.86xy] [~snuyanzin] > ROW function can not work with RewriteIntersectAllRule > ------------------------------------------------------ > > Key: FLINK-31755 > URL: https://issues.apache.org/jira/browse/FLINK-31755 > Project: Flink > Issue Type: Bug > Components: Table SQL / Planner > Reporter: Aitozi > Priority: Major > Labels: pull-request-available > > Reproduce case: > {code:java} > create table row_sink ( > `b` ROW<b1 STRING, b2 INT> > ) with ( > 'connector' = 'values' > ) > util.verifyRelPlanInsert( > "INSERT INTO row_sink " + > "SELECT ROW(a, b) FROM complex_type_src intersect all " + > "SELECT ROW(c, d) FROM complex_type_src ") > {code} > It will fails with > {code:java} > Caused by: java.lang.IllegalArgumentException: Type mismatch: > rel rowtype: RecordType(RecordType:peek_no_expand(VARCHAR(2147483647) > CHARACTER SET "UTF-16LE" EXPR$0, INTEGER EXPR$1) NOT NULL EXPR$0) NOT NULL > equiv rowtype: RecordType(RecordType(VARCHAR(2147483647) CHARACTER SET > "UTF-16LE" EXPR$0, INTEGER EXPR$1) NOT NULL EXPR$0) NOT NULL > Difference: > EXPR$0: RecordType:peek_no_expand(VARCHAR(2147483647) CHARACTER SET > "UTF-16LE" EXPR$0, INTEGER EXPR$1) NOT NULL -> RecordType(VARCHAR(2147483647) > CHARACTER SET "UTF-16LE" EXPR$0, INTEGER EXPR$1) NOT NULL > at > org.apache.calcite.plan.volcano.VolcanoPlanner.register(VolcanoPlanner.java:592) > at > org.apache.calcite.plan.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:613) > at > org.apache.calcite.plan.volcano.VolcanoRuleCall.transformTo(VolcanoRuleCall.java:144) > ... 68 more > {code} > The reason is: > ROW function will generates the {{FULLY_QUALIFIED}} type. But after the > {{RewriteIntersectAllRule}} optimization, it will produce the > {{PEEK_FIELDS_NO_EXPAND}}. So the volcano planner complains with type > mismatch. -- This message was sent by Atlassian Jira (v8.20.10#820010)