Robert Metzger created FLINK-33217:
--------------------------------------

             Summary: Flink SQL: UNNEST fails with on LEFT JOIN with NOT NULL 
type in array
                 Key: FLINK-33217
                 URL: https://issues.apache.org/jira/browse/FLINK-33217
             Project: Flink
          Issue Type: Bug
          Components: Table SQL / Planner
    Affects Versions: 1.15.3, 1.18.0, 1.19.0
            Reporter: Robert Metzger


Steps to reproduce:

Take a column of type 
{code:java}
business_data ROW<`id` STRING, `updateEvent` ARRAY<ROW<`name` STRING NOT NULL> 
NOT NULL>> {code}
Take this query
{code:java}
select id, ue_name from reproduce_unnest LEFT JOIN 
UNNEST(reproduce_unnest.business_data.updateEvent) AS exploded_ue(ue_name) ON 
true {code}
And get this error
{code:java}
Caused by: java.lang.AssertionError: Type mismatch:rowtype of rel before 
registration: RecordType(RecordType:peek_no_expand(VARCHAR(2147483647) 
CHARACTER SET "UTF-16LE" id, RecordType:peek_no_expand(VARCHAR(2147483647) 
CHARACTER SET "UTF-16LE" NOT NULL name) NOT NULL ARRAY updateEvent) 
business_data, VARCHAR(2147483647) CHARACTER SET "UTF-16LE" ue_name) NOT 
NULLrowtype of rel after registration: 
RecordType(RecordType:peek_no_expand(VARCHAR(2147483647) CHARACTER SET 
"UTF-16LE" id, RecordType:peek_no_expand(VARCHAR(2147483647) CHARACTER SET 
"UTF-16LE" NOT NULL name) NOT NULL ARRAY updateEvent) business_data, 
VARCHAR(2147483647) CHARACTER SET "UTF-16LE" NOT NULL name) NOT 
NULLDifference:ue_name: VARCHAR(2147483647) CHARACTER SET "UTF-16LE" -> 
VARCHAR(2147483647) CHARACTER SET "UTF-16LE" NOT NULL
        at org.apache.calcite.util.Litmus$1.fail(Litmus.java:32)        at 
org.apache.calcite.plan.RelOptUtil.equal(RelOptUtil.java:2206)       at 
org.apache.calcite.rel.AbstractRelNode.onRegister(AbstractRelNode.java:275)  at 
org.apache.calcite.plan.volcano.VolcanoPlanner.registerImpl(VolcanoPlanner.java:1270)
        at 
org.apache.calcite.plan.volcano.VolcanoPlanner.register(VolcanoPlanner.java:598)
     at 
org.apache.calcite.plan.volcano.VolcanoPlanner.ensureRegistered(VolcanoPlanner.java:613)
     at 
org.apache.calcite.plan.volcano.VolcanoPlanner.changeTraits(VolcanoPlanner.java:498)
 at org.apache.calcite.tools.Programs$RuleSetProgram.run(Programs.java:315)     
 at 
org.apache.flink.table.planner.plan.optimize.program.FlinkVolcanoProgram.optimize(FlinkVolcanoProgram.scala:62)
 {code}
I have implemented a small test case, which fails against Flink 1.15, 1.8 and 
the latest master branch.



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

Reply via email to