From 27c165e25565e9961c741c567e49c1e401ab5ef3 Mon Sep 17 00:00:00 2001 From: Chase Zhang Date: Wed, 10 Jun 2026 22:10:29 +0900 Subject: [PATCH 1/3] [FLINK-39899][table-planner] Fix rowtime attribute after Window TVF --- .../functions/sql/SqlWindowTableFunction.java | 41 +++++++++++++++---- ...ojectWindowTableFunctionTransposeRule.java | 7 +++- ...xpandWindowTableFunctionTransposeRule.java | 9 ++-- .../stream/sql/WindowTableFunctionTest.scala | 22 ++++++++++ 4 files changed, 67 insertions(+), 12 deletions(-) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/sql/SqlWindowTableFunction.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/sql/SqlWindowTableFunction.java index d9571d599a0b1..c5cc333860ace 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/sql/SqlWindowTableFunction.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/sql/SqlWindowTableFunction.java @@ -19,6 +19,7 @@ import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.planner.calcite.FlinkTypeFactory; +import org.apache.flink.table.planner.plan.schema.TimeIndicatorRelDataType; import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.util.Preconditions; @@ -137,18 +138,44 @@ private static RelDataType inferRowType(SqlOperatorBinding opBinding) { final RelDataTypeField timeAttributeField = inputRowType.getField(timeField, false, false); assert timeAttributeField != null; return inferRowType( - callBinding.getTypeFactory(), inputRowType, timeAttributeField.getType()); + callBinding.getTypeFactory(), + inputRowType, + timeAttributeField.getType(), + timeAttributeField.getIndex()); } + /** + * Infers the row type of a window TVF call. + * + *

The output is the row type of the input table plus the {@code window_start}, {@code + * window_end} and {@code window_time} columns. Following FLIP-145, the original rowtime + * attribute column (the one referenced by the time descriptor at {@code timeColumnIndex}) is + * materialized to a regular timestamp, so that {@code window_time} becomes the only rowtime + * attribute after applying the window TVF (see FLINK-39899). Processing-time descriptor columns + * are kept as-is. + */ public static RelDataType inferRowType( RelDataTypeFactory typeFactory, RelDataType inputRowType, - RelDataType timeAttributeType) { - return typeFactory - .builder() - .kind(inputRowType.getStructKind()) - .addAll(inputRowType.getFieldList()) - .add("window_start", SqlTypeName.TIMESTAMP, 3) + RelDataType timeAttributeType, + int timeColumnIndex) { + final RelDataTypeFactory.Builder builder = + typeFactory.builder().kind(inputRowType.getStructKind()); + final List inputFields = inputRowType.getFieldList(); + for (int i = 0; i < inputFields.size(); i++) { + final RelDataTypeField field = inputFields.get(i); + if (i == timeColumnIndex && FlinkTypeFactory.isRowtimeIndicatorType(field.getType())) { + // materialize the original rowtime attribute to a regular timestamp + builder.add( + field.getName(), + typeFactory.createTypeWithNullability( + ((TimeIndicatorRelDataType) field.getType()).originalType(), + field.getType().isNullable())); + } else { + builder.add(field); + } + } + return builder.add("window_start", SqlTypeName.TIMESTAMP, 3) .add("window_end", SqlTypeName.TIMESTAMP, 3) .add("window_time", typeFactory.createTypeWithNullability(timeAttributeType, false)) .build(); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/ProjectWindowTableFunctionTransposeRule.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/ProjectWindowTableFunctionTransposeRule.java index 57c4e6c78af00..f23127272e9c9 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/ProjectWindowTableFunctionTransposeRule.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/ProjectWindowTableFunctionTransposeRule.java @@ -111,11 +111,14 @@ public void onMatch(RelOptRuleCall call) { scan.getRowType().getFieldCount(), scanInputFieldCount, toPushFields); // 3. create new window table function scan + // the original time attribute column is shifted to a new position on the pushed-down input + int newTimeColumnIndex = mapping.get(windowingStrategy.getTimeAttributeIndex()); LogicalTableFunctionScan newScan = createNewTableFunctionScan( relBuilder, scan, windowingStrategy.getTimeAttributeType(), + newTimeColumnIndex, newScanInput, mapping, toPushFields); @@ -155,6 +158,7 @@ private LogicalTableFunctionScan createNewTableFunctionScan( RelBuilder relBuilder, LogicalTableFunctionScan oldScan, LogicalType timeAttributeType, + int timeColumnIndex, RelNode newInput, Map mapping, ImmutableBitSet toPushFields) { @@ -165,7 +169,8 @@ private LogicalTableFunctionScan createNewTableFunctionScan( SqlWindowTableFunction.inferRowType( typeFactory, newInput.getRowType(), - typeFactory.createFieldTypeFromLogicalType(timeAttributeType)); + typeFactory.createFieldTypeFromLogicalType(timeAttributeType), + timeColumnIndex); RexNode newCall = rewriteWindowCall( (RexCall) oldScan.getCall(), diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/stream/ExpandWindowTableFunctionTransposeRule.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/stream/ExpandWindowTableFunctionTransposeRule.java index ddba3c17591b9..a31faff8f8fde 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/stream/ExpandWindowTableFunctionTransposeRule.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/stream/ExpandWindowTableFunctionTransposeRule.java @@ -189,15 +189,16 @@ public void onMatch(RelOptRuleCall call) { // ------------------------------------------------------------------------- // 3. Apply WindowTVF on the new Expand node // ------------------------------------------------------------------------- + // the time attribute ref is appended + int timeAttributeOnExpand = + timeFieldAdded ? newExpand.getRowType().getFieldCount() - 1 : newTimeField; RelDataType newOutputType = SqlWindowTableFunction.inferRowType( typeFactory, newExpand.getRowType(), typeFactory.createFieldTypeFromLogicalType( - windowTVF.windowing().getTimeAttributeType())); - // the time attribute ref is appended - int timeAttributeOnExpand = - timeFieldAdded ? newExpand.getRowType().getFieldCount() - 1 : newTimeField; + windowTVF.windowing().getTimeAttributeType()), + timeAttributeOnExpand); TimeAttributeWindowingStrategy newWindowing = new TimeAttributeWindowingStrategy( windowTVF.windowing().getWindow(), diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/WindowTableFunctionTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/WindowTableFunctionTest.scala index fc4125bd54206..8007eeecc77b9 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/WindowTableFunctionTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/WindowTableFunctionTest.scala @@ -129,6 +129,28 @@ class WindowTableFunctionTest extends TableTestBase { "The window function requires the timecol is a time attribute type, but is TIMESTAMP(3).")) } + @Test + def testWindowTVFMaterializesOriginalRowtime(): Unit = { + // FLINK-39899: after applying a window TVF, the original rowtime attribute is materialized to + // a regular timestamp and only window_time stays a time attribute. Hence applying another + // window TVF on the original time column is no longer allowed. + util.tableEnv.executeSql( + """ + |CREATE VIEW windowed AS + |SELECT a, rowtime + |FROM TABLE(TUMBLE(TABLE MyTable, DESCRIPTOR(rowtime), INTERVAL '15' MINUTE)) + |""".stripMargin) + val sql = + """ + |SELECT * + |FROM TABLE(TUMBLE(TABLE windowed, DESCRIPTOR(rowtime), INTERVAL '15' MINUTE)) + |""".stripMargin + + assertThatThrownBy(() => util.verifyRelPlan(sql)) + .hasCause(new ValidationException( + "The window function requires the timecol is a time attribute type, but is TIMESTAMP(3).")) + } + @Test def testConflictingFieldNames(): Unit = { util.tableEnv.executeSql(""" From 9be913f81982f14aac698147e0ae5608ba650d9a Mon Sep 17 00:00:00 2001 From: Chase Zhang Date: Wed, 10 Jun 2026 22:10:48 +0900 Subject: [PATCH 2/3] [FLINK-39899][table-planner] Update unit tests --- .../planner/catalog/JavaCatalogTableTest.xml | 2 +- .../exec/operator/StreamOperatorNameTest.xml | 40 ++-- .../planner/plan/optimize/ScanReuseTest.xml | 8 +- ...ctWindowTableFunctionTransposeRuleTest.xml | 8 +- .../stream/DuplicateChangesInferRuleTest.xml | 16 +- ...ndWindowTableFunctionTransposeRuleTest.xml | 6 +- .../plan/stream/sql/MatchRecognizeTest.xml | 6 +- .../plan/stream/sql/WindowDeduplicateTest.xml | 26 +-- .../plan/stream/sql/WindowRankTest.xml | 26 +-- .../stream/sql/WindowTableFunctionTest.xml | 32 +-- .../stream/sql/agg/WindowAggregateTest.xml | 208 +++++++++--------- .../plan/stream/sql/join/WindowJoinTest.xml | 138 ++++++------ 12 files changed, 258 insertions(+), 258 deletions(-) diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/catalog/JavaCatalogTableTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/catalog/JavaCatalogTableTest.xml index b4b39fd374203..1967408a83b59 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/catalog/JavaCatalogTableTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/catalog/JavaCatalogTableTest.xml @@ -187,7 +187,7 @@ GROUP BY window_start, window_end]]> LogicalProject(EXPR$0=[$2], window_start=[$0]) +- LogicalAggregate(group=[{0, 1}], EXPR$0=[SUM($2)]) +- LogicalProject(window_start=[$2], window_end=[$3], i=[$0]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'ts'), 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER i, TIMESTAMP_LTZ(3) *ROWTIME* ts, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP_LTZ(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'ts'), 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER i, TIMESTAMP_WITH_LOCAL_TIME_ZONE(3) ts, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP_LTZ(3) *ROWTIME* window_time)]) +- LogicalProject(i=[$0], ts=[$1]) +- LogicalProject(i=[$0], ts=[$1]) +- LogicalWatermarkAssigner(rowtime=[ts], watermark=[$1]) diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/operator/StreamOperatorNameTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/operator/StreamOperatorNameTest.xml index 598dc4fd3da96..b351ede7ab769 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/operator/StreamOperatorNameTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/operator/StreamOperatorNameTest.xml @@ -1729,7 +1729,7 @@ Join(joinType=[InnerJoin], where=[(a = d0)], select=[a, b, c, d, a0, b0, c0, d0] ($1, 1000))], b=[$1], e=[$4], c=[$2]) - +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) @@ -47,7 +47,7 @@ LogicalAggregate(group=[{0, 1, 2}], EXPR$3=[COUNT()], EXPR$4=[SUM($3)], EXPR$5=[ ($1, 1000))], b=[$1], e=[$4], c=[$2]) - +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) @@ -73,7 +73,7 @@ GROUP BY window_start, window_end ($1, 1000))], c=[$2]) - +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) @@ -83,7 +83,7 @@ LogicalAggregate(group=[{0, 1}], EXPR$2=[COUNT()], EXPR$3=[SUM($2)], EXPR$4=[MAX ($0, 1000))], c=[$1]) - +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, TIMESTAMP(3) rowtime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(b=[$1], c=[$2], d=[$3], rowtime=[$5]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/stream/DuplicateChangesInferRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/stream/DuplicateChangesInferRuleTest.xml index b7b793aec3243..8aa309a894629 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/stream/DuplicateChangesInferRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/stream/DuplicateChangesInferRuleTest.xml @@ -708,7 +708,7 @@ LogicalSink(table=[default_catalog.default_database.no_pk_snk], fields=[a, EXPR$ +- LogicalProject(a=[$0], EXPR$1=[$3], EXPR$2=[$4]) +- LogicalAggregate(group=[{0, 1, 2}], EXPR$1=[MAX($3)], EXPR$2=[MAX($4)]) +- LogicalProject(a=[$0], window_start=[$4], window_end=[$5], b=[$1], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rt'), 60000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rt, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rt'), 60000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) rt, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rt=[$3]) +- LogicalWatermarkAssigner(rowtime=[rt], watermark=[-($3, 1000:INTERVAL SECOND)]) +- LogicalTableScan(table=[[default_catalog, default_database, append_src1]]) @@ -732,7 +732,7 @@ LogicalSink(table=[default_catalog.default_database.pk_upsert_snk], fields=[a, E +- LogicalProject(a=[$0], EXPR$1=[$3], EXPR$2=[$4]) +- LogicalAggregate(group=[{0, 1, 2}], EXPR$1=[MAX($3)], EXPR$2=[MAX($4)]) +- LogicalProject(a=[$0], window_start=[$4], window_end=[$5], b=[$1], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rt'), 60000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rt, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rt'), 60000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) rt, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rt=[$3]) +- LogicalWatermarkAssigner(rowtime=[rt], watermark=[-($3, 1000:INTERVAL SECOND)]) +- LogicalTableScan(table=[[default_catalog, default_database, append_src1]]) @@ -876,7 +876,7 @@ LogicalSink(table=[default_catalog.default_database.no_pk_snk], fields=[a, EXPR$ +- LogicalProject(a=[$0], EXPR$1=[$3], EXPR$2=[$4]) +- LogicalAggregate(group=[{0, 1, 2}], EXPR$1=[MAX($3)], EXPR$2=[MAX($4)]) +- LogicalProject(a=[$0], window_start=[$4], window_end=[$5], b=[$1], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rt'), 60000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rt, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rt'), 60000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) rt, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rt=[$3]) +- LogicalWatermarkAssigner(rowtime=[rt], watermark=[-($3, 1000:INTERVAL SECOND)]) +- LogicalTableScan(table=[[default_catalog, default_database, append_src1]]) @@ -901,7 +901,7 @@ LogicalSink(table=[default_catalog.default_database.pk_upsert_snk], fields=[a, E +- LogicalProject(a=[$0], EXPR$1=[$3], EXPR$2=[$4]) +- LogicalAggregate(group=[{0, 1, 2}], EXPR$1=[MAX($3)], EXPR$2=[MAX($4)]) +- LogicalProject(a=[$0], window_start=[$4], window_end=[$5], b=[$1], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rt'), 60000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rt, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rt'), 60000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) rt, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rt=[$3]) +- LogicalWatermarkAssigner(rowtime=[rt], watermark=[-($3, 1000:INTERVAL SECOND)]) +- LogicalTableScan(table=[[default_catalog, default_database, append_src1]]) @@ -1008,12 +1008,12 @@ LogicalSink(table=[default_catalog.default_database.pk_upsert_snk], fields=[a, b +- LogicalProject(a=[$0], b=[$8], c=[$2]) +- LogicalJoin(condition=[AND(=($0, $7), =($4, $11), =($5, $12))], joinType=[inner]) :- LogicalProject(a=[$0], b=[$1], c=[$2], rt=[$3], window_start=[$4], window_end=[$5], window_time=[$6]) - : +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rt'), 60000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rt, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + : +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rt'), 60000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) rt, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rt=[$3]) : +- LogicalWatermarkAssigner(rowtime=[rt], watermark=[-($3, 1000:INTERVAL SECOND)]) : +- LogicalTableScan(table=[[default_catalog, default_database, append_src1]]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rt=[$3], window_start=[$4], window_end=[$5], window_time=[$6]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rt'), 60000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rt, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rt'), 60000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) rt, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rt=[$3]) +- LogicalWatermarkAssigner(rowtime=[rt], watermark=[-($3, 1000:INTERVAL SECOND)]) +- LogicalTableScan(table=[[default_catalog, default_database, append_src2]]) @@ -1044,7 +1044,7 @@ LogicalSink(table=[default_catalog.default_database.pk_upsert_snk_with_time_col] +- LogicalProject(a=[$0], b=[$1], c=[$2], window_start=[$3], window_end=[$4]) +- LogicalFilter(condition=[<=($5, 1)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], window_start=[$4], window_end=[$5], rank_num=[ROW_NUMBER() OVER (PARTITION BY $0, $4, $5 ORDER BY $2 DESC NULLS LAST)]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rt'), 60000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rt, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rt'), 60000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) rt, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rt=[$3]) +- LogicalWatermarkAssigner(rowtime=[rt], watermark=[-($3, 1000:INTERVAL SECOND)]) +- LogicalTableScan(table=[[default_catalog, default_database, append_src1]]) @@ -1067,7 +1067,7 @@ Sink(table=[default_catalog.default_database.pk_upsert_snk_with_time_col], field ($1, 1000))], c=[$2]) - +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -80,7 +80,7 @@ GROUP BY a, window_start, window_end ($1, 1000))], c=[$2]) - +- LogicalTableFunctionScan(invocation=[HOP(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[HOP(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -123,7 +123,7 @@ GROUP BY a, window_start, window_end ($1, 1000))], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/MatchRecognizeTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/MatchRecognizeTest.xml index f1c3ad35fb0df..ac501ddbf3f35 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/MatchRecognizeTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/MatchRecognizeTest.xml @@ -217,8 +217,8 @@ FROM TABLE(TUMBLE(TABLE T, DESCRIPTOR(matchRowtime), INTERVAL '3' second)) (PREV(A.$2, 0), 0)]], inputFields=[[symbol, ts_ltz, price, tax]]), rowType=[RecordType(VARCHAR(2147483647) symbol, INTEGER price, INTEGER tax, TIMESTAMP_LTZ(3) *ROWTIME* matchRowtime)] @@ -228,7 +228,7 @@ LogicalProject(symbol=[$0], price=[$1], tax=[$2], matchRowtime=[$3], window_star (PREV(A.$2, 0), 0)}]), rowType=[RecordType(VARCHAR(2147483647) symbol, INTEGER price, INTEGER tax, TIMESTAMP_LTZ(3) *ROWTIME* matchRowtime)] +- Exchange(distribution=[hash[symbol]]), rowType=[RecordType(VARCHAR(2147483647) symbol, TIMESTAMP_LTZ(3) *ROWTIME* ts_ltz, INTEGER price, INTEGER tax)] +- WatermarkAssigner(rowtime=[ts_ltz], watermark=[-(ts_ltz, 1000:INTERVAL SECOND)]), rowType=[RecordType(VARCHAR(2147483647) symbol, TIMESTAMP_LTZ(3) *ROWTIME* ts_ltz, INTEGER price, INTEGER tax)] diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/WindowDeduplicateTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/WindowDeduplicateTest.xml index 0f121f15cbc48..3c5c0d72b4ba8 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/WindowDeduplicateTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/WindowDeduplicateTest.xml @@ -34,7 +34,7 @@ WHERE rownum < 3 LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6], window_start=[$7], window_end=[$8], window_time=[$9], rownum=[$10]) +- LogicalFilter(condition=[<($10, 3)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6], window_start=[$7], window_end=[$8], window_time=[$9], rownum=[ROW_NUMBER() OVER (PARTITION BY $0, $7, $8 ORDER BY $5 DESC NULLS LAST)]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -71,7 +71,7 @@ WHERE rownum <= 1 LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6], window_start=[$7], window_end=[$8], window_time=[$9], rownum=[$10]) +- LogicalFilter(condition=[<=($10, 1)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6], window_start=[$7], window_end=[$8], window_time=[$9], rownum=[ROW_NUMBER() OVER (PARTITION BY $0, $7, $8 ORDER BY $1 DESC NULLS LAST)]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -108,7 +108,7 @@ WHERE rownum <= 1 LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6], window_start=[$7], window_end=[$8], window_time=[$9], rownum=[$10]) +- LogicalFilter(condition=[<=($10, 1)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6], window_start=[$7], window_end=[$8], window_time=[$9], rownum=[ROW_NUMBER() OVER (PARTITION BY $0, $7, $8 ORDER BY $5 DESC NULLS LAST)]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -118,7 +118,7 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$ ($2, 1000))], b=[$2], e=[$5], c=[$3]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(TIMESTAMP(3) *ROWTIME* rowtime, INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(TIMESTAMP(3) rowtime, INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(rowtime=[$0], a=[$1], b=[$2], c=[$3], d=[$4], e=[$5]) +- LogicalProject(rowtime=[$9], a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]) +- LogicalFilter(condition=[<=($10, 1)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6], window_start=[$7], window_end=[$8], window_time=[$9], rownum=[ROW_NUMBER() OVER (PARTITION BY $0, $7, $8 ORDER BY $5 DESC NULLS LAST)]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -277,7 +277,7 @@ Calc(select=[window_start, window_end, EXPR$2, EXPR$3, EXPR$4, wAvg, uv]) +- Exchange(distribution=[single]) +- LocalWindowAggregate(window=[TUMBLE(time_col=[rowtime], size=[15 min])], select=[COUNT(*) AS count1$0, SUM(d) AS sum$1, MAX(d) FILTER $f3 AS max$2, weightedAvg(b, e) AS weightedavg$3, COUNT(distinct$0 c) AS count$4, DISTINCT(c) AS distinct$0, slice_end('w$) AS $slice_end]) +- Calc(select=[d, IS TRUE(>(b, 1000)) AS $f3, b, e, c, window_time AS rowtime]) - +- WindowDeduplicate(window=[TUMBLE(win_start=[window_start], win_end=[window_end], size=[15 min])], keep=[LastRow], partitionKeys=[a], orderKey=[rowtime], order=[ROWTIME]) + +- WindowRank(window=[TUMBLE(win_start=[window_start], win_end=[window_end], size=[15 min])], rankType=[ROW_NUMBER], rankRange=[rankStart=1, rankEnd=1], partitionBy=[a], orderBy=[rowtime DESC], select=[a, b, c, d, e, rowtime, window_start, window_end, window_time]) +- Exchange(distribution=[hash[a]]) +- Calc(select=[a, b, c, d, e, rowtime, window_start, window_end, window_time]) +- WindowTableFunction(window=[TUMBLE(time_col=[rowtime], size=[15 min])]) diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/WindowRankTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/WindowRankTest.xml index 07098bba99697..c6e85b5870ac7 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/WindowRankTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/WindowRankTest.xml @@ -51,7 +51,7 @@ LogicalProject(window_start=[$1], window_end=[$2], window_time=[$3], a=[$0], cnt +- LogicalProject(a=[$0], window_start=[$1], window_end=[$2], window_time=[$3], cnt=[$4], sum_d=[$5], max_d=[$6], wAvg=[$7], uv=[$8], rownum=[ROW_NUMBER() OVER (PARTITION BY $1, $2 ORDER BY $4 DESC NULLS LAST)]) +- LogicalAggregate(group=[{0, 1, 2, 3}], cnt=[COUNT()], sum_d=[SUM($4)], max_d=[MAX($4) FILTER $5], wAvg=[weightedAvg($6, $7)], uv=[COUNT(DISTINCT $8)]) +- LogicalProject(a=[$0], window_start=[$7], window_end=[$8], window_time=[$9], d=[$3], $f5=[IS TRUE(>($1, 1000))], b=[$1], e=[$4], c=[$2]) - +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -106,7 +106,7 @@ LogicalProject(window_start=[$1], window_end=[$2], window_time=[$3], a=[$0], cnt +- LogicalProject(a=[$0], window_start=[$1], window_end=[$2], window_time=[$3], cnt=[$4], sum_d=[$5], max_d=[$6], wAvg=[$7], uv=[$8], rownum=[ROW_NUMBER() OVER (PARTITION BY $0, $1, $2 ORDER BY $4 DESC NULLS LAST)]) +- LogicalAggregate(group=[{0, 1, 2, 3}], cnt=[COUNT()], sum_d=[SUM($4)], max_d=[MAX($4) FILTER $5], wAvg=[weightedAvg($6, $7)], uv=[COUNT(DISTINCT $8)]) +- LogicalProject(a=[$0], window_start=[$7], window_end=[$8], window_time=[$9], d=[$3], $f5=[IS TRUE(>($1, 1000))], b=[$1], e=[$4], c=[$2]) - +- LogicalTableFunctionScan(invocation=[HOP(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[HOP(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -160,7 +160,7 @@ LogicalProject(window_start=[$1], window_end=[$2], window_time=[$3], a=[$0], cnt +- LogicalProject(a=[$0], window_start=[$1], window_end=[$2], window_time=[$3], cnt=[$4], sum_d=[$5], max_d=[$6], wAvg=[$7], uv=[$8], rownum=[ROW_NUMBER() OVER (PARTITION BY $1, $2 ORDER BY $4 DESC NULLS LAST)]) +- LogicalAggregate(group=[{0, 1, 2, 3}], cnt=[COUNT()], sum_d=[SUM($4)], max_d=[MAX($4) FILTER $5], wAvg=[weightedAvg($6, $7)], uv=[COUNT(DISTINCT $8)]) +- LogicalProject(a=[$0], window_start=[$7], window_end=[$8], window_time=[$9], d=[$3], $f5=[IS TRUE(>($1, 1000))], b=[$1], e=[$4], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -200,7 +200,7 @@ WHERE rownum <= 3 LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6], window_start=[$7], window_end=[$8], window_time=[$9], rownum=[$10]) +- LogicalFilter(condition=[<=($10, 3)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6], window_start=[$7], window_end=[$8], window_time=[$9], rownum=[ROW_NUMBER() OVER (PARTITION BY $0, $7, $8 ORDER BY $1 DESC NULLS LAST)]) - +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -237,7 +237,7 @@ WHERE rownum <= 3 LogicalProject(window_start=[$7], window_end=[$8], window_time=[$9], a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]) +- LogicalFilter(condition=[<=($10, 3)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6], window_start=[$7], window_end=[$8], window_time=[$9], rownum=[ROW_NUMBER() OVER (PARTITION BY $0, $7, $8 ORDER BY $1 DESC NULLS LAST)]) - +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -275,7 +275,7 @@ WHERE rownum <= 3 LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6], window_start=[$7], window_end=[$8], window_time=[$9], rownum=[$10]) +- LogicalFilter(condition=[<=($10, 3)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6], window_start=[$7], window_end=[$8], window_time=[$9], rownum=[ROW_NUMBER() OVER (PARTITION BY $0, $7, $8 ORDER BY $1 DESC NULLS LAST)]) - +- LogicalTableFunctionScan(invocation=[HOP(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[HOP(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -312,7 +312,7 @@ WHERE rownum <= 3 LogicalProject(window_start=[$7], window_end=[$8], window_time=[$9], a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]) +- LogicalFilter(condition=[<=($10, 3)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6], window_start=[$7], window_end=[$8], window_time=[$9], rownum=[ROW_NUMBER() OVER (PARTITION BY $0, $7, $8 ORDER BY $1 DESC NULLS LAST)]) - +- LogicalTableFunctionScan(invocation=[HOP(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[HOP(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -349,7 +349,7 @@ WHERE rownum <= 3 LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6], window_start=[$7], window_end=[$8], window_time=[$9], rownum=[$10]) +- LogicalFilter(condition=[<=($10, 3)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6], window_start=[$7], window_end=[$8], window_time=[$9], rownum=[ROW_NUMBER() OVER (PARTITION BY $0, $7, $8 ORDER BY $1 DESC NULLS LAST)]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -385,7 +385,7 @@ WHERE rownum <= 3 LogicalProject(window_start=[$7], window_end=[$8], window_time=[$9], a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]) +- LogicalFilter(condition=[<=($10, 3)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6], window_start=[$7], window_end=[$8], window_time=[$9], rownum=[ROW_NUMBER() OVER (PARTITION BY $0, $7, $8 ORDER BY $1 DESC NULLS LAST)]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -425,12 +425,12 @@ GROUP BY a, window_start, window_end ($2, 1000))], b=[$2], e=[$5], c=[$3]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(TIMESTAMP(3) *ROWTIME* rowtime, INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(TIMESTAMP(3) rowtime, INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(rowtime=[$0], a=[$1], b=[$2], c=[$3], d=[$4], e=[$5]) +- LogicalProject(rowtime=[$9], a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]) +- LogicalFilter(condition=[<=($10, 3)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6], window_start=[$7], window_end=[$8], window_time=[$9], rownum=[ROW_NUMBER() OVER (PARTITION BY $0, $7, $8 ORDER BY $1 DESC NULLS LAST)]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -472,14 +472,14 @@ GROUP BY a, window_start, window_end ($1, 1000))], b=[$1], e=[$4], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/WindowTableFunctionTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/WindowTableFunctionTest.xml index e583fc3f0a035..af0b57c8a7762 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/WindowTableFunctionTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/WindowTableFunctionTest.xml @@ -27,7 +27,7 @@ FROM TABLE( ($5, 1000))], b=[$5], e=[$4], c=[$6]) +- LogicalProject(window_start=[$7], rowtime=[$5], d=[$3], proctime=[$6], e=[$4], b=[$1], c=[$2], window_end=[$8], window_time=[$9], a=[$0]) +- LogicalFilter(condition=[>=($7, 2021-01-01 10:10:00)]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -85,7 +85,7 @@ LogicalAggregate(group=[{0, 1}], EXPR$2=[COUNT()], EXPR$3=[SUM($2)], EXPR$4=[MAX +- LogicalProject(window_start=[$0], window_end=[$7], d=[$2], $f3=[IS TRUE(>($5, 1000))], b=[$5], e=[$4], c=[$6]) +- LogicalProject(window_start=[$7], rowtime=[$5], d=[$3], proctime=[$6], e=[$4], b=[$1], c=[$2], window_end=[$8], window_time=[$9], a=[$0]) +- LogicalFilter(condition=[>=($7, 2021-01-01 10:10:00)]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -123,7 +123,7 @@ GROUP BY GROUPING SETS ((a), (b)), window_start, window_end LogicalProject(a=[$0], b=[$1], EXPR$2=[$4], EXPR$3=[$5], uv=[$6]) +- LogicalAggregate(group=[{0, 1, 2, 3}], groups=[[{0, 2, 3}, {1, 2, 3}]], EXPR$2=[COUNT()], EXPR$3=[MAX($4) FILTER $5], uv=[COUNT(DISTINCT $6)]) +- LogicalProject(a=[$0], b=[$1], window_start=[$7], window_end=[$8], d=[$3], $f5=[IS TRUE(>($1, 1000))], window_time=[$9]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -167,7 +167,7 @@ GROUP BY GROUPING SETS ((a), (b)), window_start, window_end LogicalProject(a=[$0], b=[$1], EXPR$2=[$4], EXPR$3=[$5], uv=[$6]) +- LogicalAggregate(group=[{0, 1, 2, 3}], groups=[[{0, 2, 3}, {1, 2, 3}]], EXPR$2=[COUNT()], EXPR$3=[MAX($4) FILTER $5], uv=[COUNT(DISTINCT $6)]) +- LogicalProject(a=[$0], b=[$1], window_start=[$7], window_end=[$8], d=[$3], $f5=[IS TRUE(>($1, 1000))], window_time=[$9]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -220,7 +220,7 @@ LogicalAggregate(group=[{0, 1}], EXPR$2=[COUNT()], EXPR$3=[SUM($2)], EXPR$4=[MAX +- LogicalProject(window_start=[$7], window_end=[$8], len=[$10], d=[$3], $f4=[IS TRUE(>($1, 1000))], b=[$1], e=[$4], c=[$2]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6], window_start=[$7], window_end=[$8], window_time=[$9], len=[$10]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{2}]) - :- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + :- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) : +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -234,7 +234,7 @@ Calc(select=[window_start, window_end, EXPR$2, EXPR$3, EXPR$4, wAvg, uv]) +- WindowAggregate(window=[TUMBLE(win_start=[window_start], win_end=[window_end], size=[15 min])], select=[COUNT(*) AS EXPR$2, SUM(len) AS EXPR$3, MAX(d) FILTER $f4 AS EXPR$4, weightedAvg(b, e) AS wAvg, COUNT(DISTINCT c) AS uv, start('w$) AS window_start, end('w$) AS window_end]) +- Exchange(distribution=[single]) +- Calc(select=[window_start, window_end, EXPR$0 AS len, d, IS TRUE(>(b, 1000)) AS $f4, b, e, c]) - +- Correlate(invocation=[len_udtf($cor0.c)], correlate=[table(len_udtf($cor0.c))], select=[a,b,c,d,e,rowtime,proctime,window_start,window_end,window_time,EXPR$0], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time, INTEGER EXPR$0)], joinType=[INNER]) + +- Correlate(invocation=[len_udtf($cor0.c)], correlate=[table(len_udtf($cor0.c))], select=[a,b,c,d,e,rowtime,proctime,window_start,window_end,window_time,EXPR$0], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time, INTEGER EXPR$0)], joinType=[INNER]) +- WindowTableFunction(window=[TUMBLE(time_col=[rowtime], size=[15 min])]) +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)]) +- Calc(select=[a, b, c, d, e, rowtime, PROCTIME() AS proctime]) @@ -267,7 +267,7 @@ LogicalAggregate(group=[{0, 1}], EXPR$2=[COUNT()], EXPR$3=[SUM($2)], EXPR$4=[MAX +- LogicalProject(window_start=[$7], window_end=[$8], len=[$10], d=[$3], $f4=[IS TRUE(>($1, 1000))], b=[$1], e=[$4], c=[$2]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6], window_start=[$7], window_end=[$8], window_time=[$9], len=[$10]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{2}]) - :- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + :- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) : +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -282,7 +282,7 @@ Calc(select=[window_start, window_end, EXPR$2, EXPR$3, EXPR$4, wAvg, uv]) +- Exchange(distribution=[single]) +- LocalWindowAggregate(window=[TUMBLE(win_start=[window_start], win_end=[window_end], size=[15 min])], select=[COUNT(*) AS count1$0, SUM(len) AS sum$1, MAX(d) FILTER $f4 AS max$2, weightedAvg(b, e) AS weightedavg$3, COUNT(distinct$0 c) AS count$4, DISTINCT(c) AS distinct$0, slice_end('w$) AS $window_end]) +- Calc(select=[window_start, window_end, EXPR$0 AS len, d, IS TRUE(>(b, 1000)) AS $f4, b, e, c]) - +- Correlate(invocation=[len_udtf($cor0.c)], correlate=[table(len_udtf($cor0.c))], select=[a,b,c,d,e,rowtime,proctime,window_start,window_end,window_time,EXPR$0], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time, INTEGER EXPR$0)], joinType=[INNER]) + +- Correlate(invocation=[len_udtf($cor0.c)], correlate=[table(len_udtf($cor0.c))], select=[a,b,c,d,e,rowtime,proctime,window_start,window_end,window_time,EXPR$0], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time, INTEGER EXPR$0)], joinType=[INNER]) +- WindowTableFunction(window=[TUMBLE(time_col=[rowtime], size=[15 min])]) +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)]) +- Calc(select=[a, b, c, d, e, rowtime, PROCTIME() AS proctime]) @@ -306,7 +306,7 @@ GROUP BY CUBE (a, b, window_start, window_end) LogicalProject(a=[$0], b=[$1], uv=[$4]) +- LogicalAggregate(group=[{0, 1, 2, 3}], groups=[[{0, 1, 2, 3}, {0, 1, 2}, {0, 1, 3}, {0, 1}, {0, 2, 3}, {0, 2}, {0, 3}, {0}, {1, 2, 3}, {1, 2}, {1, 3}, {1}, {2, 3}, {2}, {3}, {}]], uv=[COUNT(DISTINCT $4)]) +- LogicalProject(a=[$0], b=[$1], window_start=[$7], window_end=[$8], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -344,7 +344,7 @@ GROUP BY window_start ($1, 1000))], b=[$1], e=[$4], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -377,7 +377,7 @@ GROUP BY GROUPING SETS ((a, window_start), (window_start)) LogicalProject(a=[$0], uv=[$2]) +- LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {1}]], uv=[COUNT(DISTINCT $2)]) +- LogicalProject(a=[$0], window_start=[$7], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -412,7 +412,7 @@ GROUP BY GROUPING SETS ((a), (window_start), (window_end)) LogicalProject(a=[$0], uv=[$3]) +- LogicalAggregate(group=[{0, 1, 2}], groups=[[{0}, {1}, {2}]], uv=[COUNT(DISTINCT $3)]) +- LogicalProject(a=[$0], window_start=[$7], window_end=[$8], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -447,7 +447,7 @@ GROUP BY window_start, window_end ($1, 1000))], c=[$2]) - +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -635,7 +635,7 @@ GROUP BY a, window_start, window_end ($1, 1000))], c=[$2]) - +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -677,7 +677,7 @@ GROUP BY GROUPING SETS ((a), (b)), window_start, window_end LogicalProject(a=[$0], b=[$1], uv=[$4]) +- LogicalAggregate(group=[{0, 1, 2, 3}], groups=[[{0, 2, 3}, {1, 2, 3}]], uv=[COUNT(DISTINCT $4)]) +- LogicalProject(a=[$0], b=[$1], window_start=[$7], window_end=[$8], c=[$2]) - +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 1200000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 1200000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -712,7 +712,7 @@ GROUP BY GROUPING SETS ((a), (b)), window_start, window_end LogicalProject(a=[$0], b=[$1], uv=[$4]) +- LogicalAggregate(group=[{0, 1, 2, 3}], groups=[[{0, 2, 3}, {1, 2, 3}]], uv=[COUNT(DISTINCT $4)]) +- LogicalProject(a=[$0], b=[$1], window_start=[$7], window_end=[$8], c=[$2]) - +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 1200000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 1200000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -749,7 +749,7 @@ GROUP BY GROUPING SETS ((a), (b)), window_start, window_end LogicalProject(a=[$0], b=[$1], EXPR$2=[$4], uv=[$5]) +- LogicalAggregate(group=[{0, 1, 2, 3}], groups=[[{0, 2, 3}, {1, 2, 3}]], EXPR$2=[COUNT()], uv=[COUNT(DISTINCT $4)]) +- LogicalProject(a=[$0], b=[$1], window_start=[$7], window_end=[$8], c=[$2]) - +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -789,7 +789,7 @@ GROUP BY GROUPING SETS ((a), (b)), window_start, window_end LogicalProject(a=[$0], b=[$1], EXPR$2=[$4], uv=[$5]) +- LogicalAggregate(group=[{0, 1, 2, 3}], groups=[[{0, 2, 3}, {1, 2, 3}]], EXPR$2=[COUNT()], uv=[COUNT(DISTINCT $4)]) +- LogicalProject(a=[$0], b=[$1], window_start=[$7], window_end=[$8], c=[$2]) - +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -914,7 +914,7 @@ GROUP BY a, window_start, window_end ($1, 1000))], b=[$1], e=[$4], c=[$2]) - +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -953,7 +953,7 @@ GROUP BY a, window_start, window_end ($1, 1000))], b=[$1], e=[$4], c=[$2]) - +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -993,7 +993,7 @@ GROUP BY a, window_start, window_end ($1, 1000))], b=[$1], e=[$4], c=[$2]) - +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -1032,7 +1032,7 @@ GROUP BY a, window_start, window_end ($1, 1000))], b=[$1], e=[$4], c=[$2]) - +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -1068,7 +1068,7 @@ GROUP BY ROLLUP (a, b), window_start, window_end LogicalProject(a=[$0], b=[$1], uv=[$4]) +- LogicalAggregate(group=[{0, 1, 2, 3}], groups=[[{0, 1, 2, 3}, {0, 2, 3}, {2, 3}]], uv=[COUNT(DISTINCT $4)]) +- LogicalProject(a=[$0], b=[$1], window_start=[$7], window_end=[$8], c=[$2]) - +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -1103,7 +1103,7 @@ GROUP BY ROLLUP (a, b), window_start, window_end LogicalProject(a=[$0], b=[$1], uv=[$4]) +- LogicalAggregate(group=[{0, 1, 2, 3}], groups=[[{0, 1, 2, 3}, {0, 2, 3}, {2, 3}]], uv=[COUNT(DISTINCT $4)]) +- LogicalProject(a=[$0], b=[$1], window_start=[$7], window_end=[$8], c=[$2]) - +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -1137,7 +1137,7 @@ GROUP BY window_start, window_end, window_time LogicalProject(window_time=[$2], start_time=[$3], end_time=[$4]) +- LogicalAggregate(group=[{0, 1, 2}], start_time=[MIN($3)], end_time=[MAX($3)]) +- LogicalProject(window_start=[$7], window_end=[$8], window_time=[$9], rowtime=[$5]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -1147,9 +1147,9 @@ LogicalProject(window_time=[$2], start_time=[$3], end_time=[$4]) @@ -1170,7 +1170,7 @@ GROUP BY window_start, window_end, window_time LogicalProject(window_time=[$2], start_time=[$3], end_time=[$4]) +- LogicalAggregate(group=[{0, 1, 2}], start_time=[MIN($3)], end_time=[MAX($3)]) +- LogicalProject(window_start=[$7], window_end=[$8], window_time=[$9], rowtime=[$5]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -1180,10 +1180,10 @@ LogicalProject(window_time=[$2], start_time=[$3], end_time=[$4]) @@ -1195,7 +1195,7 @@ Calc(select=[window_time, start_time, end_time]) LogicalProject(window_start=[$1], window_end=[$2], EXPR$2=[$3], EXPR$3=[$4], EXPR$4=[$5], wAvg=[$6], uv=[$7]) +- LogicalAggregate(group=[{0, 1, 2}], EXPR$2=[COUNT()], EXPR$3=[SUM($3)], EXPR$4=[MAX($3) FILTER $4], wAvg=[weightedAvg($0, $5)], uv=[COUNT(DISTINCT $6)]) +- LogicalProject(b=[$1], window_start=[$7], window_end=[$8], d=[$3], $f4=[IS TRUE(>($1, 1000))], e=[$4], c=[$2]) - +- LogicalTableFunctionScan(invocation=[SESSION(TABLE(#0) PARTITION BY($1, $0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[SESSION(TABLE(#0) PARTITION BY($1, $0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -1229,7 +1229,7 @@ Calc(select=[window_start, window_end, EXPR$2, EXPR$3, EXPR$4, wAvg, uv]) LogicalProject(window_start=[$1], window_end=[$2], EXPR$2=[$3], EXPR$3=[$4], EXPR$4=[$5], wAvg=[$6], uv=[$7]) +- LogicalAggregate(group=[{0, 1, 2}], EXPR$2=[COUNT()], EXPR$3=[SUM($3)], EXPR$4=[MAX($3) FILTER $4], wAvg=[weightedAvg($0, $5)], uv=[COUNT(DISTINCT $6)]) +- LogicalProject(b=[$1], window_start=[$7], window_end=[$8], d=[$3], $f4=[IS TRUE(>($1, 1000))], e=[$4], c=[$2]) - +- LogicalTableFunctionScan(invocation=[SESSION(TABLE(#0) PARTITION BY($1, $0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[SESSION(TABLE(#0) PARTITION BY($1, $0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -1262,7 +1262,7 @@ Calc(select=[window_start, window_end, EXPR$2, EXPR$3, EXPR$4, wAvg, uv]) ($1, 1000))], b=[$1], e=[$4], c=[$2]) - +- LogicalTableFunctionScan(invocation=[SESSION(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[SESSION(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -1293,7 +1293,7 @@ GroupAggregate(groupBy=[a, window_start, window_end], select=[a, window_start, w ($1, 1000))], b=[$1], e=[$4], c=[$2]) - +- LogicalTableFunctionScan(invocation=[SESSION(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[SESSION(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -1400,7 +1400,7 @@ GROUP BY CUBE (a, b), window_start, window_end LogicalProject(a=[$0], b=[$1], uv=[$4]) +- LogicalAggregate(group=[{0, 1, 2, 3}], groups=[[{0, 1, 2, 3}, {0, 2, 3}, {1, 2, 3}, {2, 3}]], uv=[COUNT(DISTINCT $4)]) +- LogicalProject(a=[$0], b=[$1], window_start=[$7], window_end=[$8], c=[$2]) - +- LogicalTableFunctionScan(invocation=[HOP(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[HOP(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -1435,7 +1435,7 @@ GROUP BY CUBE (a, b), window_start, window_end LogicalProject(a=[$0], b=[$1], uv=[$4]) +- LogicalAggregate(group=[{0, 1, 2, 3}], groups=[[{0, 1, 2, 3}, {0, 2, 3}, {1, 2, 3}, {2, 3}]], uv=[COUNT(DISTINCT $4)]) +- LogicalProject(a=[$0], b=[$1], window_start=[$7], window_end=[$8], c=[$2]) - +- LogicalTableFunctionScan(invocation=[HOP(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[HOP(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -1474,7 +1474,7 @@ GROUP BY a, window_start, window_end ($1, 1000))], c=[$2]) - +- LogicalTableFunctionScan(invocation=[HOP(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[HOP(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -1517,7 +1517,7 @@ GROUP BY a, window_start, window_end ($1, 1000))], c=[$2]) - +- LogicalTableFunctionScan(invocation=[HOP(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[HOP(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -1559,7 +1559,7 @@ GROUP BY GROUPING SETS ((a), (b)), window_start, window_end LogicalProject(a=[$0], b=[$1], uv=[$4]) +- LogicalAggregate(group=[{0, 1, 2, 3}], groups=[[{0, 2, 3}, {1, 2, 3}]], uv=[COUNT(DISTINCT $4)]) +- LogicalProject(a=[$0], b=[$1], window_start=[$7], window_end=[$8], c=[$2]) - +- LogicalTableFunctionScan(invocation=[HOP(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[HOP(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -1594,7 +1594,7 @@ GROUP BY GROUPING SETS ((a), (b)), window_start, window_end LogicalProject(a=[$0], b=[$1], uv=[$4]) +- LogicalAggregate(group=[{0, 1, 2, 3}], groups=[[{0, 2, 3}, {1, 2, 3}]], uv=[COUNT(DISTINCT $4)]) +- LogicalProject(a=[$0], b=[$1], window_start=[$7], window_end=[$8], c=[$2]) - +- LogicalTableFunctionScan(invocation=[HOP(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[HOP(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -1631,7 +1631,7 @@ GROUP BY GROUPING SETS ((a), (b)), window_start, window_end LogicalProject(a=[$0], b=[$1], EXPR$2=[$4], uv=[$5]) +- LogicalAggregate(group=[{0, 1, 2, 3}], groups=[[{0, 2, 3}, {1, 2, 3}]], EXPR$2=[COUNT()], uv=[COUNT(DISTINCT $4)]) +- LogicalProject(a=[$0], b=[$1], window_start=[$7], window_end=[$8], c=[$2]) - +- LogicalTableFunctionScan(invocation=[HOP(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[HOP(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -1671,7 +1671,7 @@ GROUP BY GROUPING SETS ((a), (b)), window_start, window_end LogicalProject(a=[$0], b=[$1], EXPR$2=[$4], uv=[$5]) +- LogicalAggregate(group=[{0, 1, 2, 3}], groups=[[{0, 2, 3}, {1, 2, 3}]], EXPR$2=[COUNT()], uv=[COUNT(DISTINCT $4)]) +- LogicalProject(a=[$0], b=[$1], window_start=[$7], window_end=[$8], c=[$2]) - +- LogicalTableFunctionScan(invocation=[HOP(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[HOP(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -1796,7 +1796,7 @@ GROUP BY a, window_start, window_end ($1, 1000))], b=[$1], e=[$4], c=[$2]) - +- LogicalTableFunctionScan(invocation=[HOP(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[HOP(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -1835,7 +1835,7 @@ GROUP BY a, window_start, window_end ($1, 1000))], b=[$1], e=[$4], c=[$2]) - +- LogicalTableFunctionScan(invocation=[HOP(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[HOP(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -1875,7 +1875,7 @@ GROUP BY a, window_start, window_end ($1, 1000))], b=[$1], e=[$4], c=[$2]) - +- LogicalTableFunctionScan(invocation=[HOP(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[HOP(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -1914,7 +1914,7 @@ GROUP BY a, window_start, window_end ($1, 1000))], b=[$1], e=[$4], c=[$2]) - +- LogicalTableFunctionScan(invocation=[HOP(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[HOP(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -1950,7 +1950,7 @@ GROUP BY ROLLUP (a, b), window_start, window_end LogicalProject(a=[$0], b=[$1], uv=[$4]) +- LogicalAggregate(group=[{0, 1, 2, 3}], groups=[[{0, 1, 2, 3}, {0, 2, 3}, {2, 3}]], uv=[COUNT(DISTINCT $4)]) +- LogicalProject(a=[$0], b=[$1], window_start=[$7], window_end=[$8], c=[$2]) - +- LogicalTableFunctionScan(invocation=[HOP(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[HOP(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -1985,7 +1985,7 @@ GROUP BY ROLLUP (a, b), window_start, window_end LogicalProject(a=[$0], b=[$1], uv=[$4]) +- LogicalAggregate(group=[{0, 1, 2, 3}], groups=[[{0, 1, 2, 3}, {0, 2, 3}, {2, 3}]], uv=[COUNT(DISTINCT $4)]) +- LogicalProject(a=[$0], b=[$1], window_start=[$7], window_end=[$8], c=[$2]) - +- LogicalTableFunctionScan(invocation=[HOP(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[HOP(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -2190,7 +2190,7 @@ LogicalSink(table=[default_catalog.default_database.s1], fields=[window_start, w +- LogicalAggregate(group=[{0, 1}], wAvg=[weightedAvg($2, $3)]) +- LogicalProject(window_start=[$7], window_end=[$8], b=[$1], e=[$4]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6], window_start=[$7], window_end=[$8], window_time=[$9]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -2200,7 +2200,7 @@ LogicalSink(table=[default_catalog.default_database.s1], fields=[window_start, w +- LogicalAggregate(group=[{0, 1}], EXPR$2=[COUNT()]) +- LogicalProject(window_start=[$7], window_end=[$8]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6], window_start=[$7], window_end=[$8], window_time=[$9]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -2232,7 +2232,7 @@ LogicalSink(table=[default_catalog.default_database.s1], fields=[window_start, w +- LogicalAggregate(group=[{0, 1}], wAvg=[weightedAvg($2, $3)]) +- LogicalProject(window_start=[$7], window_end=[$8], b=[$1], e=[$4]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6], window_start=[$7], window_end=[$8], window_time=[$9]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -2242,7 +2242,7 @@ LogicalSink(table=[default_catalog.default_database.s1], fields=[window_start, w +- LogicalAggregate(group=[{0, 1}], EXPR$2=[COUNT()]) +- LogicalProject(window_start=[$7], window_end=[$8]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6], window_start=[$7], window_end=[$8], window_time=[$9]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -3002,7 +3002,7 @@ LogicalProject(window_start=[$1], window_end=[$2], a=[$0], EXPR$3=[$3], EXPR$4=[ +- LogicalProject(a=[$9], window_start=[$0], window_end=[$7], d=[$2], $f4=[IS TRUE(>($5, 1000))], b=[$5], e=[$4], c=[$6]) +- LogicalProject(window_start=[$7], rowtime=[$5], d=[$3], proctime=[$6], e=[$4], b=[$1], c=[$2], window_end=[$8], window_time=[$9], a=[$0]) +- LogicalFilter(condition=[>=($7, 2021-01-01 10:10:00)]) - +- LogicalTableFunctionScan(invocation=[SESSION(TABLE(#0) PARTITION BY($0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[SESSION(TABLE(#0) PARTITION BY($0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -3049,7 +3049,7 @@ LogicalProject(window_start=[$1], window_end=[$2], a=[$0], EXPR$3=[$3], EXPR$4=[ +- LogicalProject(a=[$9], window_start=[$0], window_end=[$7], d=[$2], $f4=[IS TRUE(>($5, 1000))], b=[$5], e=[$4], c=[$6]) +- LogicalProject(window_start=[$7], rowtime=[$5], d=[$3], proctime=[$6], e=[$4], b=[$1], c=[$2], window_end=[$8], window_time=[$9], a=[$0]) +- LogicalFilter(condition=[>=($7, 2021-01-01 10:10:00)]) - +- LogicalTableFunctionScan(invocation=[SESSION(TABLE(#0) PARTITION BY($0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[SESSION(TABLE(#0) PARTITION BY($0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -3094,7 +3094,7 @@ LogicalAggregate(group=[{0, 1}], EXPR$2=[COUNT()], EXPR$3=[SUM($2)], EXPR$4=[MAX +- LogicalProject(window_start=[$0], window_end=[$7], d=[$2], $f3=[IS TRUE(>($5, 1000))], b=[$5], e=[$4], c=[$6]) +- LogicalProject(window_start=[$7], rowtime=[$5], d=[$3], proctime=[$6], e=[$4], b=[$1], c=[$2], window_end=[$8], window_time=[$9], a=[$0]) +- LogicalFilter(condition=[>=($7, 2021-01-01 10:10:00)]) - +- LogicalTableFunctionScan(invocation=[SESSION(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[SESSION(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -3139,7 +3139,7 @@ LogicalAggregate(group=[{0, 1}], EXPR$2=[COUNT()], EXPR$3=[SUM($2)], EXPR$4=[MAX +- LogicalProject(window_start=[$0], window_end=[$7], d=[$2], $f3=[IS TRUE(>($5, 1000))], b=[$5], e=[$4], c=[$6]) +- LogicalProject(window_start=[$7], rowtime=[$5], d=[$3], proctime=[$6], e=[$4], b=[$1], c=[$2], window_end=[$8], window_time=[$9], a=[$0]) +- LogicalFilter(condition=[>=($7, 2021-01-01 10:10:00)]) - +- LogicalTableFunctionScan(invocation=[SESSION(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[SESSION(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -3165,7 +3165,7 @@ Calc(select=[window_start, window_end, EXPR$2, EXPR$3, EXPR$4, wAvg, uv]) LogicalProject(a=[$1], b=[$0], window_start=[$2], window_end=[$3], EXPR$4=[$4], EXPR$5=[$5], EXPR$6=[$6], wAvg=[$7], uv=[$8]) +- LogicalAggregate(group=[{0, 1, 2, 3}], EXPR$4=[COUNT()], EXPR$5=[SUM($4)], EXPR$6=[MAX($4) FILTER $5], wAvg=[weightedAvg($0, $6)], uv=[COUNT(DISTINCT $7)]) +- LogicalProject(b=[$1], a=[$0], window_start=[$7], window_end=[$8], d=[$3], $f5=[IS TRUE(>($1, 1000))], e=[$4], c=[$2]) - +- LogicalTableFunctionScan(invocation=[SESSION(TABLE(#0) PARTITION BY($1, $0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[SESSION(TABLE(#0) PARTITION BY($1, $0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -3195,7 +3195,7 @@ Calc(select=[a, b, window_start, window_end, EXPR$4, EXPR$5, EXPR$6, wAvg, uv]) LogicalProject(a=[$1], b=[$0], window_start=[$2], window_end=[$3], EXPR$4=[$4], EXPR$5=[$5], EXPR$6=[$6], wAvg=[$7], uv=[$8]) +- LogicalAggregate(group=[{0, 1, 2, 3}], EXPR$4=[COUNT()], EXPR$5=[SUM($4)], EXPR$6=[MAX($4) FILTER $5], wAvg=[weightedAvg($0, $6)], uv=[COUNT(DISTINCT $7)]) +- LogicalProject(b=[$1], a=[$0], window_start=[$7], window_end=[$8], d=[$3], $f5=[IS TRUE(>($1, 1000))], e=[$4], c=[$2]) - +- LogicalTableFunctionScan(invocation=[SESSION(TABLE(#0) PARTITION BY($1, $0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[SESSION(TABLE(#0) PARTITION BY($1, $0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -3358,7 +3358,7 @@ GROUP BY a, window_start, window_end ($1, 1000))], b=[$1], e=[$4], c=[$2]) - +- LogicalTableFunctionScan(invocation=[SESSION(TABLE(#0) PARTITION BY($0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[SESSION(TABLE(#0) PARTITION BY($0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -3397,7 +3397,7 @@ GROUP BY a, window_start, window_end ($1, 1000))], b=[$1], e=[$4], c=[$2]) - +- LogicalTableFunctionScan(invocation=[SESSION(TABLE(#0) PARTITION BY($0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[SESSION(TABLE(#0) PARTITION BY($0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -3441,7 +3441,7 @@ LogicalAggregate(group=[{0, 1, 2}], EXPR$3=[COUNT()], EXPR$4=[SUM($3)], EXPR$5=[ +- LogicalProject(a=[$9], window_start=[$0], window_end=[$7], d=[$2], $f4=[IS TRUE(>($5, 1000))], b=[$5], e=[$4], c=[$6]) +- LogicalProject(window_start=[$7], rowtime=[$5], d=[$3], proctime=[$6], e=[$4], b=[$1], c=[$2], window_end=[$8], window_time=[$9], a=[$0]) +- LogicalFilter(condition=[>($1, 1000)]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -3485,7 +3485,7 @@ LogicalAggregate(group=[{0, 1, 2}], EXPR$3=[COUNT()], EXPR$4=[SUM($3)], EXPR$5=[ +- LogicalProject(a=[$9], window_start=[$0], window_end=[$7], d=[$2], $f4=[IS TRUE(>($5, 1000))], b=[$5], e=[$4], c=[$6]) +- LogicalProject(window_start=[$7], rowtime=[$5], d=[$3], proctime=[$6], e=[$4], b=[$1], c=[$2], window_end=[$8], window_time=[$9], a=[$0]) +- LogicalFilter(condition=[>($1, 1000)]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -3670,12 +3670,12 @@ GROUP BY a, window_start, window_end ($1, 1000))], window_time=[$9]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -3986,7 +3986,7 @@ GROUP BY a, window_start, window_end ($1, 1000))], window_time=[$9]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -4031,7 +4031,7 @@ GROUP BY a, window_start, window_end ($1, 1000))], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -4073,7 +4073,7 @@ GROUP BY a, window_start, window_end ($1, 1000))], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -4157,7 +4157,7 @@ GROUP BY window_start, a, window_end, b LogicalProject(a=[$1], b=[$3], window_start=[$0], window_end=[$2], EXPR$4=[$4], EXPR$5=[$5], EXPR$6=[$6], uv=[$7]) +- LogicalAggregate(group=[{0, 1, 2, 3}], EXPR$4=[COUNT()], EXPR$5=[SUM($4)], EXPR$6=[MAX($4) FILTER $5], uv=[COUNT(DISTINCT $6)]) +- LogicalProject(window_start=[$7], a=[$0], window_end=[$8], b=[$1], d=[$3], $f5=[IS TRUE(>($1, 1000))], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -4196,7 +4196,7 @@ GROUP BY window_start, a, window_end, b LogicalProject(a=[$1], b=[$3], window_start=[$0], window_end=[$2], EXPR$4=[$4], EXPR$5=[$5], EXPR$6=[$6], uv=[$7]) +- LogicalAggregate(group=[{0, 1, 2, 3}], EXPR$4=[COUNT()], EXPR$5=[SUM($4)], EXPR$6=[MAX($4) FILTER $5], uv=[COUNT(DISTINCT $6)]) +- LogicalProject(window_start=[$7], a=[$0], window_end=[$8], b=[$1], d=[$3], $f5=[IS TRUE(>($1, 1000))], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -4242,7 +4242,7 @@ LogicalProject(a=[$0], window_start=[$1], ws=[$3], window_end=[$2], window_time= +- LogicalAggregate(group=[{0, 1, 2, 3, 4}], EXPR$5=[COUNT()], EXPR$6=[SUM($5)], EXPR$7=[MAX($5) FILTER $6], wAvg=[weightedAvg($7, $8)], uv=[COUNT(DISTINCT $9)]) +- LogicalProject(a=[$0], window_start=[$7], window_end=[$8], ws=[$10], window_time=[$9], d=[$3], $f6=[IS TRUE(>($1, 1000))], b=[$1], e=[$4], c=[$2]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6], window_start=[$7], window_end=[$8], window_time=[$9], ws=[$7]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -4287,7 +4287,7 @@ LogicalProject(a=[$0], window_start=[$1], ws=[$3], window_end=[$2], window_time= +- LogicalAggregate(group=[{0, 1, 2, 3, 4}], EXPR$5=[COUNT()], EXPR$6=[SUM($5)], EXPR$7=[MAX($5) FILTER $6], wAvg=[weightedAvg($7, $8)], uv=[COUNT(DISTINCT $9)]) +- LogicalProject(a=[$0], window_start=[$7], window_end=[$8], ws=[$10], window_time=[$9], d=[$3], $f6=[IS TRUE(>($1, 1000))], b=[$1], e=[$4], c=[$2]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6], window_start=[$7], window_end=[$8], window_time=[$9], ws=[$7]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -4325,7 +4325,7 @@ GROUP BY 'literal', window_start, window_end LogicalProject(window_start=[$1], window_end=[$2], EXPR$2=[$3], EXPR$3=[$4], EXPR$4=[$5], uv=[$6]) +- LogicalAggregate(group=[{0, 1, 2}], EXPR$2=[COUNT()], EXPR$3=[SUM($3)], EXPR$4=[MAX($3) FILTER $4], uv=[COUNT(DISTINCT $5)]) +- LogicalProject($f0=[_UTF-16LE'literal'], window_start=[$7], window_end=[$8], d=[$3], $f4=[IS TRUE(>($1, 1000))], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -4362,7 +4362,7 @@ GROUP BY 'literal', window_start, window_end LogicalProject(window_start=[$1], window_end=[$2], EXPR$2=[$3], EXPR$3=[$4], EXPR$4=[$5], uv=[$6]) +- LogicalAggregate(group=[{0, 1, 2}], EXPR$2=[COUNT()], EXPR$3=[SUM($3)], EXPR$4=[MAX($3) FILTER $4], uv=[COUNT(DISTINCT $5)]) +- LogicalProject($f0=[_UTF-16LE'literal'], window_start=[$7], window_end=[$8], d=[$3], $f4=[IS TRUE(>($1, 1000))], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -4399,7 +4399,7 @@ GROUP BY window_start, window_end ($1, 1000))], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -4435,7 +4435,7 @@ GROUP BY window_start, window_end ($1, 1000))], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -4470,7 +4470,7 @@ GROUP BY GROUPING SETS ((a), (b)), window_start, window_end LogicalProject(a=[$0], b=[$1], uv=[$4]) +- LogicalAggregate(group=[{0, 1, 2, 3}], groups=[[{0, 2, 3}, {1, 2, 3}]], uv=[COUNT(DISTINCT $4)]) +- LogicalProject(a=[$0], b=[$1], window_start=[$7], window_end=[$8], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -4504,7 +4504,7 @@ GROUP BY GROUPING SETS ((a), (b)), window_start, window_end LogicalProject(a=[$0], b=[$1], uv=[$4]) +- LogicalAggregate(group=[{0, 1, 2, 3}], groups=[[{0, 2, 3}, {1, 2, 3}]], uv=[COUNT(DISTINCT $4)]) +- LogicalProject(a=[$0], b=[$1], window_start=[$7], window_end=[$8], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -4542,7 +4542,7 @@ GROUP BY GROUPING SETS ((a), (b)), window_start, window_end LogicalProject(a=[$0], b=[$1], EXPR$2=[$4], EXPR$3=[$5], EXPR$4=[$6], uv=[$7]) +- LogicalAggregate(group=[{0, 1, 2, 3}], groups=[[{0, 2, 3}, {1, 2, 3}]], EXPR$2=[COUNT()], EXPR$3=[SUM($4)], EXPR$4=[MAX($4) FILTER $5], uv=[COUNT(DISTINCT $6)]) +- LogicalProject(a=[$0], b=[$1], window_start=[$7], window_end=[$8], d=[$3], $f5=[IS TRUE(>($1, 1000))], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -4586,7 +4586,7 @@ GROUP BY GROUPING SETS ((a), (b)), window_start, window_end LogicalProject(a=[$0], b=[$1], EXPR$2=[$4], EXPR$3=[$5], EXPR$4=[$6], uv=[$7]) +- LogicalAggregate(group=[{0, 1, 2, 3}], groups=[[{0, 2, 3}, {1, 2, 3}]], EXPR$2=[COUNT()], EXPR$3=[SUM($4)], EXPR$4=[MAX($4) FILTER $5], uv=[COUNT(DISTINCT $6)]) +- LogicalProject(a=[$0], b=[$1], window_start=[$7], window_end=[$8], d=[$3], $f5=[IS TRUE(>($1, 1000))], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -4629,7 +4629,7 @@ GROUP BY GROUPING SETS ((a, window_start, window_end), (b, window_start, window_ LogicalProject(a=[$0], b=[$3], uv=[$4]) +- LogicalAggregate(group=[{0, 1, 2, 3}], groups=[[{0, 1, 2}, {1, 2, 3}]], uv=[COUNT(DISTINCT $4)]) +- LogicalProject(a=[$0], window_start=[$7], window_end=[$8], b=[$1], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -4663,7 +4663,7 @@ GROUP BY GROUPING SETS ((a, window_start, window_end), (b, window_start, window_ LogicalProject(a=[$0], b=[$3], uv=[$4]) +- LogicalAggregate(group=[{0, 1, 2, 3}], groups=[[{0, 1, 2}, {1, 2, 3}]], uv=[COUNT(DISTINCT $4)]) +- LogicalProject(a=[$0], window_start=[$7], window_end=[$8], b=[$1], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -4700,7 +4700,7 @@ GROUP BY window_start, window_end LogicalProject(EXPR$0=[$2], EXPR$1=[$3], EXPR$2=[$4], wAvg=[$5], uv=[$6]) +- LogicalAggregate(group=[{0, 1}], EXPR$0=[COUNT()], EXPR$1=[SUM($2)], EXPR$2=[MAX($2) FILTER $3], wAvg=[weightedAvg($4, $5)], uv=[COUNT(DISTINCT $6)]) +- LogicalProject(window_start=[$7], window_end=[$8], d=[$3], $f3=[IS TRUE(>($1, 1000))], b=[$1], e=[$4], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -4736,7 +4736,7 @@ GROUP BY window_start, window_end LogicalProject(EXPR$0=[$2], EXPR$1=[$3], EXPR$2=[$4], wAvg=[$5], uv=[$6]) +- LogicalAggregate(group=[{0, 1}], EXPR$0=[COUNT()], EXPR$1=[SUM($2)], EXPR$2=[MAX($2) FILTER $3], wAvg=[weightedAvg($4, $5)], uv=[COUNT(DISTINCT $6)]) +- LogicalProject(window_start=[$7], window_end=[$8], d=[$3], $f3=[IS TRUE(>($1, 1000))], b=[$1], e=[$4], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -4853,7 +4853,7 @@ GROUP BY a, window_start, window_end ($1, 1000))], b=[$1], e=[$4], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -4891,7 +4891,7 @@ GROUP BY a, window_start, window_end ($1, 1000))], b=[$1], e=[$4], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -4930,7 +4930,7 @@ GROUP BY a, window_start, window_end ($1, 1000))], b=[$1], e=[$4], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -4968,7 +4968,7 @@ GROUP BY a, window_start, window_end ($1, 1000))], b=[$1], e=[$4], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -5004,7 +5004,7 @@ GROUP BY a, window_start, window_end ($1, 1000))], b=[$1], e=[$4], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -5184,7 +5184,7 @@ GROUP BY a, window_start, window_end LogicalProject(a=[$0], EXPR$1=[$3], EXPR$2=[$4], EXPR$3=[$5], wAvg=[$6], uv=[$7], window_start=[$1], window_end=[$2]) +- LogicalAggregate(group=[{0, 1, 2}], EXPR$1=[COUNT()], EXPR$2=[SUM($3)], EXPR$3=[MAX($3) FILTER $4], wAvg=[weightedAvg($5, $6)], uv=[COUNT(DISTINCT $7)]) +- LogicalProject(a=[$0], window_start=[$7], window_end=[$8], d=[$3], $f4=[IS TRUE(>($1, 1000))], b=[$1], e=[$4], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -5222,7 +5222,7 @@ GROUP BY a, window_start, window_end LogicalProject(a=[$0], EXPR$1=[$3], EXPR$2=[$4], EXPR$3=[$5], wAvg=[$6], uv=[$7], window_start=[$1], window_end=[$2]) +- LogicalAggregate(group=[{0, 1, 2}], EXPR$1=[COUNT()], EXPR$2=[SUM($3)], EXPR$3=[MAX($3) FILTER $4], wAvg=[weightedAvg($5, $6)], uv=[COUNT(DISTINCT $7)]) +- LogicalProject(a=[$0], window_start=[$7], window_end=[$8], d=[$3], $f4=[IS TRUE(>($1, 1000))], b=[$1], e=[$4], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/join/WindowJoinTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/join/WindowJoinTest.xml index 77a528d3fb3ea..92829b641d0d3 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/join/WindowJoinTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/join/WindowJoinTest.xml @@ -51,7 +51,7 @@ LogicalProject(a=[$0], window_start=[$1], window_end=[$2], window_time=[$3], cnt LogicalFilter(condition=[AND(=($cor0.window_start, $1), =($cor0.window_end, $2), =($cor0.a, $0))]) LogicalAggregate(group=[{0, 1, 2, 3}], cnt=[COUNT()], uv=[COUNT(DISTINCT $4)]) LogicalProject(a=[$0], window_start=[$5], window_end=[$6], window_time=[$7], c=[$2]) - LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) @@ -59,7 +59,7 @@ LogicalFilter(condition=[AND(=($cor0.window_start, $1), =($cor0.window_end, $2), }))], variablesSet=[[$cor0]]) +- LogicalAggregate(group=[{0, 1, 2, 3}], cnt=[COUNT()], uv=[COUNT(DISTINCT $4)]) +- LogicalProject(a=[$0], window_start=[$5], window_end=[$6], window_time=[$7], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) @@ -122,7 +122,7 @@ LogicalProject(a=[$0]) LogicalFilter(condition=[AND(=($cor0.window_start, $1), =($cor0.window_end, $2))]) LogicalAggregate(group=[{0, 1, 2, 3}], cnt=[COUNT()], uv=[COUNT(DISTINCT $4)]) LogicalProject(a=[$0], window_start=[$5], window_end=[$6], window_time=[$7], c=[$2]) - LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) @@ -130,7 +130,7 @@ LogicalProject(a=[$0]) }))], variablesSet=[[$cor0]]) +- LogicalAggregate(group=[{0, 1, 2, 3}], cnt=[COUNT()], uv=[COUNT(DISTINCT $4)]) +- LogicalProject(a=[$0], window_start=[$5], window_end=[$6], window_time=[$7], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) @@ -166,13 +166,13 @@ LogicalSink(table=[default_catalog.default_database.sink1], fields=[window_start +- LogicalJoin(condition=[AND(=($0, $3), =($1, $4), =($2, $5))], joinType=[inner]) :- LogicalAggregate(group=[{0, 1, 2}]) : +- LogicalProject(user_id=[$0], window_start=[$4], window_end=[$5]) - : +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'event_time'), 60000:INTERVAL MINUTE)], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) order_id, INTEGER amount, TIMESTAMP(3) *ROWTIME* event_time, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + : +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'event_time'), 60000:INTERVAL MINUTE)], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) order_id, INTEGER amount, TIMESTAMP(3) event_time, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) : +- LogicalProject(user_id=[$0], order_id=[$1], amount=[$2], event_time=[$3]) : +- LogicalWatermarkAssigner(rowtime=[event_time], watermark=[$3]) : +- LogicalTableScan(table=[[default_catalog, default_database, food_order]]) +- LogicalAggregate(group=[{0, 1, 2}]) +- LogicalProject(user_id=[$0], window_start=[$4], window_end=[$5]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'event_time'), 60000:INTERVAL MINUTE)], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) order_id, INTEGER amount, TIMESTAMP(3) *ROWTIME* event_time, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'event_time'), 60000:INTERVAL MINUTE)], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) order_id, INTEGER amount, TIMESTAMP(3) event_time, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(user_id=[$0], order_id=[$1], amount=[$2], event_time=[$3]) +- LogicalWatermarkAssigner(rowtime=[event_time], watermark=[$3]) +- LogicalTableScan(table=[[default_catalog, default_database, food_order]]) @@ -183,13 +183,13 @@ LogicalSink(table=[default_catalog.default_database.sink2], fields=[window_start +- LogicalJoin(condition=[AND(=($0, $3), =($1, $4), =($2, $5))], joinType=[inner]) :- LogicalAggregate(group=[{0, 1, 2}]) : +- LogicalProject(user_id=[$0], window_start=[$4], window_end=[$5]) - : +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'event_time'), 60000:INTERVAL MINUTE)], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) order_id, INTEGER amount, TIMESTAMP(3) *ROWTIME* event_time, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + : +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'event_time'), 60000:INTERVAL MINUTE)], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) order_id, INTEGER amount, TIMESTAMP(3) event_time, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) : +- LogicalProject(user_id=[$0], order_id=[$1], amount=[$2], event_time=[$3]) : +- LogicalWatermarkAssigner(rowtime=[event_time], watermark=[$3]) : +- LogicalTableScan(table=[[default_catalog, default_database, food_order]]) +- LogicalAggregate(group=[{0, 1, 2}]) +- LogicalProject(user_id=[$0], window_start=[$4], window_end=[$5]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'event_time'), 60000:INTERVAL MINUTE)], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) order_id, INTEGER amount, TIMESTAMP(3) *ROWTIME* event_time, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'event_time'), 60000:INTERVAL MINUTE)], rowType=[RecordType(VARCHAR(2147483647) user_id, VARCHAR(2147483647) order_id, INTEGER amount, TIMESTAMP(3) event_time, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(user_id=[$0], order_id=[$1], amount=[$2], event_time=[$3]) +- LogicalWatermarkAssigner(rowtime=[event_time], watermark=[$3]) +- LogicalTableScan(table=[[default_catalog, default_database, food_order]]) @@ -267,14 +267,14 @@ LogicalProject(a=[$0], window_start=[$1], window_end=[$2], window_time=[$3], cnt +- LogicalJoin(condition=[AND(=($1, $7), =($2, $8), IS NOT DISTINCT FROM($0, $6))], joinType=[inner]) :- LogicalAggregate(group=[{0, 1, 2, 3}], cnt=[COUNT()], uv=[COUNT(DISTINCT $4)]) : +- LogicalProject(a=[$0], window_start=[$5], window_end=[$6], window_time=[$7], c=[$2]) - : +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + : +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) : +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalAggregate(group=[{0, 1, 2, 3}], cnt=[COUNT()], uv=[COUNT(DISTINCT $4)]) +- LogicalProject(a=[$0], window_start=[$5], window_end=[$6], window_time=[$7], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) @@ -340,14 +340,14 @@ LogicalProject(a=[$0], window_start=[$1], window_end=[$2], window_time=[$3], cnt +- LogicalJoin(condition=[AND(=($1, $7), =($0, $6))], joinType=[inner]) :- LogicalAggregate(group=[{0, 1, 2, 3}], cnt=[COUNT()], uv=[COUNT(DISTINCT $4)]) : +- LogicalProject(a=[$0], window_start=[$5], window_end=[$6], window_time=[$7], c=[$2]) - : +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + : +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) : +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalAggregate(group=[{0, 1, 2, 3}], cnt=[COUNT()], uv=[COUNT(DISTINCT $4)]) +- LogicalProject(a=[$0], window_start=[$5], window_end=[$6], window_time=[$7], c=[$2]) - +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) @@ -411,14 +411,14 @@ LogicalProject(a=[$0], window_start=[$1], window_end=[$2], window_time=[$3], cnt +- LogicalJoin(condition=[AND(=($1, $7), =($0, $6))], joinType=[inner]) :- LogicalAggregate(group=[{0, 1, 2, 3}], cnt=[COUNT()], uv=[COUNT(DISTINCT $4)]) : +- LogicalProject(a=[$0], window_start=[$5], window_end=[$6], window_time=[$7], c=[$2]) - : +- LogicalTableFunctionScan(invocation=[HOP(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + : +- LogicalTableFunctionScan(invocation=[HOP(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) : +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalAggregate(group=[{0, 1, 2, 3}], cnt=[COUNT()], uv=[COUNT(DISTINCT $4)]) +- LogicalProject(a=[$0], window_start=[$5], window_end=[$6], window_time=[$7], c=[$2]) - +- LogicalTableFunctionScan(invocation=[HOP(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[HOP(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) @@ -480,14 +480,14 @@ LogicalProject(a=[$0], window_start=[$1], window_end=[$2], window_time=[$3], cnt +- LogicalJoin(condition=[AND(=($1, $7), =($0, $6))], joinType=[inner]) :- LogicalAggregate(group=[{0, 1, 2, 3}], cnt=[COUNT()], uv=[COUNT(DISTINCT $4)]) : +- LogicalProject(a=[$0], window_start=[$5], window_end=[$6], window_time=[$7], c=[$2]) - : +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + : +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) : +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalAggregate(group=[{0, 1, 2, 3}], cnt=[COUNT()], uv=[COUNT(DISTINCT $4)]) +- LogicalProject(a=[$0], window_start=[$5], window_end=[$6], window_time=[$7], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) @@ -553,14 +553,14 @@ LogicalProject(a=[$0], window_start=[$1], window_end=[$2], window_time=[$3], cnt +- LogicalJoin(condition=[AND(=($2, $8), =($0, $6))], joinType=[inner]) :- LogicalAggregate(group=[{0, 1, 2, 3}], cnt=[COUNT()], uv=[COUNT(DISTINCT $4)]) : +- LogicalProject(a=[$0], window_start=[$5], window_end=[$6], window_time=[$7], c=[$2]) - : +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + : +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) : +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalAggregate(group=[{0, 1, 2, 3}], cnt=[COUNT()], uv=[COUNT(DISTINCT $4)]) +- LogicalProject(a=[$0], window_start=[$5], window_end=[$6], window_time=[$7], c=[$2]) - +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) @@ -624,14 +624,14 @@ LogicalProject(a=[$0], window_start=[$1], window_end=[$2], window_time=[$3], cnt +- LogicalJoin(condition=[AND(=($2, $8), =($0, $6))], joinType=[inner]) :- LogicalAggregate(group=[{0, 1, 2, 3}], cnt=[COUNT()], uv=[COUNT(DISTINCT $4)]) : +- LogicalProject(a=[$0], window_start=[$5], window_end=[$6], window_time=[$7], c=[$2]) - : +- LogicalTableFunctionScan(invocation=[HOP(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + : +- LogicalTableFunctionScan(invocation=[HOP(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) : +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalAggregate(group=[{0, 1, 2, 3}], cnt=[COUNT()], uv=[COUNT(DISTINCT $4)]) +- LogicalProject(a=[$0], window_start=[$5], window_end=[$6], window_time=[$7], c=[$2]) - +- LogicalTableFunctionScan(invocation=[HOP(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[HOP(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) @@ -693,14 +693,14 @@ LogicalProject(a=[$0], window_start=[$1], window_end=[$2], window_time=[$3], cnt +- LogicalJoin(condition=[AND(=($2, $8), =($0, $6))], joinType=[inner]) :- LogicalAggregate(group=[{0, 1, 2, 3}], cnt=[COUNT()], uv=[COUNT(DISTINCT $4)]) : +- LogicalProject(a=[$0], window_start=[$5], window_end=[$6], window_time=[$7], c=[$2]) - : +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + : +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) : +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalAggregate(group=[{0, 1, 2, 3}], cnt=[COUNT()], uv=[COUNT(DISTINCT $4)]) +- LogicalProject(a=[$0], window_start=[$5], window_end=[$6], window_time=[$7], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) @@ -773,7 +773,7 @@ LogicalProject(a=[$0], window_start=[$1], window_end=[$2], window_time=[$3], cnt : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalAggregate(group=[{0, 1, 2, 3}], cnt=[COUNT()], uv=[COUNT(DISTINCT $4)]) +- LogicalProject(a=[$0], window_start=[$5], window_end=[$6], window_time=[$7], c=[$2]) - +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) @@ -838,14 +838,14 @@ LogicalProject(a=[$0], window_start=[$1], window_end=[$2], window_time=[$3], cnt +- LogicalJoin(condition=[AND(=($1, $7), =($2, $8), =($0, $6))], joinType=[inner]) :- LogicalAggregate(group=[{0, 1, 2, 3}], cnt=[COUNT()], uv=[COUNT(DISTINCT $4)]) : +- LogicalProject(a=[$0], window_start=[$5], window_end=[$6], window_time=[$7], c=[$2]) - : +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 7200000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + : +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 7200000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) : +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalAggregate(group=[{0, 1, 2, 3}], cnt=[COUNT()], uv=[COUNT(DISTINCT $4)]) +- LogicalProject(a=[$0], window_start=[$5], window_end=[$6], window_time=[$7], c=[$2]) - +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) @@ -909,14 +909,14 @@ LogicalProject(a=[$0], window_start=[$1], window_end=[$2], window_time=[$3], cnt +- LogicalJoin(condition=[AND(=($1, $7), =($2, $8), =($0, $6))], joinType=[inner]) :- LogicalAggregate(group=[{0, 1, 2, 3}], cnt=[COUNT()], uv=[COUNT(DISTINCT $4)]) : +- LogicalProject(a=[$0], window_start=[$5], window_end=[$6], window_time=[$7], c=[$2]) - : +- LogicalTableFunctionScan(invocation=[HOP(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + : +- LogicalTableFunctionScan(invocation=[HOP(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) : +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalAggregate(group=[{0, 1, 2, 3}], cnt=[COUNT()], uv=[COUNT(DISTINCT $4)]) +- LogicalProject(a=[$0], window_start=[$5], window_end=[$6], window_time=[$7], c=[$2]) - +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) @@ -982,14 +982,14 @@ LogicalProject(a=[$0], window_start=[$1], window_end=[$2], window_time=[$3], cnt +- LogicalJoin(condition=[AND(=($1, $7), =($2, $8), =($0, $6))], joinType=[inner]) :- LogicalAggregate(group=[{0, 1, 2, 3}], cnt=[COUNT()], uv=[COUNT(DISTINCT $4)]) : +- LogicalProject(a=[$0], window_start=[$5], window_end=[$6], window_time=[$7], c=[$2]) - : +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + : +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) : +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalAggregate(group=[{0, 1, 2, 3}], cnt=[COUNT()], uv=[COUNT(DISTINCT $4)]) +- LogicalProject(a=[$0], window_start=[$5], window_end=[$6], window_time=[$7], c=[$2]) - +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) @@ -1129,14 +1129,14 @@ LogicalProject(a=[$0], window_start=[$1], window_end=[$2], window_time=[$3], cnt +- LogicalJoin(condition=[AND(=($1, $7), =($2, $8), =($0, $6))], joinType=[inner]) :- LogicalAggregate(group=[{0, 1, 2, 3}], cnt=[COUNT()], uv=[COUNT(DISTINCT $4)]) : +- LogicalProject(a=[$0], window_start=[$5], window_end=[$6], window_time=[$7], c=[$2]) - : +- LogicalTableFunctionScan(invocation=[HOP(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + : +- LogicalTableFunctionScan(invocation=[HOP(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) : +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalAggregate(group=[{0, 1, 2, 3}], cnt=[COUNT()], uv=[COUNT(DISTINCT $4)]) +- LogicalProject(a=[$0], window_start=[$5], window_end=[$6], window_time=[$7], c=[$2]) - +- LogicalTableFunctionScan(invocation=[HOP(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[HOP(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) @@ -1272,14 +1272,14 @@ LogicalProject(a=[$0], window_start=[$1], window_end=[$2], window_time=[$3], cnt +- LogicalJoin(condition=[AND(=($1, $7), =($2, $8), =($0, $6))], joinType=[inner]) :- LogicalAggregate(group=[{0, 1, 2, 3}], cnt=[COUNT()], uv=[COUNT(DISTINCT $4)]) : +- LogicalProject(a=[$0], window_start=[$5], window_end=[$6], window_time=[$7], c=[$2]) - : +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + : +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) : +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalAggregate(group=[{0, 1, 2, 3}], cnt=[COUNT()], uv=[COUNT(DISTINCT $4)]) +- LogicalProject(a=[$0], window_start=[$5], window_end=[$6], window_time=[$7], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) @@ -1413,7 +1413,7 @@ LogicalProject(a=[$0], window_start=[$1], window_end=[$2], window_time=[$3], cnt LogicalFilter(condition=[AND(=($cor0.window_start, $1), =($cor0.window_end, $2), =($cor0.a, $0))]) LogicalAggregate(group=[{0, 1, 2, 3}], cnt=[COUNT()], uv=[COUNT(DISTINCT $4)]) LogicalProject(a=[$0], window_start=[$5], window_end=[$6], window_time=[$7], c=[$2]) - LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) @@ -1421,7 +1421,7 @@ LogicalFilter(condition=[AND(=($cor0.window_start, $1), =($cor0.window_end, $2), })], variablesSet=[[$cor0]]) +- LogicalAggregate(group=[{0, 1, 2, 3}], cnt=[COUNT()], uv=[COUNT(DISTINCT $4)]) +- LogicalProject(a=[$0], window_start=[$5], window_end=[$6], window_time=[$7], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) @@ -1484,7 +1484,7 @@ LogicalProject(a=[$0]) LogicalFilter(condition=[AND(=($cor0.window_start, $1), =($cor0.window_end, $2))]) LogicalAggregate(group=[{0, 1, 2, 3}], cnt=[COUNT()], uv=[COUNT(DISTINCT $4)]) LogicalProject(a=[$0], window_start=[$5], window_end=[$6], window_time=[$7], c=[$2]) - LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) @@ -1492,7 +1492,7 @@ LogicalProject(a=[$0]) })], variablesSet=[[$cor0]]) +- LogicalAggregate(group=[{0, 1, 2, 3}], cnt=[COUNT()], uv=[COUNT(DISTINCT $4)]) +- LogicalProject(a=[$0], window_start=[$5], window_end=[$6], window_time=[$7], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) @@ -1541,13 +1541,13 @@ ON L.window_start = R.window_start AND L.window_end = R.window_end AND L.a = R.a LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4], window_start=[$5], window_end=[$6], window_time=[$7], a0=[$8], b0=[$9], c0=[$10], rowtime0=[$11], proctime0=[$12], window_start0=[$13], window_end0=[$14], window_time0=[$15]) +- LogicalJoin(condition=[AND(=($5, $13), =($6, $14), =($0, $8))], joinType=[inner]) :- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4], window_start=[$5], window_end=[$6], window_time=[$7]) - : +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + : +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) : +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4], window_start=[$5], window_end=[$6], window_time=[$7]) - +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) @@ -1595,13 +1595,13 @@ LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4], window_start +- LogicalJoin(condition=[AND(=($5, $13), =($6, $14), =($0, $8))], joinType=[inner]) :- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4], window_start=[$5], window_end=[$6], window_time=[$7]) : +- LogicalFilter(condition=[>($2, 10)]) - : +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + : +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) : +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4], window_start=[$5], window_end=[$6], window_time=[$7]) - +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) @@ -1649,14 +1649,14 @@ ON L.window_start = R.window_start AND L.window_end = R.window_end AND L.a = R.a LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4], window_start=[$5], window_end=[$6], window_time=[$7], a0=[$8], b0=[$9], c0=[$10], rowtime0=[$11], proctime0=[$12], window_start0=[$13], window_end0=[$14], window_time0=[$15]) +- LogicalJoin(condition=[AND(=($5, $13), =($6, $14), =($0, $8))], joinType=[inner]) :- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4], window_start=[$5], window_end=[$6], window_time=[$7]) - : +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + : +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) : +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4], window_start=[$5], window_end=[$6], window_time=[$7]) +- LogicalFilter(condition=[>($2, 10)]) - +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) @@ -1706,14 +1706,14 @@ LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4], window_start +- LogicalJoin(condition=[AND(=($5, $13), =($6, $14), =($0, $8))], joinType=[inner]) :- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4], window_start=[$5], window_end=[$6], window_time=[$7]) : +- LogicalFilter(condition=[>($2, 10)]) - : +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + : +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) : +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4], window_start=[$5], window_end=[$6], window_time=[$7]) +- LogicalFilter(condition=[>($2, 10)]) - +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) @@ -1761,13 +1761,13 @@ ON L.window_start = R.window_start AND L.window_end = R.window_end AND L.a = R.a LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4], window_start=[$5], window_end=[$6], window_time=[$7], a0=[$8], b0=[$9], c0=[$10], rowtime0=[$11], proctime0=[$12], window_start0=[$13], window_end0=[$14], window_time0=[$15]) +- LogicalJoin(condition=[AND(=($5, $13), =($6, $14), =($0, $8))], joinType=[inner]) :- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4], window_start=[$5], window_end=[$6], window_time=[$7]) - : +- LogicalTableFunctionScan(invocation=[HOP(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + : +- LogicalTableFunctionScan(invocation=[HOP(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) : +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4], window_start=[$5], window_end=[$6], window_time=[$7]) - +- LogicalTableFunctionScan(invocation=[HOP(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[HOP(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) @@ -1815,13 +1815,13 @@ LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4], window_start +- LogicalJoin(condition=[AND(=($5, $13), =($6, $14), =($0, $8))], joinType=[inner]) :- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4], window_start=[$5], window_end=[$6], window_time=[$7]) : +- LogicalFilter(condition=[>($2, 10)]) - : +- LogicalTableFunctionScan(invocation=[HOP(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + : +- LogicalTableFunctionScan(invocation=[HOP(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) : +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4], window_start=[$5], window_end=[$6], window_time=[$7]) - +- LogicalTableFunctionScan(invocation=[HOP(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[HOP(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) @@ -1869,14 +1869,14 @@ ON L.window_start = R.window_start AND L.window_end = R.window_end AND L.a = R.a LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4], window_start=[$5], window_end=[$6], window_time=[$7], a0=[$8], b0=[$9], c0=[$10], rowtime0=[$11], proctime0=[$12], window_start0=[$13], window_end0=[$14], window_time0=[$15]) +- LogicalJoin(condition=[AND(=($5, $13), =($6, $14), =($0, $8))], joinType=[inner]) :- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4], window_start=[$5], window_end=[$6], window_time=[$7]) - : +- LogicalTableFunctionScan(invocation=[HOP(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + : +- LogicalTableFunctionScan(invocation=[HOP(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) : +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4], window_start=[$5], window_end=[$6], window_time=[$7]) +- LogicalFilter(condition=[>($2, 10)]) - +- LogicalTableFunctionScan(invocation=[HOP(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[HOP(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) @@ -1926,14 +1926,14 @@ LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4], window_start +- LogicalJoin(condition=[AND(=($5, $13), =($6, $14), =($0, $8))], joinType=[inner]) :- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4], window_start=[$5], window_end=[$6], window_time=[$7]) : +- LogicalFilter(condition=[>($2, 10)]) - : +- LogicalTableFunctionScan(invocation=[HOP(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + : +- LogicalTableFunctionScan(invocation=[HOP(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) : +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4], window_start=[$5], window_end=[$6], window_time=[$7]) +- LogicalFilter(condition=[>($2, 10)]) - +- LogicalTableFunctionScan(invocation=[HOP(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[HOP(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) @@ -1979,13 +1979,13 @@ ON L.window_start = R.window_start AND L.window_end = R.window_end AND L.a = R.a LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4], window_start=[$5], window_end=[$6], window_time=[$7], a0=[$8], b0=[$9], c0=[$10], rowtime0=[$11], proctime0=[$12], window_start0=[$13], window_end0=[$14], window_time0=[$15]) +- LogicalJoin(condition=[AND(=($5, $13), =($6, $14), =($0, $8))], joinType=[inner]) :- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4], window_start=[$5], window_end=[$6], window_time=[$7]) - : +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + : +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) : +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4], window_start=[$5], window_end=[$6], window_time=[$7]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) @@ -2031,13 +2031,13 @@ LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4], window_start +- LogicalJoin(condition=[AND(=($5, $13), =($6, $14), =($0, $8))], joinType=[inner]) :- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4], window_start=[$5], window_end=[$6], window_time=[$7]) : +- LogicalFilter(condition=[>($2, 10)]) - : +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + : +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) : +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4], window_start=[$5], window_end=[$6], window_time=[$7]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) @@ -2083,14 +2083,14 @@ ON L.window_start = R.window_start AND L.window_end = R.window_end AND L.a = R.a LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4], window_start=[$5], window_end=[$6], window_time=[$7], a0=[$8], b0=[$9], c0=[$10], rowtime0=[$11], proctime0=[$12], window_start0=[$13], window_end0=[$14], window_time0=[$15]) +- LogicalJoin(condition=[AND(=($5, $13), =($6, $14), =($0, $8))], joinType=[inner]) :- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4], window_start=[$5], window_end=[$6], window_time=[$7]) - : +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + : +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) : +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4], window_start=[$5], window_end=[$6], window_time=[$7]) +- LogicalFilter(condition=[>($2, 10)]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) @@ -2138,14 +2138,14 @@ LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4], window_start +- LogicalJoin(condition=[AND(=($5, $13), =($6, $14), =($0, $8))], joinType=[inner]) :- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4], window_start=[$5], window_end=[$6], window_time=[$7]) : +- LogicalFilter(condition=[>($2, 10)]) - : +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + : +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) : +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4], window_start=[$5], window_end=[$6], window_time=[$7]) +- LogicalFilter(condition=[>($2, 10)]) - +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) @@ -2188,13 +2188,13 @@ LogicalProject(rowtime=[$0], a=[$1], l_b=[$2], l_c=[$3], r_b=[$4], r_c=[$5], a0= :- LogicalProject(rowtime=[$7], a=[$0], l_b=[$1], l_c=[$2], r_b=[$9], r_c=[$10]) : +- LogicalJoin(condition=[AND(=($5, $13), =($6, $14), =($0, $8))], joinType=[inner]) : :- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4], window_start=[$5], window_end=[$6], window_time=[$7]) - : : +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + : : +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) : : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) : : +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) : : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) : : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4], window_start=[$5], window_end=[$6], window_time=[$7]) - : +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + : +- LogicalTableFunctionScan(invocation=[TUMBLE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) : +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) @@ -2246,14 +2246,14 @@ LogicalProject(rowtime=[$0], a=[$1], l_cnt=[$2], l_uv=[$3], r_cnt=[$4], r_uv=[$5 : +- LogicalJoin(condition=[AND(=($1, $7), =($2, $8), =($0, $6))], joinType=[inner]) : :- LogicalAggregate(group=[{0, 1, 2, 3}], cnt=[COUNT()], uv=[COUNT(DISTINCT $4)]) : : +- LogicalProject(a=[$0], window_start=[$5], window_end=[$6], window_time=[$7], c=[$2]) - : : +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + : : +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) : : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) : : +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) : : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) : : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) : +- LogicalAggregate(group=[{0, 1, 2, 3}], cnt=[COUNT()], uv=[COUNT(DISTINCT $4)]) : +- LogicalProject(a=[$0], window_start=[$5], window_end=[$6], window_time=[$7], c=[$2]) - : +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) + : +- LogicalTableFunctionScan(invocation=[CUMULATE(TABLE(#0), DESCRIPTOR(_UTF-16LE'rowtime'), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) : +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) @@ -2383,12 +2383,12 @@ ON L.window_start = R.window_start AND L.window_end = R.window_end AND L.a = R.a Date: Thu, 11 Jun 2026 22:36:32 +0900 Subject: [PATCH 3/3] [FLINK-39899][table-planner] Preserve window deduplicate after materializing the original time attribute --- .../plan/trait/RelWindowProperties.java | 26 +++++++++- .../metadata/FlinkRelMdWindowProperties.scala | 24 ++++++++-- .../StreamPhysicalWindowDeduplicateRule.scala | 2 +- .../stream/StreamPhysicalWindowRankRule.scala | 2 +- .../table/planner/plan/utils/RankUtil.scala | 29 ++++++++++-- .../exec/operator/StreamOperatorNameTest.xml | 16 +++---- .../plan/stream/sql/WindowDeduplicateTest.xml | 47 +++++++++++++++++-- .../FlinkRelMdWindowPropertiesTest.scala | 4 +- .../plan/metadata/MetadataTestUtil.scala | 4 +- .../stream/sql/WindowDeduplicateTest.scala | 20 ++++++++ 10 files changed, 149 insertions(+), 25 deletions(-) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/trait/RelWindowProperties.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/trait/RelWindowProperties.java index 1966234a9b48b..a5551bdcf46e7 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/trait/RelWindowProperties.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/trait/RelWindowProperties.java @@ -38,6 +38,16 @@ public class RelWindowProperties { private final ImmutableBitSet windowStartColumns; private final ImmutableBitSet windowEndColumns; private final ImmutableBitSet windowTimeColumns; + + /** + * The columns that hold the window's original time attribute (the column referenced by the + * window TVF time descriptor). After applying the window TVF this column is materialized to a + * regular timestamp (see FLINK-39899), so it can no longer be recognized as a time attribute by + * its type; we track its position here so that window operators (e.g. window deduplicate) can + * still identify it. + */ + private final ImmutableBitSet timeAttributeColumns; + private final WindowSpec windowSpec; private final LogicalType timeAttributeType; @@ -49,6 +59,7 @@ public class RelWindowProperties { ImmutableBitSet windowStartColumns, ImmutableBitSet windowEndColumns, ImmutableBitSet windowTimeColumns, + ImmutableBitSet timeAttributeColumns, WindowSpec windowSpec, LogicalType timeAttributeType) { if (windowStartColumns.isEmpty() || windowEndColumns.isEmpty()) { @@ -59,6 +70,7 @@ public class RelWindowProperties { windowStartColumns, windowEndColumns, windowTimeColumns, + timeAttributeColumns, windowSpec, timeAttributeType); } @@ -68,6 +80,7 @@ private RelWindowProperties( ImmutableBitSet windowStartColumns, ImmutableBitSet windowEndColumns, ImmutableBitSet windowTimeColumns, + ImmutableBitSet timeAttributeColumns, WindowSpec windowSpec, LogicalType timeAttributeType) { checkArgument( @@ -77,6 +90,7 @@ private RelWindowProperties( this.windowStartColumns = checkNotNull(windowStartColumns); this.windowEndColumns = checkNotNull(windowEndColumns); this.windowTimeColumns = checkNotNull(windowTimeColumns); + this.timeAttributeColumns = checkNotNull(timeAttributeColumns); this.windowSpec = checkNotNull(windowSpec); this.timeAttributeType = checkNotNull(timeAttributeType); } @@ -84,11 +98,13 @@ private RelWindowProperties( public @Nullable RelWindowProperties copy( ImmutableBitSet windowStartColumns, ImmutableBitSet windowEndColumns, - ImmutableBitSet windowTimeColumns) { + ImmutableBitSet windowTimeColumns, + ImmutableBitSet timeAttributeColumns) { return create( windowStartColumns, windowEndColumns, windowTimeColumns, + timeAttributeColumns, windowSpec, this.timeAttributeType); } @@ -97,11 +113,13 @@ private RelWindowProperties( ImmutableBitSet windowStartColumns, ImmutableBitSet windowEndColumns, ImmutableBitSet windowTimeColumns, + ImmutableBitSet timeAttributeColumns, WindowSpec windowSpec) { return create( windowStartColumns, windowEndColumns, windowTimeColumns, + timeAttributeColumns, windowSpec, this.timeAttributeType); } @@ -118,6 +136,10 @@ public ImmutableBitSet getWindowTimeColumns() { return windowTimeColumns; } + public ImmutableBitSet getTimeAttributeColumns() { + return timeAttributeColumns; + } + public ImmutableBitSet getWindowColumns() { return windowStartColumns.union(windowEndColumns).union(windowTimeColumns); } @@ -151,6 +173,8 @@ public boolean equals(Object o) { @Override public int hashCode() { + // timeAttributeColumns is intentionally excluded (like timeAttributeType): it is derived + // deterministically from the same window, so it cannot differ when the other fields match. return Objects.hash(windowStartColumns, windowEndColumns, windowTimeColumns, windowSpec); } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdWindowProperties.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdWindowProperties.scala index 80cce27a55938..eacf6c5edd159 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdWindowProperties.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdWindowProperties.scala @@ -104,6 +104,7 @@ class FlinkRelMdWindowProperties private extends MetadataHandler[FlinkMetadata.W transformColumnIndex(childProps.getWindowStartColumns, mapInToOutPos), transformColumnIndex(childProps.getWindowEndColumns, mapInToOutPos), transformColumnIndex(childProps.getWindowTimeColumns, mapInToOutPos), + transformColumnIndex(childProps.getTimeAttributeColumns, mapInToOutPos), updateWindowSpec(windowSpec, mapInToOutPos) ) } @@ -209,7 +210,9 @@ class FlinkRelMdWindowProperties private extends MetadataHandler[FlinkMetadata.W val starts = inferWindowPropertyAfterExpand(inputWindowProperties.getWindowStartColumns) val ends = inferWindowPropertyAfterExpand(inputWindowProperties.getWindowEndColumns) val times = inferWindowPropertyAfterExpand(inputWindowProperties.getWindowTimeColumns) - inputWindowProperties.copy(starts, ends, times) + val timeAttributes = inferWindowPropertyAfterExpand( + inputWindowProperties.getTimeAttributeColumns) + inputWindowProperties.copy(starts, ends, times, timeAttributes) } def getWindowProperties(rel: Exchange, mq: RelMetadataQuery): RelWindowProperties = { @@ -229,7 +232,9 @@ class FlinkRelMdWindowProperties private extends MetadataHandler[FlinkMetadata.W val starts = properties.map(_.getWindowStartColumns).reduce((l, r) => l.intersect(r)) val ends = properties.map(_.getWindowEndColumns).reduce((l, r) => l.intersect(r)) val times = properties.map(_.getWindowTimeColumns).reduce((l, r) => l.intersect(r)) - properties.head.copy(starts, ends, times) + val timeAttributes = + properties.map(_.getTimeAttributeColumns).reduce((l, r) => l.intersect(r)) + properties.head.copy(starts, ends, times, timeAttributes) } else { // window properties is lost if windows are not equal null @@ -246,6 +251,7 @@ class FlinkRelMdWindowProperties private extends MetadataHandler[FlinkMetadata.W ImmutableBitSet.of(fieldCount - 3), ImmutableBitSet.of(fieldCount - 2), ImmutableBitSet.of(fieldCount - 1), + ImmutableBitSet.of(windowingStrategy.getTimeAttributeIndex), windowingStrategy.getWindow, windowingStrategy.getTimeAttributeType ) @@ -274,11 +280,16 @@ class FlinkRelMdWindowProperties private extends MetadataHandler[FlinkMetadata.W .intersect(grouping) .map(grouping.indexOf(_)) .toList + val timeAttributeColumns = windowProperties.getTimeAttributeColumns + .intersect(grouping) + .map(grouping.indexOf(_)) + .toList RelWindowProperties.create( ImmutableBitSet.of(startColumns: _*), ImmutableBitSet.of(endColumns: _*), ImmutableBitSet.of(timeColumns: _*), + ImmutableBitSet.of(timeAttributeColumns: _*), windowProperties.getWindowSpec, windowProperties.getTimeAttributeType ) @@ -292,6 +303,7 @@ class FlinkRelMdWindowProperties private extends MetadataHandler[FlinkMetadata.W ImmutableBitSet.of(fieldCount - 3), ImmutableBitSet.of(fieldCount - 2), ImmutableBitSet.of(fieldCount - 1), + ImmutableBitSet.of(rel.windowing.getTimeAttributeIndex), rel.windowing.getWindow, rel.windowing.getTimeAttributeType ) @@ -344,6 +356,9 @@ class FlinkRelMdWindowProperties private extends MetadataHandler[FlinkMetadata.W ImmutableBitSet.of(starts: _*), ImmutableBitSet.of(ends: _*), ImmutableBitSet.of(times: _*), + // the original time attribute column is consumed by the window aggregate; only window_time + // remains as the time attribute of the aggregate output + ImmutableBitSet.of(), windowSpec, timeAttributeType ) @@ -454,7 +469,10 @@ class FlinkRelMdWindowProperties private extends MetadataHandler[FlinkMetadata.W val timeColumns = inferWindowPropertyAfterWindowJoin( leftWindowProperties.getWindowTimeColumns, rightWindowProperties.getWindowTimeColumns) - leftWindowProperties.copy(startColumns, endColumns, timeColumns) + val timeAttributeColumns = inferWindowPropertyAfterWindowJoin( + leftWindowProperties.getTimeAttributeColumns, + rightWindowProperties.getTimeAttributeColumns) + leftWindowProperties.copy(startColumns, endColumns, timeColumns, timeAttributeColumns) } def getWindowProperties(hepRelVertex: HepRelVertex, mq: RelMetadataQuery): RelWindowProperties = { diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamPhysicalWindowDeduplicateRule.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamPhysicalWindowDeduplicateRule.scala index c45229062d978..1cde13de58f73 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamPhysicalWindowDeduplicateRule.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamPhysicalWindowDeduplicateRule.scala @@ -40,7 +40,7 @@ class StreamPhysicalWindowDeduplicateRule(config: Config) extends ConverterRule( val windowProperties = fmq.getRelWindowProperties(rank.getInput) val partitionKey = rank.partitionKey WindowUtil.groupingContainsWindowStartEnd(partitionKey, windowProperties) && - RankUtil.canConvertToDeduplicate(rank) + RankUtil.canConvertToDeduplicate(rank, windowProperties) } override def convert(rel: RelNode): RelNode = { diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamPhysicalWindowRankRule.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamPhysicalWindowRankRule.scala index 7259e3495e0b2..a9d6477e89e6c 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamPhysicalWindowRankRule.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamPhysicalWindowRankRule.scala @@ -40,7 +40,7 @@ class StreamPhysicalWindowRankRule(config: Config) extends ConverterRule(config) val windowProperties = fmq.getRelWindowProperties(rank.getInput) val partitionKey = rank.partitionKey WindowUtil.groupingContainsWindowStartEnd(partitionKey, windowProperties) && - !RankUtil.canConvertToDeduplicate(rank) + !RankUtil.canConvertToDeduplicate(rank, windowProperties) } override def convert(rel: RelNode): RelNode = { diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/RankUtil.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/RankUtil.scala index 2314eb5a88fe9..eae0ac6a10142 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/RankUtil.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/RankUtil.scala @@ -22,6 +22,7 @@ import org.apache.flink.table.api.TableConfig import org.apache.flink.table.api.config.ExecutionConfigOptions import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.codegen.ExpressionReducer +import org.apache.flink.table.planner.plan.`trait`.RelWindowProperties import org.apache.flink.table.planner.plan.nodes.calcite.Rank import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalRank import org.apache.flink.table.planner.plan.nodes.physical.stream.{StreamPhysicalRank, StreamPhysicalWindowDeduplicate} @@ -339,7 +340,9 @@ object RankUtil { * @return * True if the input rank could be converted to [[StreamPhysicalWindowDeduplicate]] */ - def canConvertToDeduplicate(rank: FlinkLogicalRank): Boolean = { + def canConvertToDeduplicate( + rank: FlinkLogicalRank, + windowProperties: RelWindowProperties): Boolean = { val sortCollation = rank.orderKey val rankRange = rank.rankRange @@ -351,10 +354,28 @@ object RankUtil { case _ => false } - val inputRowType = rank.getInput.getRowType - val isSortOnTimeAttribute = sortOnTimeAttributeOnly(sortCollation, inputRowType) + val isSortOnWindowTimeAttribute = + sortOnWindowTimeAttributeColumn(sortCollation, windowProperties) - !rank.outputRankNumber && isLimit1 && isSortOnTimeAttribute && isRowNumberType + !rank.outputRankNumber && isLimit1 && isSortOnWindowTimeAttribute && isRowNumberType + } + + /** + * Window deduplication must sort on the window's original time attribute column, i.e. the column + * referenced by the window TVF time descriptor (tracked in + * [[RelWindowProperties.getTimeAttributeColumns]]). Sorting on window_time is forbidden because + * window_time is constant within a window and cannot distinguish the first/last row. After + * FLINK-39899 the original time attribute is materialized to a regular timestamp once it leaves + * the window TVF, so we identify it by column position instead of by time-indicator type. + */ + private def sortOnWindowTimeAttributeColumn( + sortCollation: RelCollation, + windowProperties: RelWindowProperties): Boolean = { + if (windowProperties == null || sortCollation.getFieldCollations.size() != 1) { + return false + } + val firstSortField = sortCollation.getFieldCollations.get(0) + windowProperties.getTimeAttributeColumns.get(firstSortField.getFieldIndex) } private def sortOnTimeAttributeOnly( diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/operator/StreamOperatorNameTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/operator/StreamOperatorNameTest.xml index b351ede7ab769..d6dc31ac18901 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/operator/StreamOperatorNameTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/operator/StreamOperatorNameTest.xml @@ -3264,7 +3264,7 @@ LogicalProject(window_start=[$6], window_end=[$7], a=[$0], b=[$1], c=[$2]) == Optimized Physical Plan == Calc(select=[window_start, window_end, a, b, c]) -+- WindowRank(window=[TUMBLE(win_start=[window_start], win_end=[window_end], size=[15 min])], rankType=[ROW_NUMBER], rankRange=[rankStart=1, rankEnd=1], partitionBy=[a], orderBy=[rowtime DESC], select=[a, b, c, rowtime, window_start, window_end]) ++- WindowDeduplicate(window=[TUMBLE(win_start=[window_start], win_end=[window_end], size=[15 min])], keep=[LastRow], partitionKeys=[a], orderKey=[rowtime], order=[ROWTIME]) +- Exchange(distribution=[hash[a]]) +- Calc(select=[a, b, c, rowtime, window_start, window_end]) +- WindowTableFunction(window=[TUMBLE(time_col=[rowtime], size=[15 min])]) @@ -3274,7 +3274,7 @@ Calc(select=[window_start, window_end, a, b, c]) == Optimized Execution Plan == Calc(select=[window_start, window_end, a, b, c]) -+- WindowRank(window=[TUMBLE(win_start=[window_start], win_end=[window_end], size=[15 min])], rankType=[ROW_NUMBER], rankRange=[rankStart=1, rankEnd=1], partitionBy=[a], orderBy=[rowtime DESC], select=[a, b, c, rowtime, window_start, window_end]) ++- WindowDeduplicate(window=[TUMBLE(win_start=[window_start], win_end=[window_end], size=[15 min])], keep=[LastRow], partitionKeys=[a], orderKey=[rowtime], order=[ROWTIME]) +- Exchange(distribution=[hash[a]]) +- Calc(select=[a, b, c, rowtime, window_start, window_end]) +- WindowTableFunction(window=[TUMBLE(time_col=[rowtime], size=[15 min])]) @@ -3336,9 +3336,9 @@ Calc(select=[window_start, window_end, a, b, c]) } ] }, { "id" : , - "type" : "WindowRank(window=[TUMBLE(win_start=[window_start], win_end=[window_end], size=[15 min])], rankType=[ROW_NUMBER], rankRange=[rankStart=1, rankEnd=1], partitionBy=[a], orderBy=[rowtime DESC], select=[a, b, c, rowtime, window_start, window_end])", + "type" : "WindowDeduplicate(window=[TUMBLE(win_start=[window_start], win_end=[window_end], size=[15 min])], keep=[LastRow], partitionKeys=[a], orderKey=[rowtime], order=[ROWTIME])", "pact" : "Operator", - "contents" : "WindowRank(window=[TUMBLE(win_start=[window_start], win_end=[window_end], size=[15 min])], rankType=[ROW_NUMBER], rankRange=[rankStart=1, rankEnd=1], partitionBy=[a], orderBy=[rowtime DESC], select=[a, b, c, rowtime, window_start, window_end])", + "contents" : "WindowDeduplicate(window=[TUMBLE(win_start=[window_start], win_end=[window_end], size=[15 min])], keep=[LastRow], partitionKeys=[a], orderKey=[rowtime], order=[ROWTIME])", "parallelism" : 2, "predecessors" : [ { "id" : , @@ -3374,7 +3374,7 @@ LogicalProject(window_start=[$6], window_end=[$7], a=[$0], b=[$1], c=[$2]) == Optimized Physical Plan == Calc(select=[window_start, window_end, a, b, c]) -+- WindowRank(window=[TUMBLE(win_start=[window_start], win_end=[window_end], size=[15 min])], rankType=[ROW_NUMBER], rankRange=[rankStart=1, rankEnd=1], partitionBy=[a], orderBy=[rowtime DESC], select=[a, b, c, rowtime, window_start, window_end]) ++- WindowDeduplicate(window=[TUMBLE(win_start=[window_start], win_end=[window_end], size=[15 min])], keep=[LastRow], partitionKeys=[a], orderKey=[rowtime], order=[ROWTIME]) +- Exchange(distribution=[hash[a]]) +- Calc(select=[a, b, c, rowtime, window_start, window_end]) +- WindowTableFunction(window=[TUMBLE(time_col=[rowtime], size=[15 min])]) @@ -3384,7 +3384,7 @@ Calc(select=[window_start, window_end, a, b, c]) == Optimized Execution Plan == Calc(select=[window_start, window_end, a, b, c]) -+- WindowRank(window=[TUMBLE(win_start=[window_start], win_end=[window_end], size=[15 min])], rankType=[ROW_NUMBER], rankRange=[rankStart=1, rankEnd=1], partitionBy=[a], orderBy=[rowtime DESC], select=[a, b, c, rowtime, window_start, window_end]) ++- WindowDeduplicate(window=[TUMBLE(win_start=[window_start], win_end=[window_end], size=[15 min])], keep=[LastRow], partitionKeys=[a], orderKey=[rowtime], order=[ROWTIME]) +- Exchange(distribution=[hash[a]]) +- Calc(select=[a, b, c, rowtime, window_start, window_end]) +- WindowTableFunction(window=[TUMBLE(time_col=[rowtime], size=[15 min])]) @@ -3446,9 +3446,9 @@ Calc(select=[window_start, window_end, a, b, c]) } ] }, { "id" : , - "type" : "WindowRank[]", + "type" : "WindowDeduplicate[]", "pact" : "Operator", - "contents" : "[]:WindowRank(window=[TUMBLE(win_start=[window_start], win_end=[window_end], size=[15 min])], rankType=[ROW_NUMBER], rankRange=[rankStart=1, rankEnd=1], partitionBy=[a], orderBy=[rowtime DESC], select=[a, b, c, rowtime, window_start, window_end])", + "contents" : "[]:WindowDeduplicate(window=[TUMBLE(win_start=[window_start], win_end=[window_end], size=[15 min])], keep=[LastRow], partitionKeys=[a], orderKey=[rowtime], order=[ROWTIME])", "parallelism" : 2, "predecessors" : [ { "id" : , diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/WindowDeduplicateTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/WindowDeduplicateTest.xml index 3c5c0d72b4ba8..010890098be68 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/WindowDeduplicateTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/WindowDeduplicateTest.xml @@ -118,7 +118,7 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$ + + + + + + + + + + + (b, 1000)) AS $f3, b, e, c, window_time AS rowtime]) - +- WindowRank(window=[TUMBLE(win_start=[window_start], win_end=[window_end], size=[15 min])], rankType=[ROW_NUMBER], rankRange=[rankStart=1, rankEnd=1], partitionBy=[a], orderBy=[rowtime DESC], select=[a, b, c, d, e, rowtime, window_start, window_end, window_time]) + +- WindowDeduplicate(window=[TUMBLE(win_start=[window_start], win_end=[window_end], size=[15 min])], keep=[LastRow], partitionKeys=[a], orderKey=[rowtime], order=[ROWTIME]) +- Exchange(distribution=[hash[a]]) +- Calc(select=[a, b, c, d, e, rowtime, window_start, window_end, window_time]) +- WindowTableFunction(window=[TUMBLE(time_col=[rowtime], size=[15 min])]) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdWindowPropertiesTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdWindowPropertiesTest.scala index ba6637c85a633..be391f78d6a18 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdWindowPropertiesTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdWindowPropertiesTest.scala @@ -177,8 +177,10 @@ class FlinkRelMdWindowPropertiesTest extends FlinkRelMdHandlerTestBase { ImmutableBitSet.of(start), ImmutableBitSet.of(end), if (time >= 0) ImmutableBitSet.of(time) else ImmutableBitSet.of(), + ImmutableBitSet.of(), spec, - timeAttributeType) + timeAttributeType + ) } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/MetadataTestUtil.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/MetadataTestUtil.scala index 8f5f94572e5a1..92d0483ea72de 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/MetadataTestUtil.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/MetadataTestUtil.scala @@ -289,8 +289,10 @@ object MetadataTestUtil { ImmutableBitSet.of(0), ImmutableBitSet.of(1), ImmutableBitSet.of(2), + ImmutableBitSet.of(), new TumblingWindowSpec(Duration.ofMinutes(10L), null), - fieldTypes.apply(2)) + fieldTypes.apply(2) + ) val colStatsMap = Map[String, ColumnStats]( "a" -> new ColumnStats(3740000000L, 0L, 4d, 4, null, null), diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/WindowDeduplicateTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/WindowDeduplicateTest.scala index 38385a6989543..8669621c174a8 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/WindowDeduplicateTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/WindowDeduplicateTest.scala @@ -80,6 +80,26 @@ class WindowDeduplicateTest extends TableTestBase { util.verifyRelPlan(sql) } + @Test + def testOnWindowTVFOrderByWindowTime(): Unit = { + // FLINK-39899: window_time is constant within a window and cannot distinguish the first/last + // row, so a top-1 ROW_NUMBER ordered by window_time must NOT be recognized as a window + // deduplication; it falls back to WindowRank. Only the original time attribute column + // (here 'rowtime') is a valid window-deduplicate order key. + val sql = + """ + |SELECT * + |FROM ( + |SELECT *, + | ROW_NUMBER() OVER(PARTITION BY a, window_start, window_end + | ORDER BY window_time DESC) as rownum + |FROM TABLE(TUMBLE(TABLE MyTable, DESCRIPTOR(rowtime), INTERVAL '15' MINUTE)) + |) + |WHERE rownum <= 1 + """.stripMargin + util.verifyRelPlan(sql) + } + @Test def testOnWindowTVFWithValidCondition(): Unit = { val sql =