xuyangzhong commented on code in PR #26051: URL: https://github.com/apache/flink/pull/26051#discussion_r1944682832
########## flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecDeduplicate.java: ########## @@ -339,25 +362,39 @@ OneInputStreamOperator<RowData, RowData> createDeduplicateOperator() { } } else { if (isAsyncStateEnabled()) { - AsyncStateRowTimeDeduplicateFunction processFunction = - new AsyncStateRowTimeDeduplicateFunction( - rowTypeInfo, - stateRetentionTime, - rowtimeIndex, - generateUpdateBefore, - generateInsert(), - keepLastRow); - return new AsyncKeyedProcessOperator<>(processFunction); + if (!keepLastRow && outputInsertOnly) { + checkState(canBeInsertOnly(config, keepLastRow)); Review Comment: nit: can we need to double check the field `outputInsertOnly`? The same goes for the following. ########## flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalRank.scala: ########## @@ -109,28 +109,34 @@ class StreamPhysicalRank( .item("select", getRowType.getFieldNames.mkString(", ")) } - private def getDeduplicateDescription(isRowtime: Boolean, isLastRow: Boolean): String = { + private def getDeduplicateDescription( + isRowtime: Boolean, + isLastRow: Boolean, + insertOnly: Boolean): String = { val fieldNames = getRowType.getFieldNames val orderString = if (isRowtime) "ROWTIME" else "PROCTIME" val keep = if (isLastRow) "LastRow" else "FirstRow" - s"Deduplicate(keep=[$keep], key=[${partitionKey.toArray.map(fieldNames.get).mkString(", ")}], order=[$orderString])" + s"Deduplicate(keep=[$keep], key=[${partitionKey.toArray.map(fieldNames.get).mkString(", ")}], order=[$orderString], outputInsertOnly=[$insertOnly])" } override def translateToExecNode(): ExecNode[_] = { val generateUpdateBefore = ChangelogPlanUtils.generateUpdateBefore(this) if (RankUtil.canConvertToDeduplicate(this)) { val keepLastRow = RankUtil.keepLastDeduplicateRow(orderKey) + val tableConfig = unwrapTableConfig(this) + val outputInsertOnly = StreamExecDeduplicate.canBeInsertOnly(tableConfig, keepLastRow) Review Comment: nit: use `val outputInsertOnly = ChangelogPlanUtils.isInsertOnly(this)`? ########## flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkChangelogModeInferenceProgram.scala: ########## @@ -220,6 +221,38 @@ class FlinkChangelogModeInferenceProgram extends FlinkOptimizeProgram[StreamOpti val providedTrait = ModifyKindSetTrait.INSERT_ONLY createNewNode(rel, children, providedTrait, requiredTrait, requester) + case rank: StreamPhysicalRank if RankUtil.isDeduplication(rank) => + val children = visitChildren(rel, ModifyKindSetTrait.ALL_CHANGES) + val tableConfig = unwrapTableConfig(rank) + + // if the rank is deduplication and can be executed as insert-only, forward that information + val insertOnly = children Review Comment: nit: using `val insertOnly = children.forall(ChangelogPlanUtils.isInsertOnly)` to resolve idea warning ########## flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/DeduplicateTest.scala: ########## @@ -139,14 +139,17 @@ class DeduplicateTest extends TableTestBase { @Test def testSimpleFirstRowOnRowtime(): Unit = { + // indirectly check output insert only via used SUM or SUM_RETRACT aggregation function Review Comment: Tips, use `util.verifyExplain(sql, ExplainDetail.CHANGELOG_MODE)` can also print the changelog mode in physical nodes. For example: ``` @Test def test(): Unit = { val sql = """ | SELECT a, b, c | FROM ( | SELECT *, | ROW_NUMBER() OVER (PARTITION BY a ORDER BY rowtime ASC) as rank_num | FROM MyTable) | WHERE rank_num <= 1 """.stripMargin util.verifyExplain(sql, ExplainDetail.CHANGELOG_MODE) } ``` Before this pr: ``` == Abstract Syntax Tree == LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[<=($5, 1)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], proctime=[$3], rowtime=[$4], rank_num=[ROW_NUMBER() OVER (PARTITION BY $0 ORDER BY $4 NULLS FIRST)]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) == Optimized Physical Plan == Calc(select=[a, b, c], changelogMode=[I,UA,D]) +- Rank(strategy=[AppendFastStrategy], rankType=[ROW_NUMBER], rankRange=[rankStart=1, rankEnd=1], partitionBy=[a], orderBy=[ROWTIME rowtime ASC], select=[a, b, c, rowtime], changelogMode=[I,UA,D]) +- Exchange(distribution=[hash[a]], changelogMode=[I]) +- Calc(select=[a, b, c, rowtime], changelogMode=[I]) +- DataStreamScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, proctime, rowtime], changelogMode=[I]) == Optimized Execution Plan == Calc(select=[a, b, c]) +- Deduplicate(keep=[FirstRow], key=[a], order=[ROWTIME], outputInsertOnly=[false]) +- Exchange(distribution=[hash[a]]) +- Calc(select=[a, b, c, rowtime]) +- DataStreamScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, proctime, rowtime]) ``` After applying this pr: ``` == Abstract Syntax Tree == LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[<=($5, 1)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], proctime=[$3], rowtime=[$4], rank_num=[ROW_NUMBER() OVER (PARTITION BY $0 ORDER BY $4 NULLS FIRST)]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) == Optimized Physical Plan == Calc(select=[a, b, c], changelogMode=[I]) +- Rank(strategy=[AppendFastStrategy], rankType=[ROW_NUMBER], rankRange=[rankStart=1, rankEnd=1], partitionBy=[a], orderBy=[ROWTIME rowtime ASC], select=[a, b, c, rowtime], changelogMode=[I]) +- Exchange(distribution=[hash[a]], changelogMode=[I]) +- Calc(select=[a, b, c, rowtime], changelogMode=[I]) +- DataStreamScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, proctime, rowtime], changelogMode=[I]) == Optimized Execution Plan == Calc(select=[a, b, c]) +- Deduplicate(keep=[FirstRow], key=[a], order=[ROWTIME], outputInsertOnly=[true]) +- Exchange(distribution=[hash[a]]) +- Calc(select=[a, b, c, rowtime]) +- DataStreamScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, proctime, rowtime]) ``` ########## flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkChangelogModeInferenceProgram.scala: ########## @@ -220,6 +221,38 @@ class FlinkChangelogModeInferenceProgram extends FlinkOptimizeProgram[StreamOpti val providedTrait = ModifyKindSetTrait.INSERT_ONLY createNewNode(rel, children, providedTrait, requiredTrait, requester) + case rank: StreamPhysicalRank if RankUtil.isDeduplication(rank) => + val children = visitChildren(rel, ModifyKindSetTrait.ALL_CHANGES) + val tableConfig = unwrapTableConfig(rank) + + // if the rank is deduplication and can be executed as insert-only, forward that information + val insertOnly = children + .filterNot( + rel => { + rel.getTraitSet.contains(ModifyKindSetTrait.INSERT_ONLY) + }) + .isEmpty + + val providedTrait = { + if ( + insertOnly && StreamExecDeduplicate.canBeInsertOnly( Review Comment: It's a bit strange to use the exec node method in a place that only handles physical nodes. How about moving this method to `RankUtil` and naming it something like `RankUtil#outputInsertOnlyInDeduplicate`? ########## flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/deduplicate/RowTimeDeduplicateKeepFirstRowFunction.java: ########## @@ -0,0 +1,115 @@ +/* + * 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.runtime.operators.deduplicate; + +import org.apache.flink.api.common.functions.OpenContext; +import org.apache.flink.api.common.state.StateTtlConfig; +import org.apache.flink.api.common.state.ValueState; +import org.apache.flink.api.common.state.ValueStateDescriptor; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.streaming.api.functions.KeyedProcessFunction; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; +import org.apache.flink.util.Collector; + +import static org.apache.flink.table.runtime.operators.deduplicate.utils.DeduplicateFunctionHelper.checkInsertOnly; +import static org.apache.flink.table.runtime.operators.deduplicate.utils.DeduplicateFunctionHelper.shouldKeepCurrentRow; +import static org.apache.flink.table.runtime.util.StateConfigUtil.createTtlConfig; + +/** + * This function is used to deduplicate on keys and keeps only first row on row time. It produces + * append only stream thanks to emitting results only via firing the timers. + */ +public class RowTimeDeduplicateKeepFirstRowFunction + extends KeyedProcessFunction<RowData, RowData, RowData> { + + private static final long serialVersionUID = 1L; + + // the TypeInformation of the values in the state. + private final TypeInformation<RowData> typeInfo; + private final long stateRetentionTime; + private final int rowtimeIndex; + + // state stores previous message under the key. + protected ValueState<RowData> waitingToEmitOnTimerState; + protected ValueState<Boolean> alreadyEmittedState; + + public RowTimeDeduplicateKeepFirstRowFunction( + InternalTypeInfo<RowData> typeInfo, long minRetentionTime, int rowtimeIndex) { + this.typeInfo = typeInfo; + this.stateRetentionTime = minRetentionTime; + this.rowtimeIndex = rowtimeIndex; + } + + @Override + public void open(OpenContext openContext) throws Exception { + super.open(openContext); + + // We don't enable TTL on the timer's state, because we rely on the state cleaning up on + // watermark. Also otherwise TTL clean up before firing the watermark would cause a data + // loss. + ValueStateDescriptor<RowData> timerStateDesc = + new ValueStateDescriptor<>("waiting-to-emit-on-timer", typeInfo); + waitingToEmitOnTimerState = getRuntimeContext().getState(timerStateDesc); + + ValueStateDescriptor<Boolean> stateDesc = + new ValueStateDescriptor<>("already-emitted-state-boolean", Types.BOOLEAN); + StateTtlConfig ttlConfig = createTtlConfig(stateRetentionTime); + if (ttlConfig.isEnabled()) { + stateDesc.enableTimeToLive(ttlConfig); + } + alreadyEmittedState = getRuntimeContext().getState(stateDesc); + } + + @Override + public void processElement(RowData input, Context ctx, Collector<RowData> out) + throws Exception { + checkInsertOnly(input); + Boolean allreadyEmitted = alreadyEmittedState.value(); + if (allreadyEmitted != null && allreadyEmitted) { + // result has already been emitted, we can not retract/emit anything different. + return; + } + long rowtime = input.getLong(rowtimeIndex); + if (rowtime < ctx.timerService().currentWatermark()) { Review Comment: What about introducing a metric to log late num used for debugging just like other window operators? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org