Dawid Wysakowicz created FLINK-35619: ----------------------------------------
Summary: Window rank query fails with "must call validate first" Key: FLINK-35619 URL: https://issues.apache.org/jira/browse/FLINK-35619 Project: Flink Issue Type: Bug Components: Table SQL / Planner Reporter: Dawid Wysakowicz Assignee: Dawid Wysakowicz Fix For: 1.20.0 A program: {code} static final TableTestProgram WINDOW_RANK_HOP_TVF_NAMED_MIN_TOP_1 = TableTestProgram.of( "window-rank-hop-tvf-named-min-top-n", "validates window min top-n follows after hop window") .setupTableSource(SourceTestStep.newBuilder("bid_t") .addSchema( "ts STRING", "price DECIMAL(10,2)", "supplier_id STRING", "`bid_time` AS TO_TIMESTAMP(`ts`)", "WATERMARK for `bid_time` AS `bid_time` - INTERVAL '1' SECOND") .producedValues( Row.of( "2020-04-15 08:00:05", new BigDecimal(4.00), "supplier1")) .build()) .setupTableSink( SinkTestStep.newBuilder("sink_t") .addSchema("bid_time TIMESTAMP(3)", "supplier_id STRING") .consumedValues( "+I[2020-04-15T08:00:05, supplier1]", "+I[2020-04-15T08:00:05, supplier1]") .build()) .runSql("INSERT INTO sink_t(bid_time, supplier_id) " + "SELECT bid_time, supplier_id\n" + " FROM (\n" + " SELECT\n" + " bid_time,\n" + " supplier_id,\n" + " ROW_NUMBER() OVER (PARTITION BY window_start, window_end ORDER BY price ASC) AS row_num\n" + " FROM TABLE(HOP(\n" + " DATA => TABLE bid_t,\n" + " TIMECOL => DESCRIPTOR(`bid_time`),\n" + " SLIDE => INTERVAL '5' SECOND,\n" + " SIZE => INTERVAL '10' SECOND))\n" + " ) WHERE row_num <= 3") .build(); {code} fails with: {code} java.lang.AssertionError: must call validate first at org.apache.calcite.sql.validate.IdentifierNamespace.resolve(IdentifierNamespace.java:256) at org.apache.calcite.sql2rel.SqlToRelConverter.convertIdentifier(SqlToRelConverter.java:2871) at org.apache.calcite.sql2rel.SqlToRelConverter.convertFrom(SqlToRelConverter.java:2464) at org.apache.calcite.sql2rel.SqlToRelConverter.convertFrom(SqlToRelConverter.java:2378) at org.apache.calcite.sql2rel.SqlToRelConverter.convertFrom(SqlToRelConverter.java:2323) at org.apache.calcite.sql2rel.SqlToRelConverter.convertSelectImpl(SqlToRelConverter.java:730) at org.apache.calcite.sql2rel.SqlToRelConverter.convertSelect(SqlToRelConverter.java:716) at org.apache.calcite.sql2rel.SqlToRelConverter.convertQueryRecursive(SqlToRelConverter.java:3880) at org.apache.calcite.sql2rel.SqlToRelConverter.convertQueryOrInList(SqlToRelConverter.java:1912) at org.apache.calcite.sql2rel.SqlToRelConverter.convertExists(SqlToRelConverter.java:1895) at org.apache.calcite.sql2rel.SqlToRelConverter.substituteSubQuery(SqlToRelConverter.java:1421) at org.apache.calcite.sql2rel.SqlToRelConverter.replaceSubQueries(SqlToRelConverter.java:1161) at org.apache.calcite.sql2rel.SqlToRelConverter.convertCollectionTable(SqlToRelConverter.java:2928) at org.apache.calcite.sql2rel.SqlToRelConverter.convertFrom(SqlToRelConverter.java:2511) at org.apache.calcite.sql2rel.SqlToRelConverter.convertFrom(SqlToRelConverter.java:2378) at org.apache.calcite.sql2rel.SqlToRelConverter.convertFrom(SqlToRelConverter.java:2323) at org.apache.calcite.sql2rel.SqlToRelConverter.convertSelectImpl(SqlToRelConverter.java:730) at org.apache.calcite.sql2rel.SqlToRelConverter.convertSelect(SqlToRelConverter.java:716) at org.apache.calcite.sql2rel.SqlToRelConverter.convertQueryRecursive(SqlToRelConverter.java:3880) at org.apache.calcite.sql2rel.SqlToRelConverter.convertFrom(SqlToRelConverter.java:2490) at org.apache.calcite.sql2rel.SqlToRelConverter.convertFrom(SqlToRelConverter.java:2378) at org.apache.calcite.sql2rel.SqlToRelConverter.convertFrom(SqlToRelConverter.java:2323) at org.apache.calcite.sql2rel.SqlToRelConverter.convertSelectImpl(SqlToRelConverter.java:730) at org.apache.calcite.sql2rel.SqlToRelConverter.convertSelect(SqlToRelConverter.java:716) at org.apache.calcite.sql2rel.SqlToRelConverter.convertQueryRecursive(SqlToRelConverter.java:3880) at org.apache.calcite.sql2rel.SqlToRelConverter.convertQuery(SqlToRelConverter.java:620) at org.apache.flink.table.planner.calcite.FlinkPlannerImpl.org$apache$flink$table$planner$calcite$FlinkPlannerImpl$$rel(FlinkPlannerImpl.scala:233) at org.apache.flink.table.planner.calcite.FlinkPlannerImpl.rel(FlinkPlannerImpl.scala:208) at org.apache.flink.table.planner.operations.SqlNodeConvertContext.toRelRoot(SqlNodeConvertContext.java:77) at org.apache.flink.table.planner.operations.converters.SqlQueryConverter.convertSqlNode(SqlQueryConverter.java:48) at org.apache.flink.table.planner.operations.converters.SqlNodeConverters.convertSqlNode(SqlNodeConverters.java:83) at org.apache.flink.table.planner.operations.SqlNodeToOperationConversion.convertValidatedSqlNode(SqlNodeToOperationConversion.java:270) at org.apache.flink.table.planner.operations.SqlNodeToOperationConversion.convertValidatedSqlNodeOrFail(SqlNodeToOperationConversion.java:386) at org.apache.flink.table.planner.operations.SqlNodeToOperationConversion.convertSqlInsert(SqlNodeToOperationConversion.java:743) at org.apache.flink.table.planner.operations.SqlNodeToOperationConversion.convertValidatedSqlNode(SqlNodeToOperationConversion.java:349) at org.apache.flink.table.planner.operations.SqlNodeToOperationConversion.convert(SqlNodeToOperationConversion.java:260) 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.test.program.SqlTestStep.apply(SqlTestStep.java:44) at org.apache.flink.table.planner.plan.nodes.exec.stream.MiscTests.generateTestSetupFiles(MiscTests.java:98) at java.base/java.lang.reflect.Method.invoke(Method.java:566) at java.base/java.util.stream.ForEachOps$ForEachOp$OfRef.accept(ForEachOps.java:183) at java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:195) at java.base/java.util.stream.ReferencePipeline$2$1.accept(ReferencePipeline.java:177) at java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:195) at java.base/java.util.stream.ForEachOps$ForEachOp$OfRef.accept(ForEachOps.java:183) at java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:195) at java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:195) at java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:195) at java.base/java.util.stream.ForEachOps$ForEachOp$OfRef.accept(ForEachOps.java:183) at java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:195) at java.base/java.util.AbstractList$RandomAccessSpliterator.forEachRemaining(AbstractList.java:720) at java.base/java.util.stream.ReferencePipeline$Head.forEach(ReferencePipeline.java:658) at java.base/java.util.stream.ReferencePipeline$7$1.accept(ReferencePipeline.java:274) at java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:195) at java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:195) at java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:195) at java.base/java.util.Spliterators$ArraySpliterator.forEachRemaining(Spliterators.java:948) at java.base/java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:484) at java.base/java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:474) at java.base/java.util.stream.ForEachOps$ForEachOp.evaluateSequential(ForEachOps.java:150) at java.base/java.util.stream.ForEachOps$ForEachOp$OfRef.evaluateSequential(ForEachOps.java:173) at java.base/java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234) at java.base/java.util.stream.ReferencePipeline.forEach(ReferencePipeline.java:497) at java.base/java.util.stream.ReferencePipeline$7$1.accept(ReferencePipeline.java:274) at java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:195) at java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:195) at java.base/java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:195) at java.base/java.util.ArrayList$ArrayListSpliterator.forEachRemaining(ArrayList.java:1655) at java.base/java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:484) at java.base/java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:474) at java.base/java.util.stream.ForEachOps$ForEachOp.evaluateSequential(ForEachOps.java:150) at java.base/java.util.stream.ForEachOps$ForEachOp$OfRef.evaluateSequential(ForEachOps.java:173) at java.base/java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234) at java.base/java.util.stream.ReferencePipeline.forEach(ReferencePipeline.java:497) at java.base/java.util.stream.ReferencePipeline$7$1.accept(ReferencePipeline.java:274) at java.base/java.util.ArrayList$ArrayListSpliterator.forEachRemaining(ArrayList.java:1655) at java.base/java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:484) at java.base/java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:474) at java.base/java.util.stream.ForEachOps$ForEachOp.evaluateSequential(ForEachOps.java:150) at java.base/java.util.stream.ForEachOps$ForEachOp$OfRef.evaluateSequential(ForEachOps.java:173) at java.base/java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234) at java.base/java.util.stream.ReferencePipeline.forEach(ReferencePipeline.java:497) at java.base/java.util.ArrayList.forEach(ArrayList.java:1541) at java.base/java.util.ArrayList.forEach(ArrayList.java:1541) {code} -- This message was sent by Atlassian Jira (v8.20.10#820010)