lincoln-lil commented on code in PR #25680:
URL: https://github.com/apache/flink/pull/25680#discussion_r1908251627


##########
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}.

Review Comment:
   nit: -> AsyncStateGroupAggFunction



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGroupAggregate.java:
##########
@@ -227,6 +229,7 @@ protected Transformation<RowData> translateToPlanInternal(
                         .generateRecordEqualiser("GroupAggValueEqualiser");
         final int inputCountIndex = aggInfoList.getIndexOfCountStar();
         final boolean isMiniBatchEnabled = 
MinibatchUtil.isMiniBatchEnabled(config);
+        final boolean enableAsyncState = 
AggregateUtil.enableAsyncState(config, aggInfoList);

Review Comment:
   nit: align the naming with `isMiniBatchEnabled`



##########
flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/harness/GroupAggregateHarnessTest.scala:
##########
@@ -397,17 +450,19 @@ class GroupAggregateHarnessTest(mode: StateBackendMode, 
miniBatch: MiniBatchMode
     // expect no exception happens
     testHarness.close()
   }
+
 }
 
 object GroupAggregateHarnessTest {
 
-  @Parameters(name = "StateBackend={0}, MiniBatch={1}")
+  @Parameters(name = "StateBackend={0}, MiniBatch={1},  EnableAsyncState={2}")
   def parameters(): JCollection[Array[java.lang.Object]] = {
     Seq[Array[AnyRef]](
-      Array(HEAP_BACKEND, MiniBatchOff),
-      Array(HEAP_BACKEND, MiniBatchOn),
-      Array(ROCKSDB_BACKEND, MiniBatchOff),
-      Array(ROCKSDB_BACKEND, MiniBatchOn)
+      Array(HEAP_BACKEND, MiniBatchOff, Boolean.box(false)),
+      Array(HEAP_BACKEND, MiniBatchOff, Boolean.box(true)),
+      Array(HEAP_BACKEND, MiniBatchOn, Boolean.box(false)),
+      Array(ROCKSDB_BACKEND, MiniBatchOff, Boolean.box(false)),
+      Array(ROCKSDB_BACKEND, MiniBatchOn, Boolean.box(false))

Review Comment:
   Is it possible to integrate the new forst statebackend for the new async 
testing?



##########
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:
   Actually we can reuse one unified GroupAggHelper to replace the current 
`AsyncStateGroupAggHelper` and  `SyncStateGroupAggHelper` (since they two have 
no different logic)?



##########
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);

Review Comment:
   Is there any implicit limitation to make the runtimeContext final, or it can 
be simplified to `getRuntimeContext()...` to omit the local variable?



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