From 18d238ca53d39dfec70da98871f442c29f3e0233 Mon Sep 17 00:00:00 2001 From: Dennis-Mircea Ciupitu Date: Sat, 2 May 2026 21:49:11 +0300 Subject: [PATCH 1/4] [FLINK-14621][flink-table-planner] Do not generate watermark assigner operator if no time attribute operations on rowtime --- .../calcite/RelTimeIndicatorConverter.java | 114 +--- .../calcite/RexTimeIndicatorMaterializer.java | 185 +++++++ .../stream/MiniBatchIntervalInferRule.java | 31 +- .../RedundantWatermarkAssignerRemoveRule.java | 518 ++++++++++++++++++ .../optimize/program/FlinkStreamProgram.scala | 8 + .../plan/rules/FlinkStreamRuleSets.scala | 5 + ...undantWatermarkAssignerRemoveRuleTest.java | 292 ++++++++++ .../planner/catalog/JavaCatalogTableTest.xml | 11 +- .../plan/hints/stream/StateTtlHintTest.xml | 18 +- .../exec/operator/StreamOperatorNameTest.xml | 338 ++---------- .../testProcTimeHopWindow.out | 144 +---- .../testProcTimeSessionWindow.out | 144 +---- .../testProcTimeTumbleWindow.out | 146 +---- ...ProcTimeBoundedNonPartitionedRangeOver.out | 123 +---- ...estProcTimeBoundedPartitionedRangeOver.out | 121 +--- ...PartitionedRowsOverWithBuiltinProctime.out | 49 +- ...tProcTimeUnboundedPartitionedRangeOver.out | 121 +--- .../stream/DuplicateChangesInferRuleTest.xml | 153 ++---- ...dundantWatermarkAssignerRemoveRuleTest.xml | 378 +++++++++++++ .../planner/plan/stream/sql/DeltaJoinTest.xml | 5 +- .../stream/sql/MiniBatchIntervalInferTest.xml | 31 +- .../planner/plan/stream/sql/MultiJoinTest.xml | 15 +- .../stream/sql/NonDeterministicDagTest.xml | 60 +- .../planner/plan/stream/sql/TableScanTest.xml | 16 +- .../plan/stream/sql/TableSourceTest.xml | 6 +- .../planner/plan/stream/sql/UnionTest.xml | 6 +- .../stream/sql/WindowTableFunctionTest.xml | 20 +- .../plan/stream/sql/agg/AggregateTest.xml | 49 +- .../stream/sql/agg/WindowAggregateTest.xml | 246 +++------ .../plan/stream/sql/join/TemporalJoinTest.xml | 32 +- .../plan/stream/sql/join/WindowJoinTest.xml | 54 +- .../table/GroupWindowTableAggregateTest.xml | 5 +- .../plan/stream/table/TableAggregateTest.xml | 30 +- .../plan/stream/table/TableSourceTest.xml | 6 +- .../sql/MiniBatchIntervalInferTest.scala | 26 +- .../stream/sql/WindowTableFunctionTest.scala | 26 +- .../plan/stream/sql/agg/AggregateTest.scala | 19 +- 37 files changed, 1844 insertions(+), 1707 deletions(-) create mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/calcite/RexTimeIndicatorMaterializer.java create mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/stream/RedundantWatermarkAssignerRemoveRule.java create mode 100644 flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/physical/stream/RedundantWatermarkAssignerRemoveRuleTest.java create mode 100644 flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/stream/RedundantWatermarkAssignerRemoveRuleTest.xml diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/calcite/RelTimeIndicatorConverter.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/calcite/RelTimeIndicatorConverter.java index 68fb063e3eb1e..7905b2638dc66 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/calcite/RelTimeIndicatorConverter.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/calcite/RelTimeIndicatorConverter.java @@ -71,17 +71,13 @@ import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.rel.type.RelDataTypeField; import org.apache.calcite.rex.RexBuilder; -import org.apache.calcite.rex.RexCall; import org.apache.calcite.rex.RexInputRef; import org.apache.calcite.rex.RexLocalRef; import org.apache.calcite.rex.RexNode; -import org.apache.calcite.rex.RexPatternFieldRef; import org.apache.calcite.rex.RexProgram; import org.apache.calcite.rex.RexProgramBuilder; import org.apache.calcite.rex.RexShuttle; import org.apache.calcite.sql.SqlAggFunction; -import org.apache.calcite.sql.SqlOperator; -import org.apache.calcite.sql.fun.SqlStdOperatorTable; import org.apache.calcite.sql.type.SqlTypeName; import org.apache.calcite.util.Pair; @@ -100,7 +96,6 @@ import static org.apache.flink.table.planner.calcite.FlinkTypeFactory.isProctimeIndicatorType; import static org.apache.flink.table.planner.calcite.FlinkTypeFactory.isRowtimeIndicatorType; import static org.apache.flink.table.planner.calcite.FlinkTypeFactory.isTimeIndicatorType; -import static org.apache.flink.table.planner.plan.utils.MatchUtil.isFinalOnMatchTimeIndicator; import static org.apache.flink.table.planner.plan.utils.WindowUtil.groupingContainsWindowStartEnd; /** @@ -189,7 +184,8 @@ public RelNode visit(LogicalTableModify modify) { private RelNode visitMatch(FlinkLogicalMatch match) { RelNode newInput = match.getInput().accept(this); - RexTimeIndicatorMaterializer materializer = new RexTimeIndicatorMaterializer(newInput); + RexTimeIndicatorMaterializer materializer = + new RexTimeIndicatorMaterializer(rexBuilder, newInput); Function, Map> materializeExprs = rexNodesMap -> @@ -295,7 +291,8 @@ private RelNode visitCalc(FlinkLogicalCalc calc) { // check if input field contains time indicator type // materialize field if no time indicator is present anymore // if input field is already materialized, change to timestamp type - RexTimeIndicatorMaterializer materializer = new RexTimeIndicatorMaterializer(newInput); + RexTimeIndicatorMaterializer materializer = + new RexTimeIndicatorMaterializer(rexBuilder, newInput); List newProjects = program.getProjectList().stream() .map(project -> program.expandLocalRef(project).accept(materializer)) @@ -379,7 +376,8 @@ private RelNode visitCorrelate(FlinkLogicalCorrelate correlate) { // check if input field contains time indicator type // materialize field if no time indicator is present anymore // if input field is already materialized, change to timestamp type - RexTimeIndicatorMaterializer materializer = new RexTimeIndicatorMaterializer(newLeft); + RexTimeIndicatorMaterializer materializer = + new RexTimeIndicatorMaterializer(rexBuilder, newLeft); RexNode newScanCall = newScan.getCall().accept(materializer); newRight = @@ -585,7 +583,8 @@ private RelNode visitMultiJoin(FlinkLogicalMultiJoin multiJoin) { .flatMap(input -> RelOptUtil.getFieldTypeList(input.getRowType()).stream()) .collect(Collectors.toList()); - RexTimeIndicatorMaterializer materializer = new RexTimeIndicatorMaterializer(allFields); + RexTimeIndicatorMaterializer materializer = + new RexTimeIndicatorMaterializer(rexBuilder, allFields); final RexNode newJoinFilter = multiJoin.getJoinFilter().accept(materializer); @@ -789,101 +788,4 @@ private RelDataType timestamp(boolean isNullable, boolean isTimestampLtzIndicato private boolean isTimestampLtzType(RelDataType type) { return type.getSqlTypeName().equals(SqlTypeName.TIMESTAMP_WITH_LOCAL_TIME_ZONE); } - - // ---------------------------------------------------------------------------------------- - // Materializer for RexNode including time indicator - // ---------------------------------------------------------------------------------------- - - private class RexTimeIndicatorMaterializer extends RexShuttle { - - private final List inputFieldTypes; - - private RexTimeIndicatorMaterializer(RelNode node) { - this(RelOptUtil.getFieldTypeList(node.getRowType())); - } - - private RexTimeIndicatorMaterializer(List inputFieldTypes) { - this.inputFieldTypes = inputFieldTypes; - } - - @Override - public RexNode visitCall(RexCall call) { - RexCall updatedCall = (RexCall) super.visitCall(call); - - // materialize operands with time indicators - List materializedOperands; - SqlOperator updatedCallOp = updatedCall.getOperator(); - if (updatedCallOp == FlinkSqlOperatorTable.SESSION_OLD - || updatedCallOp == FlinkSqlOperatorTable.HOP_OLD - || updatedCallOp == FlinkSqlOperatorTable.TUMBLE_OLD) { - // skip materialization for special operators - materializedOperands = updatedCall.getOperands(); - } else { - materializedOperands = - updatedCall.getOperands().stream() - .map(RelTimeIndicatorConverter.this::materializeTimeIndicators) - .collect(Collectors.toList()); - } - - // All calls in MEASURES and DEFINE are wrapped with FINAL/RUNNING, therefore - // we should treat FINAL(MATCH_ROWTIME) and FINAL(MATCH_PROCTIME) as a time attribute - // extraction - if (isFinalOnMatchTimeIndicator(call)) { - return updatedCall; - } else if (isTimeIndicatorType(updatedCall.getType())) { - // do not modify window time attributes and some special operators - if (updatedCallOp == FlinkSqlOperatorTable.TUMBLE_ROWTIME - || updatedCallOp == FlinkSqlOperatorTable.TUMBLE_PROCTIME - || updatedCallOp == FlinkSqlOperatorTable.HOP_ROWTIME - || updatedCallOp == FlinkSqlOperatorTable.HOP_PROCTIME - || updatedCallOp == FlinkSqlOperatorTable.SESSION_ROWTIME - || updatedCallOp == FlinkSqlOperatorTable.SESSION_PROCTIME - || updatedCallOp == FlinkSqlOperatorTable.MATCH_ROWTIME - || updatedCallOp == FlinkSqlOperatorTable.MATCH_PROCTIME - || updatedCallOp == FlinkSqlOperatorTable.PROCTIME - || updatedCallOp == SqlStdOperatorTable.AS - || updatedCallOp == SqlStdOperatorTable.CAST - || updatedCallOp == FlinkSqlOperatorTable.REINTERPRET) { - return updatedCall; - } else { - // materialize function's result and operands - return updatedCall.clone( - timestamp( - updatedCall.getType().isNullable(), - isTimestampLtzType(updatedCall.getType())), - materializedOperands); - } - } else { - // materialize function's operands only - return updatedCall.clone(updatedCall.getType(), materializedOperands); - } - } - - @Override - public RexNode visitInputRef(RexInputRef inputRef) { - RelDataType oldType = inputRef.getType(); - if (isTimeIndicatorType(oldType)) { - RelDataType resolvedRefType = inputFieldTypes.get(inputRef.getIndex()); - if (!isTimeIndicatorType(resolvedRefType)) { - // input has been materialized - return new RexInputRef(inputRef.getIndex(), resolvedRefType); - } - } - return super.visitInputRef(inputRef); - } - - @Override - public RexNode visitPatternFieldRef(RexPatternFieldRef fieldRef) { - RelDataType oldType = fieldRef.getType(); - if (isTimeIndicatorType(oldType)) { - RelDataType resolvedRefType = inputFieldTypes.get(fieldRef.getIndex()); - if (!isTimeIndicatorType(resolvedRefType)) { - // input has been materialized - return new RexPatternFieldRef( - fieldRef.getAlpha(), fieldRef.getIndex(), resolvedRefType); - } - } - return super.visitPatternFieldRef(fieldRef); - } - } } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/calcite/RexTimeIndicatorMaterializer.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/calcite/RexTimeIndicatorMaterializer.java new file mode 100644 index 0000000000000..fd2e68f3be4c7 --- /dev/null +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/calcite/RexTimeIndicatorMaterializer.java @@ -0,0 +1,185 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.planner.calcite; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.planner.functions.sql.FlinkSqlOperatorTable; +import org.apache.flink.table.types.logical.LocalZonedTimestampType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.TimestampType; + +import org.apache.calcite.plan.RelOptUtil; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rex.RexBuilder; +import org.apache.calcite.rex.RexCall; +import org.apache.calcite.rex.RexInputRef; +import org.apache.calcite.rex.RexNode; +import org.apache.calcite.rex.RexPatternFieldRef; +import org.apache.calcite.rex.RexShuttle; +import org.apache.calcite.sql.SqlOperator; +import org.apache.calcite.sql.fun.SqlStdOperatorTable; +import org.apache.calcite.sql.type.SqlTypeName; + +import java.util.List; +import java.util.stream.Collectors; + +import static org.apache.flink.table.planner.calcite.FlinkTypeFactory.isProctimeIndicatorType; +import static org.apache.flink.table.planner.calcite.FlinkTypeFactory.isRowtimeIndicatorType; +import static org.apache.flink.table.planner.calcite.FlinkTypeFactory.isTimeIndicatorType; +import static org.apache.flink.table.planner.plan.utils.MatchUtil.isFinalOnMatchTimeIndicator; + +/** + * A {@link RexShuttle} that rewrites {@link RexInputRef}s and {@link RexCall}s whose declared type + * carries a {@link TimeIndicatorRelDataType} marker so that they match a (possibly demoted) input + * row type, materializing time-indicator values to plain timestamps when required. + * + *

This is shared between {@link RelTimeIndicatorConverter} (where it is used to materialize time + * indicators consumed by calculations) and downstream rules that drop a node which previously + * stamped a column as a time indicator (e.g. removing a redundant {@code WatermarkAssigner}). + */ +@Internal +public class RexTimeIndicatorMaterializer extends RexShuttle { + + private final RexBuilder rexBuilder; + private final List inputFieldTypes; + + public RexTimeIndicatorMaterializer(RexBuilder rexBuilder, RelNode input) { + this(rexBuilder, RelOptUtil.getFieldTypeList(input.getRowType())); + } + + public RexTimeIndicatorMaterializer(RexBuilder rexBuilder, List inputFieldTypes) { + this.rexBuilder = rexBuilder; + this.inputFieldTypes = inputFieldTypes; + } + + @Override + public RexNode visitCall(RexCall call) { + RexCall updatedCall = (RexCall) super.visitCall(call); + + // materialize operands with time indicators + List materializedOperands; + SqlOperator updatedCallOp = updatedCall.getOperator(); + if (updatedCallOp == FlinkSqlOperatorTable.SESSION_OLD + || updatedCallOp == FlinkSqlOperatorTable.HOP_OLD + || updatedCallOp == FlinkSqlOperatorTable.TUMBLE_OLD) { + // skip materialization for special operators + materializedOperands = updatedCall.getOperands(); + } else { + materializedOperands = + updatedCall.getOperands().stream() + .map(this::materializeTimeIndicators) + .collect(Collectors.toList()); + } + + // All calls in MEASURES and DEFINE are wrapped with FINAL/RUNNING, therefore + // we should treat FINAL(MATCH_ROWTIME) and FINAL(MATCH_PROCTIME) as a time attribute + // extraction + if (isFinalOnMatchTimeIndicator(call)) { + return updatedCall; + } else if (isTimeIndicatorType(updatedCall.getType())) { + // do not modify window time attributes and some special operators + if (updatedCallOp == FlinkSqlOperatorTable.TUMBLE_ROWTIME + || updatedCallOp == FlinkSqlOperatorTable.TUMBLE_PROCTIME + || updatedCallOp == FlinkSqlOperatorTable.HOP_ROWTIME + || updatedCallOp == FlinkSqlOperatorTable.HOP_PROCTIME + || updatedCallOp == FlinkSqlOperatorTable.SESSION_ROWTIME + || updatedCallOp == FlinkSqlOperatorTable.SESSION_PROCTIME + || updatedCallOp == FlinkSqlOperatorTable.MATCH_ROWTIME + || updatedCallOp == FlinkSqlOperatorTable.MATCH_PROCTIME + || updatedCallOp == FlinkSqlOperatorTable.PROCTIME + || updatedCallOp == SqlStdOperatorTable.AS + || updatedCallOp == SqlStdOperatorTable.CAST + || updatedCallOp == FlinkSqlOperatorTable.REINTERPRET) { + return updatedCall; + } else { + // materialize function's result and operands + return updatedCall.clone( + timestamp( + updatedCall.getType().isNullable(), + isTimestampLtzType(updatedCall.getType())), + materializedOperands); + } + } else { + // materialize function's operands only + return updatedCall.clone(updatedCall.getType(), materializedOperands); + } + } + + @Override + public RexNode visitInputRef(RexInputRef inputRef) { + RelDataType oldType = inputRef.getType(); + if (isTimeIndicatorType(oldType)) { + RelDataType resolvedRefType = inputFieldTypes.get(inputRef.getIndex()); + if (!isTimeIndicatorType(resolvedRefType)) { + // input has been materialized + return new RexInputRef(inputRef.getIndex(), resolvedRefType); + } + } + return super.visitInputRef(inputRef); + } + + @Override + public RexNode visitPatternFieldRef(RexPatternFieldRef fieldRef) { + RelDataType oldType = fieldRef.getType(); + if (isTimeIndicatorType(oldType)) { + RelDataType resolvedRefType = inputFieldTypes.get(fieldRef.getIndex()); + if (!isTimeIndicatorType(resolvedRefType)) { + // input has been materialized + return new RexPatternFieldRef( + fieldRef.getAlpha(), fieldRef.getIndex(), resolvedRefType); + } + } + return super.visitPatternFieldRef(fieldRef); + } + + /** + * Wraps an expression with the appropriate cast/proctime call to materialize a time-indicator + * value into a plain timestamp value. + */ + private RexNode materializeTimeIndicators(RexNode expr) { + if (isRowtimeIndicatorType(expr.getType())) { + // cast rowTime indicator to regular timestamp + return rexBuilder.makeAbstractCast( + timestamp(expr.getType().isNullable(), isTimestampLtzType(expr.getType())), + expr, + false); + } else if (isProctimeIndicatorType(expr.getType())) { + // generate procTime access + return rexBuilder.makeCall(FlinkSqlOperatorTable.PROCTIME_MATERIALIZE, expr); + } else { + return expr; + } + } + + private RelDataType timestamp(boolean isNullable, boolean isTimestampLtzIndicator) { + LogicalType logicalType; + if (isTimestampLtzIndicator) { + logicalType = new LocalZonedTimestampType(isNullable, 3); + } else { + logicalType = new TimestampType(isNullable, 3); + } + return ((FlinkTypeFactory) rexBuilder.getTypeFactory()) + .createFieldTypeFromLogicalType(logicalType); + } + + private boolean isTimestampLtzType(RelDataType type) { + return type.getSqlTypeName().equals(SqlTypeName.TIMESTAMP_WITH_LOCAL_TIME_ZONE); + } +} diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/stream/MiniBatchIntervalInferRule.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/stream/MiniBatchIntervalInferRule.java index b7717accc995b..848d1e48176d4 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/stream/MiniBatchIntervalInferRule.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/stream/MiniBatchIntervalInferRule.java @@ -47,14 +47,18 @@ /** * Planner rule that infers the mini-batch interval of minibatch assigner. * - *

This rule could handle the following two kinds of operator: 1. supports operators which - * supports mini-batch and does not require watermark, e.g. group aggregate. In this case, {@link - * StreamPhysicalMiniBatchAssigner} with Proctime mode will be created if not exist, and the - * interval value will be set as {@link ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY}. - * 2. supports operators which requires watermark, e.g. window join, window aggregate. In this case, - * {@link StreamPhysicalWatermarkAssigner} already exists, and its MiniBatchIntervalTrait will be - * updated as the merged intervals from its outputs. Currently, mini-batched window aggregate is not - * supported, and will be introduced later. + *

This rule could handle the following two kinds of operator: + * + *

    + *
  1. Operators which support mini-batch and do not require watermark, e.g. group aggregate. In + * this case, {@link StreamPhysicalMiniBatchAssigner} with Proctime mode will be created if + * not exist, and the interval value will be set as {@link + * ExecutionConfigOptions#TABLE_EXEC_MINIBATCH_ALLOW_LATENCY}. + *
  2. Operators which require watermark, e.g. window join, window aggregate. In this case, {@link + * StreamPhysicalWatermarkAssigner} already exists, and its {@code MiniBatchIntervalTrait} + * will be updated as the merged intervals from its outputs. Currently, mini-batched window + * aggregate is not supported, and will be introduced later. + *
* *

NOTES: This rule only supports HepPlanner with TOP_DOWN match order. */ @@ -178,12 +182,11 @@ private boolean shouldAppendMiniBatchAssignerNode(RelNode node) { return mode == MiniBatchMode.RowTime || mode == MiniBatchMode.ProcTime; } if (node instanceof StreamPhysicalWatermarkAssigner) { - // append minibatch node if it is rowtime mode and the child is watermark assigner - // TODO: if it is ProcTime mode, we also append a minibatch node for now. - // Because the downstream can be a regular aggregate and the watermark assigner - // might be redundant. In FLINK-14621, we will remove redundant watermark assigner, - // then we can remove the ProcTime condition. - return mode == MiniBatchMode.RowTime || mode == MiniBatchMode.ProcTime; + // Append minibatch node only when the watermark assigner is rowtime-driven. Any + // assigner whose watermarks are not consumed by an event-time operator is removed + // upstream by RedundantWatermarkAssignerRemoveRule (FLINK-14621), so a surviving + // assigner here always corresponds to a real rowtime consumer downstream. + return mode == MiniBatchMode.RowTime; } // others do not append minibatch node return false; diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/stream/RedundantWatermarkAssignerRemoveRule.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/stream/RedundantWatermarkAssignerRemoveRule.java new file mode 100644 index 0000000000000..c9837100864de --- /dev/null +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/stream/RedundantWatermarkAssignerRemoveRule.java @@ -0,0 +1,518 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.planner.plan.rules.physical.stream; + +import org.apache.flink.table.planner.plan.nodes.calcite.WatermarkAssigner; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCalc; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRel; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTemporalSort; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWatermarkAssigner; +import org.apache.flink.table.planner.plan.schema.TimeIndicatorRelDataType; +import org.apache.flink.table.planner.plan.utils.FlinkRelUtil; + +import org.apache.calcite.plan.RelOptRule; +import org.apache.calcite.plan.RelOptRuleCall; +import org.apache.calcite.plan.RelRule; +import org.apache.calcite.plan.hep.HepPlanner; +import org.apache.calcite.plan.hep.HepRelVertex; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.core.Aggregate; +import org.apache.calcite.rel.core.Calc; +import org.apache.calcite.rel.core.Join; +import org.apache.calcite.rel.core.JoinRelType; +import org.apache.calcite.rel.core.Project; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rex.RexBuilder; +import org.apache.calcite.rex.RexCall; +import org.apache.calcite.rex.RexInputRef; +import org.apache.calcite.rex.RexNode; +import org.apache.calcite.rex.RexProgram; +import org.apache.calcite.rex.RexShuttle; +import org.immutables.value.Value; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * Removes {@link StreamPhysicalWatermarkAssigner} nodes whose watermarks are not consumed by any + * downstream operator that requires watermarks (e.g. windowed aggregates, event-time interval / + * temporal joins, event-time temporal sort, or {@code CURRENT_WATERMARK} SQL calls). + * + *

Anchoring

+ * + *

The rule is anchored on the HEP planner's root rel so that a single firing handles the whole + * tree per optimization (per sink in a statement set, per query in sink-less optimization paths + * such as {@code EXPLAIN} or non-insert Table API). The operand matches any {@link + * StreamPhysicalRel}, but {@link #matches(RelOptRuleCall)} guards that only the current root + * triggers the rewrite. This avoids re-firing the rule on intermediate sub-trees during HEP + * top-down matching, which would either over-remove (if we drop a watermark assigner without + * knowing the full ancestry) or thrash (if we walked the same sub-tree repeatedly). + * + *

Walk and consumer detection

+ * + *

The rule walks the plan from the root towards the sources. While descending, it tracks whether + * any ancestor node has already declared {@link StreamPhysicalRel#requireWatermark()}, embeds a + * {@code CURRENT_WATERMARK} call in any of its expressions, or is a rowtime {@link + * StreamPhysicalTemporalSort} (whose runtime operator consumes watermarks even though it does not + * formally implement {@code requireWatermark()}). A watermark assigner is dropped only when no + * ancestor up to the root requires watermarks. + * + *

Type-marker demotion

+ * + *

{@link WatermarkAssigner} stamps a {@link TimeIndicatorRelDataType} marker (e.g. {@code + * TIMESTAMP(3) *ROWTIME*}) onto the watermark column. When the assigner is dropped, the column + * reverts to plain {@code TIMESTAMP(3)} along the path from the former assigner to the root. The + * rule rewrites every parent on that path accordingly: + * + *

    + *
  • {@link StreamPhysicalCalc} parents have their {@link RexProgram} rebuilt against the new + * input row type. Any {@link RexInputRef} whose column type changed is replaced with a fresh + * ref of the new type. The rest of the {@link RexNode} tree is preserved as-is — we + * deliberately do not re-run the full time-indicator materializer here, because it + * would re-wrap already-materialized {@code PROCTIME_MATERIALIZE(...)} calls on unrelated + * proctime columns. If dropping the assigner leaves two adjacent Calcs (the parent and the + * former child of the assigner), they are folded into a single Calc via {@link + * FlinkRelUtil#merge(Calc, Calc)} so the rule produces the same compact plan as a follow-up + * {@code FlinkCalcMergeRule} pass would. + *
  • Other parents (pass-through rels such as Exchange, Union, ChangelogNormalize, Sink, …) + * derive their row types from their inputs and are recreated via {@link + * RelNode#copy(org.apache.calcite.plan.RelTraitSet, java.util.List)}. + *
+ * + *

Root-rowtime guard (per-branch)

+ * + *

If the root's output row type carries an event-time {@code *ROWTIME*} time-indicator on a + * field, an upstream rowtime column is consumed by the root (e.g. forwarded through a sink for + * external watermark propagation, or surfaced as a rowtime in the result of a {@code SELECT}). The + * watermark assigner that ultimately produces that field must be kept. + * + *

The rule is precise about this: rather than refusing to fire across the whole sub-tree, it + * tracks the set of "protected" field indexes through the rewrite and only refuses to drop the + * specific assigner(s) whose rowtime field reaches a protected root field. Branches whose rowtime + * column does not reach the root's protected fields are still rewritten. The protected set + * is propagated downward as follows: + * + *

    + *
  • Pass-through rels (Filter, Sort, Union, Exchange, Sink, …): preserved. + *
  • {@link StreamPhysicalCalc} / {@link Project}: a parent's protected output index maps to its + * input index iff the corresponding projection is a bare {@link RexInputRef}. Composite + * expressions cannot carry a time-indicator marker through, so their inputs are not + * protected. + *
  • {@link Join}: indexes split between the left and right halves of the output schema (semi / + * anti joins only project the left side). + *
  • {@link Aggregate} and window aggregates: clear the protected set; aggregations consume time + * and produce non-rowtime aggregate results. + *
  • {@link WatermarkAssigner}: positional pass-through. + *
+ * + *

See FLINK-14621. + * + *

Known limitations

+ * + *
    + *
  • Computed-column rowtime forwarding to sinks. If a sink projects a column derived + * from a rowtime via an expression (e.g. {@code sink_rt AS rt + INTERVAL '1' SECOND}) rather + * than forwarding the rowtime column directly, the result type is plain {@code TIMESTAMP} and + * not a {@link TimeIndicatorRelDataType}. The protected-index propagation cannot trace the + * rowtime through the composite expression and the assigner may be dropped. The semantics of + * "watermark forwarding" through such a derived column are not well-defined in Flink today; + * if watermark propagation matters, project the rowtime column directly. + *
  • Per-branch protection across unions. When the sink's protected rowtime column is fed + * by a {@link org.apache.calcite.rel.core.Union}, every branch's corresponding rowtime input + * is protected. This is required for correctness: at runtime a downstream operator advances + * watermarks as the minimum across all union inputs, so dropping the assigner on one branch + * would block (not skip) progress on that input. + *
+ */ +@Value.Enclosing +public class RedundantWatermarkAssignerRemoveRule + extends RelRule { + + public static final RelOptRule INSTANCE = + new RedundantWatermarkAssignerRemoveRule(Config.DEFAULT); + + protected RedundantWatermarkAssignerRemoveRule(Config config) { + super(config); + } + + @Override + public boolean matches(RelOptRuleCall call) { + if (!isHepRoot(call)) { + return false; + } + final StreamPhysicalRel root = call.rel(0); + final Set protectedIndexes = rowtimeFieldIndexes(root.getRowType()); + return hasRedundantWatermarkAssigner(root, false, protectedIndexes); + } + + @Override + public void onMatch(RelOptRuleCall call) { + final StreamPhysicalRel root = call.rel(0); + final RexBuilder rexBuilder = root.getCluster().getRexBuilder(); + final Set protectedIndexes = rowtimeFieldIndexes(root.getRowType()); + final RelNode rewritten = rewrite(root, false, protectedIndexes, rexBuilder); + if (rewritten != root) { + call.transformTo(rewritten); + } + } + + /** + * Returns {@code true} if the matched rel is the current HEP root. HEP rules cannot otherwise + * detect "I'm at the root" — we ask the planner directly. + */ + private static boolean isHepRoot(RelOptRuleCall call) { + if (!(call.getPlanner() instanceof HepPlanner)) { + return false; + } + final HepPlanner planner = (HepPlanner) call.getPlanner(); + return unwrap(planner.getRoot()) == call.rel(0); + } + + private static Set rowtimeFieldIndexes(RelDataType rowType) { + if (!rowType.isStruct()) { + // Some HEP roots in intermediate optimization stages can be non-struct (e.g. RAW row + // types). They cannot carry a rowtime time-indicator marker. + return Collections.emptySet(); + } + Set indexes = null; + final List fields = + rowType.getFieldList().stream().map(f -> f.getType()).collect(Collectors.toList()); + for (int i = 0; i < fields.size(); i++) { + final RelDataType t = fields.get(i); + if (t instanceof TimeIndicatorRelDataType + && ((TimeIndicatorRelDataType) t).isEventTime()) { + if (indexes == null) { + indexes = new HashSet<>(); + } + indexes.add(i); + } + } + return indexes == null ? Collections.emptySet() : indexes; + } + + /** + * Returns {@code true} if the subtree rooted at {@code node} contains at least one watermark + * assigner whose watermarks are not consumed by any operator on the path to the root and whose + * rowtime field is not in the protected set. + */ + private static boolean hasRedundantWatermarkAssigner( + RelNode node, boolean requiredAbove, Set protectedIndexes) { + if (node instanceof StreamPhysicalWatermarkAssigner && !requiredAbove) { + final int rtIdx = ((StreamPhysicalWatermarkAssigner) node).rowtimeFieldIndex(); + if (!protectedIndexes.contains(rtIdx)) { + return true; + } + } + final boolean requiredBelow = requiredAbove || consumesWatermarkAtRuntime(node); + final List inputs = node.getInputs(); + for (int i = 0; i < inputs.size(); i++) { + final RelNode child = unwrap(inputs.get(i)); + final Set childProtected = mapProtectedDown(node, i, protectedIndexes); + if (hasRedundantWatermarkAssigner(child, requiredBelow, childProtected)) { + return true; + } + } + return false; + } + + /** + * Recursively rewrites the subtree rooted at {@code node}, dropping any {@link + * StreamPhysicalWatermarkAssigner} that is not needed by an ancestor and whose rowtime column + * is not in the protected set, and demoting downstream references to the formerly-stamped + * time-indicator column. + */ + private static RelNode rewrite( + RelNode node, + boolean requiredAbove, + Set protectedIndexes, + RexBuilder rexBuilder) { + if (node instanceof StreamPhysicalWatermarkAssigner && !requiredAbove) { + final StreamPhysicalWatermarkAssigner assigner = (StreamPhysicalWatermarkAssigner) node; + if (!protectedIndexes.contains(assigner.rowtimeFieldIndex())) { + // Drop: replace with the (rewritten) input. The TimeIndicator marker on the + // watermark column is lost; parents above will be retyped on the way back up. + return rewrite(unwrap(assigner.getInput()), false, protectedIndexes, rexBuilder); + } + // Keep this assigner — its rowtime column is protected (reaches a root rowtime + // field). Fall through to the generic recursion to continue processing below. + } + + final boolean requiredBelow = requiredAbove || consumesWatermarkAtRuntime(node); + + final List inputs = node.getInputs(); + if (inputs == null || inputs.isEmpty()) { + return node; + } + final List newInputs = new ArrayList<>(inputs.size()); + boolean changed = false; + for (int i = 0; i < inputs.size(); i++) { + final RelNode unwrapped = unwrap(inputs.get(i)); + final Set childProtected = mapProtectedDown(node, i, protectedIndexes); + final RelNode rewritten = rewrite(unwrapped, requiredBelow, childProtected, rexBuilder); + changed |= rewritten != unwrapped; + newInputs.add(rewritten); + } + if (!changed) { + return node; + } + return rebuildParent(node, newInputs, rexBuilder); + } + + /** + * Maps a parent's protected output-field indexes to one of its inputs' field-index space. See + * the class Javadoc for the propagation rules. + */ + private static Set mapProtectedDown( + RelNode node, int inputIndex, Set parentProtected) { + if (parentProtected.isEmpty()) { + return Collections.emptySet(); + } + if (node instanceof StreamPhysicalCalc) { + final RexProgram program = ((StreamPhysicalCalc) node).getProgram(); + final Set mapped = new HashSet<>(); + for (Integer outIdx : parentProtected) { + final RexNode expanded = + program.expandLocalRef(program.getProjectList().get(outIdx)); + if (expanded instanceof RexInputRef) { + mapped.add(((RexInputRef) expanded).getIndex()); + } + } + return mapped; + } + if (node instanceof Project) { + final List projects = ((Project) node).getProjects(); + final Set mapped = new HashSet<>(); + for (Integer outIdx : parentProtected) { + final RexNode rex = projects.get(outIdx); + if (rex instanceof RexInputRef) { + mapped.add(((RexInputRef) rex).getIndex()); + } + } + return mapped; + } + if (node instanceof Join) { + final Join join = (Join) node; + final int leftFieldCount = join.getLeft().getRowType().getFieldCount(); + final boolean leftOnlyOutput = + join.getJoinType() == JoinRelType.SEMI + || join.getJoinType() == JoinRelType.ANTI; + final Set mapped = new HashSet<>(); + for (Integer outIdx : parentProtected) { + if (leftOnlyOutput) { + if (inputIndex == 0) { + mapped.add(outIdx); + } + } else if (inputIndex == 0 && outIdx < leftFieldCount) { + mapped.add(outIdx); + } else if (inputIndex == 1 && outIdx >= leftFieldCount) { + mapped.add(outIdx - leftFieldCount); + } + } + return mapped; + } + if (node instanceof Aggregate) { + return Collections.emptySet(); + } + // Pass-through default for rels that preserve input column positions at the start of + // their output (WatermarkAssigner, Filter, Sort, Union, Exchange, Sink, + // ChangelogNormalize, …). Some rels (e.g. window TVF) append columns at the end; bound + // the propagated indexes by the child input's field count to stay within range. + final int inputFieldCount = + unwrap(node.getInputs().get(inputIndex)).getRowType().getFieldCount(); + final Set bounded = new HashSet<>(); + for (Integer idx : parentProtected) { + if (idx < inputFieldCount) { + bounded.add(idx); + } + } + return bounded; + } + + /** + * Rebuilds {@code node} with {@code newInputs}, accounting for the fact that a formerly + * time-indicator column on the inputs may now be a plain timestamp. + */ + private static RelNode rebuildParent( + RelNode node, List newInputs, RexBuilder rexBuilder) { + if (node instanceof StreamPhysicalCalc) { + return rebuildCalc((StreamPhysicalCalc) node, newInputs.get(0), rexBuilder); + } + // Pass-through: row type is derived from the inputs, so a regular copy demotes the + // rowtime field automatically. + return node.copy(node.getTraitSet(), newInputs); + } + + private static RelNode rebuildCalc( + StreamPhysicalCalc calc, RelNode newInput, RexBuilder rexBuilder) { + final RelDataType oldInputType = calc.getInput().getRowType(); + final RelDataType newInputType = newInput.getRowType(); + // Nothing to retype if the input row-type is unchanged. The input rel may still have + // been replaced (e.g. an inner Calc was rebuilt below); merge if it is also a Calc to + // collapse the two adjacent Calcs that result from dropping a WatermarkAssigner. + if (oldInputType.equals(newInputType)) { + final RelNode top = calc.copy(calc.getTraitSet(), newInput, calc.getProgram()); + return maybeMergeCalc((StreamPhysicalCalc) top); + } + final RexProgram program = calc.getProgram(); + // Retype any RexInputRef whose input column type changed (rowtime demoted to plain + // TIMESTAMP). We deliberately do NOT run the full RexTimeIndicatorMaterializer here: + // it would re-wrap already-materialized PROCTIME_MATERIALIZE(...) calls on other + // (proctime) columns, producing PROCTIME_MATERIALIZE(PROCTIME_MATERIALIZE(...)). + final RexShuttle retyper = + new RexShuttle() { + @Override + public RexNode visitInputRef(RexInputRef ref) { + final RelDataType newFieldType = + newInputType.getFieldList().get(ref.getIndex()).getType(); + if (!ref.getType().equals(newFieldType)) { + return rexBuilder.makeInputRef(newFieldType, ref.getIndex()); + } + return ref; + } + }; + final List projects = + program.getProjectList().stream() + .map(p -> program.expandLocalRef(p).accept(retyper)) + .collect(Collectors.toList()); + final RexNode condition = + program.getCondition() != null + ? program.expandLocalRef(program.getCondition()).accept(retyper) + : null; + final RexProgram newProgram = + RexProgram.create( + newInputType, + projects, + condition, + program.getOutputRowType().getFieldNames(), + rexBuilder); + final StreamPhysicalCalc top = + new StreamPhysicalCalc( + calc.getCluster(), + calc.getTraitSet(), + newInput, + newProgram, + newProgram.getOutputRowType()); + return maybeMergeCalc(top); + } + + /** + * If {@code top}'s input is itself a {@link Calc}, merge the two programs into a single Calc. + * Dropping a {@link StreamPhysicalWatermarkAssigner} that sits between two Calcs naturally + * leaves them adjacent; folding them keeps the plan compact and matches what {@code + * FlinkCalcMergeRule} would produce on a subsequent pass. + */ + private static RelNode maybeMergeCalc(StreamPhysicalCalc top) { + final RelNode input = unwrap(top.getInput()); + if (input instanceof Calc) { + return FlinkRelUtil.merge(top, (Calc) input); + } + return top; + } + + private static RelNode unwrap(RelNode node) { + return node instanceof HepRelVertex ? ((HepRelVertex) node).getCurrentRel() : node; + } + + /** + * Returns {@code true} if {@code node} consumes upstream watermarks at runtime, either through + * a declared {@link StreamPhysicalRel#requireWatermark()}, a {@code CURRENT_WATERMARK} SQL call + * embedded in any of its expressions, or as a special-case rel whose runtime operator depends + * on watermarks while {@code requireWatermark()} happens to return {@code false}. + */ + private static boolean consumesWatermarkAtRuntime(RelNode node) { + if (node instanceof StreamPhysicalRel && ((StreamPhysicalRel) node).requireWatermark()) { + return true; + } + if (isRowtimeTemporalSort(node)) { + return true; + } + return containsCurrentWatermarkCall(node); + } + + /** + * Returns {@code true} if {@code node} is a {@link StreamPhysicalTemporalSort} whose primary + * sort field is a rowtime time-indicator column. Such a sort is implemented at runtime by an + * operator that consumes watermarks. + */ + private static boolean isRowtimeTemporalSort(RelNode node) { + if (!(node instanceof StreamPhysicalTemporalSort)) { + return false; + } + final StreamPhysicalTemporalSort sort = (StreamPhysicalTemporalSort) node; + if (sort.getCollation().getFieldCollations().isEmpty()) { + return false; + } + final int fieldIndex = sort.getCollation().getFieldCollations().get(0).getFieldIndex(); + final RelDataType fieldType = + sort.getInput().getRowType().getFieldList().get(fieldIndex).getType(); + return fieldType instanceof TimeIndicatorRelDataType + && ((TimeIndicatorRelDataType) fieldType).isEventTime(); + } + + /** + * Defensively scans every {@link RexNode} that {@code node} exposes via {@link + * RelNode#accept(RexShuttle)}. This relies on the standard Calcite contract that a rel + * implementation routes its internal expressions through the shuttle (Calc, Project, Filter, + * Join, Aggregate filter args, …), so any future rel type that exposes a {@code + * CURRENT_WATERMARK} call is automatically classified as a watermark consumer. + */ + private static boolean containsCurrentWatermarkCall(RelNode node) { + final boolean[] found = {false}; + final RexShuttle shuttle = + new RexShuttle() { + @Override + public RexNode visitCall(RexCall call) { + if (!found[0] + && "CURRENT_WATERMARK" + .equalsIgnoreCase(call.getOperator().getName())) { + found[0] = true; + } + return super.visitCall(call); + } + }; + try { + node.accept(shuttle); + } catch (Exception ignored) { + // Some rels' accept(RexShuttle) implementations may not be defensive; treat scan + // failures as "no CURRENT_WATERMARK found" — in the worst case we miss a marker, but + // the same rel will be examined again from the parent side via the recursion. + } + return found[0]; + } + + /** Rule configuration. */ + @Value.Immutable + public interface Config extends RelRule.Config { + Config DEFAULT = + ImmutableRedundantWatermarkAssignerRemoveRule.Config.builder() + .description("RedundantWatermarkAssignerRemoveRule") + .operandSupplier( + builder -> builder.operand(StreamPhysicalRel.class).anyInputs()) + .build(); + + @Override + default RedundantWatermarkAssignerRemoveRule toRule() { + return new RedundantWatermarkAssignerRemoveRule(this); + } + } +} diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkStreamProgram.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkStreamProgram.scala index 8ec32c242fa48..148821df00070 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkStreamProgram.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkStreamProgram.scala @@ -304,6 +304,14 @@ object FlinkStreamProgram { PHYSICAL_REWRITE, FlinkGroupProgramBuilder .newBuilder[StreamOptimizeContext] + .addProgram( + FlinkHepRuleSetProgramBuilder.newBuilder + .setHepRulesExecutionType(HEP_RULES_EXECUTION_TYPE.RULE_SEQUENCE) + .setHepMatchOrder(HepMatchOrder.TOP_DOWN) + .add(FlinkStreamRuleSets.REMOVE_REDUNDANT_WATERMARK_RULES) + .build(), + "remove redundant watermarks" + ) .addProgram( new FlinkMarkChangelogNormalizeProgram, "mark changelog normalize reusing same source") diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkStreamRuleSets.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkStreamRuleSets.scala index 05de12a6950a9..a6438d4899fe8 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkStreamRuleSets.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkStreamRuleSets.scala @@ -520,6 +520,11 @@ object FlinkStreamRuleSets { StreamPhysicalLegacySinkRule.INSTANCE ) + /** RuleSet related to optimizing watermark assigners. */ + val REMOVE_REDUNDANT_WATERMARK_RULES: RuleSet = RuleSets.ofList( + RedundantWatermarkAssignerRemoveRule.INSTANCE + ) + /** * RuleSet related to optimizing ChangelogNormalize: * 1. transpose watermark to be close to source 2. transpose projections 3. push filter either diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/physical/stream/RedundantWatermarkAssignerRemoveRuleTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/physical/stream/RedundantWatermarkAssignerRemoveRuleTest.java new file mode 100644 index 0000000000000..f787464a2f279 --- /dev/null +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/physical/stream/RedundantWatermarkAssignerRemoveRuleTest.java @@ -0,0 +1,292 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.planner.plan.rules.physical.stream; + +import org.apache.flink.table.api.StatementSet; +import org.apache.flink.table.api.TableConfig; +import org.apache.flink.table.planner.utils.StreamTableTestUtil; +import org.apache.flink.table.planner.utils.TableTestBase; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +/** + * Tests for {@link RedundantWatermarkAssignerRemoveRule}. + * + *

The rule drops {@link + * org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWatermarkAssigner} + * operators whose watermarks are not consumed by any downstream operator that requires watermarks + * (windowed aggregates, event-time interval / temporal joins, event-time temporal sort, …). This + * test exercises positive cases (assigner removed) and negative cases (assigner kept). + */ +class RedundantWatermarkAssignerRemoveRuleTest extends TableTestBase { + + private final StreamTableTestUtil util = streamTestUtil(TableConfig.getDefault()); + + @BeforeEach + void setup() { + util.tableEnv() + .executeSql( + "CREATE TABLE src (\n" + + " a INT,\n" + + " b BIGINT,\n" + + " c STRING,\n" + + " rt TIMESTAMP(3),\n" + + " WATERMARK FOR rt AS rt - INTERVAL '1' SECOND\n" + + ") WITH (\n" + + " 'connector' = 'values',\n" + + " 'bounded' = 'false'\n" + + ")"); + + util.tableEnv().executeSql("CREATE TABLE src2 LIKE src"); + + // A source that supports watermark push-down: when push-down kicks in, the runtime + // operator handles watermarks itself; the rule must still produce a consistent plan. + util.tableEnv() + .executeSql( + "CREATE TABLE src_pushdown (\n" + + " a INT,\n" + + " b BIGINT,\n" + + " c STRING,\n" + + " rt TIMESTAMP(3),\n" + + " WATERMARK FOR rt AS rt - INTERVAL '1' SECOND\n" + + ") WITH (\n" + + " 'connector' = 'values',\n" + + " 'bounded' = 'false',\n" + + " 'enable-watermark-push-down' = 'true'\n" + + ")"); + + util.tableEnv() + .executeSql( + "CREATE TABLE rowtime_sink (\n" + + " a INT,\n" + + " b BIGINT,\n" + + " c STRING,\n" + + " rt TIMESTAMP(3)\n" + + ") WITH (\n" + + " 'connector' = 'values',\n" + + " 'sink-insert-only' = 'true'\n" + + ")"); + + util.tableEnv() + .executeSql( + "CREATE TABLE plain_sink (\n" + + " a INT,\n" + + " b BIGINT,\n" + + " c STRING\n" + + ") WITH (\n" + + " 'connector' = 'values',\n" + + " 'sink-insert-only' = 'true'\n" + + ")"); + + util.tableEnv() + .executeSql( + "CREATE TABLE window_sink (\n" + + " window_start TIMESTAMP(3),\n" + + " window_end TIMESTAMP(3),\n" + + " a INT,\n" + + " cnt BIGINT\n" + + ") WITH (\n" + + " 'connector' = 'values',\n" + + " 'sink-insert-only' = 'true'\n" + + ")"); + + // A versioned table for event-time temporal joins. + util.tableEnv() + .executeSql( + "CREATE TABLE rates (\n" + + " currency STRING,\n" + + " rate BIGINT,\n" + + " rt TIMESTAMP(3),\n" + + " WATERMARK FOR rt AS rt,\n" + + " PRIMARY KEY(currency) NOT ENFORCED\n" + + ") WITH (\n" + + " 'connector' = 'values',\n" + + " 'bounded' = 'false',\n" + + " 'changelog-mode' = 'I,UA,UB,D'\n" + + ")"); + + // A processing-time-only lookup table for lookup joins. + util.tableEnv() + .executeSql( + "CREATE TABLE lookup_dim (\n" + + " id INT,\n" + + " name STRING\n" + + ") WITH (\n" + + " 'connector' = 'values',\n" + + " 'bounded' = 'true'\n" + + ")"); + } + + /** No downstream event-time operator: a plain SELECT * INSERT drops the assigner. */ + @Test + void testSimpleSelectRemovesAssigner() { + util.verifyRelPlanInsert("INSERT INTO plain_sink SELECT a, b, c FROM src"); + } + + /** A Calc chain above the assigner is preserved; the assigner is removed underneath. */ + @Test + void testCalcChainRemovesAssigner() { + util.verifyRelPlanInsert( + "INSERT INTO plain_sink SELECT a, b + 1, UPPER(c) FROM src WHERE a > 0"); + } + + /** + * If a rowtime ({@code *ROWTIME*}) column reaches the sink, an upstream rowtime attribute is + * being consumed as a watermark-bearing column (e.g. for forwarding to an external system). The + * rule must keep the assigner so the watermark remains available at runtime. + */ + @Test + void testRowtimeForwardingSinkKeepsAssigner() { + util.verifyRelPlanInsert( + "INSERT INTO rowtime_sink SELECT a, b, c, rt FROM src WHERE a > 0"); + } + + /** Tumble window aggregation requires watermarks. */ + @Test + void testTumbleWindowKeepsAssigner() { + util.verifyRelPlanInsert( + "INSERT INTO window_sink " + + "SELECT window_start, window_end, a, COUNT(*) FROM TABLE(" + + " TUMBLE(TABLE src, DESCRIPTOR(rt), INTERVAL '5' SECOND)) " + + "GROUP BY window_start, window_end, a"); + } + + /** Event-time interval join requires watermarks on both sides. */ + @Test + void testRowtimeIntervalJoinKeepsAssigner() { + util.verifyRelPlanInsert( + "INSERT INTO plain_sink " + + "SELECT s1.a, s1.b, s1.c FROM src s1 JOIN src2 s2 " + + "ON s1.a = s2.a AND s1.rt BETWEEN s2.rt - INTERVAL '5' SECOND " + + "AND s2.rt + INTERVAL '5' SECOND"); + } + + /** + * {@code CURRENT_WATERMARK(rt)} reads the upstream watermark at runtime even though its hosting + * Calc does not declare {@link + * org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRel#requireWatermark()}. + * The rule must keep the assigner in this case. + */ + @Test + void testCurrentWatermarkInProjectionKeepsAssigner() { + util.verifyRelPlanInsert( + "INSERT INTO plain_sink " + + "SELECT a, b, CAST(CURRENT_WATERMARK(rt) AS STRING) FROM src"); + } + + /** Same as above but the {@code CURRENT_WATERMARK} call appears inside a WHERE filter. */ + @Test + void testCurrentWatermarkInFilterKeepsAssigner() { + util.verifyRelPlanInsert( + "INSERT INTO plain_sink " + + "SELECT a, b, c FROM src " + + "WHERE CURRENT_WATERMARK(rt) IS NULL OR rt > CURRENT_WATERMARK(rt)"); + } + + /** + * Union where only the right branch feeds a window aggregation. The left branch's assigner is + * redundant and must be removed; the right branch's assigner must be kept. + */ + @Test + void testUnionMixedBranches() { + util.verifyRelPlanInsert( + "INSERT INTO window_sink " + + "SELECT CAST(NULL AS TIMESTAMP(3)), CAST(NULL AS TIMESTAMP(3)), a, b " + + "FROM src " + + "UNION ALL " + + "SELECT window_start, window_end, a, COUNT(*) FROM TABLE(" + + " TUMBLE(TABLE src2, DESCRIPTOR(rt), INTERVAL '5' SECOND)) " + + "GROUP BY window_start, window_end, a"); + } + + // ------------------------------------------------------------------------ + // FLINK-14621 review follow-ups + // ------------------------------------------------------------------------ + + /** + * A statement set with two branches: one drops the assigner (plain select), one keeps it + * (window aggregation). The rule fires once per sink, so each branch is optimized + * independently. + */ + @Test + void testStatementSetMixedBranches() { + StatementSet stmtSet = util.tableEnv().createStatementSet(); + stmtSet.addInsertSql("INSERT INTO plain_sink SELECT a, b, c FROM src"); + stmtSet.addInsertSql( + "INSERT INTO window_sink " + + "SELECT window_start, window_end, a, COUNT(*) FROM TABLE(" + + " TUMBLE(TABLE src, DESCRIPTOR(rt), INTERVAL '5' SECOND)) " + + "GROUP BY window_start, window_end, a"); + util.verifyRelPlan(stmtSet); + } + + /** + * Event-time temporal table join (versioned table). The probe-side assigner must be kept + * because the join is fired by the rowtime watermark. This exercises the {@code + * containsCurrentWatermarkCall}-style consumer detection that walks any rel's expressions (Join + * condition lives on a Join, not a Calc/Project/Filter). + */ + @Test + void testEventTimeTemporalJoinKeepsAssigner() { + util.verifyRelPlanInsert( + "INSERT INTO plain_sink " + + "SELECT o.a, o.b * r.rate, r.currency " + + "FROM src AS o " + + "JOIN rates FOR SYSTEM_TIME AS OF o.rt AS r " + + "ON CAST(o.a AS STRING) = r.currency"); + } + + /** + * Process-time lookup join does not consume the probe stream's watermark, so the + * assigner is redundant and gets removed. + */ + @Test + void testProctimeLookupJoinRemovesAssigner() { + util.verifyRelPlanInsert( + "INSERT INTO plain_sink " + + "SELECT s.a, s.b, d.name " + + "FROM (SELECT *, PROCTIME() AS pt FROM src) s " + + "JOIN lookup_dim FOR SYSTEM_TIME AS OF s.pt AS d " + + "ON s.a = d.id"); + } + + /** + * Source that supports watermark push-down. Even when push-down applies, the rule must produce + * a consistent plan (no spurious type demotions, no removal that would break runtime + * watermarking when it is actually needed). + */ + @Test + void testWatermarkPushDownSourceWithoutEventTimeRemovesAssigner() { + util.verifyRelPlanInsert("INSERT INTO plain_sink SELECT a, b, c FROM src_pushdown"); + } + + /** + * Watermark push-down source feeding a window aggregate: push-down + window means watermarks + * must remain wired up end-to-end; the rule must not interfere. + */ + @Test + void testWatermarkPushDownSourceWithWindowKeepsWatermarks() { + util.verifyRelPlanInsert( + "INSERT INTO window_sink " + + "SELECT window_start, window_end, a, COUNT(*) FROM TABLE(" + + " TUMBLE(TABLE src_pushdown, DESCRIPTOR(rt), INTERVAL '5' SECOND)) " + + "GROUP BY window_start, window_end, a"); + } +} 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 3cb0ff64a9ac1..e4556c0fc58aa 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 @@ -34,10 +34,8 @@ LogicalProject(EXPR$0=[$1]) @@ -56,9 +54,8 @@ LogicalProject(EXPR$0=[$0]) diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/hints/stream/StateTtlHintTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/hints/stream/StateTtlHintTest.xml index 91f28026eaeb2..e36326c4eb360 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/hints/stream/StateTtlHintTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/hints/stream/StateTtlHintTest.xml @@ -520,16 +520,13 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) Calc(select=[a, b, c]) +- Join(joinType=[InnerJoin], where=[=(b, b0)], select=[a, b, c, b0], leftInputSpec=[NoUniqueKey], rightInputSpec=[JoinKeyContainsUniqueKey], stateTtlHints=[[[STATE_TTL options:{LEFT=1d, RIGHT=3d}]]]) :- Exchange(distribution=[hash[b]]) - : +- Calc(select=[a, b, CAST(c AS TIMESTAMP(3)) AS c], where=[IS NOT NULL(b)]) - : +- WatermarkAssigner(rowtime=[c], watermark=[c]) - : +- TableSourceScan(table=[[default_catalog, default_database, tableWithWatermark1]], fields=[a, b, c]) + : +- Calc(select=[a, b, c], where=[IS NOT NULL(b)]) + : +- TableSourceScan(table=[[default_catalog, default_database, tableWithWatermark1]], fields=[a, b, c]) +- Exchange(distribution=[hash[b]]) +- GroupAggregate(groupBy=[b], select=[b]) +- Exchange(distribution=[hash[b]]) +- Calc(select=[b], where=[IS NOT NULL(b)]) - +- WatermarkAssigner(rowtime=[d], watermark=[-(d, 5000:INTERVAL SECOND)]) - +- Calc(select=[b, +(c.c1, 5000:INTERVAL SECOND) AS d]) - +- TableSourceScan(table=[[default_catalog, default_database, tableWithWatermark2, project=[b, c], metadata=[]]], fields=[b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, tableWithWatermark2, project=[b, c], metadata=[]]], fields=[b, c]) ]]> @@ -559,16 +556,13 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) Calc(select=[a, b, c]) +- Join(joinType=[InnerJoin], where=[=(b, b0)], select=[a, b, c, b0], leftInputSpec=[NoUniqueKey], rightInputSpec=[JoinKeyContainsUniqueKey], stateTtlHints=[[[STATE_TTL options:{LEFT=1d, RIGHT=3d}]]]) :- Exchange(distribution=[hash[b]]) - : +- Calc(select=[a, b, CAST(c AS TIMESTAMP(3)) AS c], where=[IS NOT NULL(b)]) - : +- WatermarkAssigner(rowtime=[c], watermark=[c]) - : +- TableSourceScan(table=[[default_catalog, default_database, tableWithWatermark1]], fields=[a, b, c]) + : +- Calc(select=[a, b, c], where=[IS NOT NULL(b)]) + : +- TableSourceScan(table=[[default_catalog, default_database, tableWithWatermark1]], fields=[a, b, c]) +- Exchange(distribution=[hash[b]]) +- GroupAggregate(groupBy=[b], select=[b]) +- Exchange(distribution=[hash[b]]) +- Calc(select=[b], where=[IS NOT NULL(b)]) - +- WatermarkAssigner(rowtime=[d], watermark=[-(d, 5000:INTERVAL SECOND)]) - +- Calc(select=[b, +(c.c1, 5000:INTERVAL SECOND) AS d]) - +- TableSourceScan(table=[[default_catalog, default_database, tableWithWatermark2, project=[b, c], metadata=[]]], fields=[b, c]) + +- TableSourceScan(table=[[default_catalog, default_database, tableWithWatermark2, project=[b, c], metadata=[]]], fields=[b, c]) ]]> 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..37cc45608c81a 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 @@ -1006,29 +1006,21 @@ LogicalProject(a=[$0], b=[$7]) Calc(select=[a, b]) +- IntervalJoin(joinType=[LeftOuterJoin], windowBounds=[isRowTime=false, leftLowerBound=7200000, leftUpperBound=3600000, leftTimeIndex=1, rightTimeIndex=2], where=[AND(=(a, a0), >=(proctime, +(proctime0, 7200000:INTERVAL HOUR)), <=(proctime, +(proctime0, 3600000:INTERVAL HOUR)))], select=[a, proctime, a0, b, proctime0]) :- Exchange(distribution=[hash[a]]) - : +- Calc(select=[a, proctime]) - : +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)]) - : +- Calc(select=[a, PROCTIME() AS proctime, rowtime]) - : +- TableSourceScan(table=[[default_catalog, default_database, A, project=[a, rowtime], metadata=[]]], fields=[a, rowtime]) + : +- Calc(select=[a, PROCTIME() AS proctime]) + : +- TableSourceScan(table=[[default_catalog, default_database, A, project=[a, rowtime], metadata=[]]], fields=[a, rowtime]) +- Exchange(distribution=[hash[a]]) - +- Calc(select=[a, b, proctime]) - +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)]) - +- Calc(select=[a, b, PROCTIME() AS proctime, rowtime]) - +- TableSourceScan(table=[[default_catalog, default_database, B, project=[a, b, rowtime], metadata=[]]], fields=[a, b, rowtime]) + +- Calc(select=[a, b, PROCTIME() AS proctime]) + +- TableSourceScan(table=[[default_catalog, default_database, B, project=[a, b, rowtime], metadata=[]]], fields=[a, b, rowtime]) == Optimized Execution Plan == Calc(select=[a, b]) +- IntervalJoin(joinType=[LeftOuterJoin], windowBounds=[isRowTime=false, leftLowerBound=7200000, leftUpperBound=3600000, leftTimeIndex=1, rightTimeIndex=2], where=[((a = a0) AND (proctime >= (proctime0 + 7200000:INTERVAL HOUR)) AND (proctime <= (proctime0 + 3600000:INTERVAL HOUR)))], select=[a, proctime, a0, b, proctime0]) :- Exchange(distribution=[hash[a]]) - : +- Calc(select=[a, proctime]) - : +- WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)]) - : +- Calc(select=[a, PROCTIME() AS proctime, rowtime]) - : +- TableSourceScan(table=[[default_catalog, default_database, A, project=[a, rowtime], metadata=[]]], fields=[a, rowtime]) + : +- Calc(select=[a, PROCTIME() AS proctime]) + : +- TableSourceScan(table=[[default_catalog, default_database, A, project=[a, rowtime], metadata=[]]], fields=[a, rowtime]) +- Exchange(distribution=[hash[a]]) - +- Calc(select=[a, b, proctime]) - +- WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)]) - +- Calc(select=[a, b, PROCTIME() AS proctime, rowtime]) - +- TableSourceScan(table=[[default_catalog, default_database, B, project=[a, b, rowtime], metadata=[]]], fields=[a, b, rowtime]) + +- Calc(select=[a, b, PROCTIME() AS proctime]) + +- TableSourceScan(table=[[default_catalog, default_database, B, project=[a, b, rowtime], metadata=[]]], fields=[a, b, rowtime]) == Physical Execution Plan == { @@ -1040,31 +1032,9 @@ Calc(select=[a, b]) "parallelism" : 1 }, { "id" : , - "type" : "Calc(select=[a, PROCTIME() AS proctime, rowtime])", + "type" : "Calc(select=[a, PROCTIME() AS proctime])", "pact" : "Operator", - "contents" : "Calc(select=[a, PROCTIME() AS proctime, rowtime])", - "parallelism" : 1, - "predecessors" : [ { - "id" : , - "ship_strategy" : "FORWARD", - "side" : "second" - } ] - }, { - "id" : , - "type" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])", - "pact" : "Operator", - "contents" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])", - "parallelism" : 1, - "predecessors" : [ { - "id" : , - "ship_strategy" : "FORWARD", - "side" : "second" - } ] - }, { - "id" : , - "type" : "Calc(select=[a, proctime])", - "pact" : "Operator", - "contents" : "Calc(select=[a, proctime])", + "contents" : "Calc(select=[a, PROCTIME() AS proctime])", "parallelism" : 1, "predecessors" : [ { "id" : , @@ -1079,31 +1049,9 @@ Calc(select=[a, b]) "parallelism" : 1 }, { "id" : , - "type" : "Calc(select=[a, b, PROCTIME() AS proctime, rowtime])", - "pact" : "Operator", - "contents" : "Calc(select=[a, b, PROCTIME() AS proctime, rowtime])", - "parallelism" : 1, - "predecessors" : [ { - "id" : , - "ship_strategy" : "FORWARD", - "side" : "second" - } ] - }, { - "id" : , - "type" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])", + "type" : "Calc(select=[a, b, PROCTIME() AS proctime])", "pact" : "Operator", - "contents" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])", - "parallelism" : 1, - "predecessors" : [ { - "id" : , - "ship_strategy" : "FORWARD", - "side" : "second" - } ] - }, { - "id" : , - "type" : "Calc(select=[a, b, proctime])", - "pact" : "Operator", - "contents" : "Calc(select=[a, b, proctime])", + "contents" : "Calc(select=[a, b, PROCTIME() AS proctime])", "parallelism" : 1, "predecessors" : [ { "id" : , @@ -1167,29 +1115,21 @@ LogicalProject(a=[$0], b=[$7]) Calc(select=[a, b]) +- IntervalJoin(joinType=[LeftOuterJoin], windowBounds=[isRowTime=false, leftLowerBound=7200000, leftUpperBound=3600000, leftTimeIndex=1, rightTimeIndex=2], where=[AND(=(a, a0), >=(proctime, +(proctime0, 7200000:INTERVAL HOUR)), <=(proctime, +(proctime0, 3600000:INTERVAL HOUR)))], select=[a, proctime, a0, b, proctime0]) :- Exchange(distribution=[hash[a]]) - : +- Calc(select=[a, proctime]) - : +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)]) - : +- Calc(select=[a, PROCTIME() AS proctime, rowtime]) - : +- TableSourceScan(table=[[default_catalog, default_database, A, project=[a, rowtime], metadata=[]]], fields=[a, rowtime]) + : +- Calc(select=[a, PROCTIME() AS proctime]) + : +- TableSourceScan(table=[[default_catalog, default_database, A, project=[a, rowtime], metadata=[]]], fields=[a, rowtime]) +- Exchange(distribution=[hash[a]]) - +- Calc(select=[a, b, proctime]) - +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)]) - +- Calc(select=[a, b, PROCTIME() AS proctime, rowtime]) - +- TableSourceScan(table=[[default_catalog, default_database, B, project=[a, b, rowtime], metadata=[]]], fields=[a, b, rowtime]) + +- Calc(select=[a, b, PROCTIME() AS proctime]) + +- TableSourceScan(table=[[default_catalog, default_database, B, project=[a, b, rowtime], metadata=[]]], fields=[a, b, rowtime]) == Optimized Execution Plan == Calc(select=[a, b]) +- IntervalJoin(joinType=[LeftOuterJoin], windowBounds=[isRowTime=false, leftLowerBound=7200000, leftUpperBound=3600000, leftTimeIndex=1, rightTimeIndex=2], where=[((a = a0) AND (proctime >= (proctime0 + 7200000:INTERVAL HOUR)) AND (proctime <= (proctime0 + 3600000:INTERVAL HOUR)))], select=[a, proctime, a0, b, proctime0]) :- Exchange(distribution=[hash[a]]) - : +- Calc(select=[a, proctime]) - : +- WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)]) - : +- Calc(select=[a, PROCTIME() AS proctime, rowtime]) - : +- TableSourceScan(table=[[default_catalog, default_database, A, project=[a, rowtime], metadata=[]]], fields=[a, rowtime]) + : +- Calc(select=[a, PROCTIME() AS proctime]) + : +- TableSourceScan(table=[[default_catalog, default_database, A, project=[a, rowtime], metadata=[]]], fields=[a, rowtime]) +- Exchange(distribution=[hash[a]]) - +- Calc(select=[a, b, proctime]) - +- WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)]) - +- Calc(select=[a, b, PROCTIME() AS proctime, rowtime]) - +- TableSourceScan(table=[[default_catalog, default_database, B, project=[a, b, rowtime], metadata=[]]], fields=[a, b, rowtime]) + +- Calc(select=[a, b, PROCTIME() AS proctime]) + +- TableSourceScan(table=[[default_catalog, default_database, B, project=[a, b, rowtime], metadata=[]]], fields=[a, b, rowtime]) == Physical Execution Plan == { @@ -1203,29 +1143,7 @@ Calc(select=[a, b]) "id" : , "type" : "Calc[]", "pact" : "Operator", - "contents" : "[]:Calc(select=[a, PROCTIME() AS proctime, rowtime])", - "parallelism" : 1, - "predecessors" : [ { - "id" : , - "ship_strategy" : "FORWARD", - "side" : "second" - } ] - }, { - "id" : , - "type" : "WatermarkAssigner[]", - "pact" : "Operator", - "contents" : "[]:WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])", - "parallelism" : 1, - "predecessors" : [ { - "id" : , - "ship_strategy" : "FORWARD", - "side" : "second" - } ] - }, { - "id" : , - "type" : "Calc[]", - "pact" : "Operator", - "contents" : "[]:Calc(select=[a, proctime])", + "contents" : "[]:Calc(select=[a, PROCTIME() AS proctime])", "parallelism" : 1, "predecessors" : [ { "id" : , @@ -1242,29 +1160,7 @@ Calc(select=[a, b]) "id" : , "type" : "Calc[]", "pact" : "Operator", - "contents" : "[]:Calc(select=[a, b, PROCTIME() AS proctime, rowtime])", - "parallelism" : 1, - "predecessors" : [ { - "id" : , - "ship_strategy" : "FORWARD", - "side" : "second" - } ] - }, { - "id" : , - "type" : "WatermarkAssigner[]", - "pact" : "Operator", - "contents" : "[]:WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])", - "parallelism" : 1, - "predecessors" : [ { - "id" : , - "ship_strategy" : "FORWARD", - "side" : "second" - } ] - }, { - "id" : , - "type" : "Calc[]", - "pact" : "Operator", - "contents" : "[]:Calc(select=[a, b, proctime])", + "contents" : "[]:Calc(select=[a, b, PROCTIME() AS proctime])", "parallelism" : 1, "predecessors" : [ { "id" : , @@ -1328,29 +1224,21 @@ LogicalProject(a=[$0], b=[$7]) Calc(select=[a, b]) +- IntervalJoin(joinType=[InnerJoin], windowBounds=[isRowTime=false, leftLowerBound=-3600000, leftUpperBound=3600000, leftTimeIndex=1, rightTimeIndex=2], where=[AND(=(a, a0), >=(proctime, -(proctime0, 3600000:INTERVAL HOUR)), <=(proctime, +(proctime0, 3600000:INTERVAL HOUR)))], select=[a, proctime, a0, b, proctime0]) :- Exchange(distribution=[hash[a]]) - : +- Calc(select=[a, proctime]) - : +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)]) - : +- Calc(select=[a, PROCTIME() AS proctime, rowtime]) - : +- TableSourceScan(table=[[default_catalog, default_database, A, project=[a, rowtime], metadata=[]]], fields=[a, rowtime]) + : +- Calc(select=[a, PROCTIME() AS proctime]) + : +- TableSourceScan(table=[[default_catalog, default_database, A, project=[a, rowtime], metadata=[]]], fields=[a, rowtime]) +- Exchange(distribution=[hash[a]]) - +- Calc(select=[a, b, proctime]) - +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)]) - +- Calc(select=[a, b, PROCTIME() AS proctime, rowtime]) - +- TableSourceScan(table=[[default_catalog, default_database, B, project=[a, b, rowtime], metadata=[]]], fields=[a, b, rowtime]) + +- Calc(select=[a, b, PROCTIME() AS proctime]) + +- TableSourceScan(table=[[default_catalog, default_database, B, project=[a, b, rowtime], metadata=[]]], fields=[a, b, rowtime]) == Optimized Execution Plan == Calc(select=[a, b]) +- IntervalJoin(joinType=[InnerJoin], windowBounds=[isRowTime=false, leftLowerBound=-3600000, leftUpperBound=3600000, leftTimeIndex=1, rightTimeIndex=2], where=[((a = a0) AND (proctime >= (proctime0 - 3600000:INTERVAL HOUR)) AND (proctime <= (proctime0 + 3600000:INTERVAL HOUR)))], select=[a, proctime, a0, b, proctime0]) :- Exchange(distribution=[hash[a]]) - : +- Calc(select=[a, proctime]) - : +- WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)]) - : +- Calc(select=[a, PROCTIME() AS proctime, rowtime]) - : +- TableSourceScan(table=[[default_catalog, default_database, A, project=[a, rowtime], metadata=[]]], fields=[a, rowtime]) + : +- Calc(select=[a, PROCTIME() AS proctime]) + : +- TableSourceScan(table=[[default_catalog, default_database, A, project=[a, rowtime], metadata=[]]], fields=[a, rowtime]) +- Exchange(distribution=[hash[a]]) - +- Calc(select=[a, b, proctime]) - +- WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)]) - +- Calc(select=[a, b, PROCTIME() AS proctime, rowtime]) - +- TableSourceScan(table=[[default_catalog, default_database, B, project=[a, b, rowtime], metadata=[]]], fields=[a, b, rowtime]) + +- Calc(select=[a, b, PROCTIME() AS proctime]) + +- TableSourceScan(table=[[default_catalog, default_database, B, project=[a, b, rowtime], metadata=[]]], fields=[a, b, rowtime]) == Physical Execution Plan == { @@ -1362,31 +1250,9 @@ Calc(select=[a, b]) "parallelism" : 1 }, { "id" : , - "type" : "Calc(select=[a, PROCTIME() AS proctime, rowtime])", - "pact" : "Operator", - "contents" : "Calc(select=[a, PROCTIME() AS proctime, rowtime])", - "parallelism" : 1, - "predecessors" : [ { - "id" : , - "ship_strategy" : "FORWARD", - "side" : "second" - } ] - }, { - "id" : , - "type" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])", - "pact" : "Operator", - "contents" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])", - "parallelism" : 1, - "predecessors" : [ { - "id" : , - "ship_strategy" : "FORWARD", - "side" : "second" - } ] - }, { - "id" : , - "type" : "Calc(select=[a, proctime])", + "type" : "Calc(select=[a, PROCTIME() AS proctime])", "pact" : "Operator", - "contents" : "Calc(select=[a, proctime])", + "contents" : "Calc(select=[a, PROCTIME() AS proctime])", "parallelism" : 1, "predecessors" : [ { "id" : , @@ -1401,31 +1267,9 @@ Calc(select=[a, b]) "parallelism" : 1 }, { "id" : , - "type" : "Calc(select=[a, b, PROCTIME() AS proctime, rowtime])", - "pact" : "Operator", - "contents" : "Calc(select=[a, b, PROCTIME() AS proctime, rowtime])", - "parallelism" : 1, - "predecessors" : [ { - "id" : , - "ship_strategy" : "FORWARD", - "side" : "second" - } ] - }, { - "id" : , - "type" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])", - "pact" : "Operator", - "contents" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])", - "parallelism" : 1, - "predecessors" : [ { - "id" : , - "ship_strategy" : "FORWARD", - "side" : "second" - } ] - }, { - "id" : , - "type" : "Calc(select=[a, b, proctime])", + "type" : "Calc(select=[a, b, PROCTIME() AS proctime])", "pact" : "Operator", - "contents" : "Calc(select=[a, b, proctime])", + "contents" : "Calc(select=[a, b, PROCTIME() AS proctime])", "parallelism" : 1, "predecessors" : [ { "id" : , @@ -1478,29 +1322,21 @@ LogicalProject(a=[$0], b=[$7]) Calc(select=[a, b]) +- IntervalJoin(joinType=[InnerJoin], windowBounds=[isRowTime=false, leftLowerBound=-3600000, leftUpperBound=3600000, leftTimeIndex=1, rightTimeIndex=2], where=[AND(=(a, a0), >=(proctime, -(proctime0, 3600000:INTERVAL HOUR)), <=(proctime, +(proctime0, 3600000:INTERVAL HOUR)))], select=[a, proctime, a0, b, proctime0]) :- Exchange(distribution=[hash[a]]) - : +- Calc(select=[a, proctime]) - : +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)]) - : +- Calc(select=[a, PROCTIME() AS proctime, rowtime]) - : +- TableSourceScan(table=[[default_catalog, default_database, A, project=[a, rowtime], metadata=[]]], fields=[a, rowtime]) + : +- Calc(select=[a, PROCTIME() AS proctime]) + : +- TableSourceScan(table=[[default_catalog, default_database, A, project=[a, rowtime], metadata=[]]], fields=[a, rowtime]) +- Exchange(distribution=[hash[a]]) - +- Calc(select=[a, b, proctime]) - +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)]) - +- Calc(select=[a, b, PROCTIME() AS proctime, rowtime]) - +- TableSourceScan(table=[[default_catalog, default_database, B, project=[a, b, rowtime], metadata=[]]], fields=[a, b, rowtime]) + +- Calc(select=[a, b, PROCTIME() AS proctime]) + +- TableSourceScan(table=[[default_catalog, default_database, B, project=[a, b, rowtime], metadata=[]]], fields=[a, b, rowtime]) == Optimized Execution Plan == Calc(select=[a, b]) +- IntervalJoin(joinType=[InnerJoin], windowBounds=[isRowTime=false, leftLowerBound=-3600000, leftUpperBound=3600000, leftTimeIndex=1, rightTimeIndex=2], where=[((a = a0) AND (proctime >= (proctime0 - 3600000:INTERVAL HOUR)) AND (proctime <= (proctime0 + 3600000:INTERVAL HOUR)))], select=[a, proctime, a0, b, proctime0]) :- Exchange(distribution=[hash[a]]) - : +- Calc(select=[a, proctime]) - : +- WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)]) - : +- Calc(select=[a, PROCTIME() AS proctime, rowtime]) - : +- TableSourceScan(table=[[default_catalog, default_database, A, project=[a, rowtime], metadata=[]]], fields=[a, rowtime]) + : +- Calc(select=[a, PROCTIME() AS proctime]) + : +- TableSourceScan(table=[[default_catalog, default_database, A, project=[a, rowtime], metadata=[]]], fields=[a, rowtime]) +- Exchange(distribution=[hash[a]]) - +- Calc(select=[a, b, proctime]) - +- WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)]) - +- Calc(select=[a, b, PROCTIME() AS proctime, rowtime]) - +- TableSourceScan(table=[[default_catalog, default_database, B, project=[a, b, rowtime], metadata=[]]], fields=[a, b, rowtime]) + +- Calc(select=[a, b, PROCTIME() AS proctime]) + +- TableSourceScan(table=[[default_catalog, default_database, B, project=[a, b, rowtime], metadata=[]]], fields=[a, b, rowtime]) == Physical Execution Plan == { @@ -1514,29 +1350,7 @@ Calc(select=[a, b]) "id" : , "type" : "Calc[]", "pact" : "Operator", - "contents" : "[]:Calc(select=[a, PROCTIME() AS proctime, rowtime])", - "parallelism" : 1, - "predecessors" : [ { - "id" : , - "ship_strategy" : "FORWARD", - "side" : "second" - } ] - }, { - "id" : , - "type" : "WatermarkAssigner[]", - "pact" : "Operator", - "contents" : "[]:WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])", - "parallelism" : 1, - "predecessors" : [ { - "id" : , - "ship_strategy" : "FORWARD", - "side" : "second" - } ] - }, { - "id" : , - "type" : "Calc[]", - "pact" : "Operator", - "contents" : "[]:Calc(select=[a, proctime])", + "contents" : "[]:Calc(select=[a, PROCTIME() AS proctime])", "parallelism" : 1, "predecessors" : [ { "id" : , @@ -1553,29 +1367,7 @@ Calc(select=[a, b]) "id" : , "type" : "Calc[]", "pact" : "Operator", - "contents" : "[]:Calc(select=[a, b, PROCTIME() AS proctime, rowtime])", - "parallelism" : 1, - "predecessors" : [ { - "id" : , - "ship_strategy" : "FORWARD", - "side" : "second" - } ] - }, { - "id" : , - "type" : "WatermarkAssigner[]", - "pact" : "Operator", - "contents" : "[]:WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])", - "parallelism" : 1, - "predecessors" : [ { - "id" : , - "ship_strategy" : "FORWARD", - "side" : "second" - } ] - }, { - "id" : , - "type" : "Calc[]", - "pact" : "Operator", - "contents" : "[]:Calc(select=[a, b, proctime])", + "contents" : "[]:Calc(select=[a, b, PROCTIME() AS proctime])", "parallelism" : 1, "predecessors" : [ { "id" : , @@ -2934,17 +2726,15 @@ LogicalProject(a=[$0]) Calc(select=[a]) +- TemporalSort(orderBy=[proctime ASC, c ASC]) +- Exchange(distribution=[single]) - +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)]) - +- Calc(select=[a, PROCTIME() AS proctime, c, rowtime]) - +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, c, rowtime], metadata=[]]], fields=[a, c, rowtime]) + +- Calc(select=[a, PROCTIME() AS proctime, c, rowtime]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, c, rowtime], metadata=[]]], fields=[a, c, rowtime]) == Optimized Execution Plan == Calc(select=[a]) +- TemporalSort(orderBy=[proctime ASC, c ASC]) +- Exchange(distribution=[single]) - +- WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)]) - +- Calc(select=[a, PROCTIME() AS proctime, c, rowtime]) - +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, c, rowtime], metadata=[]]], fields=[a, c, rowtime]) + +- Calc(select=[a, PROCTIME() AS proctime, c, rowtime]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, c, rowtime], metadata=[]]], fields=[a, c, rowtime]) == Physical Execution Plan == { @@ -2965,17 +2755,6 @@ Calc(select=[a]) "ship_strategy" : "FORWARD", "side" : "second" } ] - }, { - "id" : , - "type" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])", - "pact" : "Operator", - "contents" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])", - "parallelism" : 1, - "predecessors" : [ { - "id" : , - "ship_strategy" : "FORWARD", - "side" : "second" - } ] }, { "id" : , "type" : "TemporalSort(orderBy=[proctime ASC, c ASC])", @@ -3016,17 +2795,15 @@ LogicalProject(a=[$0]) Calc(select=[a]) +- TemporalSort(orderBy=[proctime ASC, c ASC]) +- Exchange(distribution=[single]) - +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)]) - +- Calc(select=[a, PROCTIME() AS proctime, c, rowtime]) - +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, c, rowtime], metadata=[]]], fields=[a, c, rowtime]) + +- Calc(select=[a, PROCTIME() AS proctime, c, rowtime]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, c, rowtime], metadata=[]]], fields=[a, c, rowtime]) == Optimized Execution Plan == Calc(select=[a]) +- TemporalSort(orderBy=[proctime ASC, c ASC]) +- Exchange(distribution=[single]) - +- WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)]) - +- Calc(select=[a, PROCTIME() AS proctime, c, rowtime]) - +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, c, rowtime], metadata=[]]], fields=[a, c, rowtime]) + +- Calc(select=[a, PROCTIME() AS proctime, c, rowtime]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, c, rowtime], metadata=[]]], fields=[a, c, rowtime]) == Physical Execution Plan == { @@ -3047,17 +2824,6 @@ Calc(select=[a]) "ship_strategy" : "FORWARD", "side" : "second" } ] - }, { - "id" : , - "type" : "WatermarkAssigner[]", - "pact" : "Operator", - "contents" : "[]:WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])", - "parallelism" : 1, - "predecessors" : [ { - "id" : , - "ship_strategy" : "FORWARD", - "side" : "second" - } ] }, { "id" : , "type" : "TemporalSort[]", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonGroupWindowAggregateJsonPlanTest_jsonplan/testProcTimeHopWindow.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonGroupWindowAggregateJsonPlanTest_jsonplan/testProcTimeHopWindow.out index e0334458a780e..e60e8ee8fb83f 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonGroupWindowAggregateJsonPlanTest_jsonplan/testProcTimeHopWindow.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonGroupWindowAggregateJsonPlanTest_jsonplan/testProcTimeHopWindow.out @@ -84,26 +84,9 @@ "id" : 2, "type" : "stream-exec-calc_1", "projection" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 0, - "type" : "INT NOT NULL" - }, { "kind" : "INPUT_REF", "inputIndex" : 1, "type" : "BIGINT" - }, { - "kind" : "INPUT_REF", - "inputIndex" : 2, - "type" : "VARCHAR(2147483647)" - }, { - "kind" : "CALL", - "internalName" : "$TO_TIMESTAMP$1", - "operands" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 2, - "type" : "VARCHAR(2147483647)" - } ], - "type" : "TIMESTAMP(3)" }, { "kind" : "CALL", "internalName" : "$PROCTIME$1", @@ -113,111 +96,6 @@ "precision" : 3, "kind" : "PROCTIME" } - } ], - "condition" : null, - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : { - "type" : "ROW", - "fields" : [ { - "name" : "a", - "fieldType" : "INT NOT NULL" - }, { - "name" : "b", - "fieldType" : "BIGINT" - }, { - "name" : "c", - "fieldType" : "VARCHAR(2147483647)" - }, { - "name" : "rowtime", - "fieldType" : "TIMESTAMP(3)" - }, { - "name" : "proctime", - "fieldType" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - } ] - }, - "description" : "Calc(select=[a, b, c, TO_TIMESTAMP(c) AS rowtime, PROCTIME() AS proctime])" - }, { - "id" : 3, - "type" : "stream-exec-watermark-assigner_1", - "watermarkExpr" : { - "kind" : "CALL", - "syntax" : "SPECIAL", - "internalName" : "$-$1", - "operands" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 3, - "type" : "TIMESTAMP(3)" - }, { - "kind" : "LITERAL", - "value" : "1000", - "type" : "INTERVAL SECOND(6) NOT NULL" - } ], - "type" : "TIMESTAMP(3)" - }, - "rowtimeFieldIndex" : 3, - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : { - "type" : "ROW", - "fields" : [ { - "name" : "a", - "fieldType" : "INT NOT NULL" - }, { - "name" : "b", - "fieldType" : "BIGINT" - }, { - "name" : "c", - "fieldType" : "VARCHAR(2147483647)" - }, { - "name" : "rowtime", - "fieldType" : { - "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", - "precision" : 3, - "kind" : "ROWTIME" - } - }, { - "name" : "proctime", - "fieldType" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - } ] - }, - "description" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])" - }, { - "id" : 4, - "type" : "stream-exec-calc_1", - "projection" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 1, - "type" : "BIGINT" - }, { - "kind" : "INPUT_REF", - "inputIndex" : 4, - "type" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } }, { "kind" : "INPUT_REF", "inputIndex" : 0, @@ -266,9 +144,9 @@ "fieldType" : "INT NOT NULL" } ] }, - "description" : "Calc(select=[b, proctime, a, (a + 1) AS $f3])" + "description" : "Calc(select=[b, PROCTIME() AS proctime, a, (a + 1) AS $f3])" }, { - "id" : 5, + "id" : 3, "type" : "stream-exec-exchange_1", "inputProperties" : [ { "requiredDistribution" : { @@ -301,7 +179,7 @@ }, "description" : "Exchange(distribution=[hash[b]])" }, { - "id" : 6, + "id" : 4, "type" : "stream-exec-python-group-window-aggregate_1", "grouping" : [ 0 ], "aggCalls" : [ { @@ -353,7 +231,7 @@ "outputType" : "ROW<`b` BIGINT, `EXPR$1` BIGINT>", "description" : "PythonGroupWindowAggregate(groupBy=[b], window=[SlidingGroupWindow('w$, proctime, 600000, 300000)], select=[b, pyFunc(a, $f3) AS EXPR$1])" }, { - "id" : 7, + "id" : 5, "type" : "stream-exec-sink_2", "configuration" : { "table.exec.sink.keyed-shuffle" : "AUTO", @@ -420,19 +298,5 @@ "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" - }, { - "source" : 5, - "target" : 6, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 6, - "target" : 7, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" } ] } \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonGroupWindowAggregateJsonPlanTest_jsonplan/testProcTimeSessionWindow.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonGroupWindowAggregateJsonPlanTest_jsonplan/testProcTimeSessionWindow.out index cd2e2b67f300c..124c75f3d2413 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonGroupWindowAggregateJsonPlanTest_jsonplan/testProcTimeSessionWindow.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonGroupWindowAggregateJsonPlanTest_jsonplan/testProcTimeSessionWindow.out @@ -84,26 +84,9 @@ "id" : 2, "type" : "stream-exec-calc_1", "projection" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 0, - "type" : "INT NOT NULL" - }, { "kind" : "INPUT_REF", "inputIndex" : 1, "type" : "BIGINT" - }, { - "kind" : "INPUT_REF", - "inputIndex" : 2, - "type" : "VARCHAR(2147483647)" - }, { - "kind" : "CALL", - "internalName" : "$TO_TIMESTAMP$1", - "operands" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 2, - "type" : "VARCHAR(2147483647)" - } ], - "type" : "TIMESTAMP(3)" }, { "kind" : "CALL", "internalName" : "$PROCTIME$1", @@ -113,111 +96,6 @@ "precision" : 3, "kind" : "PROCTIME" } - } ], - "condition" : null, - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : { - "type" : "ROW", - "fields" : [ { - "name" : "a", - "fieldType" : "INT NOT NULL" - }, { - "name" : "b", - "fieldType" : "BIGINT" - }, { - "name" : "c", - "fieldType" : "VARCHAR(2147483647)" - }, { - "name" : "rowtime", - "fieldType" : "TIMESTAMP(3)" - }, { - "name" : "proctime", - "fieldType" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - } ] - }, - "description" : "Calc(select=[a, b, c, TO_TIMESTAMP(c) AS rowtime, PROCTIME() AS proctime])" - }, { - "id" : 3, - "type" : "stream-exec-watermark-assigner_1", - "watermarkExpr" : { - "kind" : "CALL", - "syntax" : "SPECIAL", - "internalName" : "$-$1", - "operands" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 3, - "type" : "TIMESTAMP(3)" - }, { - "kind" : "LITERAL", - "value" : "1000", - "type" : "INTERVAL SECOND(6) NOT NULL" - } ], - "type" : "TIMESTAMP(3)" - }, - "rowtimeFieldIndex" : 3, - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : { - "type" : "ROW", - "fields" : [ { - "name" : "a", - "fieldType" : "INT NOT NULL" - }, { - "name" : "b", - "fieldType" : "BIGINT" - }, { - "name" : "c", - "fieldType" : "VARCHAR(2147483647)" - }, { - "name" : "rowtime", - "fieldType" : { - "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", - "precision" : 3, - "kind" : "ROWTIME" - } - }, { - "name" : "proctime", - "fieldType" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - } ] - }, - "description" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])" - }, { - "id" : 4, - "type" : "stream-exec-calc_1", - "projection" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 1, - "type" : "BIGINT" - }, { - "kind" : "INPUT_REF", - "inputIndex" : 4, - "type" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } }, { "kind" : "INPUT_REF", "inputIndex" : 0, @@ -266,9 +144,9 @@ "fieldType" : "INT NOT NULL" } ] }, - "description" : "Calc(select=[b, proctime, a, (a + 1) AS $f3])" + "description" : "Calc(select=[b, PROCTIME() AS proctime, a, (a + 1) AS $f3])" }, { - "id" : 5, + "id" : 3, "type" : "stream-exec-exchange_1", "inputProperties" : [ { "requiredDistribution" : { @@ -301,7 +179,7 @@ }, "description" : "Exchange(distribution=[hash[b]])" }, { - "id" : 6, + "id" : 4, "type" : "stream-exec-python-group-window-aggregate_1", "grouping" : [ 0 ], "aggCalls" : [ { @@ -351,7 +229,7 @@ "outputType" : "ROW<`b` BIGINT, `EXPR$1` BIGINT>", "description" : "PythonGroupWindowAggregate(groupBy=[b], window=[SessionGroupWindow('w$, proctime, 600000)], select=[b, pyFunc(a, $f3) AS EXPR$1])" }, { - "id" : 7, + "id" : 5, "type" : "stream-exec-sink_2", "configuration" : { "table.exec.sink.keyed-shuffle" : "AUTO", @@ -418,19 +296,5 @@ "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" - }, { - "source" : 5, - "target" : 6, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 6, - "target" : 7, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" } ] } \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonGroupWindowAggregateJsonPlanTest_jsonplan/testProcTimeTumbleWindow.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonGroupWindowAggregateJsonPlanTest_jsonplan/testProcTimeTumbleWindow.out index 32b9614edc985..348ab0818bf01 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonGroupWindowAggregateJsonPlanTest_jsonplan/testProcTimeTumbleWindow.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonGroupWindowAggregateJsonPlanTest_jsonplan/testProcTimeTumbleWindow.out @@ -84,26 +84,9 @@ "id" : 2, "type" : "stream-exec-calc_1", "projection" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 0, - "type" : "INT NOT NULL" - }, { "kind" : "INPUT_REF", "inputIndex" : 1, "type" : "BIGINT" - }, { - "kind" : "INPUT_REF", - "inputIndex" : 2, - "type" : "VARCHAR(2147483647)" - }, { - "kind" : "CALL", - "internalName" : "$TO_TIMESTAMP$1", - "operands" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 2, - "type" : "VARCHAR(2147483647)" - } ], - "type" : "TIMESTAMP(3)" }, { "kind" : "CALL", "internalName" : "$PROCTIME$1", @@ -113,111 +96,6 @@ "precision" : 3, "kind" : "PROCTIME" } - } ], - "condition" : null, - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : { - "type" : "ROW", - "fields" : [ { - "name" : "a", - "fieldType" : "INT NOT NULL" - }, { - "name" : "b", - "fieldType" : "BIGINT" - }, { - "name" : "c", - "fieldType" : "VARCHAR(2147483647)" - }, { - "name" : "rowtime", - "fieldType" : "TIMESTAMP(3)" - }, { - "name" : "proctime", - "fieldType" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - } ] - }, - "description" : "Calc(select=[a, b, c, TO_TIMESTAMP(c) AS rowtime, PROCTIME() AS proctime])" - }, { - "id" : 3, - "type" : "stream-exec-watermark-assigner_1", - "watermarkExpr" : { - "kind" : "CALL", - "syntax" : "SPECIAL", - "internalName" : "$-$1", - "operands" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 3, - "type" : "TIMESTAMP(3)" - }, { - "kind" : "LITERAL", - "value" : "1000", - "type" : "INTERVAL SECOND(6) NOT NULL" - } ], - "type" : "TIMESTAMP(3)" - }, - "rowtimeFieldIndex" : 3, - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : { - "type" : "ROW", - "fields" : [ { - "name" : "a", - "fieldType" : "INT NOT NULL" - }, { - "name" : "b", - "fieldType" : "BIGINT" - }, { - "name" : "c", - "fieldType" : "VARCHAR(2147483647)" - }, { - "name" : "rowtime", - "fieldType" : { - "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", - "precision" : 3, - "kind" : "ROWTIME" - } - }, { - "name" : "proctime", - "fieldType" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - } ] - }, - "description" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])" - }, { - "id" : 4, - "type" : "stream-exec-calc_1", - "projection" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 1, - "type" : "BIGINT" - }, { - "kind" : "INPUT_REF", - "inputIndex" : 4, - "type" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } }, { "kind" : "INPUT_REF", "inputIndex" : 0, @@ -266,9 +144,9 @@ "fieldType" : "INT NOT NULL" } ] }, - "description" : "Calc(select=[b, proctime, a, (a + 1) AS $f3])" + "description" : "Calc(select=[b, PROCTIME() AS proctime, a, (a + 1) AS $f3])" }, { - "id" : 5, + "id" : 3, "type" : "stream-exec-exchange_1", "inputProperties" : [ { "requiredDistribution" : { @@ -301,7 +179,7 @@ }, "description" : "Exchange(distribution=[hash[b]])" }, { - "id" : 6, + "id" : 4, "type" : "stream-exec-python-group-window-aggregate_1", "grouping" : [ 0 ], "aggCalls" : [ { @@ -416,7 +294,7 @@ }, "description" : "PythonGroupWindowAggregate(groupBy=[b], window=[TumblingGroupWindow('w$, proctime, 900000)], properties=[w$start, w$end, w$proctime], select=[b, pyFunc(a, $f3) AS EXPR$2, start('w$) AS w$start, end('w$) AS w$end, proctime('w$) AS w$proctime])" }, { - "id" : 7, + "id" : 5, "type" : "stream-exec-calc_1", "projection" : [ { "kind" : "INPUT_REF", @@ -442,7 +320,7 @@ "outputType" : "ROW<`b` BIGINT, `window_end` TIMESTAMP(3) NOT NULL, `EXPR$2` BIGINT>", "description" : "Calc(select=[b, w$end AS window_end, EXPR$2])" }, { - "id" : 8, + "id" : 6, "type" : "stream-exec-sink_2", "configuration" : { "table.exec.sink.keyed-shuffle" : "AUTO", @@ -519,19 +397,5 @@ "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" - }, { - "source" : 6, - "target" : 7, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 7, - "target" : 8, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" } ] } \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonOverAggregateJsonPlanTest_jsonplan/testProcTimeBoundedNonPartitionedRangeOver.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonOverAggregateJsonPlanTest_jsonplan/testProcTimeBoundedNonPartitionedRangeOver.out index 670a374ed8d34..d7730cf5c7be2 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonOverAggregateJsonPlanTest_jsonplan/testProcTimeBoundedNonPartitionedRangeOver.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonOverAggregateJsonPlanTest_jsonplan/testProcTimeBoundedNonPartitionedRangeOver.out @@ -87,110 +87,13 @@ "precision" : 3, "kind" : "PROCTIME" } - }, { - "kind" : "INPUT_REF", - "inputIndex" : 1, - "type" : "INT" - }, { - "kind" : "INPUT_REF", - "inputIndex" : 2, - "type" : "TIMESTAMP(3)" - } ], - "condition" : null, - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : { - "type" : "ROW", - "fields" : [ { - "name" : "c", - "fieldType" : "INT NOT NULL" - }, { - "name" : "proctime", - "fieldType" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - }, { - "name" : "a", - "fieldType" : "INT" - }, { - "name" : "rowtime", - "fieldType" : "TIMESTAMP(3)" - } ] - }, - "description" : "Calc(select=[c, PROCTIME() AS proctime, a, rowtime])" - }, { - "id" : 3, - "type" : "stream-exec-watermark-assigner_1", - "watermarkExpr" : { - "kind" : "INPUT_REF", - "inputIndex" : 3, - "type" : "TIMESTAMP(3)" - }, - "rowtimeFieldIndex" : 3, - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : { - "type" : "ROW", - "fields" : [ { - "name" : "c", - "fieldType" : "INT NOT NULL" - }, { - "name" : "proctime", - "fieldType" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - }, { - "name" : "a", - "fieldType" : "INT" - }, { - "name" : "rowtime", - "fieldType" : { - "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", - "precision" : 3, - "kind" : "ROWTIME" - } - } ] - }, - "description" : "WatermarkAssigner(rowtime=[rowtime], watermark=[rowtime])" - }, { - "id" : 4, - "type" : "stream-exec-calc_1", - "projection" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 0, - "type" : "INT NOT NULL" - }, { - "kind" : "INPUT_REF", - "inputIndex" : 1, - "type" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } }, { "kind" : "CALL", "syntax" : "SPECIAL", "internalName" : "$CAST$1", "operands" : [ { "kind" : "INPUT_REF", - "inputIndex" : 2, + "inputIndex" : 1, "type" : "INT" } ], "type" : "BIGINT" @@ -221,9 +124,9 @@ "fieldType" : "BIGINT" } ] }, - "description" : "Calc(select=[c, proctime, CAST(a AS BIGINT) AS $2])" + "description" : "Calc(select=[c, PROCTIME() AS proctime, CAST(a AS BIGINT) AS $2])" }, { - "id" : 5, + "id" : 3, "type" : "stream-exec-exchange_1", "inputProperties" : [ { "requiredDistribution" : { @@ -252,7 +155,7 @@ }, "description" : "Exchange(distribution=[single])" }, { - "id" : 6, + "id" : 4, "type" : "stream-exec-python-over-aggregate_1", "overSpec" : { "partition" : { @@ -327,7 +230,7 @@ }, "description" : "PythonOverAggregate(orderBy=[proctime ASC], window=[ RANGE BETWEEN 10000 PRECEDING AND CURRENT ROW], select=[c, proctime, $2, pyFunc(c, c) AS w0$o0])" }, { - "id" : 7, + "id" : 5, "type" : "stream-exec-calc_1", "projection" : [ { "kind" : "INPUT_REF", @@ -349,7 +252,7 @@ "outputType" : "ROW<`$0` BIGINT, `$1` BIGINT>", "description" : "Calc(select=[$2 AS $0, w0$o0 AS $1])" }, { - "id" : 8, + "id" : 6, "type" : "stream-exec-sink_2", "configuration" : { "table.exec.sink.keyed-shuffle" : "AUTO", @@ -425,19 +328,5 @@ "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" - }, { - "source" : 6, - "target" : 7, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 7, - "target" : 8, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" } ] } \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonOverAggregateJsonPlanTest_jsonplan/testProcTimeBoundedPartitionedRangeOver.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonOverAggregateJsonPlanTest_jsonplan/testProcTimeBoundedPartitionedRangeOver.out index 6cd81cf7c4be8..0f49ee8ac5301 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonOverAggregateJsonPlanTest_jsonplan/testProcTimeBoundedPartitionedRangeOver.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonOverAggregateJsonPlanTest_jsonplan/testProcTimeBoundedPartitionedRangeOver.out @@ -91,103 +91,6 @@ "precision" : 3, "kind" : "PROCTIME" } - }, { - "kind" : "INPUT_REF", - "inputIndex" : 2, - "type" : "TIMESTAMP(3)" - } ], - "condition" : null, - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : { - "type" : "ROW", - "fields" : [ { - "name" : "a", - "fieldType" : "INT" - }, { - "name" : "c", - "fieldType" : "INT NOT NULL" - }, { - "name" : "proctime", - "fieldType" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - }, { - "name" : "rowtime", - "fieldType" : "TIMESTAMP(3)" - } ] - }, - "description" : "Calc(select=[a, c, PROCTIME() AS proctime, rowtime])" - }, { - "id" : 3, - "type" : "stream-exec-watermark-assigner_1", - "watermarkExpr" : { - "kind" : "INPUT_REF", - "inputIndex" : 3, - "type" : "TIMESTAMP(3)" - }, - "rowtimeFieldIndex" : 3, - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : { - "type" : "ROW", - "fields" : [ { - "name" : "a", - "fieldType" : "INT" - }, { - "name" : "c", - "fieldType" : "INT NOT NULL" - }, { - "name" : "proctime", - "fieldType" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - }, { - "name" : "rowtime", - "fieldType" : { - "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", - "precision" : 3, - "kind" : "ROWTIME" - } - } ] - }, - "description" : "WatermarkAssigner(rowtime=[rowtime], watermark=[rowtime])" - }, { - "id" : 4, - "type" : "stream-exec-calc_1", - "projection" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 0, - "type" : "INT" - }, { - "kind" : "INPUT_REF", - "inputIndex" : 1, - "type" : "INT NOT NULL" - }, { - "kind" : "INPUT_REF", - "inputIndex" : 2, - "type" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } }, { "kind" : "CALL", "syntax" : "SPECIAL", @@ -228,9 +131,9 @@ "fieldType" : "BIGINT" } ] }, - "description" : "Calc(select=[a, c, proctime, CAST(a AS BIGINT) AS $3])" + "description" : "Calc(select=[a, c, PROCTIME() AS proctime, CAST(a AS BIGINT) AS $3])" }, { - "id" : 5, + "id" : 3, "type" : "stream-exec-exchange_1", "inputProperties" : [ { "requiredDistribution" : { @@ -263,7 +166,7 @@ }, "description" : "Exchange(distribution=[hash[a]])" }, { - "id" : 6, + "id" : 4, "type" : "stream-exec-python-over-aggregate_1", "overSpec" : { "partition" : { @@ -341,7 +244,7 @@ }, "description" : "PythonOverAggregate(partitionBy=[a], orderBy=[proctime ASC], window=[ RANGE BETWEEN 7200000 PRECEDING AND CURRENT ROW], select=[a, c, proctime, $3, pyFunc(c, c) AS w0$o0])" }, { - "id" : 7, + "id" : 5, "type" : "stream-exec-calc_1", "projection" : [ { "kind" : "INPUT_REF", @@ -363,7 +266,7 @@ "outputType" : "ROW<`$0` BIGINT, `$1` BIGINT>", "description" : "Calc(select=[$3 AS $0, w0$o0 AS $1])" }, { - "id" : 8, + "id" : 6, "type" : "stream-exec-sink_2", "configuration" : { "table.exec.sink.keyed-shuffle" : "AUTO", @@ -439,19 +342,5 @@ "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" - }, { - "source" : 6, - "target" : 7, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 7, - "target" : 8, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" } ] } \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonOverAggregateJsonPlanTest_jsonplan/testProcTimeBoundedPartitionedRowsOverWithBuiltinProctime.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonOverAggregateJsonPlanTest_jsonplan/testProcTimeBoundedPartitionedRowsOverWithBuiltinProctime.out index 87db9c2ee918b..9be14b4fa9d2f 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonOverAggregateJsonPlanTest_jsonplan/testProcTimeBoundedPartitionedRowsOverWithBuiltinProctime.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonOverAggregateJsonPlanTest_jsonplan/testProcTimeBoundedPartitionedRowsOverWithBuiltinProctime.out @@ -73,40 +73,6 @@ "description" : "TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, c, rowtime], metadata=[]]], fields=[a, c, rowtime])" }, { "id" : 2, - "type" : "stream-exec-watermark-assigner_1", - "watermarkExpr" : { - "kind" : "INPUT_REF", - "inputIndex" : 2, - "type" : "TIMESTAMP(3)" - }, - "rowtimeFieldIndex" : 2, - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : { - "type" : "ROW", - "fields" : [ { - "name" : "a", - "fieldType" : "INT" - }, { - "name" : "c", - "fieldType" : "INT NOT NULL" - }, { - "name" : "rowtime", - "fieldType" : { - "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", - "precision" : 3, - "kind" : "ROWTIME" - } - } ] - }, - "description" : "WatermarkAssigner(rowtime=[rowtime], watermark=[rowtime])" - }, { - "id" : 3, "type" : "stream-exec-calc_1", "projection" : [ { "kind" : "INPUT_REF", @@ -167,7 +133,7 @@ }, "description" : "Calc(select=[a, c, CAST(a AS BIGINT) AS $2, PROCTIME() AS $3])" }, { - "id" : 4, + "id" : 3, "type" : "stream-exec-exchange_1", "inputProperties" : [ { "requiredDistribution" : { @@ -200,7 +166,7 @@ }, "description" : "Exchange(distribution=[hash[a]])" }, { - "id" : 5, + "id" : 4, "type" : "stream-exec-python-over-aggregate_1", "overSpec" : { "partition" : { @@ -278,7 +244,7 @@ }, "description" : "PythonOverAggregate(partitionBy=[a], orderBy=[$3 ASC], window=[ ROWS BETWEEN 4 PRECEDING AND CURRENT ROW], select=[a, c, $2, $3, pyFunc(c, c) AS w0$o0])" }, { - "id" : 6, + "id" : 5, "type" : "stream-exec-calc_1", "projection" : [ { "kind" : "INPUT_REF", @@ -300,7 +266,7 @@ "outputType" : "ROW<`$0` BIGINT, `$1` BIGINT>", "description" : "Calc(select=[$2 AS $0, w0$o0 AS $1])" }, { - "id" : 7, + "id" : 6, "type" : "stream-exec-sink_2", "configuration" : { "table.exec.sink.keyed-shuffle" : "AUTO", @@ -376,12 +342,5 @@ "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" - }, { - "source" : 6, - "target" : 7, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" } ] } \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonOverAggregateJsonPlanTest_jsonplan/testProcTimeUnboundedPartitionedRangeOver.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonOverAggregateJsonPlanTest_jsonplan/testProcTimeUnboundedPartitionedRangeOver.out index 52a537df698c8..558c72ddb6fad 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonOverAggregateJsonPlanTest_jsonplan/testProcTimeUnboundedPartitionedRangeOver.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/PythonOverAggregateJsonPlanTest_jsonplan/testProcTimeUnboundedPartitionedRangeOver.out @@ -91,103 +91,6 @@ "precision" : 3, "kind" : "PROCTIME" } - }, { - "kind" : "INPUT_REF", - "inputIndex" : 2, - "type" : "TIMESTAMP(3)" - } ], - "condition" : null, - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : { - "type" : "ROW", - "fields" : [ { - "name" : "a", - "fieldType" : "INT" - }, { - "name" : "c", - "fieldType" : "INT NOT NULL" - }, { - "name" : "proctime", - "fieldType" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - }, { - "name" : "rowtime", - "fieldType" : "TIMESTAMP(3)" - } ] - }, - "description" : "Calc(select=[a, c, PROCTIME() AS proctime, rowtime])" - }, { - "id" : 3, - "type" : "stream-exec-watermark-assigner_1", - "watermarkExpr" : { - "kind" : "INPUT_REF", - "inputIndex" : 3, - "type" : "TIMESTAMP(3)" - }, - "rowtimeFieldIndex" : 3, - "inputProperties" : [ { - "requiredDistribution" : { - "type" : "UNKNOWN" - }, - "damBehavior" : "PIPELINED", - "priority" : 0 - } ], - "outputType" : { - "type" : "ROW", - "fields" : [ { - "name" : "a", - "fieldType" : "INT" - }, { - "name" : "c", - "fieldType" : "INT NOT NULL" - }, { - "name" : "proctime", - "fieldType" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } - }, { - "name" : "rowtime", - "fieldType" : { - "type" : "TIMESTAMP_WITHOUT_TIME_ZONE", - "precision" : 3, - "kind" : "ROWTIME" - } - } ] - }, - "description" : "WatermarkAssigner(rowtime=[rowtime], watermark=[rowtime])" - }, { - "id" : 4, - "type" : "stream-exec-calc_1", - "projection" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 0, - "type" : "INT" - }, { - "kind" : "INPUT_REF", - "inputIndex" : 1, - "type" : "INT NOT NULL" - }, { - "kind" : "INPUT_REF", - "inputIndex" : 2, - "type" : { - "type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", - "nullable" : false, - "precision" : 3, - "kind" : "PROCTIME" - } }, { "kind" : "CALL", "syntax" : "SPECIAL", @@ -228,9 +131,9 @@ "fieldType" : "BIGINT" } ] }, - "description" : "Calc(select=[a, c, proctime, CAST(a AS BIGINT) AS $3])" + "description" : "Calc(select=[a, c, PROCTIME() AS proctime, CAST(a AS BIGINT) AS $3])" }, { - "id" : 5, + "id" : 3, "type" : "stream-exec-exchange_1", "inputProperties" : [ { "requiredDistribution" : { @@ -263,7 +166,7 @@ }, "description" : "Exchange(distribution=[hash[a]])" }, { - "id" : 6, + "id" : 4, "type" : "stream-exec-python-over-aggregate_1", "overSpec" : { "partition" : { @@ -331,7 +234,7 @@ }, "description" : "PythonOverAggregate(partitionBy=[a], orderBy=[proctime ASC], window=[ RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, c, proctime, $3, pyFunc(c, c) AS w0$o0])" }, { - "id" : 7, + "id" : 5, "type" : "stream-exec-calc_1", "projection" : [ { "kind" : "INPUT_REF", @@ -353,7 +256,7 @@ "outputType" : "ROW<`$0` BIGINT, `$1` BIGINT>", "description" : "Calc(select=[$3 AS $0, w0$o0 AS $1])" }, { - "id" : 8, + "id" : 6, "type" : "stream-exec-sink_2", "configuration" : { "table.exec.sink.keyed-shuffle" : "AUTO", @@ -429,19 +332,5 @@ "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" - }, { - "source" : 6, - "target" : 7, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" - }, { - "source" : 7, - "target" : 8, - "shuffle" : { - "type" : "FORWARD" - }, - "shuffleMode" : "PIPELINED" } ] } \ No newline at end of file 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..98137f592e453 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 @@ -32,8 +32,7 @@ Sink(table=[default_catalog.default_database.pk_upsert_snk], fields=[a, EXPR$1, +- GroupAggregate(groupBy=[a], select=[a, MAX(b) AS EXPR$1, SUM(c) AS EXPR$2], duplicateChanges=[ALLOW]) +- Exchange(distribution=[hash[a]], duplicateChanges=[DISALLOW]) +- Calc(select=[a, b, c], duplicateChanges=[DISALLOW]) - +- WatermarkAssigner(rowtime=[rt], watermark=[-(rt, 1000:INTERVAL SECOND)], duplicateChanges=[DISALLOW]) - +- TableSourceScan(table=[[default_catalog, default_database, append_src1]], fields=[a, b, c, rt], duplicateChanges=[DISALLOW]) + +- TableSourceScan(table=[[default_catalog, default_database, append_src1]], fields=[a, b, c, rt], duplicateChanges=[DISALLOW]) ]]> @@ -50,8 +49,7 @@ LogicalSink(table=[default_catalog.default_database.pk_append_sink], fields=[a, @@ -69,8 +67,7 @@ LogicalSink(table=[default_catalog.default_database.no_pk_snk], fields=[a, b, c] @@ -88,8 +85,7 @@ LogicalSink(table=[default_catalog.default_database.pk_upsert_snk], fields=[a, b @@ -106,8 +102,7 @@ LogicalSink(table=[default_catalog.default_database.no_pk_snk], fields=[a, b, EX @@ -124,8 +119,7 @@ LogicalSink(table=[default_catalog.default_database.pk_upsert_snk], fields=[a, b @@ -143,8 +137,7 @@ LogicalSink(table=[default_catalog.default_database.no_pk_snk], fields=[a, b, c] @@ -162,8 +155,7 @@ LogicalSink(table=[default_catalog.default_database.pk_upsert_snk], fields=[a, b @@ -181,8 +173,7 @@ LogicalSink(table=[default_catalog.default_database.no_pk_snk], fields=[a, b, c] (a, 1), <(c, CAST(NOW() AS BIGINT)))], duplicateChanges=[DISALLOW]) - +- WatermarkAssigner(rowtime=[rt], watermark=[-(rt, 1000:INTERVAL SECOND)], duplicateChanges=[DISALLOW]) - +- TableSourceScan(table=[[default_catalog, default_database, append_src1]], fields=[a, b, c, rt], duplicateChanges=[DISALLOW]) + +- TableSourceScan(table=[[default_catalog, default_database, append_src1]], fields=[a, b, c, rt], duplicateChanges=[DISALLOW]) ]]> @@ -200,8 +191,7 @@ LogicalSink(table=[default_catalog.default_database.pk_upsert_snk], fields=[a, b (a, 1), <(c, CAST(NOW() AS BIGINT)))], duplicateChanges=[ALLOW]) - +- WatermarkAssigner(rowtime=[rt], watermark=[-(rt, 1000:INTERVAL SECOND)], duplicateChanges=[ALLOW]) - +- TableSourceScan(table=[[default_catalog, default_database, append_src1]], fields=[a, b, c, rt], duplicateChanges=[ALLOW]) + +- TableSourceScan(table=[[default_catalog, default_database, append_src1]], fields=[a, b, c, rt], duplicateChanges=[ALLOW]) ]]> @@ -218,8 +208,7 @@ LogicalSink(table=[default_catalog.default_database.no_pk_snk], fields=[a, b, EX @@ -236,8 +225,7 @@ LogicalSink(table=[default_catalog.default_database.pk_upsert_snk], fields=[a, b @@ -254,8 +242,7 @@ LogicalSink(table=[default_catalog.default_database.no_pk_snk], fields=[a, b, c] @@ -272,8 +259,7 @@ LogicalSink(table=[default_catalog.default_database.pk_upsert_snk], fields=[a, b @@ -292,8 +278,7 @@ Sink(table=[default_catalog.default_database.pk_retract_snk], fields=[a, b, c], +- ChangelogNormalize(key=[a], duplicateChanges=[DISALLOW]) +- Exchange(distribution=[hash[a]], duplicateChanges=[DISALLOW]) +- Calc(select=[a, b, c], duplicateChanges=[DISALLOW]) - +- WatermarkAssigner(rowtime=[rt], watermark=[-(rt, 1000:INTERVAL SECOND)], duplicateChanges=[DISALLOW]) - +- TableSourceScan(table=[[default_catalog, default_database, upsert_src]], fields=[a, b, c, rt], duplicateChanges=[DISALLOW]) + +- TableSourceScan(table=[[default_catalog, default_database, upsert_src]], fields=[a, b, c, rt], duplicateChanges=[DISALLOW]) ]]> @@ -310,9 +295,8 @@ LogicalSink(table=[default_catalog.default_database.pk_upsert_snk], fields=[a, b @@ -339,8 +323,7 @@ Sink(table=[default_catalog.default_database.pk_upsert_snk], fields=[a, $f1, $f2 +- Expand(projects=[{a, b, c, $f3, null AS $f4, 1 AS $e}, {a, b, c, null AS $f3, $f4, 2 AS $e}], duplicateChanges=[DISALLOW]) +- Calc(select=[a, b, c, MOD(HASH_CODE(b), 1024) AS $f3, MOD(HASH_CODE(c), 1024) AS $f4], duplicateChanges=[DISALLOW]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime], duplicateChanges=[DISALLOW]) - +- WatermarkAssigner(rowtime=[rt], watermark=[-(rt, 1000:INTERVAL SECOND)], duplicateChanges=[DISALLOW]) - +- TableSourceScan(table=[[default_catalog, default_database, append_src1]], fields=[a, b, c, rt], duplicateChanges=[DISALLOW]) + +- TableSourceScan(table=[[default_catalog, default_database, append_src1]], fields=[a, b, c, rt], duplicateChanges=[DISALLOW]) ]]> @@ -389,12 +372,10 @@ Sink(table=[default_catalog.default_database.no_pk_snk], fields=[a, b, c], dupli +- Join(joinType=[InnerJoin], where=[=(a, a0)], select=[a, c, a0, b], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey], duplicateChanges=[DISALLOW]) :- Exchange(distribution=[hash[a]], duplicateChanges=[DISALLOW]) : +- Calc(select=[a, c], duplicateChanges=[DISALLOW]) - : +- WatermarkAssigner(rowtime=[rt], watermark=[-(rt, 1000:INTERVAL SECOND)], duplicateChanges=[DISALLOW]) - : +- TableSourceScan(table=[[default_catalog, default_database, append_src1, project=[a, c, rt], metadata=[]]], fields=[a, c, rt], duplicateChanges=[DISALLOW]) + : +- TableSourceScan(table=[[default_catalog, default_database, append_src1, project=[a, c, rt], metadata=[]]], fields=[a, c, rt], duplicateChanges=[DISALLOW]) +- Exchange(distribution=[hash[a]], duplicateChanges=[DISALLOW]) +- Calc(select=[a, b], duplicateChanges=[DISALLOW]) - +- WatermarkAssigner(rowtime=[rt], watermark=[-(rt, 1000:INTERVAL SECOND)], duplicateChanges=[DISALLOW]) - +- TableSourceScan(table=[[default_catalog, default_database, append_src2, project=[a, b, rt], metadata=[]]], fields=[a, b, rt], duplicateChanges=[DISALLOW]) + +- TableSourceScan(table=[[default_catalog, default_database, append_src2, project=[a, b, rt], metadata=[]]], fields=[a, b, rt], duplicateChanges=[DISALLOW]) ]]> @@ -417,12 +398,10 @@ Sink(table=[default_catalog.default_database.pk_upsert_snk], fields=[a, b, c], c +- Join(joinType=[InnerJoin], where=[=(a, a0)], select=[a, c, a0, b], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey], duplicateChanges=[ALLOW]) :- Exchange(distribution=[hash[a]], duplicateChanges=[DISALLOW]) : +- Calc(select=[a, c], duplicateChanges=[DISALLOW]) - : +- WatermarkAssigner(rowtime=[rt], watermark=[-(rt, 1000:INTERVAL SECOND)], duplicateChanges=[DISALLOW]) - : +- TableSourceScan(table=[[default_catalog, default_database, append_src1, project=[a, c, rt], metadata=[]]], fields=[a, c, rt], duplicateChanges=[DISALLOW]) + : +- TableSourceScan(table=[[default_catalog, default_database, append_src1, project=[a, c, rt], metadata=[]]], fields=[a, c, rt], duplicateChanges=[DISALLOW]) +- Exchange(distribution=[hash[a]], duplicateChanges=[DISALLOW]) +- Calc(select=[a, b], duplicateChanges=[DISALLOW]) - +- WatermarkAssigner(rowtime=[rt], watermark=[-(rt, 1000:INTERVAL SECOND)], duplicateChanges=[DISALLOW]) - +- TableSourceScan(table=[[default_catalog, default_database, append_src2, project=[a, b, rt], metadata=[]]], fields=[a, b, rt], duplicateChanges=[DISALLOW]) + +- TableSourceScan(table=[[default_catalog, default_database, append_src2, project=[a, b, rt], metadata=[]]], fields=[a, b, rt], duplicateChanges=[DISALLOW]) ]]> @@ -445,12 +424,10 @@ Sink(table=[default_catalog.default_database.pk_upsert_snk], fields=[a, b, c], u +- Join(joinType=[InnerJoin], where=[AND(=(a, a0), =(b, b0))], select=[a, b, c, a0, b0], leftInputSpec=[JoinKeyContainsUniqueKey], rightInputSpec=[JoinKeyContainsUniqueKey], duplicateChanges=[DISALLOW]) :- Exchange(distribution=[hash[a, b]], duplicateChanges=[DISALLOW]) : +- Calc(select=[a, b, c], duplicateChanges=[DISALLOW]) - : +- WatermarkAssigner(rowtime=[rt], watermark=[-(rt, 1000:INTERVAL SECOND)], duplicateChanges=[DISALLOW]) - : +- TableSourceScan(table=[[default_catalog, default_database, retract_src]], fields=[a, b, c, rt], duplicateChanges=[DISALLOW]) + : +- TableSourceScan(table=[[default_catalog, default_database, retract_src]], fields=[a, b, c, rt], duplicateChanges=[DISALLOW]) +- Exchange(distribution=[hash[a, b]], duplicateChanges=[DISALLOW]) +- Calc(select=[a, b], duplicateChanges=[DISALLOW]) - +- WatermarkAssigner(rowtime=[rt], watermark=[-(rt, 1000:INTERVAL SECOND)], duplicateChanges=[DISALLOW]) - +- TableSourceScan(table=[[default_catalog, default_database, retract_src2, project=[a, b, rt], metadata=[]]], fields=[a, b, rt], duplicateChanges=[DISALLOW]) + +- TableSourceScan(table=[[default_catalog, default_database, retract_src2, project=[a, b, rt], metadata=[]]], fields=[a, b, rt], duplicateChanges=[DISALLOW]) ]]> @@ -470,8 +447,7 @@ Sink(table=[default_catalog.default_database.no_pk_snk], fields=[a, b, c], dupli +- Calc(select=[a, b, c], duplicateChanges=[DISALLOW]) +- Limit(offset=[0], fetch=[10], duplicateChanges=[DISALLOW]) +- Exchange(distribution=[single], duplicateChanges=[DISALLOW]) - +- WatermarkAssigner(rowtime=[rt], watermark=[-(rt, 1000:INTERVAL SECOND)], duplicateChanges=[DISALLOW]) - +- TableSourceScan(table=[[default_catalog, default_database, append_src1]], fields=[a, b, c, rt], duplicateChanges=[DISALLOW]) + +- TableSourceScan(table=[[default_catalog, default_database, append_src1]], fields=[a, b, c, rt], duplicateChanges=[DISALLOW]) ]]> @@ -491,8 +467,7 @@ Sink(table=[default_catalog.default_database.pk_upsert_snk], fields=[a, b, c], c +- Calc(select=[a, b, c], duplicateChanges=[ALLOW]) +- Limit(offset=[0], fetch=[10], duplicateChanges=[ALLOW]) +- Exchange(distribution=[single], duplicateChanges=[DISALLOW]) - +- WatermarkAssigner(rowtime=[rt], watermark=[-(rt, 1000:INTERVAL SECOND)], duplicateChanges=[DISALLOW]) - +- TableSourceScan(table=[[default_catalog, default_database, append_src1]], fields=[a, b, c, rt], duplicateChanges=[DISALLOW]) + +- TableSourceScan(table=[[default_catalog, default_database, append_src1]], fields=[a, b, c, rt], duplicateChanges=[DISALLOW]) ]]> @@ -516,8 +491,7 @@ Sink(table=[default_catalog.default_database.no_pk_snk], fields=[a, b, c], dupli +- Calc(select=[a, b, c], duplicateChanges=[DISALLOW]) +- LookupJoin(table=[default_catalog.default_database.dim_src], joinType=[InnerJoin], lookup=[a=a], select=[a, b, c, a0], duplicateChanges=[DISALLOW]) +- Calc(select=[a, b, c], duplicateChanges=[DISALLOW]) - +- WatermarkAssigner(rowtime=[rt], watermark=[-(rt, 1000:INTERVAL SECOND)], duplicateChanges=[DISALLOW]) - +- TableSourceScan(table=[[default_catalog, default_database, append_src1]], fields=[a, b, c, rt], duplicateChanges=[DISALLOW]) + +- TableSourceScan(table=[[default_catalog, default_database, append_src1]], fields=[a, b, c, rt], duplicateChanges=[DISALLOW]) ]]> @@ -541,8 +515,7 @@ Sink(table=[default_catalog.default_database.pk_upsert_snk], fields=[a, b, c], c +- Calc(select=[a, b, c], duplicateChanges=[ALLOW]) +- LookupJoin(table=[default_catalog.default_database.dim_src], joinType=[InnerJoin], lookup=[a=a], select=[a, b, c, a0], duplicateChanges=[ALLOW]) +- Calc(select=[a, b, c], duplicateChanges=[ALLOW]) - +- WatermarkAssigner(rowtime=[rt], watermark=[-(rt, 1000:INTERVAL SECOND)], duplicateChanges=[ALLOW]) - +- TableSourceScan(table=[[default_catalog, default_database, append_src1]], fields=[a, b, c, rt], duplicateChanges=[ALLOW]) + +- TableSourceScan(table=[[default_catalog, default_database, append_src1]], fields=[a, b, c, rt], duplicateChanges=[ALLOW]) ]]> @@ -564,8 +537,7 @@ Sink(table=[default_catalog.default_database.pk_upsert_snk], fields=[a, EXPR$1, +- LocalGroupAggregate(groupBy=[a], select=[a, MAX(b) AS max$0, SUM(c) AS sum$1], duplicateChanges=[DISALLOW]) +- Calc(select=[a, b, c], duplicateChanges=[DISALLOW]) +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime], duplicateChanges=[DISALLOW]) - +- WatermarkAssigner(rowtime=[rt], watermark=[-(rt, 1000:INTERVAL SECOND)], duplicateChanges=[DISALLOW]) - +- TableSourceScan(table=[[default_catalog, default_database, append_src1]], fields=[a, b, c, rt], duplicateChanges=[DISALLOW]) + +- TableSourceScan(table=[[default_catalog, default_database, append_src1]], fields=[a, b, c, rt], duplicateChanges=[DISALLOW]) ]]> @@ -590,15 +562,13 @@ LogicalSink(table=[default_catalog.default_database.pk_upsert_snk2], fields=[a, Sink(table=[default_catalog.default_database.pk_upsert_snk], fields=[a, b, EXPR$2], conflictStrategy=[DEDUPLICATE], duplicateChanges=[NONE]) +- Calc(select=[a, b, /(c, 2) AS EXPR$2], duplicateChanges=[ALLOW]) +- Calc(select=[a, b, +(c, 1) AS c], duplicateChanges=[DISALLOW]) - +- WatermarkAssigner(rowtime=[rt], watermark=[-(rt, 1000:INTERVAL SECOND)], duplicateChanges=[DISALLOW]) - +- TableSourceScan(table=[[default_catalog, default_database, append_src1]], fields=[a, b, c, rt], duplicateChanges=[DISALLOW]) + +- TableSourceScan(table=[[default_catalog, default_database, append_src1]], fields=[a, b, c, rt], duplicateChanges=[DISALLOW]) Sink(table=[default_catalog.default_database.pk_upsert_snk2], fields=[a, EXPR$1, EXPR$2], conflictStrategy=[DEDUPLICATE], duplicateChanges=[NONE]) +- GroupAggregate(groupBy=[a], select=[a, MAX(b) AS EXPR$1, SUM(c) AS EXPR$2], duplicateChanges=[ALLOW]) +- Exchange(distribution=[hash[a]], duplicateChanges=[DISALLOW]) +- Calc(select=[a, b, +(c, 1) AS c], duplicateChanges=[DISALLOW]) - +- WatermarkAssigner(rowtime=[rt], watermark=[-(rt, 1000:INTERVAL SECOND)], duplicateChanges=[DISALLOW]) - +- TableSourceScan(table=[[default_catalog, default_database, append_src1]], fields=[a, b, c, rt], duplicateChanges=[DISALLOW]) + +- TableSourceScan(table=[[default_catalog, default_database, append_src1]], fields=[a, b, c, rt], duplicateChanges=[DISALLOW]) ]]> @@ -624,14 +594,12 @@ Sink(table=[default_catalog.default_database.pk_upsert_snk], fields=[a, EXPR$1, +- GroupAggregate(groupBy=[a], select=[a, MAX(b) AS EXPR$1, SUM(c) AS EXPR$2], duplicateChanges=[ALLOW]) +- Exchange(distribution=[hash[a]], duplicateChanges=[DISALLOW]) +- Calc(select=[a, b, +(c, 1) AS c], duplicateChanges=[DISALLOW]) - +- WatermarkAssigner(rowtime=[rt], watermark=[-(rt, 1000:INTERVAL SECOND)], duplicateChanges=[DISALLOW]) - +- TableSourceScan(table=[[default_catalog, default_database, append_src1]], fields=[a, b, c, rt], duplicateChanges=[DISALLOW]) + +- TableSourceScan(table=[[default_catalog, default_database, append_src1]], fields=[a, b, c, rt], duplicateChanges=[DISALLOW]) Sink(table=[default_catalog.default_database.pk_upsert_snk2], fields=[a, b, EXPR$2], conflictStrategy=[DEDUPLICATE], duplicateChanges=[NONE]) +- Calc(select=[a, b, /(c, 2) AS EXPR$2], duplicateChanges=[ALLOW]) +- Calc(select=[a, b, +(c, 1) AS c], duplicateChanges=[DISALLOW]) - +- WatermarkAssigner(rowtime=[rt], watermark=[-(rt, 1000:INTERVAL SECOND)], duplicateChanges=[DISALLOW]) - +- TableSourceScan(table=[[default_catalog, default_database, append_src1]], fields=[a, b, c, rt], duplicateChanges=[DISALLOW]) + +- TableSourceScan(table=[[default_catalog, default_database, append_src1]], fields=[a, b, c, rt], duplicateChanges=[DISALLOW]) ]]> @@ -656,14 +624,12 @@ LogicalSink(table=[default_catalog.default_database.pk_upsert_snk2], fields=[a, Sink(table=[default_catalog.default_database.pk_upsert_snk], fields=[a, b, EXPR$2], conflictStrategy=[DEDUPLICATE], duplicateChanges=[NONE]) +- Calc(select=[a, b, /(c, 3) AS EXPR$2], duplicateChanges=[ALLOW]) +- Calc(select=[a, b, +(c, 1) AS c], duplicateChanges=[ALLOW]) - +- WatermarkAssigner(rowtime=[rt], watermark=[-(rt, 1000:INTERVAL SECOND)], duplicateChanges=[ALLOW]) - +- TableSourceScan(table=[[default_catalog, default_database, append_src1]], fields=[a, b, c, rt], duplicateChanges=[ALLOW]) + +- TableSourceScan(table=[[default_catalog, default_database, append_src1]], fields=[a, b, c, rt], duplicateChanges=[ALLOW]) Sink(table=[default_catalog.default_database.pk_upsert_snk2], fields=[a, b, EXPR$2], conflictStrategy=[DEDUPLICATE], duplicateChanges=[NONE]) +- Calc(select=[a, b, /(c, 2) AS EXPR$2], duplicateChanges=[ALLOW]) +- Calc(select=[a, b, +(c, 1) AS c], duplicateChanges=[ALLOW]) - +- WatermarkAssigner(rowtime=[rt], watermark=[-(rt, 1000:INTERVAL SECOND)], duplicateChanges=[ALLOW]) - +- TableSourceScan(table=[[default_catalog, default_database, append_src1]], fields=[a, b, c, rt], duplicateChanges=[ALLOW]) + +- TableSourceScan(table=[[default_catalog, default_database, append_src1]], fields=[a, b, c, rt], duplicateChanges=[ALLOW]) ]]> @@ -689,15 +655,13 @@ Sink(table=[default_catalog.default_database.pk_upsert_snk], fields=[a, EXPR$1, +- GroupAggregate(groupBy=[a], select=[a, MAX(b) AS EXPR$1, SUM(c) AS EXPR$2], duplicateChanges=[ALLOW]) +- Exchange(distribution=[hash[a]], duplicateChanges=[DISALLOW]) +- Calc(select=[a, b, +(c, 1) AS c], duplicateChanges=[DISALLOW]) - +- WatermarkAssigner(rowtime=[rt], watermark=[-(rt, 1000:INTERVAL SECOND)], duplicateChanges=[DISALLOW]) - +- TableSourceScan(table=[[default_catalog, default_database, append_src1]], fields=[a, b, c, rt], duplicateChanges=[DISALLOW]) + +- TableSourceScan(table=[[default_catalog, default_database, append_src1]], fields=[a, b, c, rt], duplicateChanges=[DISALLOW]) Sink(table=[default_catalog.default_database.pk_upsert_snk2], fields=[a, EXPR$1, EXPR$2], duplicateChanges=[NONE]) +- GroupAggregate(groupBy=[a], select=[a, MIN(b) AS EXPR$1, MAX(c) AS EXPR$2], duplicateChanges=[ALLOW]) +- Exchange(distribution=[hash[a]], duplicateChanges=[DISALLOW]) +- Calc(select=[a, b, +(c, 1) AS c], duplicateChanges=[DISALLOW]) - +- WatermarkAssigner(rowtime=[rt], watermark=[-(rt, 1000:INTERVAL SECOND)], duplicateChanges=[DISALLOW]) - +- TableSourceScan(table=[[default_catalog, default_database, append_src1]], fields=[a, b, c, rt], duplicateChanges=[DISALLOW]) + +- TableSourceScan(table=[[default_catalog, default_database, append_src1]], fields=[a, b, c, rt], duplicateChanges=[DISALLOW]) ]]> @@ -766,8 +730,7 @@ Sink(table=[default_catalog.default_database.pk_upsert_snk], fields=[a, b, c], d +- Rank(strategy=[AppendFastStrategy], rankType=[ROW_NUMBER], rankRange=[rankStart=1, rankEnd=1], partitionBy=[a], orderBy=[c DESC], select=[a, b, c], duplicateChanges=[ALLOW]) +- Exchange(distribution=[hash[a]], duplicateChanges=[DISALLOW]) +- Calc(select=[a, b, c], duplicateChanges=[DISALLOW]) - +- WatermarkAssigner(rowtime=[rt], watermark=[-(rt, 1000:INTERVAL SECOND)], duplicateChanges=[DISALLOW]) - +- TableSourceScan(table=[[default_catalog, default_database, append_src1]], fields=[a, b, c, rt], duplicateChanges=[DISALLOW]) + +- TableSourceScan(table=[[default_catalog, default_database, append_src1]], fields=[a, b, c, rt], duplicateChanges=[DISALLOW]) ]]> @@ -784,8 +747,7 @@ LogicalSink(table=[default_catalog.default_database.pk_retract_snk], fields=[a, @@ -802,8 +764,7 @@ LogicalSink(table=[default_catalog.default_database.another_pk_upsert_snk], fiel @@ -829,12 +790,10 @@ Sink(table=[default_catalog.default_database.no_pk_snk], fields=[a, b, c], dupli +- TemporalJoin(joinType=[InnerJoin], where=[AND(=(a, a0), __TEMPORAL_JOIN_CONDITION(pt, __TEMPORAL_JOIN_CONDITION_PRIMARY_KEY(a0), __TEMPORAL_JOIN_LEFT_KEY(a), __TEMPORAL_JOIN_RIGHT_KEY(a0)))], select=[a, b, c, pt, a0], duplicateChanges=[DISALLOW]) :- Exchange(distribution=[hash[a]], duplicateChanges=[DISALLOW]) : +- Calc(select=[a, b, c, PROCTIME() AS pt], duplicateChanges=[DISALLOW]) - : +- WatermarkAssigner(rowtime=[rt], watermark=[-(rt, 1000:INTERVAL SECOND)], duplicateChanges=[DISALLOW]) - : +- TableSourceScan(table=[[default_catalog, default_database, append_src1]], fields=[a, b, c, rt], duplicateChanges=[DISALLOW]) + : +- TableSourceScan(table=[[default_catalog, default_database, append_src1]], fields=[a, b, c, rt], duplicateChanges=[DISALLOW]) +- Exchange(distribution=[hash[a]], duplicateChanges=[DISALLOW]) +- Calc(select=[a], duplicateChanges=[DISALLOW]) - +- WatermarkAssigner(rowtime=[rt], watermark=[-(rt, 1000:INTERVAL SECOND)], duplicateChanges=[DISALLOW]) - +- TableSourceScan(table=[[default_catalog, default_database, dim_src, project=[a, rt], metadata=[]]], fields=[a, rt], duplicateChanges=[DISALLOW]) + +- TableSourceScan(table=[[default_catalog, default_database, dim_src, project=[a, rt], metadata=[]]], fields=[a, rt], duplicateChanges=[DISALLOW]) ]]> @@ -860,12 +819,10 @@ Sink(table=[default_catalog.default_database.pk_upsert_snk], fields=[a, b, c], c +- TemporalJoin(joinType=[InnerJoin], where=[AND(=(a, a0), __TEMPORAL_JOIN_CONDITION(pt, __TEMPORAL_JOIN_CONDITION_PRIMARY_KEY(a0), __TEMPORAL_JOIN_LEFT_KEY(a), __TEMPORAL_JOIN_RIGHT_KEY(a0)))], select=[a, b, c, pt, a0], duplicateChanges=[ALLOW]) :- Exchange(distribution=[hash[a]], duplicateChanges=[DISALLOW]) : +- Calc(select=[a, b, c, PROCTIME() AS pt], duplicateChanges=[DISALLOW]) - : +- WatermarkAssigner(rowtime=[rt], watermark=[-(rt, 1000:INTERVAL SECOND)], duplicateChanges=[DISALLOW]) - : +- TableSourceScan(table=[[default_catalog, default_database, append_src1]], fields=[a, b, c, rt], duplicateChanges=[DISALLOW]) + : +- TableSourceScan(table=[[default_catalog, default_database, append_src1]], fields=[a, b, c, rt], duplicateChanges=[DISALLOW]) +- Exchange(distribution=[hash[a]], duplicateChanges=[DISALLOW]) +- Calc(select=[a], duplicateChanges=[DISALLOW]) - +- WatermarkAssigner(rowtime=[rt], watermark=[-(rt, 1000:INTERVAL SECOND)], duplicateChanges=[DISALLOW]) - +- TableSourceScan(table=[[default_catalog, default_database, dim_src, project=[a, rt], metadata=[]]], fields=[a, rt], duplicateChanges=[DISALLOW]) + +- TableSourceScan(table=[[default_catalog, default_database, dim_src, project=[a, rt], metadata=[]]], fields=[a, rt], duplicateChanges=[DISALLOW]) ]]> @@ -937,11 +894,9 @@ LogicalSink(table=[default_catalog.default_database.no_pk_snk], fields=[a, b, c] Sink(table=[default_catalog.default_database.no_pk_snk], fields=[a, b, c], duplicateChanges=[NONE]) +- Union(all=[true], union=[a, b, c], duplicateChanges=[DISALLOW]) :- Calc(select=[a, b, c], duplicateChanges=[DISALLOW]) - : +- WatermarkAssigner(rowtime=[rt], watermark=[-(rt, 1000:INTERVAL SECOND)], duplicateChanges=[DISALLOW]) - : +- TableSourceScan(table=[[default_catalog, default_database, append_src1]], fields=[a, b, c, rt], duplicateChanges=[DISALLOW]) + : +- TableSourceScan(table=[[default_catalog, default_database, append_src1]], fields=[a, b, c, rt], duplicateChanges=[DISALLOW]) +- Calc(select=[a, b, c], duplicateChanges=[DISALLOW]) - +- WatermarkAssigner(rowtime=[rt], watermark=[-(rt, 1000:INTERVAL SECOND)], duplicateChanges=[DISALLOW]) - +- TableSourceScan(table=[[default_catalog, default_database, append_src2]], fields=[a, b, c, rt], duplicateChanges=[DISALLOW]) + +- TableSourceScan(table=[[default_catalog, default_database, append_src2]], fields=[a, b, c, rt], duplicateChanges=[DISALLOW]) ]]> @@ -963,11 +918,9 @@ LogicalSink(table=[default_catalog.default_database.pk_upsert_snk], fields=[a, b Sink(table=[default_catalog.default_database.pk_upsert_snk], fields=[a, b, c], conflictStrategy=[DEDUPLICATE], duplicateChanges=[NONE]) +- Union(all=[true], union=[a, b, c], duplicateChanges=[ALLOW]) :- Calc(select=[a, b, c], duplicateChanges=[DISALLOW]) - : +- WatermarkAssigner(rowtime=[rt], watermark=[-(rt, 1000:INTERVAL SECOND)], duplicateChanges=[DISALLOW]) - : +- TableSourceScan(table=[[default_catalog, default_database, append_src1]], fields=[a, b, c, rt], duplicateChanges=[DISALLOW]) + : +- TableSourceScan(table=[[default_catalog, default_database, append_src1]], fields=[a, b, c, rt], duplicateChanges=[DISALLOW]) +- Calc(select=[a, b, c], duplicateChanges=[DISALLOW]) - +- WatermarkAssigner(rowtime=[rt], watermark=[-(rt, 1000:INTERVAL SECOND)], duplicateChanges=[DISALLOW]) - +- TableSourceScan(table=[[default_catalog, default_database, append_src2]], fields=[a, b, c, rt], duplicateChanges=[DISALLOW]) + +- TableSourceScan(table=[[default_catalog, default_database, append_src2]], fields=[a, b, c, rt], duplicateChanges=[DISALLOW]) ]]> @@ -990,14 +943,12 @@ Sink(table=[default_catalog.default_database.pk_upsert_snk], fields=[a, b, c], c +- Join(joinType=[InnerJoin], where=[=(a, a0)], select=[a, c, a0, b], leftInputSpec=[JoinKeyContainsUniqueKey], rightInputSpec=[JoinKeyContainsUniqueKey], duplicateChanges=[ALLOW]) :- Exchange(distribution=[hash[a]], duplicateChanges=[ALLOW]) : +- Calc(select=[a, c], duplicateChanges=[ALLOW]) - : +- WatermarkAssigner(rowtime=[rt], watermark=[-(rt, 1000:INTERVAL SECOND)], duplicateChanges=[ALLOW]) - : +- DropUpdateBefore(duplicateChanges=[ALLOW]) - : +- TableSourceScan(table=[[default_catalog, default_database, retract_src, project=[a, c, rt], metadata=[]]], fields=[a, c, rt], duplicateChanges=[ALLOW]) + : +- DropUpdateBefore(duplicateChanges=[ALLOW]) + : +- TableSourceScan(table=[[default_catalog, default_database, retract_src, project=[a, c, rt], metadata=[]]], fields=[a, c, rt], duplicateChanges=[ALLOW]) +- Exchange(distribution=[hash[a]], duplicateChanges=[ALLOW]) +- Calc(select=[a, b], duplicateChanges=[ALLOW]) - +- WatermarkAssigner(rowtime=[rt], watermark=[-(rt, 1000:INTERVAL SECOND)], duplicateChanges=[ALLOW]) - +- DropUpdateBefore(duplicateChanges=[ALLOW]) - +- TableSourceScan(table=[[default_catalog, default_database, retract_src2, project=[a, b, rt], metadata=[]]], fields=[a, b, rt], duplicateChanges=[ALLOW]) + +- DropUpdateBefore(duplicateChanges=[ALLOW]) + +- TableSourceScan(table=[[default_catalog, default_database, retract_src2, project=[a, b, rt], metadata=[]]], fields=[a, b, rt], duplicateChanges=[ALLOW]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/stream/RedundantWatermarkAssignerRemoveRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/stream/RedundantWatermarkAssignerRemoveRuleTest.xml new file mode 100644 index 0000000000000..b3d9b14a99409 --- /dev/null +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/stream/RedundantWatermarkAssignerRemoveRuleTest.xml @@ -0,0 +1,378 @@ + + + + + + 0]]> + + + ($0, 0)]) + +- LogicalWatermarkAssigner(rowtime=[rt], watermark=[-($3, 1000:INTERVAL SECOND)]) + +- LogicalTableScan(table=[[default_catalog, default_database, src]]) +]]> + + + (a, 0)]) + +- TableSourceScan(table=[[default_catalog, default_database, src]], fields=[a, b, c, rt]) +]]> + + + + + CURRENT_WATERMARK(rt)]]> + + + ($3, CURRENT_WATERMARK($3)))]) + +- LogicalWatermarkAssigner(rowtime=[rt], watermark=[-($3, 1000:INTERVAL SECOND)]) + +- LogicalTableScan(table=[[default_catalog, default_database, src]]) +]]> + + + (CAST(rt AS TIMESTAMP(3)), CURRENT_WATERMARK(CAST(rt AS TIMESTAMP(3)))))]) + +- WatermarkAssigner(rowtime=[rt], watermark=[-(rt, 1000:INTERVAL SECOND)]) + +- TableSourceScan(table=[[default_catalog, default_database, src]], fields=[a, b, c, rt]) +]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + 0]]> + + + ($0, 0)]) + +- LogicalWatermarkAssigner(rowtime=[rt], watermark=[-($3, 1000:INTERVAL SECOND)]) + +- LogicalTableScan(table=[[default_catalog, default_database, src]]) +]]> + + + (a, 0)]) + +- WatermarkAssigner(rowtime=[rt], watermark=[-(rt, 1000:INTERVAL SECOND)]) + +- TableSourceScan(table=[[default_catalog, default_database, src]], fields=[a, b, c, rt]) +]]> + + + + + + + + =($3, -($7, 5000:INTERVAL SECOND)), <=($3, +($7, 5000:INTERVAL SECOND)))], joinType=[inner]) + :- LogicalWatermarkAssigner(rowtime=[rt], watermark=[-($3, 1000:INTERVAL SECOND)]) + : +- LogicalTableScan(table=[[default_catalog, default_database, src]]) + +- LogicalWatermarkAssigner(rowtime=[rt], watermark=[-($3, 1000:INTERVAL SECOND)]) + +- LogicalTableScan(table=[[default_catalog, default_database, src2]]) +]]> + + + =(rt, -(rt0, 5000:INTERVAL SECOND)), <=(rt, +(rt0, 5000:INTERVAL SECOND)))], select=[a, b, c, rt, a0, rt0]) + :- Exchange(distribution=[hash[a]]) + : +- WatermarkAssigner(rowtime=[rt], watermark=[-(rt, 1000:INTERVAL SECOND)]) + : +- TableSourceScan(table=[[default_catalog, default_database, src]], fields=[a, b, c, rt]) + +- Exchange(distribution=[hash[a]]) + +- WatermarkAssigner(rowtime=[rt], watermark=[-(rt, 1000:INTERVAL SECOND)]) + +- TableSourceScan(table=[[default_catalog, default_database, src2, project=[a, rt], metadata=[]]], fields=[a, rt]) +]]> + + + + + 0]]> + + + ($0, 0)]) + +- LogicalWatermarkAssigner(rowtime=[rt], watermark=[-($3, 1000:INTERVAL SECOND)]) + +- LogicalTableScan(table=[[default_catalog, default_database, src]]) +]]> + + + (a, 0)]) + +- WatermarkAssigner(rowtime=[rt], watermark=[-(rt, 1000:INTERVAL SECOND)]) + +- TableSourceScan(table=[[default_catalog, default_database, src]], fields=[a, b, c, rt]) +]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/DeltaJoinTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/DeltaJoinTest.xml index 7c04bf6c2aaa6..83af493403d3e 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/DeltaJoinTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/DeltaJoinTest.xml @@ -2559,9 +2559,8 @@ LogicalSink(table=[default_catalog.default_database.tmp_sink], fields=[a0, a1, a Sink(table=[default_catalog.default_database.tmp_sink], fields=[a0, a1, a2, b0, b2, b1]) +- Join(joinType=[InnerJoin], where=[=(a1, b2)], select=[a0, a1, a2, b0, b2, b1], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[hash[a1]]) - : +- Calc(select=[a0, a1, CAST(a2 AS TIMESTAMP(3)) AS a2]) - : +- WatermarkAssigner(rowtime=[a2], watermark=[-(a2, 1000:INTERVAL SECOND)]) - : +- TableSourceScan(table=[[default_catalog, default_database, wm_source]], fields=[a0, a1, a2]) + : +- Calc(select=[a0, a1, a2]) + : +- TableSourceScan(table=[[default_catalog, default_database, wm_source]], fields=[a0, a1, a2]) +- Exchange(distribution=[hash[b2]]) +- TableSourceScan(table=[[default_catalog, default_database, src2]], fields=[b0, b2, b1]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/MiniBatchIntervalInferTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/MiniBatchIntervalInferTest.xml index ab37dca03455e..311e3adb01f14 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/MiniBatchIntervalInferTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/MiniBatchIntervalInferTest.xml @@ -479,27 +479,30 @@ GlobalGroupAggregate(groupBy=[cnt], select=[cnt, COUNT(count$0) AS EXPR$1]) - + diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/MultiJoinTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/MultiJoinTest.xml index 3a5e888ca25c4..e45c452b2bd09 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/MultiJoinTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/MultiJoinTest.xml @@ -2275,9 +2275,8 @@ Calc(select=[name, proctime, $rowtime, price]) : +- Calc(select=[user_id, name, PROCTIME_MATERIALIZE(PROCTIME()) AS proctime]) : +- TableSourceScan(table=[[default_catalog, default_database, UsersWithProctime]], fields=[user_id, name]) :- Exchange(distribution=[hash[user_id]]) - : +- Calc(select=[user_id, CAST($rowtime AS TIMESTAMP(3)) AS $rowtime]) - : +- WatermarkAssigner(rowtime=[$rowtime], watermark=[$rowtime]) - : +- TableSourceScan(table=[[default_catalog, default_database, OrdersWithRowtime, project=[user_id, $rowtime], metadata=[]]], fields=[user_id, $rowtime]) + : +- Calc(select=[user_id, $rowtime]) + : +- TableSourceScan(table=[[default_catalog, default_database, OrdersWithRowtime, project=[user_id, $rowtime], metadata=[]]], fields=[user_id, $rowtime]) +- Exchange(distribution=[hash[user_id]]) +- TableSourceScan(table=[[default_catalog, default_database, Payments, project=[price, user_id], metadata=[]]], fields=[price, user_id]) ]]> @@ -2559,13 +2558,11 @@ Rank(strategy=[AppendFastStrategy], rankType=[ROW_NUMBER], rankRange=[rankStart= +- Calc(select=[id, val, $rowtime]) +- MultiJoin(commonJoinKey=[id], joinTypes=[INNER], inputUniqueKeys=[noUniqueKey, noUniqueKey], joinConditions=[=(id, id0)], select=[id,val,$rowtime,id0,price,$rowtime0], rowType=[RecordType(VARCHAR(2147483647) id, INTEGER val, TIMESTAMP(3) $rowtime, VARCHAR(2147483647) id0, DOUBLE price, TIMESTAMP(3) $rowtime0)]) :- Exchange(distribution=[hash[id]]) - : +- Calc(select=[id, val, CAST($rowtime AS TIMESTAMP(3)) AS $rowtime]) - : +- WatermarkAssigner(rowtime=[$rowtime], watermark=[-($rowtime, 5000:INTERVAL SECOND)]) - : +- TableSourceScan(table=[[default_catalog, default_database, EventTable1]], fields=[id, val, $rowtime]) + : +- Calc(select=[id, val, $rowtime]) + : +- TableSourceScan(table=[[default_catalog, default_database, EventTable1]], fields=[id, val, $rowtime]) +- Exchange(distribution=[hash[id]]) - +- Calc(select=[id, price, CAST($rowtime AS TIMESTAMP(3)) AS $rowtime]) - +- WatermarkAssigner(rowtime=[$rowtime], watermark=[-($rowtime, 5000:INTERVAL SECOND)]) - +- TableSourceScan(table=[[default_catalog, default_database, EventTable2]], fields=[id, price, $rowtime]) + +- Calc(select=[id, price, $rowtime]) + +- TableSourceScan(table=[[default_catalog, default_database, EventTable2]], fields=[id, price, $rowtime]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/NonDeterministicDagTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/NonDeterministicDagTest.xml index 568139a28ece4..8d1de30951c28 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/NonDeterministicDagTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/NonDeterministicDagTest.xml @@ -1398,17 +1398,15 @@ Sink(table=[default_catalog.default_database.sink_with_pk], fields=[a, b, c]) +- Calc(select=[a0 AS a, b, c]) +- Join(joinType=[InnerJoin], where=[((a = a0) AND (op_ts > (op_ts0 - 5000:INTERVAL SECOND)))], select=[a, b, op_ts, a0, c, op_ts0], leftInputSpec=[JoinKeyContainsUniqueKey], rightInputSpec=[JoinKeyContainsUniqueKey]) :- Exchange(distribution=[hash[a]]) - : +- Calc(select=[a, b, CAST(op_ts AS TIMESTAMP_WITH_LOCAL_TIME_ZONE(3)) AS op_ts]) - : +- WatermarkAssigner(rowtime=[op_ts], watermark=[(op_ts - 5000:INTERVAL SECOND)]) - : +- DropUpdateBefore - : +- Calc(select=[a, b, op_ts]) - : +- TableSourceScan(table=[[default_catalog, default_database, cdc_with_watermark, project=[a, b, c, op_ts], metadata=[]]], fields=[a, b, c, op_ts])(reuse_id=[1]) + : +- Calc(select=[a, b, op_ts]) + : +- DropUpdateBefore + : +- Calc(select=[a, b, op_ts]) + : +- TableSourceScan(table=[[default_catalog, default_database, cdc_with_watermark, project=[a, b, c, op_ts], metadata=[]]], fields=[a, b, c, op_ts])(reuse_id=[1]) +- Exchange(distribution=[hash[a]]) - +- Calc(select=[a, c, CAST(op_ts AS TIMESTAMP_WITH_LOCAL_TIME_ZONE(3)) AS op_ts]) - +- WatermarkAssigner(rowtime=[op_ts], watermark=[(op_ts - 5000:INTERVAL SECOND)]) - +- DropUpdateBefore - +- Calc(select=[a, c, op_ts]) - +- Reused(reference_id=[1]) + +- Calc(select=[a, c, op_ts]) + +- DropUpdateBefore + +- Calc(select=[a, c, op_ts]) + +- Reused(reference_id=[1]) ]]> @@ -1437,17 +1435,15 @@ Sink(table=[default_catalog.default_database.sink_with_pk], fields=[a, b, c]) +- Calc(select=[a0 AS a, b, c]) +- Join(joinType=[InnerJoin], where=[((a = a0) AND (op_ts > (op_ts0 - 5000:INTERVAL SECOND)))], select=[a, b, op_ts, a0, c, op_ts0], leftInputSpec=[JoinKeyContainsUniqueKey], rightInputSpec=[JoinKeyContainsUniqueKey]) :- Exchange(distribution=[hash[a]]) - : +- Calc(select=[a, b, CAST(op_ts AS TIMESTAMP_WITH_LOCAL_TIME_ZONE(3)) AS op_ts]) - : +- WatermarkAssigner(rowtime=[op_ts], watermark=[(op_ts - 5000:INTERVAL SECOND)]) - : +- DropUpdateBefore - : +- Calc(select=[a, b, op_ts]) - : +- TableSourceScan(table=[[default_catalog, default_database, cdc_with_watermark, project=[a, b, c, op_ts], metadata=[]]], fields=[a, b, c, op_ts])(reuse_id=[1]) + : +- Calc(select=[a, b, op_ts]) + : +- DropUpdateBefore + : +- Calc(select=[a, b, op_ts]) + : +- TableSourceScan(table=[[default_catalog, default_database, cdc_with_watermark, project=[a, b, c, op_ts], metadata=[]]], fields=[a, b, c, op_ts])(reuse_id=[1]) +- Exchange(distribution=[hash[a]]) - +- Calc(select=[a, c, CAST(op_ts AS TIMESTAMP_WITH_LOCAL_TIME_ZONE(3)) AS op_ts]) - +- WatermarkAssigner(rowtime=[op_ts], watermark=[(op_ts - 5000:INTERVAL SECOND)]) - +- DropUpdateBefore - +- Calc(select=[a, c, op_ts]) - +- Reused(reference_id=[1]) + +- Calc(select=[a, c, op_ts]) + +- DropUpdateBefore + +- Calc(select=[a, c, op_ts]) + +- Reused(reference_id=[1]) ]]> @@ -1476,15 +1472,11 @@ Sink(table=[default_catalog.default_database.sink_without_pk], fields=[a, b, c]) +- Calc(select=[a0 AS a, b, c]) +- Join(joinType=[InnerJoin], where=[((a = a0) AND (op_ts > (op_ts0 - 5000:INTERVAL SECOND)))], select=[a, b, op_ts, a0, c, op_ts0], leftInputSpec=[JoinKeyContainsUniqueKey], rightInputSpec=[JoinKeyContainsUniqueKey]) :- Exchange(distribution=[hash[a]]) - : +- Calc(select=[a, b, CAST(op_ts AS TIMESTAMP_WITH_LOCAL_TIME_ZONE(3)) AS op_ts]) - : +- WatermarkAssigner(rowtime=[op_ts], watermark=[(op_ts - 5000:INTERVAL SECOND)]) - : +- Calc(select=[a, b, op_ts]) - : +- TableSourceScan(table=[[default_catalog, default_database, cdc_with_watermark, project=[a, b, c, op_ts], metadata=[]]], fields=[a, b, c, op_ts])(reuse_id=[1]) + : +- Calc(select=[a, b, op_ts]) + : +- TableSourceScan(table=[[default_catalog, default_database, cdc_with_watermark, project=[a, b, c, op_ts], metadata=[]]], fields=[a, b, c, op_ts])(reuse_id=[1]) +- Exchange(distribution=[hash[a]]) - +- Calc(select=[a, c, CAST(op_ts AS TIMESTAMP_WITH_LOCAL_TIME_ZONE(3)) AS op_ts]) - +- WatermarkAssigner(rowtime=[op_ts], watermark=[(op_ts - 5000:INTERVAL SECOND)]) - +- Calc(select=[a, c, op_ts]) - +- Reused(reference_id=[1]) + +- Calc(select=[a, c, op_ts]) + +- Reused(reference_id=[1]) ]]> @@ -1513,15 +1505,11 @@ Sink(table=[default_catalog.default_database.sink_without_pk], fields=[a, b, c]) +- Calc(select=[a0 AS a, b, c]) +- Join(joinType=[InnerJoin], where=[((a = a0) AND (op_ts > (op_ts0 - 5000:INTERVAL SECOND)))], select=[a, b, op_ts, a0, c, op_ts0], leftInputSpec=[JoinKeyContainsUniqueKey], rightInputSpec=[JoinKeyContainsUniqueKey]) :- Exchange(distribution=[hash[a]]) - : +- Calc(select=[a, b, CAST(op_ts AS TIMESTAMP_WITH_LOCAL_TIME_ZONE(3)) AS op_ts]) - : +- WatermarkAssigner(rowtime=[op_ts], watermark=[(op_ts - 5000:INTERVAL SECOND)]) - : +- Calc(select=[a, b, op_ts]) - : +- TableSourceScan(table=[[default_catalog, default_database, cdc_with_watermark, project=[a, b, c, op_ts], metadata=[]]], fields=[a, b, c, op_ts])(reuse_id=[1]) + : +- Calc(select=[a, b, op_ts]) + : +- TableSourceScan(table=[[default_catalog, default_database, cdc_with_watermark, project=[a, b, c, op_ts], metadata=[]]], fields=[a, b, c, op_ts])(reuse_id=[1]) +- Exchange(distribution=[hash[a]]) - +- Calc(select=[a, c, CAST(op_ts AS TIMESTAMP_WITH_LOCAL_TIME_ZONE(3)) AS op_ts]) - +- WatermarkAssigner(rowtime=[op_ts], watermark=[(op_ts - 5000:INTERVAL SECOND)]) - +- Calc(select=[a, c, op_ts]) - +- Reused(reference_id=[1]) + +- Calc(select=[a, c, op_ts]) + +- Reused(reference_id=[1]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.xml index adf7dd0d61293..738ecca5f17f9 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.xml @@ -94,14 +94,11 @@ LogicalProject(a=[$1], b=[$2], c=[$3]) (a, 1)], changelogMode=[I,UA,D]) +- Exchange(distribution=[hash[id]], changelogMode=[I,UA,D]) - +- Calc(select=[id, a, b, c], changelogMode=[I,UA,D]) - +- WatermarkAssigner(rowtime=[ts], watermark=[-(ts, 1000:INTERVAL SECOND)], changelogMode=[I,UA,D]) - +- Calc(select=[id, a, +(a, 1) AS b, c, TO_TIMESTAMP(c) AS ts], changelogMode=[I,UA,D]) - +- DropUpdateBefore(changelogMode=[I,UA,D]) - +- TableSourceScan(table=[[default_catalog, default_database, src]], fields=[id, a, c], changelogMode=[I,UB,UA,D]) + +- DropUpdateBefore(changelogMode=[I,UA,D]) + +- TableSourceScan(table=[[default_catalog, default_database, src]], fields=[id, a, c], changelogMode=[I,UB,UA,D]) ]]> @@ -717,13 +714,10 @@ LogicalProject(a=[$1], b=[$2], c=[$3]) (a, 1)], changelogMode=[I,UA,D]) +- Exchange(distribution=[hash[id]], changelogMode=[UA,D]) - +- Calc(select=[id, a, b, c], changelogMode=[UA,D]) - +- WatermarkAssigner(rowtime=[ts], watermark=[-(ts, 1000:INTERVAL SECOND)], changelogMode=[UA,D]) - +- Calc(select=[id, a, +(a, 1) AS b, c, TO_TIMESTAMP(c) AS ts], changelogMode=[UA,D]) - +- TableSourceScan(table=[[default_catalog, default_database, src]], fields=[id, a, c], changelogMode=[UA,D]) + +- TableSourceScan(table=[[default_catalog, default_database, src]], fields=[id, a, c], changelogMode=[UA,D]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableSourceTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableSourceTest.xml index 2fad4c62450fb..c21cc1091f076 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableSourceTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableSourceTest.xml @@ -179,10 +179,8 @@ LogicalProject(ptime=[$4], name=[$3], val=[$2], id=[$0]) diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/UnionTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/UnionTest.xml index 2969cdc7486ee..5ec92a3e4e391 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/UnionTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/UnionTest.xml @@ -113,10 +113,8 @@ Union(all=[true], union=[id, ts, name, timestamp_col]), rowType=[RecordType(INTE :- Calc(select=[id, ts, CAST(name AS VARCHAR(2147483647)) AS name, timestamp_col]), rowType=[RecordType(INTEGER id, BIGINT ts, VARCHAR(2147483647) name, TIMESTAMP(3) *ROWTIME* timestamp_col)] : +- WatermarkAssigner(rowtime=[timestamp_col], watermark=[timestamp_col]), rowType=[RecordType(INTEGER id, BIGINT ts, VARCHAR(32) name, TIMESTAMP(3) *ROWTIME* timestamp_col)] : +- TableSourceScan(table=[[default_catalog, default_database, t1, project=[id, ts, name, timestamp_col], metadata=[]]], fields=[id, ts, name, timestamp_col]), rowType=[RecordType(INTEGER id, BIGINT ts, VARCHAR(32) name, TIMESTAMP(3) timestamp_col)] -+- Calc(select=[id, ts, name, CAST(timestamp_ltz_col AS TIMESTAMP(3) *ROWTIME*) AS timestamp_ltz_col]), rowType=[RecordType(INTEGER id, BIGINT ts, VARCHAR(2147483647) name, TIMESTAMP(3) *ROWTIME* timestamp_ltz_col)] - +- WatermarkAssigner(rowtime=[timestamp_ltz_col], watermark=[timestamp_ltz_col]), rowType=[RecordType(INTEGER id, BIGINT ts, VARCHAR(2147483647) name, TIMESTAMP_LTZ(3) *ROWTIME* timestamp_ltz_col)] - +- Calc(select=[id, ts, name, TO_TIMESTAMP_LTZ(ts, 3) AS timestamp_ltz_col]), rowType=[RecordType(INTEGER id, BIGINT ts, VARCHAR(2147483647) name, TIMESTAMP_WITH_LOCAL_TIME_ZONE(3) timestamp_ltz_col)] - +- TableSourceScan(table=[[default_catalog, default_database, t2, project=[id, ts, name], metadata=[]]], fields=[id, ts, name]), rowType=[RecordType(INTEGER id, BIGINT ts, VARCHAR(2147483647) name)] ++- Calc(select=[id, ts, name, CAST(TO_TIMESTAMP_LTZ(ts, 3) AS TIMESTAMP(3) *ROWTIME*) AS timestamp_ltz_col]), rowType=[RecordType(INTEGER id, BIGINT ts, VARCHAR(2147483647) name, TIMESTAMP(3) *ROWTIME* timestamp_ltz_col)] + +- TableSourceScan(table=[[default_catalog, default_database, t2, project=[id, ts, name], metadata=[]]], fields=[id, ts, name]), rowType=[RecordType(INTEGER id, BIGINT ts, VARCHAR(2147483647) name)] ]]> 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..120baa2378d74 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 @@ -294,25 +294,27 @@ Calc(select=[a, b, c, d, rowtime, PROCTIME_MATERIALIZE(proctime) AS proctime, wi @@ -243,9 +242,7 @@ LogicalAggregate(group=[{0}], EXPR$1=[SUM($1)]) GroupAggregate(groupBy=[b], select=[b, SUM(a) AS EXPR$1]) +- Exchange(distribution=[hash[b]]) +- Calc(select=[a, b], where=[SEARCH(a, Sarg[(0.1..10)])]) - +- WatermarkAssigner(rowtime=[rowtime], watermark=[rowtime]) - +- Calc(select=[a, b, rowtime]) - +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, rowtime]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, rowtime]) ]]> @@ -446,9 +443,7 @@ Calc(select=[EXPR$0]) +- GroupAggregate(groupBy=[b], select=[b, COUNT(a) AS EXPR$0]) +- Exchange(distribution=[hash[b]]) +- Calc(select=[b, a]) - +- WatermarkAssigner(rowtime=[rowtime], watermark=[rowtime]) - +- Calc(select=[b, a, rowtime]) - +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, rowtime]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, rowtime]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/agg/WindowAggregateTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/agg/WindowAggregateTest.xml index 71a9bd5df9aa3..229f4b8ac7af3 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/agg/WindowAggregateTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/agg/WindowAggregateTest.xml @@ -846,10 +846,8 @@ LogicalAggregate(group=[{0, 1, 2}], EXPR$3=[COUNT()], EXPR$4=[SUM($3)], EXPR$5=[ Calc(select=[a, window_start, window_end, EXPR$3, EXPR$4, EXPR$5, wAvg, uv], changelogMode=[I]) +- WindowAggregate(groupBy=[a], window=[CUMULATE(time_col=[proctime], max_size=[1 h], step=[10 min])], select=[a, COUNT_RETRACT(*) AS EXPR$3, SUM_RETRACT(d) AS EXPR$4, MAX_RETRACT(d) FILTER $f4 AS EXPR$5, weightedAvg_RETRACT(b, e) AS wAvg, COUNT_RETRACT(DISTINCT c) AS uv, start('w$) AS window_start, end('w$) AS window_end], changelogMode=[I]) +- Exchange(distribution=[hash[a]], changelogMode=[I,UB,UA,D]) - +- Calc(select=[a, d, IS TRUE(>(b, 1000)) AS $f4, b, e, c, proctime], changelogMode=[I,UB,UA,D]) - +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)], changelogMode=[I,UB,UA,D]) - +- Calc(select=[a, b, c, d, e, rowtime, PROCTIME() AS proctime], changelogMode=[I,UB,UA,D]) - +- TableSourceScan(table=[[default_catalog, default_database, MyCDCTable]], fields=[a, b, c, d, e, rowtime], changelogMode=[I,UB,UA,D]) + +- Calc(select=[a, d, IS TRUE(>(b, 1000)) AS $f4, b, e, c, PROCTIME() AS proctime], changelogMode=[I,UB,UA,D]) + +- TableSourceScan(table=[[default_catalog, default_database, MyCDCTable]], fields=[a, b, c, d, e, rowtime], changelogMode=[I,UB,UA,D]) ]]> @@ -886,10 +884,8 @@ LogicalAggregate(group=[{0, 1, 2}], EXPR$3=[COUNT()], EXPR$4=[SUM($3)], EXPR$5=[ Calc(select=[a, window_start, window_end, EXPR$3, EXPR$4, EXPR$5, wAvg, uv]) +- WindowAggregate(groupBy=[a], window=[CUMULATE(time_col=[proctime], max_size=[1 h], step=[10 min])], select=[a, COUNT(*) AS EXPR$3, SUM(d) AS EXPR$4, MAX(d) FILTER $f4 AS EXPR$5, weightedAvg(b, e) AS wAvg, COUNT(DISTINCT c) AS uv, start('w$) AS window_start, end('w$) AS window_end]) +- Exchange(distribution=[hash[a]]) - +- Calc(select=[a, d, IS TRUE(>(b, 1000)) AS $f4, b, e, c, proctime]) - +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)]) - +- Calc(select=[a, b, c, d, e, rowtime, PROCTIME() AS proctime]) - +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e, rowtime]) + +- Calc(select=[a, d, IS TRUE(>(b, 1000)) AS $f4, b, e, c, PROCTIME() AS proctime]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e, rowtime]) ]]> @@ -1335,19 +1331,15 @@ LogicalProject(window_start=[$0], window_end=[$1], EXPR$2=[$3], a=[$2]) Calc(select=[window_start, window_end, EXPR$2, a]) +- WindowAggregate(groupBy=[a], window=[SESSION(time_col=[proctime], gap=[10 min], partition keys=[a])], select=[a, COUNT(*) AS EXPR$2, start('w$) AS window_start, end('w$) AS window_end]) +- Exchange(distribution=[hash[a]]) - +- Calc(select=[a, proctime]) - +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)]) - +- Calc(select=[a, PROCTIME() AS proctime, rowtime]) - +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, rowtime], metadata=[]]], fields=[a, rowtime]) + +- Calc(select=[a, PROCTIME() AS proctime]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, rowtime], metadata=[]]], fields=[a, rowtime]) == Optimized Execution Plan == Calc(select=[window_start, window_end, EXPR$2, a]) +- WindowAggregate(groupBy=[a], window=[SESSION(time_col=[proctime], gap=[10 min], partition keys=[a])], select=[a, COUNT(*) AS EXPR$2, start('w$) AS window_start, end('w$) AS window_end]) +- Exchange(distribution=[hash[a]]) - +- Calc(select=[a, proctime]) - +- WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)]) - +- Calc(select=[a, PROCTIME() AS proctime, rowtime]) - +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, rowtime], metadata=[]]], fields=[a, rowtime]) + +- Calc(select=[a, PROCTIME() AS proctime]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, rowtime], metadata=[]]], fields=[a, rowtime]) ]]> @@ -1367,19 +1359,15 @@ LogicalProject(window_start=[$0], window_end=[$1], EXPR$2=[$3], a=[$2]) Calc(select=[window_start, window_end, EXPR$2, a]) +- WindowAggregate(groupBy=[a], window=[SESSION(time_col=[proctime], gap=[10 min], partition keys=[a])], select=[a, COUNT(*) AS EXPR$2, start('w$) AS window_start, end('w$) AS window_end]) +- Exchange(distribution=[hash[a]]) - +- Calc(select=[a, proctime]) - +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)]) - +- Calc(select=[a, PROCTIME() AS proctime, rowtime]) - +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, rowtime], metadata=[]]], fields=[a, rowtime]) + +- Calc(select=[a, PROCTIME() AS proctime]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, rowtime], metadata=[]]], fields=[a, rowtime]) == Optimized Execution Plan == Calc(select=[window_start, window_end, EXPR$2, a]) +- WindowAggregate(groupBy=[a], window=[SESSION(time_col=[proctime], gap=[10 min], partition keys=[a])], select=[a, COUNT(*) AS EXPR$2, start('w$) AS window_start, end('w$) AS window_end]) +- Exchange(distribution=[hash[a]]) - +- Calc(select=[a, proctime]) - +- WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)]) - +- Calc(select=[a, PROCTIME() AS proctime, rowtime]) - +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, rowtime], metadata=[]]], fields=[a, rowtime]) + +- Calc(select=[a, PROCTIME() AS proctime]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, rowtime], metadata=[]]], fields=[a, rowtime]) ]]> @@ -1728,10 +1716,8 @@ LogicalAggregate(group=[{0, 1, 2}], EXPR$3=[COUNT()], EXPR$4=[SUM($3)], EXPR$5=[ Calc(select=[a, window_start, window_end, EXPR$3, EXPR$4, EXPR$5, wAvg, uv], changelogMode=[I]) +- WindowAggregate(groupBy=[a], window=[HOP(time_col=[proctime], size=[10 min], slide=[5 min])], select=[a, COUNT_RETRACT(*) AS EXPR$3, SUM_RETRACT(d) AS EXPR$4, MAX_RETRACT(d) FILTER $f4 AS EXPR$5, weightedAvg_RETRACT(b, e) AS wAvg, COUNT_RETRACT(DISTINCT c) AS uv, start('w$) AS window_start, end('w$) AS window_end], changelogMode=[I]) +- Exchange(distribution=[hash[a]], changelogMode=[I,UB,UA,D]) - +- Calc(select=[a, d, IS TRUE(>(b, 1000)) AS $f4, b, e, c, proctime], changelogMode=[I,UB,UA,D]) - +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)], changelogMode=[I,UB,UA,D]) - +- Calc(select=[a, b, c, d, e, rowtime, PROCTIME() AS proctime], changelogMode=[I,UB,UA,D]) - +- TableSourceScan(table=[[default_catalog, default_database, MyCDCTable]], fields=[a, b, c, d, e, rowtime], changelogMode=[I,UB,UA,D]) + +- Calc(select=[a, d, IS TRUE(>(b, 1000)) AS $f4, b, e, c, PROCTIME() AS proctime], changelogMode=[I,UB,UA,D]) + +- TableSourceScan(table=[[default_catalog, default_database, MyCDCTable]], fields=[a, b, c, d, e, rowtime], changelogMode=[I,UB,UA,D]) ]]> @@ -1768,10 +1754,8 @@ LogicalAggregate(group=[{0, 1, 2}], EXPR$3=[COUNT()], EXPR$4=[SUM($3)], EXPR$5=[ Calc(select=[a, window_start, window_end, EXPR$3, EXPR$4, EXPR$5, wAvg, uv]) +- WindowAggregate(groupBy=[a], window=[HOP(time_col=[proctime], size=[10 min], slide=[5 min])], select=[a, COUNT(*) AS EXPR$3, SUM(d) AS EXPR$4, MAX(d) FILTER $f4 AS EXPR$5, weightedAvg(b, e) AS wAvg, COUNT(DISTINCT c) AS uv, start('w$) AS window_start, end('w$) AS window_end]) +- Exchange(distribution=[hash[a]]) - +- Calc(select=[a, d, IS TRUE(>(b, 1000)) AS $f4, b, e, c, proctime]) - +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)]) - +- Calc(select=[a, b, c, d, e, rowtime, PROCTIME() AS proctime]) - +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e, rowtime]) + +- Calc(select=[a, d, IS TRUE(>(b, 1000)) AS $f4, b, e, c, PROCTIME() AS proctime]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e, rowtime]) ]]> @@ -2037,10 +2021,8 @@ GroupAggregate(groupBy=[a, ws, we], select=[a, ws, we, COUNT(*) AS EXPR$3]) +- Calc(select=[a, window_start AS ws, window_end AS we]) +- WindowAggregate(groupBy=[a, b], window=[TUMBLE(time_col=[proctime], size=[5 min])], select=[a, b, start('w$) AS window_start, end('w$) AS window_end, proctime('w$) AS window_time]) +- Exchange(distribution=[hash[a, b]]) - +- Calc(select=[a, b, d, proctime]) - +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)]) - +- Calc(select=[a, b, d, PROCTIME() AS proctime, rowtime]) - +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b, d, rowtime], metadata=[]]], fields=[a, b, d, rowtime]) + +- Calc(select=[a, b, d, PROCTIME() AS proctime]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b, d, rowtime], metadata=[]]], fields=[a, b, d, rowtime]) ]]> @@ -2079,10 +2061,8 @@ GroupAggregate(groupBy=[a, ws, we], select=[a, ws, we, COUNT(*) AS EXPR$3]) +- Exchange(distribution=[single]) +- Calc(select=[a, proctime, window_start, window_end]) +- WindowTableFunction(window=[TUMBLE(time_col=[proctime], size=[5 min])]) - +- Calc(select=[a, b, proctime]) - +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)]) - +- Calc(select=[a, b, PROCTIME() AS proctime, rowtime]) - +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b, rowtime], metadata=[]]], fields=[a, b, rowtime]) + +- Calc(select=[a, b, PROCTIME() AS proctime]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b, rowtime], metadata=[]]], fields=[a, b, rowtime]) ]]> @@ -2127,17 +2107,13 @@ GroupAggregate(groupBy=[a, ws, we], select=[a, ws, we, COUNT(*) AS EXPR$3]) :- Exchange(distribution=[single]) : +- Calc(select=[a, window_start, window_end]) : +- WindowTableFunction(window=[TUMBLE(time_col=[proctime], size=[5 min])]) - : +- Calc(select=[a, proctime]) - : +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)]) - : +- Calc(select=[a, PROCTIME() AS proctime, rowtime]) - : +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, rowtime], metadata=[]]], fields=[a, rowtime]) + : +- Calc(select=[a, PROCTIME() AS proctime]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, rowtime], metadata=[]]], fields=[a, rowtime]) +- Exchange(distribution=[single]) +- Calc(select=[window_start, window_end]) +- WindowTableFunction(window=[TUMBLE(time_col=[proctime], size=[5 min])]) - +- Calc(select=[proctime]) - +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)]) - +- Calc(select=[PROCTIME() AS proctime, rowtime]) - +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, rowtime], metadata=[]]], fields=[a, rowtime]) + +- Calc(select=[PROCTIME() AS proctime]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, rowtime], metadata=[]]], fields=[a, rowtime]) ]]> @@ -2176,10 +2152,8 @@ GroupAggregate(groupBy=[a, ws, we], select=[a, ws, we, COUNT(*) AS EXPR$3]) +- Exchange(distribution=[single]) +- Calc(select=[a, proctime, window_start, window_end]) +- WindowTableFunction(window=[TUMBLE(time_col=[proctime], size=[5 min])]) - +- Calc(select=[a, b, proctime]) - +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)]) - +- Calc(select=[a, b, PROCTIME() AS proctime, rowtime]) - +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b, rowtime], metadata=[]]], fields=[a, b, rowtime]) + +- Calc(select=[a, b, PROCTIME() AS proctime]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b, rowtime], metadata=[]]], fields=[a, b, rowtime]) ]]> @@ -2299,10 +2273,8 @@ Calc(select=[c, EXPR$1]) +- Exchange(distribution=[hash[window_start, window_end, c, window_time]]) +- Calc(select=[window_start, window_end, c, window_time, a], where=[<>(window_start, '123')]) +- WindowTableFunction(window=[CUMULATE(time_col=[proctime], max_size=[5 min], step=[10 s])]) - +- Calc(select=[a, c, proctime]) - +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)]) - +- Calc(select=[a, c, PROCTIME() AS proctime, rowtime]) - +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, c, rowtime], metadata=[]]], fields=[a, c, rowtime]) + +- Calc(select=[a, c, PROCTIME() AS proctime]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, c, rowtime], metadata=[]]], fields=[a, c, rowtime]) ]]> @@ -2336,10 +2308,8 @@ Calc(select=[c, EXPR$1]) +- Exchange(distribution=[hash[window_start, window_end, c, window_time]]) +- Calc(select=[window_start, window_end, c, window_time, a], where=[<>(window_start, '123')]) +- WindowTableFunction(window=[CUMULATE(time_col=[proctime], max_size=[5 min], step=[10 s])]) - +- Calc(select=[a, c, proctime]) - +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)]) - +- Calc(select=[a, c, PROCTIME() AS proctime, rowtime]) - +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, c, rowtime], metadata=[]]], fields=[a, c, rowtime]) + +- Calc(select=[a, c, PROCTIME() AS proctime]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, c, rowtime], metadata=[]]], fields=[a, c, rowtime]) ]]> @@ -2376,9 +2346,8 @@ Calc(select=[c, EXPR$1, EXPR$2]) +- Calc(select=[window_start, window_end, c, window_time, EXPR$0, a]) +- Correlate(invocation=[str_split(_UTF-16LE'Jack,John', _UTF-16LE',')], correlate=[table(str_split('Jack,John',','))], 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_LTZ(3) *PROCTIME* window_time, VARCHAR(2147483647) EXPR$0)], joinType=[LEFT]) +- WindowTableFunction(window=[CUMULATE(time_col=[proctime], max_size=[5 min], step=[10 s])]) - +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)]) - +- Calc(select=[a, b, c, d, e, rowtime, PROCTIME() AS proctime]) - +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e, rowtime]) + +- Calc(select=[a, b, c, d, e, rowtime, PROCTIME() AS proctime]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e, rowtime]) ]]> @@ -2415,9 +2384,8 @@ Calc(select=[c, EXPR$1, EXPR$2]) +- Calc(select=[window_start, window_end, c, window_time, EXPR$0, a]) +- Correlate(invocation=[str_split(_UTF-16LE'Jack,John', _UTF-16LE',')], correlate=[table(str_split('Jack,John',','))], 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_LTZ(3) *PROCTIME* window_time, VARCHAR(2147483647) EXPR$0)], joinType=[LEFT]) +- WindowTableFunction(window=[CUMULATE(time_col=[proctime], max_size=[5 min], step=[10 s])]) - +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)]) - +- Calc(select=[a, b, c, d, e, rowtime, PROCTIME() AS proctime]) - +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e, rowtime]) + +- Calc(select=[a, b, c, d, e, rowtime, PROCTIME() AS proctime]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e, rowtime]) ]]> @@ -2458,9 +2426,8 @@ Calc(select=[c, EXPR$1]) +- Exchange(distribution=[hash[c]]) +- Calc(select=[a, c, proctime, window_start, window_end, window_time]) +- WindowTableFunction(window=[CUMULATE(time_col=[proctime], max_size=[5 min], step=[10 s])]) - +- WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)]) - +- Calc(select=[a, b, c, d, e, rowtime, PROCTIME() AS proctime]) - +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e, rowtime]) + +- Calc(select=[a, b, c, d, e, rowtime, PROCTIME() AS proctime]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e, rowtime]) ]]> @@ -2501,9 +2468,8 @@ Calc(select=[c, EXPR$1]) +- Exchange(distribution=[hash[c]]) +- Calc(select=[a, c, proctime, window_start, window_end, window_time]) +- WindowTableFunction(window=[CUMULATE(time_col=[proctime], max_size=[5 min], step=[10 s])]) - +- WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)]) - +- Calc(select=[a, b, c, d, e, rowtime, PROCTIME() AS proctime]) - +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e, rowtime]) + +- Calc(select=[a, b, c, d, e, rowtime, PROCTIME() AS proctime]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e, rowtime]) ]]> @@ -2544,15 +2510,11 @@ Calc(select=[c, EXPR$1, EXPR$2]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a, c, window_start, window_end, PROCTIME_MATERIALIZE(window_time) AS window_time]) : +- WindowTableFunction(window=[CUMULATE(time_col=[proctime], max_size=[5 min], step=[10 s])]) - : +- Calc(select=[a, c, proctime]) - : +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)]) - : +- Calc(select=[a, c, PROCTIME() AS proctime, rowtime]) - : +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, c, e, rowtime], metadata=[]]], fields=[a, c, e, rowtime]) + : +- Calc(select=[a, c, PROCTIME() AS proctime]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, c, e, rowtime], metadata=[]]], fields=[a, c, e, rowtime]) +- Exchange(distribution=[hash[a]]) +- Calc(select=[a, e]) - +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)]) - +- Calc(select=[a, e, rowtime]) - +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, c, e, rowtime], metadata=[]]], fields=[a, c, e, rowtime]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, c, e, rowtime], metadata=[]]], fields=[a, c, e, rowtime]) ]]> @@ -2593,15 +2555,11 @@ Calc(select=[c, EXPR$1, EXPR$2]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a, c, window_start, window_end, PROCTIME_MATERIALIZE(window_time) AS window_time]) : +- WindowTableFunction(window=[CUMULATE(time_col=[proctime], max_size=[5 min], step=[10 s])]) - : +- Calc(select=[a, c, proctime]) - : +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)]) - : +- Calc(select=[a, c, PROCTIME() AS proctime, rowtime]) - : +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, c, e, rowtime], metadata=[]]], fields=[a, c, e, rowtime]) + : +- Calc(select=[a, c, PROCTIME() AS proctime]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, c, e, rowtime], metadata=[]]], fields=[a, c, e, rowtime]) +- Exchange(distribution=[hash[a]]) +- Calc(select=[a, e]) - +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)]) - +- Calc(select=[a, e, rowtime]) - +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, c, e, rowtime], metadata=[]]], fields=[a, c, e, rowtime]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, c, e, rowtime], metadata=[]]], fields=[a, c, e, rowtime]) ]]> @@ -2639,9 +2597,8 @@ Calc(select=[c, EXPR$1, EXPR$2]) +- OverAggregate(partitionBy=[c], orderBy=[proctime DESC], window=[ RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, b, c, d, e, rowtime, proctime, window_start, window_end, window_time, COUNT(*) AS w0$o0]) +- Exchange(distribution=[hash[c]]) +- WindowTableFunction(window=[CUMULATE(time_col=[proctime], max_size=[5 min], step=[10 s])]) - +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)]) - +- Calc(select=[a, b, c, d, e, rowtime, PROCTIME() AS proctime]) - +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e, rowtime]) + +- Calc(select=[a, b, c, d, e, rowtime, PROCTIME() AS proctime]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e, rowtime]) ]]> @@ -2679,9 +2636,8 @@ Calc(select=[c, EXPR$1, EXPR$2]) +- OverAggregate(partitionBy=[c], orderBy=[proctime DESC], window=[ RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, b, c, d, e, rowtime, proctime, window_start, window_end, window_time, COUNT(*) AS w0$o0]) +- Exchange(distribution=[hash[c]]) +- WindowTableFunction(window=[CUMULATE(time_col=[proctime], max_size=[5 min], step=[10 s])]) - +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)]) - +- Calc(select=[a, b, c, d, e, rowtime, PROCTIME() AS proctime]) - +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e, rowtime]) + +- Calc(select=[a, b, c, d, e, rowtime, PROCTIME() AS proctime]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e, rowtime]) ]]> @@ -2722,9 +2678,8 @@ Calc(select=[c, EXPR$1]) +- Exchange(distribution=[hash[c]]) +- Calc(select=[a, c, proctime, window_start, window_end, window_time]) +- WindowTableFunction(window=[CUMULATE(time_col=[proctime], max_size=[5 min], step=[10 s])]) - +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)]) - +- Calc(select=[a, b, c, d, e, rowtime, PROCTIME() AS proctime]) - +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e, rowtime]) + +- Calc(select=[a, b, c, d, e, rowtime, PROCTIME() AS proctime]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e, rowtime]) ]]> @@ -2765,9 +2720,8 @@ Calc(select=[c, EXPR$1]) +- Exchange(distribution=[hash[c]]) +- Calc(select=[a, c, proctime, window_start, window_end, window_time]) +- WindowTableFunction(window=[CUMULATE(time_col=[proctime], max_size=[5 min], step=[10 s])]) - +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)]) - +- Calc(select=[a, b, c, d, e, rowtime, PROCTIME() AS proctime]) - +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e, rowtime]) + +- Calc(select=[a, b, c, d, e, rowtime, PROCTIME() AS proctime]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e, rowtime]) ]]> @@ -2814,16 +2768,12 @@ Calc(select=[c, EXPR$1]) +- Union(all=[true], union=[window_start, window_end, c, window_time, a]) :- Calc(select=[window_start, window_end, c, window_time, a]) : +- WindowTableFunction(window=[TUMBLE(time_col=[proctime], size=[10 s])]) - : +- Calc(select=[a, c, proctime]) - : +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)]) - : +- Calc(select=[a, c, PROCTIME() AS proctime, rowtime]) - : +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, c, rowtime], metadata=[]]], fields=[a, c, rowtime]) + : +- Calc(select=[a, c, PROCTIME() AS proctime]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, c, rowtime], metadata=[]]], fields=[a, c, rowtime]) +- Calc(select=[window_start, window_end, c, window_time, a]) +- WindowTableFunction(window=[TUMBLE(time_col=[proctime], size=[5 s])]) - +- Calc(select=[a, c, proctime]) - +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)]) - +- Calc(select=[a, c, PROCTIME() AS proctime, rowtime]) - +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, c, rowtime], metadata=[]]], fields=[a, c, rowtime]) + +- Calc(select=[a, c, PROCTIME() AS proctime]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, c, rowtime], metadata=[]]], fields=[a, c, rowtime]) ]]> @@ -2870,16 +2820,12 @@ Calc(select=[c, EXPR$1]) +- Union(all=[true], union=[window_start, window_end, c, window_time, a]) :- Calc(select=[window_start, window_end, c, window_time, a]) : +- WindowTableFunction(window=[TUMBLE(time_col=[proctime], size=[10 s])]) - : +- Calc(select=[a, c, proctime]) - : +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)]) - : +- Calc(select=[a, c, PROCTIME() AS proctime, rowtime]) - : +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, c, rowtime], metadata=[]]], fields=[a, c, rowtime]) + : +- Calc(select=[a, c, PROCTIME() AS proctime]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, c, rowtime], metadata=[]]], fields=[a, c, rowtime]) +- Calc(select=[window_start, window_end, c, window_time, a]) +- WindowTableFunction(window=[TUMBLE(time_col=[proctime], size=[5 s])]) - +- Calc(select=[a, c, proctime]) - +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)]) - +- Calc(select=[a, c, PROCTIME() AS proctime, rowtime]) - +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, c, rowtime], metadata=[]]], fields=[a, c, rowtime]) + +- Calc(select=[a, c, PROCTIME() AS proctime]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, c, rowtime], metadata=[]]], fields=[a, c, rowtime]) ]]> @@ -3251,10 +3197,8 @@ LogicalAggregate(group=[{0, 1, 2}], EXPR$3=[COUNT()], EXPR$4=[SUM($3)], EXPR$5=[ Calc(select=[a, window_start, window_end, EXPR$3, EXPR$4, EXPR$5, uv]) +- WindowAggregate(groupBy=[a], window=[SESSION(time_col=[proctime], gap=[5 min], partition keys=[a])], select=[a, COUNT(*) AS EXPR$3, SUM(d) AS EXPR$4, MAX(d) FILTER $f4 AS EXPR$5, COUNT(DISTINCT c) AS uv, start('w$) AS window_start, end('w$) AS window_end]) +- Exchange(distribution=[hash[a]]) - +- Calc(select=[a, d, IS TRUE(>(b, 1000)) AS $f4, c, proctime]) - +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)]) - +- Calc(select=[a, b, c, d, PROCTIME() AS proctime, rowtime]) - +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b, c, d, rowtime], metadata=[]]], fields=[a, b, c, d, rowtime]) + +- Calc(select=[a, d, IS TRUE(>(b, 1000)) AS $f4, c, PROCTIME() AS proctime]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b, c, d, rowtime], metadata=[]]], fields=[a, b, c, d, rowtime]) ]]> @@ -3290,10 +3234,8 @@ LogicalAggregate(group=[{0, 1, 2}], EXPR$3=[COUNT()], EXPR$4=[SUM($3)], EXPR$5=[ Calc(select=[a, window_start, window_end, EXPR$3, EXPR$4, EXPR$5, uv]) +- WindowAggregate(groupBy=[a], window=[SESSION(time_col=[proctime], gap=[5 min], partition keys=[a])], select=[a, COUNT(*) AS EXPR$3, SUM(d) AS EXPR$4, MAX(d) FILTER $f4 AS EXPR$5, COUNT(DISTINCT c) AS uv, start('w$) AS window_start, end('w$) AS window_end]) +- Exchange(distribution=[hash[a]]) - +- Calc(select=[a, d, IS TRUE(>(b, 1000)) AS $f4, c, proctime]) - +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)]) - +- Calc(select=[a, b, c, d, PROCTIME() AS proctime, rowtime]) - +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b, c, d, rowtime], metadata=[]]], fields=[a, b, c, d, rowtime]) + +- Calc(select=[a, d, IS TRUE(>(b, 1000)) AS $f4, c, PROCTIME() AS proctime]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b, c, d, rowtime], metadata=[]]], fields=[a, b, c, d, rowtime]) ]]> @@ -3330,10 +3272,8 @@ LogicalAggregate(group=[{0, 1, 2}], EXPR$3=[COUNT()], EXPR$4=[SUM($3)], EXPR$5=[ Calc(select=[a, window_start, window_end, EXPR$3, EXPR$4, EXPR$5, wAvg, uv]) +- WindowAggregate(groupBy=[a], window=[SESSION(time_col=[proctime], gap=[5 min], partition keys=[a])], select=[a, COUNT(*) AS EXPR$3, SUM(d) AS EXPR$4, MAX(d) FILTER $f4 AS EXPR$5, weightedAvg(b, e) AS wAvg, COUNT(DISTINCT c) AS uv, start('w$) AS window_start, end('w$) AS window_end]) +- Exchange(distribution=[hash[a]]) - +- Calc(select=[a, d, IS TRUE(>(b, 1000)) AS $f4, b, e, c, proctime]) - +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)]) - +- Calc(select=[a, b, c, d, e, rowtime, PROCTIME() AS proctime]) - +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e, rowtime]) + +- Calc(select=[a, d, IS TRUE(>(b, 1000)) AS $f4, b, e, c, PROCTIME() AS proctime]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e, rowtime]) ]]> @@ -3542,10 +3482,8 @@ Calc(select=[a, window_start, window_end, EXPR$3]) +- Exchange(distribution=[single]) +- Calc(select=[a, proctime, window_start, window_end, PROCTIME() AS $6]) +- WindowTableFunction(window=[TUMBLE(time_col=[proctime], size=[5 min])]) - +- Calc(select=[a, b, proctime]) - +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)]) - +- Calc(select=[a, b, PROCTIME() AS proctime, rowtime]) - +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b, rowtime], metadata=[]]], fields=[a, b, rowtime]) + +- Calc(select=[a, b, PROCTIME() AS proctime]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b, rowtime], metadata=[]]], fields=[a, b, rowtime]) ]]> @@ -3593,17 +3531,13 @@ Calc(select=[a, window_start, window_end, EXPR$3]) :- Exchange(distribution=[single]) : +- Calc(select=[a, window_start, window_end]) : +- WindowTableFunction(window=[TUMBLE(time_col=[proctime], size=[5 min])]) - : +- Calc(select=[a, proctime]) - : +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)]) - : +- Calc(select=[a, PROCTIME() AS proctime, rowtime]) - : +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, rowtime], metadata=[]]], fields=[a, rowtime]) + : +- Calc(select=[a, PROCTIME() AS proctime]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, rowtime], metadata=[]]], fields=[a, rowtime]) +- Exchange(distribution=[single]) +- Calc(select=[window_start, window_end]) +- WindowTableFunction(window=[TUMBLE(time_col=[proctime], size=[5 min])]) - +- Calc(select=[proctime]) - +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)]) - +- Calc(select=[PROCTIME() AS proctime, rowtime]) - +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, rowtime], metadata=[]]], fields=[a, rowtime]) + +- Calc(select=[PROCTIME() AS proctime]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, rowtime], metadata=[]]], fields=[a, rowtime]) ]]> @@ -3645,10 +3579,8 @@ Calc(select=[a, window_start, window_end, EXPR$3]) +- Exchange(distribution=[single]) +- Calc(select=[a, proctime, window_start, window_end, PROCTIME() AS $6]) +- WindowTableFunction(window=[TUMBLE(time_col=[proctime], size=[5 min])]) - +- Calc(select=[a, b, proctime]) - +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)]) - +- Calc(select=[a, b, PROCTIME() AS proctime, rowtime]) - +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b, rowtime], metadata=[]]], fields=[a, b, rowtime]) + +- Calc(select=[a, b, PROCTIME() AS proctime]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b, rowtime], metadata=[]]], fields=[a, b, rowtime]) ]]> @@ -3777,10 +3709,8 @@ Calc(select=[window_start, window_end, EXPR$2, EXPR$3]) +- Calc(select=[a, cnt, PROCTIME() AS new_proctime]) +- WindowAggregate(groupBy=[a, b], window=[TUMBLE(time_col=[proctime], size=[5 min])], select=[a, b, COUNT(*) AS cnt, start('w$) AS window_start, end('w$) AS window_end, proctime('w$) AS window_time]) +- Exchange(distribution=[hash[a, b]]) - +- Calc(select=[a, b, d, proctime]) - +- WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)]) - +- Calc(select=[a, b, d, PROCTIME() AS proctime, rowtime]) - +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b, d, rowtime], metadata=[]]], fields=[a, b, d, rowtime]) + +- Calc(select=[a, b, d, PROCTIME() AS proctime]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b, d, rowtime], metadata=[]]], fields=[a, b, d, rowtime]) ]]> @@ -3821,10 +3751,8 @@ Calc(select=[window_start, window_end, EXPR$2, EXPR$3]) +- Calc(select=[a, cnt, window_time AS wt]) +- WindowAggregate(groupBy=[a, b], window=[TUMBLE(time_col=[proctime], size=[5 min])], select=[a, b, COUNT(*) AS cnt, start('w$) AS window_start, end('w$) AS window_end, proctime('w$) AS window_time]) +- Exchange(distribution=[hash[a, b]]) - +- Calc(select=[a, b, d, proctime]) - +- WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)]) - +- Calc(select=[a, b, d, PROCTIME() AS proctime, rowtime]) - +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b, d, rowtime], metadata=[]]], fields=[a, b, d, rowtime]) + +- Calc(select=[a, b, d, PROCTIME() AS proctime]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b, d, rowtime], metadata=[]]], fields=[a, b, d, rowtime]) ]]> @@ -4129,10 +4057,8 @@ LogicalAggregate(group=[{0, 1, 2}], EXPR$3=[COUNT()], EXPR$4=[SUM($3)], EXPR$5=[ Calc(select=[a, window_start, window_end, EXPR$3, EXPR$4, EXPR$5, uv]) +- WindowAggregate(groupBy=[a], window=[TUMBLE(time_col=[proctime], size=[15 min])], select=[a, COUNT(*) AS EXPR$3, SUM(d) AS EXPR$4, MAX(d) FILTER $f4 AS EXPR$5, COUNT(DISTINCT c) AS uv, start('w$) AS window_start, end('w$) AS window_end]) +- Exchange(distribution=[hash[a]]) - +- Calc(select=[a, d, IS TRUE(>(b, 1000)) AS $f4, c, proctime]) - +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)]) - +- Calc(select=[a, b, c, d, PROCTIME() AS proctime, rowtime]) - +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b, c, d, rowtime], metadata=[]]], fields=[a, b, c, d, rowtime]) + +- Calc(select=[a, d, IS TRUE(>(b, 1000)) AS $f4, c, PROCTIME() AS proctime]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b, c, d, rowtime], metadata=[]]], fields=[a, b, c, d, rowtime]) ]]> @@ -4787,10 +4713,8 @@ LogicalAggregate(group=[{0, 1, 2}], EXPR$3=[COUNT()], EXPR$4=[SUM($3)], EXPR$5=[ Calc(select=[a, window_start, window_end, EXPR$3, EXPR$4, EXPR$5, wAvg, uv], changelogMode=[I]) +- WindowAggregate(groupBy=[a], window=[TUMBLE(time_col=[proctime], size=[15 min])], select=[a, COUNT_RETRACT(*) AS EXPR$3, SUM_RETRACT(d) AS EXPR$4, MAX_RETRACT(d) FILTER $f4 AS EXPR$5, weightedAvg_RETRACT(b, e) AS wAvg, COUNT_RETRACT(DISTINCT c) AS uv, start('w$) AS window_start, end('w$) AS window_end], changelogMode=[I]) +- Exchange(distribution=[hash[a]], changelogMode=[I,UB,UA,D]) - +- Calc(select=[a, d, IS TRUE(>(b, 1000)) AS $f4, b, e, c, proctime], changelogMode=[I,UB,UA,D]) - +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)], changelogMode=[I,UB,UA,D]) - +- Calc(select=[a, b, c, d, e, rowtime, PROCTIME() AS proctime], changelogMode=[I,UB,UA,D]) - +- TableSourceScan(table=[[default_catalog, default_database, MyCDCTable]], fields=[a, b, c, d, e, rowtime], changelogMode=[I,UB,UA,D]) + +- Calc(select=[a, d, IS TRUE(>(b, 1000)) AS $f4, b, e, c, PROCTIME() AS proctime], changelogMode=[I,UB,UA,D]) + +- TableSourceScan(table=[[default_catalog, default_database, MyCDCTable]], fields=[a, b, c, d, e, rowtime], changelogMode=[I,UB,UA,D]) ]]> @@ -4826,10 +4750,8 @@ LogicalAggregate(group=[{0, 1, 2}], EXPR$3=[COUNT()], EXPR$4=[SUM($3)], EXPR$5=[ Calc(select=[a, window_start, window_end, EXPR$3, EXPR$4, EXPR$5, wAvg, uv]) +- WindowAggregate(groupBy=[a], window=[TUMBLE(time_col=[proctime], size=[15 min])], select=[a, COUNT(*) AS EXPR$3, SUM(d) AS EXPR$4, MAX(d) FILTER $f4 AS EXPR$5, weightedAvg(b, e) AS wAvg, COUNT(DISTINCT c) AS uv, start('w$) AS window_start, end('w$) AS window_end]) +- Exchange(distribution=[hash[a]]) - +- Calc(select=[a, d, IS TRUE(>(b, 1000)) AS $f4, b, e, c, proctime]) - +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)]) - +- Calc(select=[a, b, c, d, e, rowtime, PROCTIME() AS proctime]) - +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e, rowtime]) + +- Calc(select=[a, d, IS TRUE(>(b, 1000)) AS $f4, b, e, c, PROCTIME() AS proctime]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, e, rowtime]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/join/TemporalJoinTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/join/TemporalJoinTest.xml index fea574dc70531..e04f4f002c8f5 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/join/TemporalJoinTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/join/TemporalJoinTest.xml @@ -455,15 +455,14 @@ LogicalProject(amount=[$0], currency=[$1], rowtime=[$2], proctime=[$3], currency @@ -489,10 +488,8 @@ LogicalProject(currency=[$1], currency0=[$4], rate1=[$6]) Calc(select=[currency, currency0, rate1]) +- TemporalJoin(joinType=[InnerJoin], where=[((currency = currency0) AND __TEMPORAL_JOIN_CONDITION(proctime, __TEMPORAL_JOIN_CONDITION_PRIMARY_KEY(currency0), __TEMPORAL_JOIN_LEFT_KEY(currency), __TEMPORAL_JOIN_RIGHT_KEY(currency0)))], select=[currency, proctime, currency0, rate1]) :- Exchange(distribution=[hash[currency]]) - : +- Calc(select=[currency, proctime], where=[(amount > 10)]) - : +- WatermarkAssigner(rowtime=[rowtime], watermark=[rowtime]) - : +- Calc(select=[amount, currency, rowtime, PROCTIME() AS proctime]) - : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[amount, currency, rowtime]) + : +- Calc(select=[currency, PROCTIME() AS proctime], where=[(amount > 10)]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[amount, currency, rowtime]) +- Exchange(distribution=[hash[currency]]) +- Calc(select=[currency, (rate + 1) AS rate1], where=[(rate < 100)]) +- TableSourceScan(table=[[default_catalog, default_database, RatesBinlogWithoutWatermark, project=[currency, rate], metadata=[], filter=[]]], fields=[currency, rate]) @@ -522,15 +519,11 @@ LogicalProject(currency=[$1], currency0=[$4], rate1=[$6]) Calc(select=[currency, currency0, rate1]) +- TemporalJoin(joinType=[InnerJoin], where=[((currency = currency0) AND __TEMPORAL_JOIN_CONDITION(proctime, __TEMPORAL_JOIN_CONDITION_PRIMARY_KEY(currency0), __TEMPORAL_JOIN_LEFT_KEY(currency), __TEMPORAL_JOIN_RIGHT_KEY(currency0)))], select=[currency, proctime, currency0, rate1]) :- Exchange(distribution=[hash[currency]]) - : +- Calc(select=[currency, proctime], where=[(amount > 10)]) - : +- WatermarkAssigner(rowtime=[rowtime], watermark=[rowtime]) - : +- Calc(select=[amount, currency, rowtime, PROCTIME() AS proctime]) - : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[amount, currency, rowtime]) + : +- Calc(select=[currency, PROCTIME() AS proctime], where=[(amount > 10)]) + : +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[amount, currency, rowtime]) +- Exchange(distribution=[hash[currency]]) - +- Calc(select=[currency, rate1], where=[(rate < 100)]) - +- WatermarkAssigner(rowtime=[rowtime], watermark=[rowtime]) - +- Calc(select=[currency, rate, (rate + 1) AS rate1, rowtime]) - +- TableSourceScan(table=[[default_catalog, default_database, RatesBinlogWithComputedColumn]], fields=[currency, rate, rowtime]) + +- Calc(select=[currency, (rate + 1) AS rate1], where=[(rate < 100)]) + +- TableSourceScan(table=[[default_catalog, default_database, RatesBinlogWithComputedColumn]], fields=[currency, rate, rowtime]) ]]> @@ -601,8 +594,7 @@ Calc(select=[amount, currency, rowtime, PROCTIME_MATERIALIZE(proctime) AS procti +- GroupAggregate(groupBy=[currency], select=[currency, LAST_VALUE(rate) AS rate]) +- Exchange(distribution=[hash[currency]]) +- Calc(select=[currency, rate]) - +- WatermarkAssigner(rowtime=[rowtime], watermark=[rowtime]) - +- TableSourceScan(table=[[default_catalog, default_database, RatesHistory]], fields=[currency, rate, rowtime]) + +- TableSourceScan(table=[[default_catalog, default_database, RatesHistory]], fields=[currency, rate, rowtime]) ]]> @@ -637,8 +629,7 @@ Calc(select=[amount, currency, rowtime, PROCTIME_MATERIALIZE(proctime) AS procti +- GroupAggregate(groupBy=[currency], select=[currency, LAST_VALUE(rate) AS rate]) +- Exchange(distribution=[hash[currency]]) +- Calc(select=[currency, rate]) - +- WatermarkAssigner(rowtime=[rowtime], watermark=[rowtime]) - +- TableSourceScan(table=[[default_catalog, default_database, RatesHistory]], fields=[currency, rate, rowtime]) + +- TableSourceScan(table=[[default_catalog, default_database, RatesHistory]], fields=[currency, rate, rowtime]) ]]> @@ -749,8 +740,7 @@ Calc(select=[amount, currency, rowtime, PROCTIME_MATERIALIZE(proctime) AS procti +- GroupAggregate(groupBy=[currency], select=[currency, LAST_VALUE(rate) AS rate]) +- Exchange(distribution=[hash[currency]]) +- Calc(select=[currency, rate]) - +- WatermarkAssigner(rowtime=[rowtime], watermark=[rowtime]) - +- TableSourceScan(table=[[default_catalog, default_database, RatesHistory]], fields=[currency, rate, rowtime]) + +- TableSourceScan(table=[[default_catalog, default_database, RatesHistory]], fields=[currency, rate, rowtime]) ]]> 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..7488d8202a639 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 @@ -787,10 +787,8 @@ Join(joinType=[InnerJoin], where=[AND(=(window_start, window_start0), =(window_e : +- Calc(select=[a, window_start, window_end, PROCTIME_MATERIALIZE(window_time) AS window_time, cnt, uv]) : +- WindowAggregate(groupBy=[a], window=[CUMULATE(time_col=[proctime], max_size=[1 h], step=[10 min])], select=[a, COUNT(*) AS cnt, COUNT(DISTINCT c) AS uv, start('w$) AS window_start, end('w$) AS window_end, proctime('w$) AS window_time]) : +- Exchange(distribution=[hash[a]]) -: +- Calc(select=[a, c, proctime]) -: +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)]) -: +- Calc(select=[a, c, PROCTIME() AS proctime, rowtime]) -: +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, c, rowtime], metadata=[]]], fields=[a, c, rowtime]) +: +- Calc(select=[a, c, PROCTIME() AS proctime]) +: +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, c, rowtime], metadata=[]]], fields=[a, c, rowtime]) +- Exchange(distribution=[hash[window_start, window_end, a]]) +- Calc(select=[a, window_start, window_end, CAST(window_time AS TIMESTAMP(3)) AS window_time, cnt, uv]) +- GlobalWindowAggregate(groupBy=[a], window=[CUMULATE(slice_end=[$slice_end], max_size=[1 h], step=[10 min])], select=[a, COUNT(count1$0) AS cnt, COUNT(distinct$0 count$1) AS uv, start('w$) AS window_start, end('w$) AS window_end, rowtime('w$) AS window_time]) @@ -1077,18 +1075,14 @@ Calc(select=[a, window_start, window_end, PROCTIME_MATERIALIZE(window_time) AS w : +- Calc(select=[a, window_start, window_end, window_time, cnt, uv]) : +- WindowAggregate(groupBy=[a], window=[CUMULATE(time_col=[proctime], max_size=[1 h], step=[10 min])], select=[a, COUNT(*) AS cnt, COUNT(DISTINCT c) AS uv, start('w$) AS window_start, end('w$) AS window_end, proctime('w$) AS window_time]) : +- Exchange(distribution=[hash[a]]) - : +- Calc(select=[a, c, proctime]) - : +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)]) - : +- Calc(select=[a, c, PROCTIME() AS proctime, rowtime]) - : +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, c, rowtime], metadata=[]]], fields=[a, c, rowtime]) + : +- Calc(select=[a, c, PROCTIME() AS proctime]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, c, rowtime], metadata=[]]], fields=[a, c, rowtime]) +- Exchange(distribution=[hash[a]]) +- Calc(select=[a, window_start, window_end, window_time, cnt, uv]) +- WindowAggregate(groupBy=[a], window=[CUMULATE(time_col=[proctime], max_size=[1 h], step=[10 min])], select=[a, COUNT(*) AS cnt, COUNT(DISTINCT c) AS uv, start('w$) AS window_start, end('w$) AS window_end, proctime('w$) AS window_time]) +- Exchange(distribution=[hash[a]]) - +- Calc(select=[a, c, proctime]) - +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)]) - +- Calc(select=[a, c, PROCTIME() AS proctime, rowtime]) - +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, project=[a, c, rowtime], metadata=[]]], fields=[a, c, rowtime]) + +- Calc(select=[a, c, PROCTIME() AS proctime]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, project=[a, c, rowtime], metadata=[]]], fields=[a, c, rowtime]) ]]> @@ -1222,18 +1216,14 @@ Calc(select=[a, window_start, window_end, PROCTIME_MATERIALIZE(window_time) AS w : +- Calc(select=[a, window_start, window_end, window_time, cnt, uv]) : +- WindowAggregate(groupBy=[a], window=[HOP(time_col=[proctime], size=[10 min], slide=[5 min])], select=[a, COUNT(*) AS cnt, COUNT(DISTINCT c) AS uv, start('w$) AS window_start, end('w$) AS window_end, proctime('w$) AS window_time]) : +- Exchange(distribution=[hash[a]]) - : +- Calc(select=[a, c, proctime]) - : +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)]) - : +- Calc(select=[a, c, PROCTIME() AS proctime, rowtime]) - : +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, c, rowtime], metadata=[]]], fields=[a, c, rowtime]) + : +- Calc(select=[a, c, PROCTIME() AS proctime]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, c, rowtime], metadata=[]]], fields=[a, c, rowtime]) +- Exchange(distribution=[hash[a]]) +- Calc(select=[a, window_start, window_end, window_time, cnt, uv]) +- WindowAggregate(groupBy=[a], window=[HOP(time_col=[proctime], size=[10 min], slide=[5 min])], select=[a, COUNT(*) AS cnt, COUNT(DISTINCT c) AS uv, start('w$) AS window_start, end('w$) AS window_end, proctime('w$) AS window_time]) +- Exchange(distribution=[hash[a]]) - +- Calc(select=[a, c, proctime]) - +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)]) - +- Calc(select=[a, c, PROCTIME() AS proctime, rowtime]) - +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, project=[a, c, rowtime], metadata=[]]], fields=[a, c, rowtime]) + +- Calc(select=[a, c, PROCTIME() AS proctime]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, project=[a, c, rowtime], metadata=[]]], fields=[a, c, rowtime]) ]]> @@ -1363,18 +1353,14 @@ Calc(select=[a, window_start, window_end, PROCTIME_MATERIALIZE(window_time) AS w : +- Calc(select=[a, window_start, window_end, window_time, cnt, uv]) : +- WindowAggregate(groupBy=[a], window=[TUMBLE(time_col=[proctime], size=[15 min])], select=[a, COUNT(*) AS cnt, COUNT(DISTINCT c) AS uv, start('w$) AS window_start, end('w$) AS window_end, proctime('w$) AS window_time]) : +- Exchange(distribution=[hash[a]]) - : +- Calc(select=[a, c, proctime]) - : +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)]) - : +- Calc(select=[a, c, PROCTIME() AS proctime, rowtime]) - : +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, c, rowtime], metadata=[]]], fields=[a, c, rowtime]) + : +- Calc(select=[a, c, PROCTIME() AS proctime]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, c, rowtime], metadata=[]]], fields=[a, c, rowtime]) +- Exchange(distribution=[hash[a]]) +- Calc(select=[a, window_start, window_end, window_time, cnt, uv]) +- WindowAggregate(groupBy=[a], window=[TUMBLE(time_col=[proctime], size=[15 min])], select=[a, COUNT(*) AS cnt, COUNT(DISTINCT c) AS uv, start('w$) AS window_start, end('w$) AS window_end, proctime('w$) AS window_time]) +- Exchange(distribution=[hash[a]]) - +- Calc(select=[a, c, proctime]) - +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)]) - +- Calc(select=[a, c, PROCTIME() AS proctime, rowtime]) - +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, project=[a, c, rowtime], metadata=[]]], fields=[a, c, rowtime]) + +- Calc(select=[a, c, PROCTIME() AS proctime]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, project=[a, c, rowtime], metadata=[]]], fields=[a, c, rowtime]) ]]> @@ -2353,18 +2339,14 @@ Calc(select=[a, window_start, window_end, PROCTIME_MATERIALIZE(window_time) AS w : +- Calc(select=[a, window_start, window_end, window_time, cnt, uv]) : +- WindowAggregate(groupBy=[a], window=[CUMULATE(time_col=[proctime], max_size=[1 h], step=[10 min])], select=[a, COUNT(*) AS cnt, COUNT(DISTINCT c) AS uv, start('w$) AS window_start, end('w$) AS window_end, proctime('w$) AS window_time]) : +- Exchange(distribution=[hash[a]]) - : +- Calc(select=[a, c, proctime]) - : +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)]) - : +- Calc(select=[a, c, PROCTIME() AS proctime, rowtime]) - : +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, c, rowtime], metadata=[]]], fields=[a, c, rowtime]) + : +- Calc(select=[a, c, PROCTIME() AS proctime]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, c, rowtime], metadata=[]]], fields=[a, c, rowtime]) +- Exchange(distribution=[hash[a]]) +- Calc(select=[a, window_start, window_end, window_time, cnt, uv]) +- WindowAggregate(groupBy=[a], window=[CUMULATE(time_col=[proctime], max_size=[1 h], step=[10 min])], select=[a, COUNT(*) AS cnt, COUNT(DISTINCT c) AS uv, start('w$) AS window_start, end('w$) AS window_end, proctime('w$) AS window_time]) +- Exchange(distribution=[hash[a]]) - +- Calc(select=[a, c, proctime]) - +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)]) - +- Calc(select=[a, c, PROCTIME() AS proctime, rowtime]) - +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, project=[a, c, rowtime], metadata=[]]], fields=[a, c, rowtime]) + +- Calc(select=[a, c, PROCTIME() AS proctime]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, project=[a, c, rowtime], metadata=[]]], fields=[a, c, rowtime]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/GroupWindowTableAggregateTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/GroupWindowTableAggregateTest.xml index 94788691eb760..3d52090c7dc13 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/GroupWindowTableAggregateTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/GroupWindowTableAggregateTest.xml @@ -36,9 +36,8 @@ Calc(select=[EXPR$0, f1]) +- Calc(select=[EXPR$0 AS proctime, c, f0, (f1 + 1) AS f1]) +- GroupWindowTableAggregate(groupBy=[c], window=[TumblingGroupWindow('w1, e, 50)], properties=[EXPR$0], select=[c, EmptyTableAggFunc(a, b) AS (f0, f1), proctime('w1) AS EXPR$0]) +- Exchange(distribution=[hash[c]]) - +- WatermarkAssigner(rowtime=[d], watermark=[d]) - +- Calc(select=[a, b, c, d, PROCTIME() AS e]) - +- TableSourceScan(table=[[default_catalog, default_database, Table1]], fields=[a, b, c, d]) + +- Calc(select=[a, b, c, d, PROCTIME() AS e]) + +- TableSourceScan(table=[[default_catalog, default_database, Table1]], fields=[a, b, c, d]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/TableAggregateTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/TableAggregateTest.xml index 79ee3d2f2676e..ba276235bb3ef 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/TableAggregateTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/TableAggregateTest.xml @@ -47,9 +47,8 @@ LogicalProject(a=[AS($0, _UTF-16LE'a')], b=[AS($1, _UTF-16LE'b')]) Calc(select=[f0 AS a, f1 AS b]) +- GroupTableAggregate(select=[*org.apache.flink.table.planner.utils.EmptyTableAggFunc*(b) AS (f0, f1)]) +- Exchange(distribution=[single]) - +- WatermarkAssigner(rowtime=[d], watermark=[d]) - +- Calc(select=[a, b, c, d, PROCTIME() AS e]) - +- TableSourceScan(table=[[default_catalog, default_database, Table1]], fields=[a, b, c, d]) + +- Calc(select=[a, b, c, d, PROCTIME() AS e]) + +- TableSourceScan(table=[[default_catalog, default_database, Table1]], fields=[a, b, c, d]) ]]> @@ -70,9 +69,7 @@ Calc(select=[bb, (f0 + 1) AS _c1, f1 AS y]) +- GroupTableAggregate(groupBy=[bb], select=[bb, *org.apache.flink.table.planner.utils.EmptyTableAggFunc*(a, b) AS (f0, f1)]) +- Exchange(distribution=[hash[bb]]) +- Calc(select=[a, b, MOD(b, 5) AS bb]) - +- WatermarkAssigner(rowtime=[d], watermark=[d]) - +- Calc(select=[a, b, d]) - +- TableSourceScan(table=[[default_catalog, default_database, Table1]], fields=[a, b, c, d]) + +- TableSourceScan(table=[[default_catalog, default_database, Table1]], fields=[a, b, c, d]) ]]> @@ -90,9 +87,8 @@ LogicalProject(f0=[$0], f0_0=[$1]) @@ -110,9 +106,8 @@ LogicalProject(f0=[$0], f1=[$1]) @@ -130,10 +125,8 @@ LogicalProject(a=[$0], b=[$1]) @@ -152,9 +145,8 @@ LogicalProject(a=[*org.apache.flink.table.planner.expressions.utils.Func0$$6ad06 Calc(select=[*org.apache.flink.table.planner.expressions.utils.Func0$$6ad060e6c46e5cd996d7b888db472ebc*(f0) AS a, f1 AS b]) +- GroupTableAggregate(select=[EmptyTableAggFunc(a, b) AS (f0, f1)]) +- Exchange(distribution=[single]) - +- WatermarkAssigner(rowtime=[d], watermark=[d]) - +- Calc(select=[a, b, c, d, PROCTIME() AS e]) - +- TableSourceScan(table=[[default_catalog, default_database, Table1]], fields=[a, b, c, d]) + +- Calc(select=[a, b, c, d, PROCTIME() AS e]) + +- TableSourceScan(table=[[default_catalog, default_database, Table1]], fields=[a, b, c, d]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/TableSourceTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/TableSourceTest.xml index c17313c118778..ab387847958b0 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/TableSourceTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/TableSourceTest.xml @@ -92,10 +92,8 @@ LogicalProject(ptime=[$4], name=[$3], val=[$2], id=[$0]) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/MiniBatchIntervalInferTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/MiniBatchIntervalInferTest.scala index a261b4ba7b09e..ee0b54d7030e1 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/MiniBatchIntervalInferTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/MiniBatchIntervalInferTest.scala @@ -76,6 +76,21 @@ class MiniBatchIntervalInferTest extends TableTestBase { |) LIKE MyTable1 (INCLUDING ALL) |""".stripMargin) + // sink used by tests that exercise the post-FLINK-14621 behavior of + // RedundantWatermarkAssignerRemoveRule (which only fires when an enclosing sink is present). + util.addTable(s""" + |CREATE TABLE aggSink ( + | `b` STRING, + | `cnt_distinct_a` BIGINT, + | `max_b` STRING, + | `sum_c` BIGINT, + | PRIMARY KEY (`b`) NOT ENFORCED + |) WITH ( + | 'connector' = 'values', + | 'sink-changelog-mode-enforced' = 'I,UA,UB,D' + |) + |""".stripMargin) + // enable mini-batch util.tableEnv.getConfig .set(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED, Boolean.box(true)) @@ -101,8 +116,15 @@ class MiniBatchIntervalInferTest extends TableTestBase { def testRedundantWatermarkDefinition(): Unit = { util.tableEnv.getConfig .set(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, Duration.ofSeconds(1)) - val sql = "SELECT b, COUNT(DISTINCT a), MAX(b), SUM(c) FROM wmTable1 GROUP BY b" - util.verifyExecPlan(sql) + // Anchor on a sink so RedundantWatermarkAssignerRemoveRule (FLINK-14621) fires and the + // redundant WatermarkAssigner is dropped from the plan. The mini-batch (ProcTime) assigner + // is then attached directly above the table source scan. + val sql = + """ + |INSERT INTO aggSink + |SELECT b, COUNT(DISTINCT a), MAX(b), SUM(c) FROM wmTable1 GROUP BY b + |""".stripMargin + util.verifyExecPlanInsert(sql) } @Test 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..b1b577f63e466 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 @@ -46,6 +46,26 @@ class WindowTableFunctionTest extends TableTestBase { |) |""".stripMargin) + // Sink for tests that need to anchor a Sink rel so RedundantWatermarkAssignerRemoveRule + // (FLINK-14621) can fire on plans where the rowtime watermark is not consumed by any + // downstream operator (e.g. proctime window TVFs). + util.tableEnv.executeSql(s""" + |CREATE TABLE windowSink ( + | a INT, + | b BIGINT, + | c STRING, + | d DECIMAL(10, 3), + | rowtime TIMESTAMP(3), + | proctime TIMESTAMP_LTZ(3), + | window_start TIMESTAMP(3), + | window_end TIMESTAMP(3), + | window_time TIMESTAMP_LTZ(3) + |) with ( + | 'connector' = 'values', + | 'sink-insert-only' = 'true' + |) + |""".stripMargin) + @Test def testTumbleTVF(): Unit = { val sql = @@ -323,12 +343,16 @@ class WindowTableFunctionTest extends TableTestBase { @Test def testProctimeWindowTVFWithMiniBatch(): Unit = { enableMiniBatch() + // Anchored on a sink so RedundantWatermarkAssignerRemoveRule (FLINK-14621) drops the + // rowtime WatermarkAssigner that is unused by the proctime window TVF; the mini-batch + // (ProcTime) assigner is then attached directly above the table source scan. val sql = """ + |INSERT INTO windowSink |SELECT * |FROM TABLE(TUMBLE(TABLE MyTable, DESCRIPTOR(proctime), INTERVAL '15' MINUTE)) |""".stripMargin - util.verifyRelPlan(sql) + util.verifyRelPlanInsert(sql) } @Test diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/AggregateTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/AggregateTest.scala index 7ee7a72223c24..238a155c822ec 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/AggregateTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/AggregateTest.scala @@ -37,6 +37,20 @@ class AggregateTest extends TableTestBase { 'c, 'proctime.proctime, 'rowtime.rowtime) + // Sink used by tests that need an explicit sink to anchor RedundantWatermarkAssignerRemoveRule + // (FLINK-14621) when the upstream watermark is unused by any event-time operator. + util.tableEnv.executeSql(s""" + |CREATE TABLE aggSink ( + | `b` STRING, + | `cnt_distinct_a` BIGINT, + | `max_b` STRING, + | `sum_c` BIGINT, + | PRIMARY KEY (`b`) NOT ENFORCED + |) WITH ( + | 'connector' = 'values', + | 'sink-changelog-mode-enforced' = 'I,UA,UB,D' + |) + |""".stripMargin) util.addTableSource[(Int, Long, String, Boolean)]("T", 'a, 'b, 'c, 'd) util.addTableSource[(Long, Int, String)]("T1", 'a, 'b, 'c) util.addTableSource[(Long, Int, String)]("T2", 'a, 'b, 'c) @@ -100,7 +114,10 @@ class AggregateTest extends TableTestBase { .set(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, Duration.ofSeconds(1)) util.tableEnv.getConfig .set(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_SIZE, Long.box(5000L)) - util.verifyExplain("SELECT b, COUNT(DISTINCT a), MAX(b), SUM(c) FROM MyTable GROUP BY b") + // Anchored on a sink so RedundantWatermarkAssignerRemoveRule (FLINK-14621) drops the + // watermark assigner that the proctime-only group aggregate does not consume. + util.verifyExplainInsert( + "INSERT INTO aggSink SELECT b, COUNT(DISTINCT a), MAX(b), SUM(c) FROM MyTable GROUP BY b") } @Test From f32c1565d63b905511614f1f7ed8b10710c80800 Mon Sep 17 00:00:00 2001 From: Dennis-Mircea Ciupitu Date: Sun, 3 May 2026 10:34:49 +0300 Subject: [PATCH 2/4] [FLINK-14621][flink-table-planner] Fix SQL Gateway failing tests --- .../src/test/resources/sql/table.q | 96 +++---------------- 1 file changed, 14 insertions(+), 82 deletions(-) diff --git a/flink-table/flink-sql-gateway/src/test/resources/sql/table.q b/flink-table/flink-sql-gateway/src/test/resources/sql/table.q index 0b201217f8f35..51734d9cd9349 100644 --- a/flink-table/flink-sql-gateway/src/test/resources/sql/table.q +++ b/flink-table/flink-sql-gateway/src/test/resources/sql/table.q @@ -682,15 +682,11 @@ LogicalProject(user=[$0], product=[$1]) == Optimized Physical Plan == Calc(select=[user, product]) -+- WatermarkAssigner(rowtime=[ts], watermark=[-(ts, 1000:INTERVAL SECOND)]) - +- Calc(select=[user, product, ts]) - +- TableSourceScan(table=[[default_catalog, default_database, orders]], fields=[user, product, amount, ts]) ++- TableSourceScan(table=[[default_catalog, default_database, orders]], fields=[user, product, amount, ts]) == Optimized Execution Plan == Calc(select=[user, product]) -+- WatermarkAssigner(rowtime=[ts], watermark=[(ts - 1000:INTERVAL SECOND)]) - +- Calc(select=[user, product, ts]) - +- TableSourceScan(table=[[default_catalog, default_database, orders]], fields=[user, product, amount, ts]) ++- TableSourceScan(table=[[default_catalog, default_database, orders]], fields=[user, product, amount, ts]) !ok # test explain plan for insert @@ -725,15 +721,11 @@ LogicalProject(user=[$0], product=[$1]) == Optimized Physical Plan == Calc(select=[user, product]) -+- WatermarkAssigner(rowtime=[ts], watermark=[-(ts, 1000:INTERVAL SECOND)]) - +- Calc(select=[user, product, ts]) - +- TableSourceScan(table=[[default_catalog, default_database, orders]], fields=[user, product, amount, ts]) ++- TableSourceScan(table=[[default_catalog, default_database, orders]], fields=[user, product, amount, ts]) == Optimized Execution Plan == Calc(select=[user, product]) -+- WatermarkAssigner(rowtime=[ts], watermark=[(ts - 1000:INTERVAL SECOND)]) - +- Calc(select=[user, product, ts]) - +- TableSourceScan(table=[[default_catalog, default_database, orders]], fields=[user, product, amount, ts]) ++- TableSourceScan(table=[[default_catalog, default_database, orders]], fields=[user, product, amount, ts]) !ok # test explain insert @@ -844,15 +836,11 @@ LogicalProject(user=[$0], product=[$1]) == Optimized Physical Plan == Calc(select=[user, product]) -+- WatermarkAssigner(rowtime=[ts], watermark=[-(ts, 1000:INTERVAL SECOND)]) - +- Calc(select=[user, product, ts]) - +- TableSourceScan(table=[[default_catalog, default_database, orders]], fields=[user, product, amount, ts]) ++- TableSourceScan(table=[[default_catalog, default_database, orders]], fields=[user, product, amount, ts]) == Optimized Execution Plan == Calc(select=[user, product]) -+- WatermarkAssigner(rowtime=[ts], watermark=[(ts - 1000:INTERVAL SECOND)]) - +- Calc(select=[user, product, ts]) - +- TableSourceScan(table=[[default_catalog, default_database, orders]], fields=[user, product, amount, ts]) ++- TableSourceScan(table=[[default_catalog, default_database, orders]], fields=[user, product, amount, ts]) == Physical Execution Plan == { @@ -862,28 +850,6 @@ Calc(select=[user, product]) "pact" : "Data Source", "contents" : "[]:TableSourceScan(table=[[default_catalog, default_database, orders]], fields=[user, product, amount, ts])", "parallelism" : 1 - }, { - "id" : , - "type" : "Calc[]", - "pact" : "Operator", - "contents" : "[]:Calc(select=[user, product, ts])", - "parallelism" : 1, - "predecessors" : [ { - "id" : , - "ship_strategy" : "FORWARD", - "side" : "second" - } ] - }, { - "id" : , - "type" : "WatermarkAssigner[]", - "pact" : "Operator", - "contents" : "[]:WatermarkAssigner(rowtime=[ts], watermark=[(ts - 1000:INTERVAL SECOND)])", - "parallelism" : 1, - "predecessors" : [ { - "id" : , - "ship_strategy" : "FORWARD", - "side" : "second" - } ] }, { "id" : , "type" : "Calc[]", @@ -909,16 +875,12 @@ LogicalProject(user=[$0], product=[$1]) +- LogicalTableScan(table=[[default_catalog, default_database, orders]]) == Optimized Physical Plan == -Calc(select=[user, product]): rowcount = 1.0E8, cumulative cost = {4.0E8 rows, 2.0E8 cpu, 3.6E9 io, 0.0 network, 0.0 memory} -+- WatermarkAssigner(rowtime=[ts], watermark=[-(ts, 1000:INTERVAL SECOND)]): rowcount = 1.0E8, cumulative cost = {3.0E8 rows, 2.0E8 cpu, 3.6E9 io, 0.0 network, 0.0 memory} - +- Calc(select=[user, product, ts]): rowcount = 1.0E8, cumulative cost = {2.0E8 rows, 1.0E8 cpu, 3.6E9 io, 0.0 network, 0.0 memory} - +- TableSourceScan(table=[[default_catalog, default_database, orders]], fields=[user, product, amount, ts]): rowcount = 1.0E8, cumulative cost = {1.0E8 rows, 1.0E8 cpu, 3.6E9 io, 0.0 network, 0.0 memory} +Calc(select=[user, product]): rowcount = 1.0E8, cumulative cost = {2.0E8 rows, 1.0E8 cpu, 3.6E9 io, 0.0 network, 0.0 memory} ++- TableSourceScan(table=[[default_catalog, default_database, orders]], fields=[user, product, amount, ts]): rowcount = 1.0E8, cumulative cost = {1.0E8 rows, 1.0E8 cpu, 3.6E9 io, 0.0 network, 0.0 memory} == Optimized Execution Plan == Calc(select=[user, product]) -+- WatermarkAssigner(rowtime=[ts], watermark=[(ts - 1000:INTERVAL SECOND)]) - +- Calc(select=[user, product, ts]) - +- TableSourceScan(table=[[default_catalog, default_database, orders]], fields=[user, product, amount, ts]) ++- TableSourceScan(table=[[default_catalog, default_database, orders]], fields=[user, product, amount, ts]) !ok # test explain select with CHANGELOG_MODE @@ -932,15 +894,11 @@ LogicalProject(user=[$0], product=[$1]) == Optimized Physical Plan == Calc(select=[user, product], changelogMode=[I]) -+- WatermarkAssigner(rowtime=[ts], watermark=[-(ts, 1000:INTERVAL SECOND)], changelogMode=[I]) - +- Calc(select=[user, product, ts], changelogMode=[I]) - +- TableSourceScan(table=[[default_catalog, default_database, orders]], fields=[user, product, amount, ts], changelogMode=[I]) ++- TableSourceScan(table=[[default_catalog, default_database, orders]], fields=[user, product, amount, ts], changelogMode=[I]) == Optimized Execution Plan == Calc(select=[user, product]) -+- WatermarkAssigner(rowtime=[ts], watermark=[(ts - 1000:INTERVAL SECOND)]) - +- Calc(select=[user, product, ts]) - +- TableSourceScan(table=[[default_catalog, default_database, orders]], fields=[user, product, amount, ts]) ++- TableSourceScan(table=[[default_catalog, default_database, orders]], fields=[user, product, amount, ts]) !ok # test explain select with all details @@ -953,16 +911,12 @@ LogicalProject(user=[$0], product=[$1]) +- LogicalTableScan(table=[[default_catalog, default_database, orders]]) == Optimized Physical Plan == -Calc(select=[user, product], changelogMode=[I]): rowcount = 1.0E8, cumulative cost = {4.0E8 rows, 2.0E8 cpu, 3.6E9 io, 0.0 network, 0.0 memory} -+- WatermarkAssigner(rowtime=[ts], watermark=[-(ts, 1000:INTERVAL SECOND)], changelogMode=[I]): rowcount = 1.0E8, cumulative cost = {3.0E8 rows, 2.0E8 cpu, 3.6E9 io, 0.0 network, 0.0 memory} - +- Calc(select=[user, product, ts], changelogMode=[I]): rowcount = 1.0E8, cumulative cost = {2.0E8 rows, 1.0E8 cpu, 3.6E9 io, 0.0 network, 0.0 memory} - +- TableSourceScan(table=[[default_catalog, default_database, orders]], fields=[user, product, amount, ts], changelogMode=[I]): rowcount = 1.0E8, cumulative cost = {1.0E8 rows, 1.0E8 cpu, 3.6E9 io, 0.0 network, 0.0 memory} +Calc(select=[user, product], changelogMode=[I]): rowcount = 1.0E8, cumulative cost = {2.0E8 rows, 1.0E8 cpu, 3.6E9 io, 0.0 network, 0.0 memory} ++- TableSourceScan(table=[[default_catalog, default_database, orders]], fields=[user, product, amount, ts], changelogMode=[I]): rowcount = 1.0E8, cumulative cost = {1.0E8 rows, 1.0E8 cpu, 3.6E9 io, 0.0 network, 0.0 memory} == Optimized Execution Plan == Calc(select=[user, product]) -+- WatermarkAssigner(rowtime=[ts], watermark=[(ts - 1000:INTERVAL SECOND)]) - +- Calc(select=[user, product, ts]) - +- TableSourceScan(table=[[default_catalog, default_database, orders]], fields=[user, product, amount, ts]) ++- TableSourceScan(table=[[default_catalog, default_database, orders]], fields=[user, product, amount, ts]) == Physical Execution Plan == { @@ -972,28 +926,6 @@ Calc(select=[user, product]) "pact" : "Data Source", "contents" : "[]:TableSourceScan(table=[[default_catalog, default_database, orders]], fields=[user, product, amount, ts])", "parallelism" : 1 - }, { - "id" : , - "type" : "Calc[]", - "pact" : "Operator", - "contents" : "[]:Calc(select=[user, product, ts])", - "parallelism" : 1, - "predecessors" : [ { - "id" : , - "ship_strategy" : "FORWARD", - "side" : "second" - } ] - }, { - "id" : , - "type" : "WatermarkAssigner[]", - "pact" : "Operator", - "contents" : "[]:WatermarkAssigner(rowtime=[ts], watermark=[(ts - 1000:INTERVAL SECOND)])", - "parallelism" : 1, - "predecessors" : [ { - "id" : , - "ship_strategy" : "FORWARD", - "side" : "second" - } ] }, { "id" : , "type" : "Calc[]", From 19060ddffea5435d83c77814dde85bf079350868 Mon Sep 17 00:00:00 2001 From: Dennis-Mircea Ciupitu Date: Sun, 3 May 2026 14:44:21 +0300 Subject: [PATCH 3/4] [FLINK-14621][flink-table-planner] Fix SQL Client failing tests --- .../src/test/resources/sql/table.q | 162 +++++------------- 1 file changed, 45 insertions(+), 117 deletions(-) diff --git a/flink-table/flink-sql-client/src/test/resources/sql/table.q b/flink-table/flink-sql-client/src/test/resources/sql/table.q index b5caeafb5cc91..7a35c94b4e525 100644 --- a/flink-table/flink-sql-client/src/test/resources/sql/table.q +++ b/flink-table/flink-sql-client/src/test/resources/sql/table.q @@ -1031,7 +1031,7 @@ CREATE TABLE IF NOT EXISTS daily_orders ( PRIMARY KEY(dt, `user`) NOT ENFORCED ) PARTITIONED BY (dt) WITH ( 'connector' = 'filesystem', - 'path' = '$VAR_BATCH_PATH', + 'path' = '/var/folders/hh/kby1w5q92h34hc76_2jbntrh0000gn/T/junit-10269433974092078821/cbfbbe71-3c0b-48db-9550-c9b4687887bb2667139526066467859', 'format' = 'csv' ); [INFO] Execute statement succeeded. @@ -1039,9 +1039,9 @@ CREATE TABLE IF NOT EXISTS daily_orders ( # test explain plan for select explain plan for select `user`, product from orders; -+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| result | -+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ ++--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| result | ++--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ | == Abstract Syntax Tree == LogicalProject(user=[$0], product=[$1]) +- LogicalWatermarkAssigner(rowtime=[ts], watermark=[-($3, 1000:INTERVAL SECOND)]) @@ -1050,17 +1050,13 @@ LogicalProject(user=[$0], product=[$1]) == Optimized Physical Plan == Calc(select=[user, product]) -+- WatermarkAssigner(rowtime=[ts], watermark=[-(ts, 1000:INTERVAL SECOND)]) - +- Calc(select=[user, product, ts]) - +- TableSourceScan(table=[[default_catalog, default_database, orders]], fields=[user, product, amount, ts]) ++- TableSourceScan(table=[[default_catalog, default_database, orders]], fields=[user, product, amount, ts]) == Optimized Execution Plan == Calc(select=[user, product]) -+- WatermarkAssigner(rowtime=[ts], watermark=[(ts - 1000:INTERVAL SECOND)]) - +- Calc(select=[user, product, ts]) - +- TableSourceScan(table=[[default_catalog, default_database, orders]], fields=[user, product, amount, ts]) ++- TableSourceScan(table=[[default_catalog, default_database, orders]], fields=[user, product, amount, ts]) | -+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ ++--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ 1 row in set !ok @@ -1139,9 +1135,9 @@ Sink(table=[default_catalog.default_database.daily_orders], fields=[user, produc # test explain select explain select `user`, product from orders; -+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| result | -+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ ++--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| result | ++--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ | == Abstract Syntax Tree == LogicalProject(user=[$0], product=[$1]) +- LogicalWatermarkAssigner(rowtime=[ts], watermark=[-($3, 1000:INTERVAL SECOND)]) @@ -1150,17 +1146,13 @@ LogicalProject(user=[$0], product=[$1]) == Optimized Physical Plan == Calc(select=[user, product]) -+- WatermarkAssigner(rowtime=[ts], watermark=[-(ts, 1000:INTERVAL SECOND)]) - +- Calc(select=[user, product, ts]) - +- TableSourceScan(table=[[default_catalog, default_database, orders]], fields=[user, product, amount, ts]) ++- TableSourceScan(table=[[default_catalog, default_database, orders]], fields=[user, product, amount, ts]) == Optimized Execution Plan == Calc(select=[user, product]) -+- WatermarkAssigner(rowtime=[ts], watermark=[(ts - 1000:INTERVAL SECOND)]) - +- Calc(select=[user, product, ts]) - +- TableSourceScan(table=[[default_catalog, default_database, orders]], fields=[user, product, amount, ts]) ++- TableSourceScan(table=[[default_catalog, default_database, orders]], fields=[user, product, amount, ts]) | -+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ ++--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ 1 row in set !ok @@ -1272,9 +1264,9 @@ Sink(table=[default_catalog.default_database.orders2], fields=[user, product, am # test explain select with json format explain json_execution_plan select `user`, product from orders; -+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| result | -+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ ++----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| result | ++----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ | == Abstract Syntax Tree == LogicalProject(user=[$0], product=[$1]) +- LogicalWatermarkAssigner(rowtime=[ts], watermark=[-($3, 1000:INTERVAL SECOND)]) @@ -1283,15 +1275,11 @@ LogicalProject(user=[$0], product=[$1]) == Optimized Physical Plan == Calc(select=[user, product]) -+- WatermarkAssigner(rowtime=[ts], watermark=[-(ts, 1000:INTERVAL SECOND)]) - +- Calc(select=[user, product, ts]) - +- TableSourceScan(table=[[default_catalog, default_database, orders]], fields=[user, product, amount, ts]) ++- TableSourceScan(table=[[default_catalog, default_database, orders]], fields=[user, product, amount, ts]) == Optimized Execution Plan == Calc(select=[user, product]) -+- WatermarkAssigner(rowtime=[ts], watermark=[(ts - 1000:INTERVAL SECOND)]) - +- Calc(select=[user, product, ts]) - +- TableSourceScan(table=[[default_catalog, default_database, orders]], fields=[user, product, amount, ts]) ++- TableSourceScan(table=[[default_catalog, default_database, orders]], fields=[user, product, amount, ts]) == Physical Execution Plan == { @@ -1301,28 +1289,6 @@ Calc(select=[user, product]) "pact" : "Data Source", "contents" : "[]:TableSourceScan(table=[[default_catalog, default_database, orders]], fields=[user, product, amount, ts])", "parallelism" : 1 - }, { - "id" : , - "type" : "Calc[]", - "pact" : "Operator", - "contents" : "[]:Calc(select=[user, product, ts])", - "parallelism" : 1, - "predecessors" : [ { - "id" : , - "ship_strategy" : "FORWARD", - "side" : "second" - } ] - }, { - "id" : , - "type" : "WatermarkAssigner[]", - "pact" : "Operator", - "contents" : "[]:WatermarkAssigner(rowtime=[ts], watermark=[(ts - 1000:INTERVAL SECOND)])", - "parallelism" : 1, - "predecessors" : [ { - "id" : , - "ship_strategy" : "FORWARD", - "side" : "second" - } ] }, { "id" : , "type" : "Calc[]", @@ -1336,15 +1302,15 @@ Calc(select=[user, product]) } ] } ] } | -+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ ++----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ 1 row in set !ok # test explain select with ESTIMATED_COST explain estimated_cost select `user`, product from orders; -+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| result | -+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ ++--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| result | ++--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ | == Abstract Syntax Tree == LogicalProject(user=[$0], product=[$1]) +- LogicalWatermarkAssigner(rowtime=[ts], watermark=[-($3, 1000:INTERVAL SECOND)]) @@ -1352,26 +1318,22 @@ LogicalProject(user=[$0], product=[$1]) +- LogicalTableScan(table=[[default_catalog, default_database, orders]]) == Optimized Physical Plan == -Calc(select=[user, product]): rowcount = 1.0E8, cumulative cost = {4.0E8 rows, 2.0E8 cpu, 3.6E9 io, 0.0 network, 0.0 memory} -+- WatermarkAssigner(rowtime=[ts], watermark=[-(ts, 1000:INTERVAL SECOND)]): rowcount = 1.0E8, cumulative cost = {3.0E8 rows, 2.0E8 cpu, 3.6E9 io, 0.0 network, 0.0 memory} - +- Calc(select=[user, product, ts]): rowcount = 1.0E8, cumulative cost = {2.0E8 rows, 1.0E8 cpu, 3.6E9 io, 0.0 network, 0.0 memory} - +- TableSourceScan(table=[[default_catalog, default_database, orders]], fields=[user, product, amount, ts]): rowcount = 1.0E8, cumulative cost = {1.0E8 rows, 1.0E8 cpu, 3.6E9 io, 0.0 network, 0.0 memory} +Calc(select=[user, product]): rowcount = 1.0E8, cumulative cost = {2.0E8 rows, 1.0E8 cpu, 3.6E9 io, 0.0 network, 0.0 memory} ++- TableSourceScan(table=[[default_catalog, default_database, orders]], fields=[user, product, amount, ts]): rowcount = 1.0E8, cumulative cost = {1.0E8 rows, 1.0E8 cpu, 3.6E9 io, 0.0 network, 0.0 memory} == Optimized Execution Plan == Calc(select=[user, product]) -+- WatermarkAssigner(rowtime=[ts], watermark=[(ts - 1000:INTERVAL SECOND)]) - +- Calc(select=[user, product, ts]) - +- TableSourceScan(table=[[default_catalog, default_database, orders]], fields=[user, product, amount, ts]) ++- TableSourceScan(table=[[default_catalog, default_database, orders]], fields=[user, product, amount, ts]) | -+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ ++--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ 1 row in set !ok # test explain select with CHANGELOG_MODE explain changelog_mode select `user`, product from orders; -+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| result | -+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ ++----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| result | ++----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ | == Abstract Syntax Tree == LogicalProject(user=[$0], product=[$1]) +- LogicalWatermarkAssigner(rowtime=[ts], watermark=[-($3, 1000:INTERVAL SECOND)]) @@ -1380,26 +1342,22 @@ LogicalProject(user=[$0], product=[$1]) == Optimized Physical Plan == Calc(select=[user, product], changelogMode=[I]) -+- WatermarkAssigner(rowtime=[ts], watermark=[-(ts, 1000:INTERVAL SECOND)], changelogMode=[I]) - +- Calc(select=[user, product, ts], changelogMode=[I]) - +- TableSourceScan(table=[[default_catalog, default_database, orders]], fields=[user, product, amount, ts], changelogMode=[I]) ++- TableSourceScan(table=[[default_catalog, default_database, orders]], fields=[user, product, amount, ts], changelogMode=[I]) == Optimized Execution Plan == Calc(select=[user, product]) -+- WatermarkAssigner(rowtime=[ts], watermark=[(ts - 1000:INTERVAL SECOND)]) - +- Calc(select=[user, product, ts]) - +- TableSourceScan(table=[[default_catalog, default_database, orders]], fields=[user, product, amount, ts]) ++- TableSourceScan(table=[[default_catalog, default_database, orders]], fields=[user, product, amount, ts]) | -+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ ++----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ 1 row in set !ok # test explain select with PLAN_ADVICE # test explain select with PLAN_ADVICE explain plan_advice select sum(`amount`) as revenue, count(distinct `user`) as buyer_cnt from orders; -+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| result | -+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ ++------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| result | ++------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ | == Abstract Syntax Tree == LogicalAggregate(group=[{}], revenue=[SUM($0)], buyer_cnt=[COUNT(DISTINCT $1)]) +- LogicalProject(amount=[$2], user=[$0]) @@ -1411,9 +1369,7 @@ LogicalAggregate(group=[{}], revenue=[SUM($0)], buyer_cnt=[COUNT(DISTINCT $1)]) GroupAggregate(advice=[1], select=[SUM(amount) AS revenue, COUNT(DISTINCT user) AS buyer_cnt]) +- Exchange(distribution=[single]) +- Calc(select=[amount, user]) - +- WatermarkAssigner(rowtime=[ts], watermark=[-(ts, 1000:INTERVAL SECOND)]) - +- Calc(select=[amount, user, ts]) - +- TableSourceScan(table=[[default_catalog, default_database, orders]], fields=[user, product, amount, ts]) + +- TableSourceScan(table=[[default_catalog, default_database, orders]], fields=[user, product, amount, ts]) advice[1]: [ADVICE] You might want to enable local-global two-phase optimization by configuring ('table.exec.mini-batch.enabled' to 'true', 'table.exec.mini-batch.allow-latency' to a positive long value, 'table.exec.mini-batch.size' to a positive long value). @@ -1421,19 +1377,17 @@ advice[1]: [ADVICE] You might want to enable local-global two-phase optimization GroupAggregate(select=[SUM(amount) AS revenue, COUNT(DISTINCT user) AS buyer_cnt]) +- Exchange(distribution=[single]) +- Calc(select=[amount, user]) - +- WatermarkAssigner(rowtime=[ts], watermark=[(ts - 1000:INTERVAL SECOND)]) - +- Calc(select=[amount, user, ts]) - +- TableSourceScan(table=[[default_catalog, default_database, orders]], fields=[user, product, amount, ts]) + +- TableSourceScan(table=[[default_catalog, default_database, orders]], fields=[user, product, amount, ts]) | -+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ ++------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ 1 row in set !ok # test explain select with all details explain changelog_mode, estimated_cost, plan_advice, json_execution_plan select `user`, product from orders; -+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| result | -+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ ++------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| result | ++------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ | == Abstract Syntax Tree == LogicalProject(user=[$0], product=[$1]) +- LogicalWatermarkAssigner(rowtime=[ts], watermark=[-($3, 1000:INTERVAL SECOND)]) @@ -1441,18 +1395,14 @@ LogicalProject(user=[$0], product=[$1]) +- LogicalTableScan(table=[[default_catalog, default_database, orders]]) == Optimized Physical Plan With Advice == -Calc(select=[user, product], changelogMode=[I]): rowcount = 1.0E8, cumulative cost = {4.0E8 rows, 2.0E8 cpu, 3.6E9 io, 0.0 network, 0.0 memory} -+- WatermarkAssigner(rowtime=[ts], watermark=[-(ts, 1000:INTERVAL SECOND)], changelogMode=[I]): rowcount = 1.0E8, cumulative cost = {3.0E8 rows, 2.0E8 cpu, 3.6E9 io, 0.0 network, 0.0 memory} - +- Calc(select=[user, product, ts], changelogMode=[I]): rowcount = 1.0E8, cumulative cost = {2.0E8 rows, 1.0E8 cpu, 3.6E9 io, 0.0 network, 0.0 memory} - +- TableSourceScan(table=[[default_catalog, default_database, orders]], fields=[user, product, amount, ts], changelogMode=[I]): rowcount = 1.0E8, cumulative cost = {1.0E8 rows, 1.0E8 cpu, 3.6E9 io, 0.0 network, 0.0 memory} +Calc(select=[user, product], changelogMode=[I]): rowcount = 1.0E8, cumulative cost = {2.0E8 rows, 1.0E8 cpu, 3.6E9 io, 0.0 network, 0.0 memory} ++- TableSourceScan(table=[[default_catalog, default_database, orders]], fields=[user, product, amount, ts], changelogMode=[I]): rowcount = 1.0E8, cumulative cost = {1.0E8 rows, 1.0E8 cpu, 3.6E9 io, 0.0 network, 0.0 memory} No available advice... == Optimized Execution Plan == Calc(select=[user, product]) -+- WatermarkAssigner(rowtime=[ts], watermark=[(ts - 1000:INTERVAL SECOND)]) - +- Calc(select=[user, product, ts]) - +- TableSourceScan(table=[[default_catalog, default_database, orders]], fields=[user, product, amount, ts]) ++- TableSourceScan(table=[[default_catalog, default_database, orders]], fields=[user, product, amount, ts]) == Physical Execution Plan == { @@ -1462,28 +1412,6 @@ Calc(select=[user, product]) "pact" : "Data Source", "contents" : "[]:TableSourceScan(table=[[default_catalog, default_database, orders]], fields=[user, product, amount, ts])", "parallelism" : 1 - }, { - "id" : , - "type" : "Calc[]", - "pact" : "Operator", - "contents" : "[]:Calc(select=[user, product, ts])", - "parallelism" : 1, - "predecessors" : [ { - "id" : , - "ship_strategy" : "FORWARD", - "side" : "second" - } ] - }, { - "id" : , - "type" : "WatermarkAssigner[]", - "pact" : "Operator", - "contents" : "[]:WatermarkAssigner(rowtime=[ts], watermark=[(ts - 1000:INTERVAL SECOND)])", - "parallelism" : 1, - "predecessors" : [ { - "id" : , - "ship_strategy" : "FORWARD", - "side" : "second" - } ] }, { "id" : , "type" : "Calc[]", @@ -1497,6 +1425,6 @@ Calc(select=[user, product]) } ] } ] } | -+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ ++------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ 1 row in set !ok From df4a8603997347ae6177101c42c42f09bf67a136 Mon Sep 17 00:00:00 2001 From: Dennis-Mircea Ciupitu Date: Wed, 6 May 2026 11:21:27 +0300 Subject: [PATCH 4/4] [FLINK-14621][flink-table-planner] Replace hardcoded value with env var --- flink-table/flink-sql-client/src/test/resources/sql/table.q | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-table/flink-sql-client/src/test/resources/sql/table.q b/flink-table/flink-sql-client/src/test/resources/sql/table.q index 7a35c94b4e525..1000ee243ed7d 100644 --- a/flink-table/flink-sql-client/src/test/resources/sql/table.q +++ b/flink-table/flink-sql-client/src/test/resources/sql/table.q @@ -1031,7 +1031,7 @@ CREATE TABLE IF NOT EXISTS daily_orders ( PRIMARY KEY(dt, `user`) NOT ENFORCED ) PARTITIONED BY (dt) WITH ( 'connector' = 'filesystem', - 'path' = '/var/folders/hh/kby1w5q92h34hc76_2jbntrh0000gn/T/junit-10269433974092078821/cbfbbe71-3c0b-48db-9550-c9b4687887bb2667139526066467859', + 'path' = '$VAR_BATCH_PATH', 'format' = 'csv' ); [INFO] Execute statement succeeded.