lincoln-lil commented on code in PR #25723:
URL: https://github.com/apache/flink/pull/25723#discussion_r1914390827
##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecRank.java:
##########
@@ -283,17 +284,31 @@ protected Transformation<RowData> translateToPlanInternal(
cacheSize);
}
} else {
- processFunction =
- new AppendOnlyTopNFunction(
- ttlConfig,
- inputRowTypeInfo,
- sortKeyComparator,
- sortKeySelector,
- rankType,
- rankRange,
- generateUpdateBefore,
- outputRankNumber,
- cacheSize);
+ if (isAsyncStateEnabled) {
Review Comment:
nit: we can remove the TODO in L362 directly (there'll be no coming
`UnaryUpdateTopNFunction`)
##########
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/rank/async/AsyncStateAppendOnlyTopNFunction.java:
##########
@@ -0,0 +1,199 @@
+/*
+ * 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.rank.async;
+
+import org.apache.flink.api.common.functions.OpenContext;
+import org.apache.flink.api.common.serialization.SerializerConfigImpl;
+import org.apache.flink.api.common.state.StateTtlConfig;
+import org.apache.flink.api.common.state.v2.MapState;
+import org.apache.flink.api.common.state.v2.StateFuture;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.java.typeutils.ListTypeInfo;
+import org.apache.flink.core.state.StateFutureUtils;
+import org.apache.flink.runtime.state.v2.MapStateDescriptor;
+import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.runtime.generated.GeneratedRecordComparator;
+import org.apache.flink.table.runtime.keyselector.RowDataKeySelector;
+import org.apache.flink.table.runtime.operators.rank.AppendOnlyTopNFunction;
+import org.apache.flink.table.runtime.operators.rank.RankRange;
+import org.apache.flink.table.runtime.operators.rank.RankType;
+import org.apache.flink.table.runtime.operators.rank.TopNBuffer;
+import
org.apache.flink.table.runtime.operators.rank.utils.AppendOnlyTopNHelper;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.util.Collector;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+/**
+ * A TopN function could handle insert-only stream.
+ *
+ * <p>The input stream should only contain INSERT messages.
+ *
+ * <p>Different with {@link AppendOnlyTopNFunction}, this function is used
with async state api.
+ */
+public class AsyncStateAppendOnlyTopNFunction extends
AbstractAsyncStateTopNFunction {
+
+ private static final long serialVersionUID = 1L;
+
+ private final InternalTypeInfo<RowData> sortKeyType;
+ private final TypeSerializer<RowData> inputRowSer;
+ private final long cacheSize;
+
+ // a map state stores mapping from sort key to records list which is in
topN
+ private transient MapState<RowData, List<RowData>> dataState;
+
+ private transient AsyncStateAppendOnlyTopNHelper helper;
+
+ public AsyncStateAppendOnlyTopNFunction(
+ StateTtlConfig ttlConfig,
+ InternalTypeInfo<RowData> inputRowType,
+ GeneratedRecordComparator sortKeyGeneratedRecordComparator,
+ RowDataKeySelector sortKeySelector,
+ RankType rankType,
+ RankRange rankRange,
+ boolean generateUpdateBefore,
+ boolean outputRankNumber,
+ long cacheSize) {
+ super(
+ ttlConfig,
+ inputRowType,
+ sortKeyGeneratedRecordComparator,
+ sortKeySelector,
+ rankType,
+ rankRange,
+ generateUpdateBefore,
+ outputRankNumber);
+ this.sortKeyType = sortKeySelector.getProducedType();
+ this.inputRowSer = inputRowType.createSerializer(new
SerializerConfigImpl());
+ this.cacheSize = cacheSize;
+ }
+
+ @Override
+ public void open(OpenContext openContext) throws Exception {
+ super.open(openContext);
+
+ ListTypeInfo<RowData> valueTypeInfo = new ListTypeInfo<>(inputRowType);
+ MapStateDescriptor<RowData, List<RowData>> mapStateDescriptor =
Review Comment:
Curious about using this deprecated `MapStateDescriptor` since we're
implementing a totally new operator base the the new api.
--
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: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]