xuyangzhong commented on code in PR #25680: URL: https://github.com/apache/flink/pull/25680#discussion_r1908333884
########## flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/asyncprocessing/AsyncStateGroupAggFunction.java: ########## @@ -0,0 +1,114 @@ +/* + * 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.aggregate.asyncprocessing; + +import org.apache.flink.api.common.functions.OpenContext; +import org.apache.flink.api.common.state.v2.ValueState; +import org.apache.flink.runtime.state.v2.ValueStateDescriptor; +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.generated.GeneratedAggsHandleFunction; +import org.apache.flink.table.runtime.generated.GeneratedRecordEqualiser; +import org.apache.flink.table.runtime.operators.aggregate.GroupAggFunction; +import org.apache.flink.table.runtime.operators.aggregate.GroupAggFunctionBase; +import org.apache.flink.table.runtime.operators.aggregate.utils.GroupAggHelper; +import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.util.Collector; + +/** Aggregate Function used for the groupby (without window) aggregate with async state api. */ +public class AsyncStateGroupAggFunction extends GroupAggFunctionBase { + + private static final long serialVersionUID = 1L; + + // stores the accumulators + private transient ValueState<RowData> accState = null; + + private transient AsyncStateGroupAggHelper aggHelper = null; + + /** + * Creates a {@link GroupAggFunction}. + * + * @param genAggsHandler The code generated function used to handle aggregates. + * @param genRecordEqualiser The code generated equaliser used to equal RowData. + * @param accTypes The accumulator types. + * @param indexOfCountStar The index of COUNT(*) in the aggregates. -1 when the input doesn't + * contain COUNT(*), i.e. doesn't contain retraction messages. We make sure there is a + * COUNT(*) if input stream contains retraction. + * @param generateUpdateBefore Whether this operator will generate UPDATE_BEFORE messages. + * @param stateRetentionTime state idle retention time which unit is MILLISECONDS. + */ + public AsyncStateGroupAggFunction( + GeneratedAggsHandleFunction genAggsHandler, + GeneratedRecordEqualiser genRecordEqualiser, + LogicalType[] accTypes, + int indexOfCountStar, + boolean generateUpdateBefore, + long stateRetentionTime) { + super( + genAggsHandler, + genRecordEqualiser, + accTypes, + indexOfCountStar, + generateUpdateBefore, + stateRetentionTime); + } + + @Override + public void open(OpenContext openContext) throws Exception { + super.open(openContext); + + final StreamingRuntimeContext runtimeContext = + (StreamingRuntimeContext) getRuntimeContext(); + + InternalTypeInfo<RowData> accTypeInfo = InternalTypeInfo.ofFields(accTypes); + ValueStateDescriptor<RowData> accDesc = new ValueStateDescriptor<>("accState", accTypeInfo); + if (ttlConfig.isEnabled()) { + accDesc.enableTimeToLive(ttlConfig); + } + + accState = runtimeContext.getValueState(accDesc); + aggHelper = new AsyncStateGroupAggHelper(); + } + + @Override + public void processElement(RowData input, Context ctx, Collector<RowData> out) + throws Exception { + RowData currentKey = ctx.getCurrentKey(); + accState.asyncValue() + .thenAccept(acc -> aggHelper.processElement(input, currentKey, acc, out)); + } + + private class AsyncStateGroupAggHelper extends GroupAggHelper { Review Comment: There are mainly two differences: 1. the ValueState in `SyncStateGroupAggHelper` is `org.apache.flink.api.common.state.ValueState` and that in `AsyncStateGroupAggHelper ` is `org.apache.flink.api.common.state.v2.ValueState` 2. I'll update the logic in `AsyncStateGroupAggHelper` to `asyncUpdate` and `asyncClear` -- 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]
