fhueske commented on code in PR #26424: URL: https://github.com/apache/flink/pull/26424#discussion_r2184645556
########## flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/over/NonTimeRowsUnboundedPrecedingFunction.java: ########## @@ -0,0 +1,421 @@ +/* + * 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.over; + +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.generated.GeneratedAggsHandleFunction; +import org.apache.flink.table.runtime.generated.GeneratedRecordComparator; +import org.apache.flink.table.runtime.generated.GeneratedRecordEqualiser; +import org.apache.flink.table.runtime.keyselector.RowDataKeySelector; +import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.types.RowKind; +import org.apache.flink.util.Collector; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.List; + +/** + * The NonTimeRowsUnboundedPrecedingFunction class is a specialized implementation for processing + * unbounded OVER window aggregations, particularly for non-time-based rows queries in Apache Flink. + * It maintains strict ordering of rows within partitions and handles the full changelog lifecycle + * (inserts, updates, deletes). + * + * <p>Key Components and Assumptions + * + * <p>Data Structure Design: (1) Maintains a sorted list of tuples containing sort keys and lists of + * IDs for each key (2) Each incoming row is assigned a unique Long ID (starting from + * Long.MIN_VALUE) (3) Uses multiple state types to track rows, sort orders, and aggregations + * + * <p>State Management: (1) idState: Counter for generating unique row IDs (2) sortedListState: + * Ordered list of sort keys with their associated row IDs (3) valueMapState: Maps IDs to their + * corresponding input rows (4) accMapState: Maps RowData(ID)s to their accumulated values + * + * <p>Processing Model: (1) For inserts/updates: Adds rows to the appropriate position based on sort + * key (2) For deletes: Removes rows by matching both sort key and row content (3) Recalculates + * aggregates for affected rows and emits the appropriate events (4) Skips redundant events when + * accumulators haven't changed to reduce network traffic + * + * <p>Optimization Assumptions: (1) Skip emitting updates when accumulators haven't changed to + * reduce network traffic (2) Uses state TTL for automatic cleanup of stale data (3) Carefully + * manages row state to support incremental calculations + * + * <p>Retraction Handling: (1) Handles retraction mode (DELETE/UPDATE_BEFORE) events properly (2) + * Supports the proper processing of changelog streams + * + * <p>Limitations + * + * <p>Linear search performance: - The current implementation uses a linear search to find the + * correct position for each sort key. This can be optimized using a binary search for large state + * sizes. + * + * <p>State size and performance: - The implementation maintains multiple state types that could + * grow large with high cardinality data + * + * <p>Linear recalculation: - When processing updates, all subsequent elements need to be + * recalculated, which could be inefficient for large windows + */ +public class NonTimeRowsUnboundedPrecedingFunction<K> + extends AbstractNonTimeUnboundedPrecedingOver<K> { + private static final long serialVersionUID = 1L; + + private static final Logger LOG = + LoggerFactory.getLogger(NonTimeRowsUnboundedPrecedingFunction.class); + + public NonTimeRowsUnboundedPrecedingFunction( + long stateRetentionTime, + GeneratedAggsHandleFunction genAggsHandler, + GeneratedRecordEqualiser genRecordEqualiser, + GeneratedRecordEqualiser genSortKeyEqualiser, + GeneratedRecordComparator genSortKeyComparator, + LogicalType[] accTypes, + LogicalType[] inputFieldTypes, + LogicalType[] sortKeyTypes, + RowDataKeySelector sortKeySelector, + InternalTypeInfo<RowData> accKeyRowTypeInfo) { Review Comment: ```suggestion RowDataKeySelector sortKeySelector) { ``` ########## flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecOverAggregate.java: ########## @@ -444,17 +440,33 @@ private KeyedProcessFunction<RowData, RowData, RowData> createUnboundedOverProce KeySelectorUtil.getRowDataSelector( ctx.classLoader(), orderKeys, inputRowTypeInfo); - // Non-Time Range Unbounded Preceding Function - return new NonTimeRangeUnboundedPrecedingFunction<>( - config.getStateRetentionTime(), - genAggsHandler, - generatedRecordEqualiser, - generatedSortKeyEqualiser, - generatedRecordComparator, - flattenAccTypes, - fieldTypes, - sortKeyTypes, - sortKeySelector); + if (isRowsClause) { + // Non-Time Rows Unbounded Preceding Function + return new NonTimeRowsUnboundedPrecedingFunction<>( + config.getStateRetentionTime(), + genAggsHandler, + generatedRecordEqualiser, + generatedSortKeyEqualiser, + generatedRecordComparator, + flattenAccTypes, + fieldTypes, + sortKeyTypes, + sortKeySelector, + InternalTypeInfo.ofFields(new BigIntType())); + } else { + // Non-Time Range Unbounded Preceding Function + return new NonTimeRangeUnboundedPrecedingFunction<>( + config.getStateRetentionTime(), + genAggsHandler, + generatedRecordEqualiser, + generatedSortKeyEqualiser, + generatedRecordComparator, + flattenAccTypes, + fieldTypes, + sortKeyTypes, + sortKeySelector, + InternalTypeInfo.ofFields(sortKeyTypes)); Review Comment: ```suggestion sortKeySelector); ``` ########## flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecOverAggregate.java: ########## @@ -444,17 +440,33 @@ private KeyedProcessFunction<RowData, RowData, RowData> createUnboundedOverProce KeySelectorUtil.getRowDataSelector( ctx.classLoader(), orderKeys, inputRowTypeInfo); - // Non-Time Range Unbounded Preceding Function - return new NonTimeRangeUnboundedPrecedingFunction<>( - config.getStateRetentionTime(), - genAggsHandler, - generatedRecordEqualiser, - generatedSortKeyEqualiser, - generatedRecordComparator, - flattenAccTypes, - fieldTypes, - sortKeyTypes, - sortKeySelector); + if (isRowsClause) { + // Non-Time Rows Unbounded Preceding Function + return new NonTimeRowsUnboundedPrecedingFunction<>( + config.getStateRetentionTime(), + genAggsHandler, + generatedRecordEqualiser, + generatedSortKeyEqualiser, + generatedRecordComparator, + flattenAccTypes, + fieldTypes, + sortKeyTypes, + sortKeySelector, + InternalTypeInfo.ofFields(new BigIntType())); Review Comment: ```suggestion sortKeySelector); ``` ########## flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/over/NonTimeRangeUnboundedPrecedingFunction.java: ########## @@ -168,127 +91,19 @@ public NonTimeRangeUnboundedPrecedingFunction( LogicalType[] accTypes, LogicalType[] inputFieldTypes, LogicalType[] sortKeyTypes, - RowDataKeySelector sortKeySelector) { - this.stateRetentionTime = stateRetentionTime; - this.generatedAggsHandler = genAggsHandler; - this.generatedRecordEqualiser = genRecordEqualiser; - this.generatedSortKeyEqualiser = genSortKeyEqualiser; - this.generatedSortKeyComparator = genSortKeyComparator; - this.accTypes = accTypes; - this.inputFieldTypes = inputFieldTypes; - this.sortKeyTypes = sortKeyTypes; - this.sortKeySelector = sortKeySelector; - } - - @Override - public void open(OpenContext openContext) throws Exception { - // Initialize agg functions - aggFuncs = generatedAggsHandler.newInstance(getRuntimeContext().getUserCodeClassLoader()); - aggFuncs.open(new PerKeyStateDataViewStore(getRuntimeContext())); - - // Initialize output record - output = new JoinedRowData(); - - // Initialize value/row equaliser - valueEqualiser = - generatedRecordEqualiser.newInstance(getRuntimeContext().getUserCodeClassLoader()); - - // Initialize sortKey equaliser - sortKeyEqualiser = - generatedSortKeyEqualiser.newInstance(getRuntimeContext().getUserCodeClassLoader()); - - // Initialize sort comparator - sortKeyComparator = - generatedSortKeyComparator.newInstance( - getRuntimeContext().getUserCodeClassLoader()); - - StateTtlConfig ttlConfig = createTtlConfig(stateRetentionTime); - - // Initialize state to maintain id counter - idStateDescriptor = new ValueStateDescriptor<Long>("idState", Long.class); - if (ttlConfig.isEnabled()) { - idStateDescriptor.enableTimeToLive(ttlConfig); - } - idState = getRuntimeContext().getState(idStateDescriptor); - - // Input elements are all binary rows as they came from network - InternalTypeInfo<RowData> inputRowTypeInfo = InternalTypeInfo.ofFields(inputFieldTypes); - InternalTypeInfo<RowData> sortKeyRowTypeInfo = InternalTypeInfo.ofFields(this.sortKeyTypes); - - // Initialize state which maintains a sorted list of tuples(sortKey, List of IDs) - ListTypeInfo<Long> idListTypeInfo = new ListTypeInfo<Long>(Types.LONG); - ListTypeInfo<Tuple2<RowData, List<Long>>> listTypeInfo = - new ListTypeInfo<>(new TupleTypeInfo<>(sortKeyRowTypeInfo, idListTypeInfo)); - sortedListStateDescriptor = - new ValueStateDescriptor<List<Tuple2<RowData, List<Long>>>>( - "sortedListState", listTypeInfo); - if (ttlConfig.isEnabled()) { - sortedListStateDescriptor.enableTimeToLive(ttlConfig); - } - sortedListState = getRuntimeContext().getState(sortedListStateDescriptor); - - // Initialize state which maintains the actual row - valueStateDescriptor = - new MapStateDescriptor<Long, RowData>( - "valueMapState", Types.LONG, inputRowTypeInfo); - if (ttlConfig.isEnabled()) { - valueStateDescriptor.enableTimeToLive(ttlConfig); - } - valueMapState = getRuntimeContext().getMapState(valueStateDescriptor); - - // Initialize accumulator state per row - InternalTypeInfo<RowData> accTypeInfo = InternalTypeInfo.ofFields(accTypes); - accStateDescriptor = - new MapStateDescriptor<RowData, RowData>( - "accMapState", sortKeyRowTypeInfo, accTypeInfo); - if (ttlConfig.isEnabled()) { - accStateDescriptor.enableTimeToLive(ttlConfig); - } - accMapState = getRuntimeContext().getMapState(accStateDescriptor); - - // metrics - this.numOfIdsNotFound = - getRuntimeContext().getMetricGroup().counter(IDS_NOT_FOUND_METRIC_NAME); - this.numOfSortKeysNotFound = - getRuntimeContext().getMetricGroup().counter(SORT_KEYS_NOT_FOUND_METRIC_NAME); - } - - /** - * Puts an element from the input stream into state or removes it from state if the input is a - * retraction. Emits the aggregated value for the newly inserted element and updates all results - * that are affected by the added or removed row. Emits the same aggregated value for all - * elements with the same sortKey to comply with the sql RANGE syntax. - * - * @param input The input value. - * @param ctx A {@link Context} that allows querying the timestamp of the element and getting - * TimerService for registering timers and querying the time. The context is only valid - * during the invocation of this method, do not store it. - * @param out The collector for returning result values. - * @throws Exception - */ - @Override - public void processElement( - RowData input, - KeyedProcessFunction<K, RowData, RowData>.Context ctx, - Collector<RowData> out) - throws Exception { - RowKind rowKind = input.getRowKind(); - - switch (rowKind) { - case INSERT: - case UPDATE_AFTER: - insertIntoSortedList(input, out); - break; - - case DELETE: - case UPDATE_BEFORE: - removeFromSortedList(input, out); - break; - } - - // Reset acc state since we can have out of order inserts into the ordered list - aggFuncs.resetAccumulators(); - aggFuncs.cleanup(); + RowDataKeySelector sortKeySelector, + InternalTypeInfo<RowData> accKeyRowTypeInfo) { + super( + stateRetentionTime, + genAggsHandler, + genRecordEqualiser, + genSortKeyEqualiser, + genSortKeyComparator, + accTypes, + inputFieldTypes, + sortKeyTypes, + sortKeySelector, + accKeyRowTypeInfo); Review Comment: ```suggestion InternalTypeInfo.ofFields(sortKeyTypes)); ``` ########## flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/over/NonTimeRowsUnboundedPrecedingFunction.java: ########## @@ -0,0 +1,421 @@ +/* + * 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.over; + +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.generated.GeneratedAggsHandleFunction; +import org.apache.flink.table.runtime.generated.GeneratedRecordComparator; +import org.apache.flink.table.runtime.generated.GeneratedRecordEqualiser; +import org.apache.flink.table.runtime.keyselector.RowDataKeySelector; +import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.types.RowKind; +import org.apache.flink.util.Collector; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.List; + +/** + * The NonTimeRowsUnboundedPrecedingFunction class is a specialized implementation for processing + * unbounded OVER window aggregations, particularly for non-time-based rows queries in Apache Flink. + * It maintains strict ordering of rows within partitions and handles the full changelog lifecycle + * (inserts, updates, deletes). + * + * <p>Key Components and Assumptions + * + * <p>Data Structure Design: (1) Maintains a sorted list of tuples containing sort keys and lists of + * IDs for each key (2) Each incoming row is assigned a unique Long ID (starting from + * Long.MIN_VALUE) (3) Uses multiple state types to track rows, sort orders, and aggregations + * + * <p>State Management: (1) idState: Counter for generating unique row IDs (2) sortedListState: + * Ordered list of sort keys with their associated row IDs (3) valueMapState: Maps IDs to their + * corresponding input rows (4) accMapState: Maps RowData(ID)s to their accumulated values + * + * <p>Processing Model: (1) For inserts/updates: Adds rows to the appropriate position based on sort + * key (2) For deletes: Removes rows by matching both sort key and row content (3) Recalculates + * aggregates for affected rows and emits the appropriate events (4) Skips redundant events when + * accumulators haven't changed to reduce network traffic + * + * <p>Optimization Assumptions: (1) Skip emitting updates when accumulators haven't changed to + * reduce network traffic (2) Uses state TTL for automatic cleanup of stale data (3) Carefully + * manages row state to support incremental calculations + * + * <p>Retraction Handling: (1) Handles retraction mode (DELETE/UPDATE_BEFORE) events properly (2) + * Supports the proper processing of changelog streams + * + * <p>Limitations + * + * <p>Linear search performance: - The current implementation uses a linear search to find the + * correct position for each sort key. This can be optimized using a binary search for large state + * sizes. + * + * <p>State size and performance: - The implementation maintains multiple state types that could + * grow large with high cardinality data + * + * <p>Linear recalculation: - When processing updates, all subsequent elements need to be + * recalculated, which could be inefficient for large windows + */ +public class NonTimeRowsUnboundedPrecedingFunction<K> + extends AbstractNonTimeUnboundedPrecedingOver<K> { + private static final long serialVersionUID = 1L; + + private static final Logger LOG = + LoggerFactory.getLogger(NonTimeRowsUnboundedPrecedingFunction.class); + + public NonTimeRowsUnboundedPrecedingFunction( + long stateRetentionTime, + GeneratedAggsHandleFunction genAggsHandler, + GeneratedRecordEqualiser genRecordEqualiser, + GeneratedRecordEqualiser genSortKeyEqualiser, + GeneratedRecordComparator genSortKeyComparator, + LogicalType[] accTypes, + LogicalType[] inputFieldTypes, + LogicalType[] sortKeyTypes, + RowDataKeySelector sortKeySelector, + InternalTypeInfo<RowData> accKeyRowTypeInfo) { + super( + stateRetentionTime, + genAggsHandler, + genRecordEqualiser, + genSortKeyEqualiser, + genSortKeyComparator, + accTypes, + inputFieldTypes, + sortKeyTypes, + sortKeySelector, + accKeyRowTypeInfo); Review Comment: ```suggestion InternalTypeInfo.ofFields(new BigIntType())); ``` ########## flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/OverWindowTestPrograms.java: ########## @@ -227,141 +227,63 @@ public class OverWindowTestPrograms { "key1", 3L, 200L, - 4L), - Row.ofKind( - RowKind.UPDATE_BEFORE, - "key1", - 5L, - 500L, - 6L), - Row.ofKind( - RowKind.UPDATE_AFTER, - "key1", - 5L, - 500L, - 9L), - Row.ofKind( - RowKind.UPDATE_BEFORE, - "key1", - 6L, - 600L, - 12L), - Row.ofKind( - RowKind.UPDATE_AFTER, - "key1", - 6L, - 600L, 15L), Row.of("key2", 1L, 100L, 1L), Row.of("key2", 2L, 200L, 3L)) .consumedAfterRestore( Row.of("key3", 1L, 100L, 1L), - Row.of("key1", 4L, 400L, 8L), + Row.of("key1", 4L, 400L, 19L), Row.ofKind( - RowKind.UPDATE_BEFORE, - "key1", - 5L, - 500L, - 9L), - Row.ofKind( - RowKind.UPDATE_AFTER, - "key1", - 5L, - 500L, - 13L), - Row.ofKind( - RowKind.UPDATE_BEFORE, - "key1", - 6L, - 600L, - 15L), - Row.ofKind( - RowKind.UPDATE_AFTER, - "key1", - 6L, - 600L, - 19L), - Row.ofKind( - RowKind.DELETE, "key1", 3L, 200L, 4L), + RowKind.DELETE, "key1", 3L, 200L, 15L), Row.ofKind( RowKind.UPDATE_BEFORE, "key1", 4L, 400L, - 8L), + 19L), Row.ofKind( RowKind.UPDATE_AFTER, "key1", 4L, 400L, - 5L), - Row.ofKind( - RowKind.UPDATE_BEFORE, - "key1", - 5L, - 500L, - 13L), - Row.ofKind( - RowKind.UPDATE_AFTER, - "key1", - 5L, - 500L, - 10L), - Row.ofKind( - RowKind.UPDATE_BEFORE, - "key1", - 6L, - 600L, - 19L), - Row.ofKind( - RowKind.UPDATE_AFTER, - "key1", - 6L, - 600L, 16L), Row.ofKind( RowKind.UPDATE_AFTER, "key1", 3L, 300L, - 4L), - Row.ofKind( - RowKind.UPDATE_BEFORE, - "key1", - 4L, - 400L, - 5L), - Row.ofKind( - RowKind.UPDATE_AFTER, - "key1", - 4L, - 400L, - 8L), - Row.ofKind( - RowKind.UPDATE_BEFORE, - "key1", - 5L, - 500L, - 10L), - Row.ofKind( - RowKind.UPDATE_AFTER, - "key1", - 5L, - 500L, - 13L), - Row.ofKind( - RowKind.UPDATE_BEFORE, - "key1", - 6L, - 600L, - 16L), - Row.ofKind( - RowKind.UPDATE_AFTER, - "key1", - 6L, - 600L, 19L)) .build()) + .runSql( + "INSERT INTO sink_t SELECT key, val, ts, SUM(val) OVER (" + + "PARTITION BY key " + + "ORDER BY key " + + "ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) " Review Comment: I think it also makes sense to add this test with a different sort key type for the RANGE operator ########## flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/OverWindowTestPrograms.java: ########## @@ -227,141 +227,63 @@ public class OverWindowTestPrograms { "key1", 3L, 200L, - 4L), - Row.ofKind( - RowKind.UPDATE_BEFORE, - "key1", - 5L, - 500L, - 6L), - Row.ofKind( - RowKind.UPDATE_AFTER, - "key1", - 5L, - 500L, - 9L), - Row.ofKind( - RowKind.UPDATE_BEFORE, - "key1", - 6L, - 600L, - 12L), - Row.ofKind( - RowKind.UPDATE_AFTER, - "key1", - 6L, - 600L, 15L), Row.of("key2", 1L, 100L, 1L), Row.of("key2", 2L, 200L, 3L)) .consumedAfterRestore( Row.of("key3", 1L, 100L, 1L), - Row.of("key1", 4L, 400L, 8L), + Row.of("key1", 4L, 400L, 19L), Row.ofKind( - RowKind.UPDATE_BEFORE, - "key1", - 5L, - 500L, - 9L), - Row.ofKind( - RowKind.UPDATE_AFTER, - "key1", - 5L, - 500L, - 13L), - Row.ofKind( - RowKind.UPDATE_BEFORE, - "key1", - 6L, - 600L, - 15L), - Row.ofKind( - RowKind.UPDATE_AFTER, - "key1", - 6L, - 600L, - 19L), - Row.ofKind( - RowKind.DELETE, "key1", 3L, 200L, 4L), + RowKind.DELETE, "key1", 3L, 200L, 15L), Row.ofKind( RowKind.UPDATE_BEFORE, "key1", 4L, 400L, - 8L), + 19L), Row.ofKind( RowKind.UPDATE_AFTER, "key1", 4L, 400L, - 5L), - Row.ofKind( - RowKind.UPDATE_BEFORE, - "key1", - 5L, - 500L, - 13L), - Row.ofKind( - RowKind.UPDATE_AFTER, - "key1", - 5L, - 500L, - 10L), - Row.ofKind( - RowKind.UPDATE_BEFORE, - "key1", - 6L, - 600L, - 19L), - Row.ofKind( - RowKind.UPDATE_AFTER, - "key1", - 6L, - 600L, 16L), Row.ofKind( RowKind.UPDATE_AFTER, "key1", 3L, 300L, - 4L), - Row.ofKind( - RowKind.UPDATE_BEFORE, - "key1", - 4L, - 400L, - 5L), - Row.ofKind( - RowKind.UPDATE_AFTER, - "key1", - 4L, - 400L, - 8L), - Row.ofKind( - RowKind.UPDATE_BEFORE, - "key1", - 5L, - 500L, - 10L), - Row.ofKind( - RowKind.UPDATE_AFTER, - "key1", - 5L, - 500L, - 13L), - Row.ofKind( - RowKind.UPDATE_BEFORE, - "key1", - 6L, - 600L, - 16L), - Row.ofKind( - RowKind.UPDATE_AFTER, - "key1", - 6L, - 600L, 19L)) .build()) + .runSql( + "INSERT INTO sink_t SELECT key, val, ts, SUM(val) OVER (" + + "PARTITION BY key " Review Comment: ```suggestion + "PARTITION BY val " ``` I would not use the same sort key as partition key because then each partition just has a single sort key. IMO it would be better to swap the partition key and sort key (compared to the other tests). ########## flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/testutils/RestoreTestBase.java: ########## @@ -248,7 +247,7 @@ private void registerSinkObserver( * Execute this test to generate test files. Remember to be using the correct branch when * generating the test files. */ - @Disabled + // @Disabled Review Comment: is this an intentional change? Enabling a test is obviously a good idea but if that's intended, we should IMO remove the attachment completely not just comment it out. ########## flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/over/NonTimeRangeUnboundedPrecedingFunction.java: ########## @@ -168,127 +91,19 @@ public NonTimeRangeUnboundedPrecedingFunction( LogicalType[] accTypes, LogicalType[] inputFieldTypes, LogicalType[] sortKeyTypes, - RowDataKeySelector sortKeySelector) { - this.stateRetentionTime = stateRetentionTime; - this.generatedAggsHandler = genAggsHandler; - this.generatedRecordEqualiser = genRecordEqualiser; - this.generatedSortKeyEqualiser = genSortKeyEqualiser; - this.generatedSortKeyComparator = genSortKeyComparator; - this.accTypes = accTypes; - this.inputFieldTypes = inputFieldTypes; - this.sortKeyTypes = sortKeyTypes; - this.sortKeySelector = sortKeySelector; - } - - @Override - public void open(OpenContext openContext) throws Exception { - // Initialize agg functions - aggFuncs = generatedAggsHandler.newInstance(getRuntimeContext().getUserCodeClassLoader()); - aggFuncs.open(new PerKeyStateDataViewStore(getRuntimeContext())); - - // Initialize output record - output = new JoinedRowData(); - - // Initialize value/row equaliser - valueEqualiser = - generatedRecordEqualiser.newInstance(getRuntimeContext().getUserCodeClassLoader()); - - // Initialize sortKey equaliser - sortKeyEqualiser = - generatedSortKeyEqualiser.newInstance(getRuntimeContext().getUserCodeClassLoader()); - - // Initialize sort comparator - sortKeyComparator = - generatedSortKeyComparator.newInstance( - getRuntimeContext().getUserCodeClassLoader()); - - StateTtlConfig ttlConfig = createTtlConfig(stateRetentionTime); - - // Initialize state to maintain id counter - idStateDescriptor = new ValueStateDescriptor<Long>("idState", Long.class); - if (ttlConfig.isEnabled()) { - idStateDescriptor.enableTimeToLive(ttlConfig); - } - idState = getRuntimeContext().getState(idStateDescriptor); - - // Input elements are all binary rows as they came from network - InternalTypeInfo<RowData> inputRowTypeInfo = InternalTypeInfo.ofFields(inputFieldTypes); - InternalTypeInfo<RowData> sortKeyRowTypeInfo = InternalTypeInfo.ofFields(this.sortKeyTypes); - - // Initialize state which maintains a sorted list of tuples(sortKey, List of IDs) - ListTypeInfo<Long> idListTypeInfo = new ListTypeInfo<Long>(Types.LONG); - ListTypeInfo<Tuple2<RowData, List<Long>>> listTypeInfo = - new ListTypeInfo<>(new TupleTypeInfo<>(sortKeyRowTypeInfo, idListTypeInfo)); - sortedListStateDescriptor = - new ValueStateDescriptor<List<Tuple2<RowData, List<Long>>>>( - "sortedListState", listTypeInfo); - if (ttlConfig.isEnabled()) { - sortedListStateDescriptor.enableTimeToLive(ttlConfig); - } - sortedListState = getRuntimeContext().getState(sortedListStateDescriptor); - - // Initialize state which maintains the actual row - valueStateDescriptor = - new MapStateDescriptor<Long, RowData>( - "valueMapState", Types.LONG, inputRowTypeInfo); - if (ttlConfig.isEnabled()) { - valueStateDescriptor.enableTimeToLive(ttlConfig); - } - valueMapState = getRuntimeContext().getMapState(valueStateDescriptor); - - // Initialize accumulator state per row - InternalTypeInfo<RowData> accTypeInfo = InternalTypeInfo.ofFields(accTypes); - accStateDescriptor = - new MapStateDescriptor<RowData, RowData>( - "accMapState", sortKeyRowTypeInfo, accTypeInfo); - if (ttlConfig.isEnabled()) { - accStateDescriptor.enableTimeToLive(ttlConfig); - } - accMapState = getRuntimeContext().getMapState(accStateDescriptor); - - // metrics - this.numOfIdsNotFound = - getRuntimeContext().getMetricGroup().counter(IDS_NOT_FOUND_METRIC_NAME); - this.numOfSortKeysNotFound = - getRuntimeContext().getMetricGroup().counter(SORT_KEYS_NOT_FOUND_METRIC_NAME); - } - - /** - * Puts an element from the input stream into state or removes it from state if the input is a - * retraction. Emits the aggregated value for the newly inserted element and updates all results - * that are affected by the added or removed row. Emits the same aggregated value for all - * elements with the same sortKey to comply with the sql RANGE syntax. - * - * @param input The input value. - * @param ctx A {@link Context} that allows querying the timestamp of the element and getting - * TimerService for registering timers and querying the time. The context is only valid - * during the invocation of this method, do not store it. - * @param out The collector for returning result values. - * @throws Exception - */ - @Override - public void processElement( - RowData input, - KeyedProcessFunction<K, RowData, RowData>.Context ctx, - Collector<RowData> out) - throws Exception { - RowKind rowKind = input.getRowKind(); - - switch (rowKind) { - case INSERT: - case UPDATE_AFTER: - insertIntoSortedList(input, out); - break; - - case DELETE: - case UPDATE_BEFORE: - removeFromSortedList(input, out); - break; - } - - // Reset acc state since we can have out of order inserts into the ordered list - aggFuncs.resetAccumulators(); - aggFuncs.cleanup(); + RowDataKeySelector sortKeySelector, + InternalTypeInfo<RowData> accKeyRowTypeInfo) { Review Comment: ```suggestion RowDataKeySelector sortKeySelector) { ``` -- 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