xishuaidelin commented on code in PR #25717:
URL: https://github.com/apache/flink/pull/25717#discussion_r1907207187


##########
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/rank/asyncprocessing/AsyncStateFastTop1Function.java:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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.asyncprocessing;
+
+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.StateFuture;
+import org.apache.flink.api.common.state.v2.ValueState;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.core.state.StateFutureUtils;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.runtime.state.FunctionSnapshotContext;
+import org.apache.flink.runtime.state.v2.ValueStateDescriptor;
+import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
+import org.apache.flink.streaming.api.functions.KeyedProcessFunction;
+import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
+import 
org.apache.flink.streaming.runtime.operators.asyncprocessing.AsyncStateProcessingOperator;
+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.FastTop1Function;
+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.UpdatableTopNFunction;
+import org.apache.flink.table.runtime.operators.rank.utils.FastTop1Helper;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.util.Collector;
+
+/**
+ * A more concise implementation for {@link AppendOnlyTopNFunction} and {@link
+ * UpdatableTopNFunction} when only Top-1 is desired. This function can handle 
updating stream
+ * because the RankProcessStrategy is inferred as UpdateFastStrategy, i.e., 1) 
the upsert key of
+ * input steam contains partition key; 2) the sort field is updated monotonely 
under the upsert key.
+ *
+ * <p>Different with {@link FastTop1Function}, this function is used with 
async state api.
+ */
+public class AsyncStateFastTop1Function extends 
AbstractAsyncSyncStateTopNFunction
+        implements CheckpointedFunction {
+
+    private static final long serialVersionUID = 1L;
+
+    private final TypeSerializer<RowData> inputRowSer;
+    private final long cacheSize;
+
+    // a map state stores list of records
+    private transient ValueState<RowData> dataState;

Review Comment:
   This comment is not accurate.



##########
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/rank/asyncprocessing/AbstractAsyncSyncStateTopNFunction.java:
##########
@@ -0,0 +1,60 @@
+/*
+ * 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.asyncprocessing;
+
+import org.apache.flink.api.common.state.StateTtlConfig;
+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.AbstractTopNFunction;
+import org.apache.flink.table.runtime.operators.rank.RankRange;
+import org.apache.flink.table.runtime.operators.rank.RankType;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+
+/**
+ * Base class for TopN Function with async state api.
+ *
+ * <p>TODO FLINK-36831 support variable rank end in async state rank later.
+ */
+public abstract class AbstractAsyncSyncStateTopNFunction extends 
AbstractTopNFunction {

Review Comment:
   rename to  AbstractAsyncStateTopNFunction?



-- 
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

Reply via email to