dianfu commented on a change in pull request #13420:
URL: https://github.com/apache/flink/pull/13420#discussion_r492000867



##########
File path: 
flink-python/src/main/java/org/apache/flink/streaming/api/runners/python/beam/BeamPythonFunctionRunner.java
##########
@@ -41,30 +60,60 @@
 import org.apache.beam.runners.fnexecution.control.StageBundleFactory;
 import org.apache.beam.runners.fnexecution.provisioning.JobInfo;
 import org.apache.beam.runners.fnexecution.state.StateRequestHandler;
+import org.apache.beam.sdk.coders.ByteArrayCoder;
+import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.fn.data.FnDataReceiver;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
 import org.apache.beam.sdk.options.PortablePipelineOptions;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
 import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
 import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.Struct;
 import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import javax.annotation.Nullable;
 
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
 import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionStage;
 import java.util.concurrent.LinkedBlockingQueue;
 
+import static org.apache.beam.runners.core.construction.BeamUrns.getUrn;
+
 /**
- * An base class for {@link PythonFunctionRunner} based on beam.
+ * A {@link BeamPythonFunctionRunner} used to execute Python stateful 
functions.

Review comment:
       ```suggestion
    * A {@link BeamPythonFunctionRunner} used to execute Python functions.
   ```

##########
File path: flink-python/pyflink/proto/flink-fn-execution.proto
##########
@@ -71,6 +71,24 @@ message UserDefinedDataStreamFunctions {
   bool metric_enabled = 2;
 }
 
+// A list of the user-defined aggregate functions to be executed in an group 
aggregate operation.

Review comment:
       ```suggestion
   // A list of the user-defined aggregate functions to be executed in a group 
aggregate operation.
   ```

##########
File path: 
flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/RowDataPythonStreamGroupAggregateOperator.java
##########
@@ -0,0 +1,397 @@
+/*
+ * 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.python.aggregate;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.state.ValueState;
+import org.apache.flink.api.common.state.ValueStateDescriptor;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.memory.ByteArrayInputStreamWithPos;
+import org.apache.flink.core.memory.ByteArrayOutputStreamWithPos;
+import org.apache.flink.core.memory.DataInputViewStreamWrapper;
+import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
+import org.apache.flink.fnexecution.v1.FlinkFnApi;
+import org.apache.flink.python.PythonFunctionRunner;
+import org.apache.flink.runtime.state.VoidNamespace;
+import org.apache.flink.runtime.state.VoidNamespaceSerializer;
+import org.apache.flink.streaming.api.SimpleTimerService;
+import org.apache.flink.streaming.api.TimerService;
+import org.apache.flink.streaming.api.operators.InternalTimer;
+import org.apache.flink.streaming.api.operators.Triggerable;
+import 
org.apache.flink.streaming.api.operators.python.AbstractOneInputPythonFunctionOperator;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.UpdatableRowData;
+import org.apache.flink.table.functions.python.PythonEnv;
+import org.apache.flink.table.functions.python.PythonFunctionInfo;
+import org.apache.flink.table.planner.plan.utils.KeySelectorUtil;
+import org.apache.flink.table.runtime.functions.CleanupState;
+import org.apache.flink.table.runtime.keyselector.RowDataKeySelector;
+import 
org.apache.flink.table.runtime.operators.python.utils.PythonOperatorUtils;
+import 
org.apache.flink.table.runtime.operators.python.utils.StreamRecordRowDataWrappingCollector;
+import 
org.apache.flink.table.runtime.runners.python.beam.BeamTableStatefulPythonFunctionRunner;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.runtime.typeutils.PythonTypeUtils;
+import org.apache.flink.table.types.logical.BigIntType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.logical.TinyIntType;
+import org.apache.flink.util.Preconditions;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import static 
org.apache.flink.table.runtime.typeutils.PythonTypeUtils.toProtoType;
+
+/**
+ *  The Python AggregateFunction operator for the blink planner.
+ */
+public class RowDataPythonStreamGroupAggregateOperator
+       extends AbstractOneInputPythonFunctionOperator<RowData, RowData>
+       implements Triggerable<RowData, VoidNamespace>, CleanupState {
+
+       @VisibleForTesting
+       protected static final String 
FLINK_AGGREGATE_FUNCTION_INPUT_SCHEMA_CODER_URN =
+               "flink:coder:schema:aggregate_function_input:v1";
+
+       @VisibleForTesting
+       protected static final String 
FLINK_AGGREGATE_FUNCTION_OUTPUT_SCHEMA_CODER_URN =
+               "flink:coder:schema:aggregate_function_output:v1";
+
+       @VisibleForTesting
+       protected static final String STREAM_GROUP_AGGREGATE_URN = 
"flink:transform:stream_group_aggregate:v1";
+
+       @VisibleForTesting
+       protected static final byte NORMAL_RECORD = 0;
+
+       private static final byte TRIGGER_TIMER = 1;
+
+       private static final long serialVersionUID = 1L;

Review comment:
       Move this line to the front of the class?

##########
File path: 
flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/RowDataPythonStreamGroupAggregateOperator.java
##########
@@ -0,0 +1,397 @@
+/*
+ * 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.python.aggregate;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.state.ValueState;
+import org.apache.flink.api.common.state.ValueStateDescriptor;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.memory.ByteArrayInputStreamWithPos;
+import org.apache.flink.core.memory.ByteArrayOutputStreamWithPos;
+import org.apache.flink.core.memory.DataInputViewStreamWrapper;
+import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
+import org.apache.flink.fnexecution.v1.FlinkFnApi;
+import org.apache.flink.python.PythonFunctionRunner;
+import org.apache.flink.runtime.state.VoidNamespace;
+import org.apache.flink.runtime.state.VoidNamespaceSerializer;
+import org.apache.flink.streaming.api.SimpleTimerService;
+import org.apache.flink.streaming.api.TimerService;
+import org.apache.flink.streaming.api.operators.InternalTimer;
+import org.apache.flink.streaming.api.operators.Triggerable;
+import 
org.apache.flink.streaming.api.operators.python.AbstractOneInputPythonFunctionOperator;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.UpdatableRowData;
+import org.apache.flink.table.functions.python.PythonEnv;
+import org.apache.flink.table.functions.python.PythonFunctionInfo;
+import org.apache.flink.table.planner.plan.utils.KeySelectorUtil;
+import org.apache.flink.table.runtime.functions.CleanupState;
+import org.apache.flink.table.runtime.keyselector.RowDataKeySelector;
+import 
org.apache.flink.table.runtime.operators.python.utils.PythonOperatorUtils;
+import 
org.apache.flink.table.runtime.operators.python.utils.StreamRecordRowDataWrappingCollector;
+import 
org.apache.flink.table.runtime.runners.python.beam.BeamTableStatefulPythonFunctionRunner;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.runtime.typeutils.PythonTypeUtils;
+import org.apache.flink.table.types.logical.BigIntType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.logical.TinyIntType;
+import org.apache.flink.util.Preconditions;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import static 
org.apache.flink.table.runtime.typeutils.PythonTypeUtils.toProtoType;
+
+/**
+ *  The Python AggregateFunction operator for the blink planner.
+ */
+public class RowDataPythonStreamGroupAggregateOperator
+       extends AbstractOneInputPythonFunctionOperator<RowData, RowData>
+       implements Triggerable<RowData, VoidNamespace>, CleanupState {
+
+       @VisibleForTesting
+       protected static final String 
FLINK_AGGREGATE_FUNCTION_INPUT_SCHEMA_CODER_URN =
+               "flink:coder:schema:aggregate_function_input:v1";
+
+       @VisibleForTesting
+       protected static final String 
FLINK_AGGREGATE_FUNCTION_OUTPUT_SCHEMA_CODER_URN =
+               "flink:coder:schema:aggregate_function_output:v1";
+
+       @VisibleForTesting
+       protected static final String STREAM_GROUP_AGGREGATE_URN = 
"flink:transform:stream_group_aggregate:v1";
+
+       @VisibleForTesting
+       protected static final byte NORMAL_RECORD = 0;
+
+       private static final byte TRIGGER_TIMER = 1;
+
+       private static final long serialVersionUID = 1L;
+
+       private static final int DEFAULT_STATE_CACHE_SIZE = 1000;
+
+       /**
+        * The input logical type.
+        */
+       protected final RowType inputType;
+
+       /**
+        * The output logical type.
+        */
+       protected final RowType outputType;
+
+       /**
+        * The options used to configure the Python worker process.
+        */
+       private final Map<String, String> jobOptions;
+
+       private final PythonFunctionInfo[] aggregateFunctions;
+
+       /**
+        * The array of the key indexes.
+        */
+       private final int[] grouping;
+
+       /**
+        * The index of a count aggregate used to calculate the number of 
accumulated rows.
+        */
+       private final int indexOfCountStar;
+
+       /**
+        * Generate retract messages if true.
+        */
+       private final boolean generateUpdateBefore;
+
+       /**
+        * The minimum time in milliseconds until state which was not updated 
will be retained.
+        */
+       final long minRetentionTime;
+
+       /**
+        * The maximum time in milliseconds until state which was not updated 
will be retained.
+        */
+       final long maxRetentionTime;

Review comment:
       ditto

##########
File path: 
flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/RowDataPythonStreamGroupAggregateOperator.java
##########
@@ -0,0 +1,397 @@
+/*
+ * 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.python.aggregate;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.state.ValueState;
+import org.apache.flink.api.common.state.ValueStateDescriptor;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.memory.ByteArrayInputStreamWithPos;
+import org.apache.flink.core.memory.ByteArrayOutputStreamWithPos;
+import org.apache.flink.core.memory.DataInputViewStreamWrapper;
+import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
+import org.apache.flink.fnexecution.v1.FlinkFnApi;
+import org.apache.flink.python.PythonFunctionRunner;
+import org.apache.flink.runtime.state.VoidNamespace;
+import org.apache.flink.runtime.state.VoidNamespaceSerializer;
+import org.apache.flink.streaming.api.SimpleTimerService;
+import org.apache.flink.streaming.api.TimerService;
+import org.apache.flink.streaming.api.operators.InternalTimer;
+import org.apache.flink.streaming.api.operators.Triggerable;
+import 
org.apache.flink.streaming.api.operators.python.AbstractOneInputPythonFunctionOperator;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.UpdatableRowData;
+import org.apache.flink.table.functions.python.PythonEnv;
+import org.apache.flink.table.functions.python.PythonFunctionInfo;
+import org.apache.flink.table.planner.plan.utils.KeySelectorUtil;
+import org.apache.flink.table.runtime.functions.CleanupState;
+import org.apache.flink.table.runtime.keyselector.RowDataKeySelector;
+import 
org.apache.flink.table.runtime.operators.python.utils.PythonOperatorUtils;
+import 
org.apache.flink.table.runtime.operators.python.utils.StreamRecordRowDataWrappingCollector;
+import 
org.apache.flink.table.runtime.runners.python.beam.BeamTableStatefulPythonFunctionRunner;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.runtime.typeutils.PythonTypeUtils;
+import org.apache.flink.table.types.logical.BigIntType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.logical.TinyIntType;
+import org.apache.flink.util.Preconditions;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import static 
org.apache.flink.table.runtime.typeutils.PythonTypeUtils.toProtoType;
+
+/**
+ *  The Python AggregateFunction operator for the blink planner.
+ */
+public class RowDataPythonStreamGroupAggregateOperator
+       extends AbstractOneInputPythonFunctionOperator<RowData, RowData>
+       implements Triggerable<RowData, VoidNamespace>, CleanupState {
+
+       @VisibleForTesting
+       protected static final String 
FLINK_AGGREGATE_FUNCTION_INPUT_SCHEMA_CODER_URN =
+               "flink:coder:schema:aggregate_function_input:v1";
+
+       @VisibleForTesting
+       protected static final String 
FLINK_AGGREGATE_FUNCTION_OUTPUT_SCHEMA_CODER_URN =
+               "flink:coder:schema:aggregate_function_output:v1";
+
+       @VisibleForTesting
+       protected static final String STREAM_GROUP_AGGREGATE_URN = 
"flink:transform:stream_group_aggregate:v1";
+
+       @VisibleForTesting
+       protected static final byte NORMAL_RECORD = 0;
+
+       private static final byte TRIGGER_TIMER = 1;
+
+       private static final long serialVersionUID = 1L;
+
+       private static final int DEFAULT_STATE_CACHE_SIZE = 1000;
+
+       /**
+        * The input logical type.
+        */
+       protected final RowType inputType;
+
+       /**
+        * The output logical type.
+        */
+       protected final RowType outputType;
+
+       /**
+        * The options used to configure the Python worker process.
+        */
+       private final Map<String, String> jobOptions;
+
+       private final PythonFunctionInfo[] aggregateFunctions;
+
+       /**
+        * The array of the key indexes.
+        */
+       private final int[] grouping;
+
+       /**
+        * The index of a count aggregate used to calculate the number of 
accumulated rows.
+        */
+       private final int indexOfCountStar;
+
+       /**
+        * Generate retract messages if true.
+        */
+       private final boolean generateUpdateBefore;
+
+       /**
+        * The minimum time in milliseconds until state which was not updated 
will be retained.
+        */
+       final long minRetentionTime;
+
+       /**
+        * The maximum time in milliseconds until state which was not updated 
will be retained.
+        */
+       final long maxRetentionTime;
+
+       /**
+        * The maximum NUMBER of the states cached in Python side.
+        */
+       private int stateCacheSize = DEFAULT_STATE_CACHE_SIZE;
+
+       /**
+        * Indicates whether state cleaning is enabled. Can be calculated from 
the `minRetentionTime`.
+        */
+       protected final boolean stateCleaningEnabled;
+
+       private transient Object keyForTimerService;
+
+       /**
+        * The user-defined function input logical type.
+        */
+       protected transient RowType userDefinedFunctionInputType;
+
+       /**
+        * The TypeSerializer for udf execution results.
+        */
+       protected transient TypeSerializer<RowData> udfOutputTypeSerializer;
+
+       /**
+        * The TypeSerializer for udf input elements.
+        */
+       protected transient TypeSerializer<RowData> udfInputTypeSerializer;
+
+       /**
+        * Reusable InputStream used to holding the execution results to be 
deserialized.
+        */
+       private transient ByteArrayInputStreamWithPos bais;
+
+       /**
+        * InputStream Wrapper.
+        */
+       private transient DataInputViewStreamWrapper baisWrapper;
+
+       /**
+        * Reusable OutputStream used to holding the serialized input elements.
+        */
+       private transient ByteArrayOutputStreamWithPos baos;
+
+       /**
+        * OutputStream Wrapper.
+        */
+       private transient DataOutputViewStreamWrapper baosWrapper;
+
+       private transient TimerService timerService;
+
+       // holds the latest registered cleanup timer
+       private transient ValueState<Long> cleanupTimeState;
+
+       private transient UpdatableRowData reuseRowData;
+       private transient UpdatableRowData reuseTimerRowData;
+
+       /**
+        * The collector used to collect records.
+        */
+       private transient StreamRecordRowDataWrappingCollector rowDataWrapper;
+
+       public RowDataPythonStreamGroupAggregateOperator(
+                       Configuration config,
+                       RowType inputType,
+                       RowType outputType,
+                       PythonFunctionInfo[] aggregateFunctions,
+                       int[] grouping,
+                       int indexOfCountStar,
+                       boolean generateUpdateBefore,
+                       long minRetentionTime,
+                       long maxRetentionTime) {
+               super(config);
+               this.inputType = Preconditions.checkNotNull(inputType);
+               this.outputType = Preconditions.checkNotNull(outputType);
+               this.aggregateFunctions = aggregateFunctions;
+               this.jobOptions = buildJobOptions(config);
+               this.grouping = grouping;
+               this.indexOfCountStar = indexOfCountStar;
+               this.generateUpdateBefore = generateUpdateBefore;
+               this.minRetentionTime = minRetentionTime;
+               this.maxRetentionTime = maxRetentionTime;
+               this.stateCleaningEnabled = minRetentionTime > 1;
+       }
+
+       /**
+        * As the beam state gRPC service will access the KeyedStateBackend in 
parallel with this operator, we must
+        * override this method to prevent changing the current key of the 
KeyedStateBackend while the beam service
+        * is handling requests.
+        */
+       @Override
+       public void setCurrentKey(Object key) {
+               keyForTimerService = key;
+       }
+
+       @Override
+       public Object getCurrentKey() {
+               return keyForTimerService;
+       }
+
+       @Override
+       public void open() throws Exception {
+               List<RowType.RowField> fields = new ArrayList<>();
+               fields.add(new RowType.RowField("record_type", new 
TinyIntType()));
+               fields.add(new RowType.RowField("row", inputType));
+               fields.add(new RowType.RowField("timestamp", new BigIntType()));
+               fields.add(new RowType.RowField("key", getKeyType()));
+               userDefinedFunctionInputType = new RowType(fields);
+               udfInputTypeSerializer = 
PythonTypeUtils.toBlinkTypeSerializer(userDefinedFunctionInputType);
+               udfOutputTypeSerializer = 
PythonTypeUtils.toBlinkTypeSerializer(outputType);
+               bais = new ByteArrayInputStreamWithPos();
+               baisWrapper = new DataInputViewStreamWrapper(bais);
+               baos = new ByteArrayOutputStreamWithPos();
+               baosWrapper = new DataOutputViewStreamWrapper(baos);
+               timerService = new SimpleTimerService(
+                       getInternalTimerService("state-clean-timer", 
VoidNamespaceSerializer.INSTANCE, this));
+               reuseRowData = new 
UpdatableRowData(GenericRowData.of(NORMAL_RECORD, null, null, null), 4);

Review comment:
       Add some comments about the meaning of each column?

##########
File path: 
flink-python/src/main/java/org/apache/flink/streaming/api/runners/python/beam/BeamPythonFunctionRunner.java
##########
@@ -309,4 +356,287 @@ private void checkInvokeStartBundle() throws Exception {
                }
        }
 
+       /**
+        * Creates a {@link ExecutableStage} which contains the Python 
user-defined functions to be executed
+        * and all the other information needed to execute them, such as the 
execution environment, the input
+        * and output coder, etc.
+        */
+       @SuppressWarnings("unchecked")
+       private ExecutableStage createExecutableStage() throws Exception {
+               RunnerApi.Components components =
+                       RunnerApi.Components.newBuilder()
+                               .putPcollections(
+                                       INPUT_ID,
+                                       RunnerApi.PCollection.newBuilder()
+                                               
.setWindowingStrategyId(WINDOW_STRATEGY)
+                                               .setCoderId(INPUT_CODER_ID)
+                                               .build())
+                               .putPcollections(
+                                       OUTPUT_ID,
+                                       RunnerApi.PCollection.newBuilder()
+                                               
.setWindowingStrategyId(WINDOW_STRATEGY)
+                                               .setCoderId(OUTPUT_CODER_ID)
+                                               .build())
+                               .putTransforms(
+                                       TRANSFORM_ID,
+                                       RunnerApi.PTransform.newBuilder()
+                                               .setUniqueName(TRANSFORM_ID)
+                                               
.setSpec(RunnerApi.FunctionSpec.newBuilder()
+                                                       .setUrn(functionUrn)
+                                                       .setPayload(
+                                                               
org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString.copyFrom(
+                                                                       
getUserDefinedFunctionsProtoBytes()))
+                                                       .build())
+                                               .putInputs(MAIN_INPUT_NAME, 
INPUT_ID)
+                                               .putOutputs(MAIN_OUTPUT_NAME, 
OUTPUT_ID)
+                                               .build())
+                               .putWindowingStrategies(
+                                       WINDOW_STRATEGY,
+                                       RunnerApi.WindowingStrategy.newBuilder()
+                                               
.setWindowCoderId(WINDOW_CODER_ID)
+                                               .build())
+                               .putCoders(
+                                       INPUT_CODER_ID,
+                                       getInputCoderProto())
+                               .putCoders(
+                                       OUTPUT_CODER_ID,
+                                       getOutputCoderProto())
+                               .putCoders(
+                                       WINDOW_CODER_ID,
+                                       getWindowCoderProto())
+                               .build();
+
+               PipelineNode.PCollectionNode input =
+                       PipelineNode.pCollection(INPUT_ID, 
components.getPcollectionsOrThrow(INPUT_ID));
+               List<SideInputReference> sideInputs = Collections.EMPTY_LIST;
+               List<UserStateReference> userStates = Collections.EMPTY_LIST;
+               List<TimerReference> timers = Collections.EMPTY_LIST;
+               List<PipelineNode.PTransformNode> transforms =
+                       Collections.singletonList(
+                               PipelineNode.pTransform(TRANSFORM_ID, 
components.getTransformsOrThrow(TRANSFORM_ID)));
+               List<PipelineNode.PCollectionNode> outputs =
+                       Collections.singletonList(
+                               PipelineNode.pCollection(OUTPUT_ID, 
components.getPcollectionsOrThrow(OUTPUT_ID)));
+               return ImmutableExecutableStage.of(
+                       components, createPythonExecutionEnvironment(), input, 
sideInputs, userStates, timers, transforms, outputs, 
createValueOnlyWireCoderSetting());
+       }
+
+       private Collection<RunnerApi.ExecutableStagePayload.WireCoderSetting> 
createValueOnlyWireCoderSetting() throws
+               IOException {
+               WindowedValue<byte[]> value = 
WindowedValue.valueInGlobalWindow(new byte[0]);
+               Coder<? extends BoundedWindow> windowCoder = 
GlobalWindow.Coder.INSTANCE;
+               WindowedValue.FullWindowedValueCoder<byte[]> windowedValueCoder 
=
+                       
WindowedValue.FullWindowedValueCoder.of(ByteArrayCoder.of(), windowCoder);
+               ByteArrayOutputStream baos = new ByteArrayOutputStream();
+               windowedValueCoder.encode(value, baos);
+
+               return Arrays.asList(
+                       
RunnerApi.ExecutableStagePayload.WireCoderSetting.newBuilder()
+                               
.setUrn(getUrn(RunnerApi.StandardCoders.Enum.PARAM_WINDOWED_VALUE))
+                               .setPayload(
+                                       
org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString.copyFrom(baos.toByteArray()))
+                               .setInputOrOutputId(INPUT_ID)
+                               .build(),
+                       
RunnerApi.ExecutableStagePayload.WireCoderSetting.newBuilder()
+                               
.setUrn(getUrn(RunnerApi.StandardCoders.Enum.PARAM_WINDOWED_VALUE))
+                               .setPayload(
+                                       
org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString.copyFrom(baos.toByteArray()))
+                               .setInputOrOutputId(OUTPUT_ID)
+                               .build()
+               );
+       }
+
+       protected abstract byte[] getUserDefinedFunctionsProtoBytes();
+
+       protected abstract RunnerApi.Coder getInputCoderProto();
+
+       protected abstract RunnerApi.Coder getOutputCoderProto();
+
+       /**
+        * Gets the proto representation of the window coder.
+        */
+       private RunnerApi.Coder getWindowCoderProto() {
+               return RunnerApi.Coder.newBuilder()
+                       .setSpec(
+                               RunnerApi.FunctionSpec.newBuilder()
+                                       
.setUrn(ModelCoders.GLOBAL_WINDOW_CODER_URN)
+                                       .build())
+                       .build();
+       }
+
+       private static StateRequestHandler getStateRequestHandler(
+                       KeyedStateBackend keyedStateBackend,
+                       TypeSerializer keySerializer) {
+               if (keyedStateBackend == null) {
+                       return StateRequestHandler.unsupported();
+               } else {
+                       assert keySerializer != null;
+                       return new SimpleStateRequestHandler(keyedStateBackend, 
keySerializer);
+               }
+       }
+
+       /**
+        * A state request handler which handles the state request from Python 
side.
+        */
+       private static class SimpleStateRequestHandler implements 
StateRequestHandler {
+
+               private final TypeSerializer keySerializer;
+               private final TypeSerializer<byte[]> valueSerializer;
+               private final KeyedStateBackend keyedStateBackend;
+
+               /**
+                * Reusable OutputStream used to holding the serialized input 
elements.
+                */
+               private final ByteArrayOutputStreamWithPos baos;

Review comment:
       Can be converted to a local variable.

##########
File path: 
flink-python/src/main/java/org/apache/flink/streaming/api/runners/python/beam/BeamPythonFunctionRunner.java
##########
@@ -41,30 +60,60 @@
 import org.apache.beam.runners.fnexecution.control.StageBundleFactory;
 import org.apache.beam.runners.fnexecution.provisioning.JobInfo;
 import org.apache.beam.runners.fnexecution.state.StateRequestHandler;
+import org.apache.beam.sdk.coders.ByteArrayCoder;
+import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.fn.data.FnDataReceiver;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
 import org.apache.beam.sdk.options.PortablePipelineOptions;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
 import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
 import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.Struct;
 import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import javax.annotation.Nullable;
 
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
 import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionStage;
 import java.util.concurrent.LinkedBlockingQueue;
 
+import static org.apache.beam.runners.core.construction.BeamUrns.getUrn;
+
 /**
- * An base class for {@link PythonFunctionRunner} based on beam.
+ * A {@link BeamPythonFunctionRunner} used to execute Python stateful 
functions.
  */
 @Internal
 public abstract class BeamPythonFunctionRunner implements PythonFunctionRunner 
{
        protected static final Logger LOG = 
LoggerFactory.getLogger(BeamPythonFunctionRunner.class);
 
-       private transient boolean bundleStarted;
+       private static final String BEAM_STATE_PREFIX = "beam-state-";

Review comment:
       What about `python-state-`? 

##########
File path: 
flink-python/src/main/java/org/apache/flink/streaming/api/runners/python/beam/BeamPythonFunctionRunner.java
##########
@@ -309,4 +356,287 @@ private void checkInvokeStartBundle() throws Exception {
                }
        }
 
+       /**
+        * Creates a {@link ExecutableStage} which contains the Python 
user-defined functions to be executed
+        * and all the other information needed to execute them, such as the 
execution environment, the input
+        * and output coder, etc.
+        */
+       @SuppressWarnings("unchecked")
+       private ExecutableStage createExecutableStage() throws Exception {
+               RunnerApi.Components components =
+                       RunnerApi.Components.newBuilder()
+                               .putPcollections(
+                                       INPUT_ID,
+                                       RunnerApi.PCollection.newBuilder()
+                                               
.setWindowingStrategyId(WINDOW_STRATEGY)
+                                               .setCoderId(INPUT_CODER_ID)
+                                               .build())
+                               .putPcollections(
+                                       OUTPUT_ID,
+                                       RunnerApi.PCollection.newBuilder()
+                                               
.setWindowingStrategyId(WINDOW_STRATEGY)
+                                               .setCoderId(OUTPUT_CODER_ID)
+                                               .build())
+                               .putTransforms(
+                                       TRANSFORM_ID,
+                                       RunnerApi.PTransform.newBuilder()
+                                               .setUniqueName(TRANSFORM_ID)
+                                               
.setSpec(RunnerApi.FunctionSpec.newBuilder()
+                                                       .setUrn(functionUrn)
+                                                       .setPayload(
+                                                               
org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString.copyFrom(
+                                                                       
getUserDefinedFunctionsProtoBytes()))
+                                                       .build())
+                                               .putInputs(MAIN_INPUT_NAME, 
INPUT_ID)
+                                               .putOutputs(MAIN_OUTPUT_NAME, 
OUTPUT_ID)
+                                               .build())
+                               .putWindowingStrategies(
+                                       WINDOW_STRATEGY,
+                                       RunnerApi.WindowingStrategy.newBuilder()
+                                               
.setWindowCoderId(WINDOW_CODER_ID)
+                                               .build())
+                               .putCoders(
+                                       INPUT_CODER_ID,
+                                       getInputCoderProto())
+                               .putCoders(
+                                       OUTPUT_CODER_ID,
+                                       getOutputCoderProto())
+                               .putCoders(
+                                       WINDOW_CODER_ID,
+                                       getWindowCoderProto())
+                               .build();
+
+               PipelineNode.PCollectionNode input =
+                       PipelineNode.pCollection(INPUT_ID, 
components.getPcollectionsOrThrow(INPUT_ID));
+               List<SideInputReference> sideInputs = Collections.EMPTY_LIST;
+               List<UserStateReference> userStates = Collections.EMPTY_LIST;
+               List<TimerReference> timers = Collections.EMPTY_LIST;
+               List<PipelineNode.PTransformNode> transforms =
+                       Collections.singletonList(
+                               PipelineNode.pTransform(TRANSFORM_ID, 
components.getTransformsOrThrow(TRANSFORM_ID)));
+               List<PipelineNode.PCollectionNode> outputs =
+                       Collections.singletonList(
+                               PipelineNode.pCollection(OUTPUT_ID, 
components.getPcollectionsOrThrow(OUTPUT_ID)));
+               return ImmutableExecutableStage.of(
+                       components, createPythonExecutionEnvironment(), input, 
sideInputs, userStates, timers, transforms, outputs, 
createValueOnlyWireCoderSetting());
+       }
+
+       private Collection<RunnerApi.ExecutableStagePayload.WireCoderSetting> 
createValueOnlyWireCoderSetting() throws
+               IOException {
+               WindowedValue<byte[]> value = 
WindowedValue.valueInGlobalWindow(new byte[0]);
+               Coder<? extends BoundedWindow> windowCoder = 
GlobalWindow.Coder.INSTANCE;
+               WindowedValue.FullWindowedValueCoder<byte[]> windowedValueCoder 
=
+                       
WindowedValue.FullWindowedValueCoder.of(ByteArrayCoder.of(), windowCoder);
+               ByteArrayOutputStream baos = new ByteArrayOutputStream();
+               windowedValueCoder.encode(value, baos);
+
+               return Arrays.asList(
+                       
RunnerApi.ExecutableStagePayload.WireCoderSetting.newBuilder()
+                               
.setUrn(getUrn(RunnerApi.StandardCoders.Enum.PARAM_WINDOWED_VALUE))
+                               .setPayload(
+                                       
org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString.copyFrom(baos.toByteArray()))
+                               .setInputOrOutputId(INPUT_ID)
+                               .build(),
+                       
RunnerApi.ExecutableStagePayload.WireCoderSetting.newBuilder()
+                               
.setUrn(getUrn(RunnerApi.StandardCoders.Enum.PARAM_WINDOWED_VALUE))
+                               .setPayload(
+                                       
org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString.copyFrom(baos.toByteArray()))
+                               .setInputOrOutputId(OUTPUT_ID)
+                               .build()
+               );
+       }
+
+       protected abstract byte[] getUserDefinedFunctionsProtoBytes();
+
+       protected abstract RunnerApi.Coder getInputCoderProto();
+
+       protected abstract RunnerApi.Coder getOutputCoderProto();
+
+       /**
+        * Gets the proto representation of the window coder.
+        */
+       private RunnerApi.Coder getWindowCoderProto() {
+               return RunnerApi.Coder.newBuilder()
+                       .setSpec(
+                               RunnerApi.FunctionSpec.newBuilder()
+                                       
.setUrn(ModelCoders.GLOBAL_WINDOW_CODER_URN)
+                                       .build())
+                       .build();
+       }
+
+       private static StateRequestHandler getStateRequestHandler(
+                       KeyedStateBackend keyedStateBackend,
+                       TypeSerializer keySerializer) {
+               if (keyedStateBackend == null) {
+                       return StateRequestHandler.unsupported();
+               } else {
+                       assert keySerializer != null;
+                       return new SimpleStateRequestHandler(keyedStateBackend, 
keySerializer);
+               }
+       }
+
+       /**
+        * A state request handler which handles the state request from Python 
side.
+        */
+       private static class SimpleStateRequestHandler implements 
StateRequestHandler {
+
+               private final TypeSerializer keySerializer;
+               private final TypeSerializer<byte[]> valueSerializer;
+               private final KeyedStateBackend keyedStateBackend;
+
+               /**
+                * Reusable OutputStream used to holding the serialized input 
elements.
+                */
+               private final ByteArrayOutputStreamWithPos baos;
+
+               /**
+                * Reusable InputStream used to holding the elements to be 
deserialized.
+                */
+               private final ByteArrayInputStreamWithPos bais;
+
+               /**
+                * OutputStream Wrapper.
+                */
+               private final DataOutputViewStreamWrapper baosWrapper;

Review comment:
       It's never used?

##########
File path: 
flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/RowDataPythonStreamGroupAggregateOperator.java
##########
@@ -0,0 +1,397 @@
+/*
+ * 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.python.aggregate;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.state.ValueState;
+import org.apache.flink.api.common.state.ValueStateDescriptor;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.memory.ByteArrayInputStreamWithPos;
+import org.apache.flink.core.memory.ByteArrayOutputStreamWithPos;
+import org.apache.flink.core.memory.DataInputViewStreamWrapper;
+import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
+import org.apache.flink.fnexecution.v1.FlinkFnApi;
+import org.apache.flink.python.PythonFunctionRunner;
+import org.apache.flink.runtime.state.VoidNamespace;
+import org.apache.flink.runtime.state.VoidNamespaceSerializer;
+import org.apache.flink.streaming.api.SimpleTimerService;
+import org.apache.flink.streaming.api.TimerService;
+import org.apache.flink.streaming.api.operators.InternalTimer;
+import org.apache.flink.streaming.api.operators.Triggerable;
+import 
org.apache.flink.streaming.api.operators.python.AbstractOneInputPythonFunctionOperator;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.UpdatableRowData;
+import org.apache.flink.table.functions.python.PythonEnv;
+import org.apache.flink.table.functions.python.PythonFunctionInfo;
+import org.apache.flink.table.planner.plan.utils.KeySelectorUtil;
+import org.apache.flink.table.runtime.functions.CleanupState;
+import org.apache.flink.table.runtime.keyselector.RowDataKeySelector;
+import 
org.apache.flink.table.runtime.operators.python.utils.PythonOperatorUtils;
+import 
org.apache.flink.table.runtime.operators.python.utils.StreamRecordRowDataWrappingCollector;
+import 
org.apache.flink.table.runtime.runners.python.beam.BeamTableStatefulPythonFunctionRunner;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.runtime.typeutils.PythonTypeUtils;
+import org.apache.flink.table.types.logical.BigIntType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.logical.TinyIntType;
+import org.apache.flink.util.Preconditions;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import static 
org.apache.flink.table.runtime.typeutils.PythonTypeUtils.toProtoType;
+
+/**
+ *  The Python AggregateFunction operator for the blink planner.
+ */
+public class RowDataPythonStreamGroupAggregateOperator

Review comment:
       Remove the RowData prefix as we will not support the old planner any 
more and so the prefix is not necessary.

##########
File path: 
flink-python/src/main/java/org/apache/flink/streaming/api/runners/python/beam/BeamPythonFunctionRunner.java
##########
@@ -309,4 +356,287 @@ private void checkInvokeStartBundle() throws Exception {
                }
        }
 
+       /**
+        * Creates a {@link ExecutableStage} which contains the Python 
user-defined functions to be executed
+        * and all the other information needed to execute them, such as the 
execution environment, the input
+        * and output coder, etc.
+        */
+       @SuppressWarnings("unchecked")
+       private ExecutableStage createExecutableStage() throws Exception {
+               RunnerApi.Components components =
+                       RunnerApi.Components.newBuilder()
+                               .putPcollections(
+                                       INPUT_ID,
+                                       RunnerApi.PCollection.newBuilder()
+                                               
.setWindowingStrategyId(WINDOW_STRATEGY)
+                                               .setCoderId(INPUT_CODER_ID)
+                                               .build())
+                               .putPcollections(
+                                       OUTPUT_ID,
+                                       RunnerApi.PCollection.newBuilder()
+                                               
.setWindowingStrategyId(WINDOW_STRATEGY)
+                                               .setCoderId(OUTPUT_CODER_ID)
+                                               .build())
+                               .putTransforms(
+                                       TRANSFORM_ID,
+                                       RunnerApi.PTransform.newBuilder()
+                                               .setUniqueName(TRANSFORM_ID)
+                                               
.setSpec(RunnerApi.FunctionSpec.newBuilder()
+                                                       .setUrn(functionUrn)
+                                                       .setPayload(
+                                                               
org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString.copyFrom(
+                                                                       
getUserDefinedFunctionsProtoBytes()))
+                                                       .build())
+                                               .putInputs(MAIN_INPUT_NAME, 
INPUT_ID)
+                                               .putOutputs(MAIN_OUTPUT_NAME, 
OUTPUT_ID)
+                                               .build())
+                               .putWindowingStrategies(
+                                       WINDOW_STRATEGY,
+                                       RunnerApi.WindowingStrategy.newBuilder()
+                                               
.setWindowCoderId(WINDOW_CODER_ID)
+                                               .build())
+                               .putCoders(
+                                       INPUT_CODER_ID,
+                                       getInputCoderProto())
+                               .putCoders(
+                                       OUTPUT_CODER_ID,
+                                       getOutputCoderProto())
+                               .putCoders(
+                                       WINDOW_CODER_ID,
+                                       getWindowCoderProto())
+                               .build();
+
+               PipelineNode.PCollectionNode input =
+                       PipelineNode.pCollection(INPUT_ID, 
components.getPcollectionsOrThrow(INPUT_ID));
+               List<SideInputReference> sideInputs = Collections.EMPTY_LIST;
+               List<UserStateReference> userStates = Collections.EMPTY_LIST;
+               List<TimerReference> timers = Collections.EMPTY_LIST;
+               List<PipelineNode.PTransformNode> transforms =
+                       Collections.singletonList(
+                               PipelineNode.pTransform(TRANSFORM_ID, 
components.getTransformsOrThrow(TRANSFORM_ID)));
+               List<PipelineNode.PCollectionNode> outputs =
+                       Collections.singletonList(
+                               PipelineNode.pCollection(OUTPUT_ID, 
components.getPcollectionsOrThrow(OUTPUT_ID)));
+               return ImmutableExecutableStage.of(
+                       components, createPythonExecutionEnvironment(), input, 
sideInputs, userStates, timers, transforms, outputs, 
createValueOnlyWireCoderSetting());
+       }
+
+       private Collection<RunnerApi.ExecutableStagePayload.WireCoderSetting> 
createValueOnlyWireCoderSetting() throws
+               IOException {
+               WindowedValue<byte[]> value = 
WindowedValue.valueInGlobalWindow(new byte[0]);
+               Coder<? extends BoundedWindow> windowCoder = 
GlobalWindow.Coder.INSTANCE;
+               WindowedValue.FullWindowedValueCoder<byte[]> windowedValueCoder 
=
+                       
WindowedValue.FullWindowedValueCoder.of(ByteArrayCoder.of(), windowCoder);
+               ByteArrayOutputStream baos = new ByteArrayOutputStream();
+               windowedValueCoder.encode(value, baos);
+
+               return Arrays.asList(
+                       
RunnerApi.ExecutableStagePayload.WireCoderSetting.newBuilder()
+                               
.setUrn(getUrn(RunnerApi.StandardCoders.Enum.PARAM_WINDOWED_VALUE))
+                               .setPayload(
+                                       
org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString.copyFrom(baos.toByteArray()))
+                               .setInputOrOutputId(INPUT_ID)
+                               .build(),
+                       
RunnerApi.ExecutableStagePayload.WireCoderSetting.newBuilder()
+                               
.setUrn(getUrn(RunnerApi.StandardCoders.Enum.PARAM_WINDOWED_VALUE))
+                               .setPayload(
+                                       
org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString.copyFrom(baos.toByteArray()))
+                               .setInputOrOutputId(OUTPUT_ID)
+                               .build()
+               );
+       }
+
+       protected abstract byte[] getUserDefinedFunctionsProtoBytes();
+
+       protected abstract RunnerApi.Coder getInputCoderProto();
+
+       protected abstract RunnerApi.Coder getOutputCoderProto();
+
+       /**
+        * Gets the proto representation of the window coder.
+        */
+       private RunnerApi.Coder getWindowCoderProto() {
+               return RunnerApi.Coder.newBuilder()
+                       .setSpec(
+                               RunnerApi.FunctionSpec.newBuilder()
+                                       
.setUrn(ModelCoders.GLOBAL_WINDOW_CODER_URN)
+                                       .build())
+                       .build();
+       }
+
+       private static StateRequestHandler getStateRequestHandler(
+                       KeyedStateBackend keyedStateBackend,
+                       TypeSerializer keySerializer) {
+               if (keyedStateBackend == null) {
+                       return StateRequestHandler.unsupported();
+               } else {
+                       assert keySerializer != null;
+                       return new SimpleStateRequestHandler(keyedStateBackend, 
keySerializer);
+               }
+       }
+
+       /**
+        * A state request handler which handles the state request from Python 
side.
+        */
+       private static class SimpleStateRequestHandler implements 
StateRequestHandler {
+
+               private final TypeSerializer keySerializer;
+               private final TypeSerializer<byte[]> valueSerializer;
+               private final KeyedStateBackend keyedStateBackend;
+
+               /**
+                * Reusable OutputStream used to holding the serialized input 
elements.
+                */
+               private final ByteArrayOutputStreamWithPos baos;
+
+               /**
+                * Reusable InputStream used to holding the elements to be 
deserialized.
+                */
+               private final ByteArrayInputStreamWithPos bais;
+
+               /**
+                * OutputStream Wrapper.
+                */
+               private final DataOutputViewStreamWrapper baosWrapper;
+
+               /**
+                * InputStream Wrapper.
+                */
+               private final DataInputViewStreamWrapper baisWrapper;
+
+               /**
+                * The cache of the stateDescriptors.
+                */
+               final Map<String, StateDescriptor> stateDescriptorCache;
+
+               SimpleStateRequestHandler(
+                       KeyedStateBackend keyedStateBackend,
+                       TypeSerializer keySerializer) {
+                       this.keySerializer = keySerializer;
+                       this.valueSerializer = 
PrimitiveArrayTypeInfo.BYTE_PRIMITIVE_ARRAY_TYPE_INFO
+                               .createSerializer(new ExecutionConfig());
+                       this.keyedStateBackend = keyedStateBackend;

Review comment:
       Move this line to the front of the constructor?

##########
File path: 
flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/RowDataPythonStreamGroupAggregateOperator.java
##########
@@ -0,0 +1,397 @@
+/*
+ * 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.python.aggregate;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.state.ValueState;
+import org.apache.flink.api.common.state.ValueStateDescriptor;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.memory.ByteArrayInputStreamWithPos;
+import org.apache.flink.core.memory.ByteArrayOutputStreamWithPos;
+import org.apache.flink.core.memory.DataInputViewStreamWrapper;
+import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
+import org.apache.flink.fnexecution.v1.FlinkFnApi;
+import org.apache.flink.python.PythonFunctionRunner;
+import org.apache.flink.runtime.state.VoidNamespace;
+import org.apache.flink.runtime.state.VoidNamespaceSerializer;
+import org.apache.flink.streaming.api.SimpleTimerService;
+import org.apache.flink.streaming.api.TimerService;
+import org.apache.flink.streaming.api.operators.InternalTimer;
+import org.apache.flink.streaming.api.operators.Triggerable;
+import 
org.apache.flink.streaming.api.operators.python.AbstractOneInputPythonFunctionOperator;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.UpdatableRowData;
+import org.apache.flink.table.functions.python.PythonEnv;
+import org.apache.flink.table.functions.python.PythonFunctionInfo;
+import org.apache.flink.table.planner.plan.utils.KeySelectorUtil;
+import org.apache.flink.table.runtime.functions.CleanupState;
+import org.apache.flink.table.runtime.keyselector.RowDataKeySelector;
+import 
org.apache.flink.table.runtime.operators.python.utils.PythonOperatorUtils;
+import 
org.apache.flink.table.runtime.operators.python.utils.StreamRecordRowDataWrappingCollector;
+import 
org.apache.flink.table.runtime.runners.python.beam.BeamTableStatefulPythonFunctionRunner;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.runtime.typeutils.PythonTypeUtils;
+import org.apache.flink.table.types.logical.BigIntType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.logical.TinyIntType;
+import org.apache.flink.util.Preconditions;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import static 
org.apache.flink.table.runtime.typeutils.PythonTypeUtils.toProtoType;
+
+/**
+ *  The Python AggregateFunction operator for the blink planner.
+ */
+public class RowDataPythonStreamGroupAggregateOperator
+       extends AbstractOneInputPythonFunctionOperator<RowData, RowData>
+       implements Triggerable<RowData, VoidNamespace>, CleanupState {
+
+       @VisibleForTesting
+       protected static final String 
FLINK_AGGREGATE_FUNCTION_INPUT_SCHEMA_CODER_URN =
+               "flink:coder:schema:aggregate_function_input:v1";
+
+       @VisibleForTesting
+       protected static final String 
FLINK_AGGREGATE_FUNCTION_OUTPUT_SCHEMA_CODER_URN =
+               "flink:coder:schema:aggregate_function_output:v1";
+
+       @VisibleForTesting
+       protected static final String STREAM_GROUP_AGGREGATE_URN = 
"flink:transform:stream_group_aggregate:v1";
+
+       @VisibleForTesting
+       protected static final byte NORMAL_RECORD = 0;
+
+       private static final byte TRIGGER_TIMER = 1;
+
+       private static final long serialVersionUID = 1L;
+
+       private static final int DEFAULT_STATE_CACHE_SIZE = 1000;
+
+       /**
+        * The input logical type.
+        */
+       protected final RowType inputType;
+
+       /**
+        * The output logical type.
+        */
+       protected final RowType outputType;
+
+       /**
+        * The options used to configure the Python worker process.
+        */
+       private final Map<String, String> jobOptions;
+
+       private final PythonFunctionInfo[] aggregateFunctions;
+
+       /**
+        * The array of the key indexes.
+        */
+       private final int[] grouping;
+
+       /**
+        * The index of a count aggregate used to calculate the number of 
accumulated rows.
+        */
+       private final int indexOfCountStar;
+
+       /**
+        * Generate retract messages if true.
+        */
+       private final boolean generateUpdateBefore;
+
+       /**
+        * The minimum time in milliseconds until state which was not updated 
will be retained.
+        */
+       final long minRetentionTime;

Review comment:
       private?

##########
File path: 
flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/RowDataPythonStreamGroupAggregateOperator.java
##########
@@ -0,0 +1,397 @@
+/*
+ * 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.python.aggregate;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.state.ValueState;
+import org.apache.flink.api.common.state.ValueStateDescriptor;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.memory.ByteArrayInputStreamWithPos;
+import org.apache.flink.core.memory.ByteArrayOutputStreamWithPos;
+import org.apache.flink.core.memory.DataInputViewStreamWrapper;
+import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
+import org.apache.flink.fnexecution.v1.FlinkFnApi;
+import org.apache.flink.python.PythonFunctionRunner;
+import org.apache.flink.runtime.state.VoidNamespace;
+import org.apache.flink.runtime.state.VoidNamespaceSerializer;
+import org.apache.flink.streaming.api.SimpleTimerService;
+import org.apache.flink.streaming.api.TimerService;
+import org.apache.flink.streaming.api.operators.InternalTimer;
+import org.apache.flink.streaming.api.operators.Triggerable;
+import 
org.apache.flink.streaming.api.operators.python.AbstractOneInputPythonFunctionOperator;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.UpdatableRowData;
+import org.apache.flink.table.functions.python.PythonEnv;
+import org.apache.flink.table.functions.python.PythonFunctionInfo;
+import org.apache.flink.table.planner.plan.utils.KeySelectorUtil;
+import org.apache.flink.table.runtime.functions.CleanupState;
+import org.apache.flink.table.runtime.keyselector.RowDataKeySelector;
+import 
org.apache.flink.table.runtime.operators.python.utils.PythonOperatorUtils;
+import 
org.apache.flink.table.runtime.operators.python.utils.StreamRecordRowDataWrappingCollector;
+import 
org.apache.flink.table.runtime.runners.python.beam.BeamTableStatefulPythonFunctionRunner;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.runtime.typeutils.PythonTypeUtils;
+import org.apache.flink.table.types.logical.BigIntType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.logical.TinyIntType;
+import org.apache.flink.util.Preconditions;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import static 
org.apache.flink.table.runtime.typeutils.PythonTypeUtils.toProtoType;
+
+/**
+ *  The Python AggregateFunction operator for the blink planner.
+ */
+public class RowDataPythonStreamGroupAggregateOperator
+       extends AbstractOneInputPythonFunctionOperator<RowData, RowData>
+       implements Triggerable<RowData, VoidNamespace>, CleanupState {
+
+       @VisibleForTesting
+       protected static final String 
FLINK_AGGREGATE_FUNCTION_INPUT_SCHEMA_CODER_URN =
+               "flink:coder:schema:aggregate_function_input:v1";
+
+       @VisibleForTesting
+       protected static final String 
FLINK_AGGREGATE_FUNCTION_OUTPUT_SCHEMA_CODER_URN =
+               "flink:coder:schema:aggregate_function_output:v1";
+
+       @VisibleForTesting
+       protected static final String STREAM_GROUP_AGGREGATE_URN = 
"flink:transform:stream_group_aggregate:v1";
+
+       @VisibleForTesting
+       protected static final byte NORMAL_RECORD = 0;
+
+       private static final byte TRIGGER_TIMER = 1;
+
+       private static final long serialVersionUID = 1L;
+
+       private static final int DEFAULT_STATE_CACHE_SIZE = 1000;
+
+       /**
+        * The input logical type.
+        */
+       protected final RowType inputType;
+
+       /**
+        * The output logical type.
+        */
+       protected final RowType outputType;
+
+       /**
+        * The options used to configure the Python worker process.
+        */
+       private final Map<String, String> jobOptions;
+
+       private final PythonFunctionInfo[] aggregateFunctions;
+
+       /**
+        * The array of the key indexes.
+        */
+       private final int[] grouping;
+
+       /**
+        * The index of a count aggregate used to calculate the number of 
accumulated rows.
+        */
+       private final int indexOfCountStar;
+
+       /**
+        * Generate retract messages if true.
+        */
+       private final boolean generateUpdateBefore;
+
+       /**
+        * The minimum time in milliseconds until state which was not updated 
will be retained.
+        */
+       final long minRetentionTime;
+
+       /**
+        * The maximum time in milliseconds until state which was not updated 
will be retained.
+        */
+       final long maxRetentionTime;
+
+       /**
+        * The maximum NUMBER of the states cached in Python side.
+        */
+       private int stateCacheSize = DEFAULT_STATE_CACHE_SIZE;
+
+       /**
+        * Indicates whether state cleaning is enabled. Can be calculated from 
the `minRetentionTime`.
+        */
+       protected final boolean stateCleaningEnabled;

Review comment:
       can be private

##########
File path: 
flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/RowDataPythonStreamGroupAggregateOperator.java
##########
@@ -0,0 +1,397 @@
+/*
+ * 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.python.aggregate;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.state.ValueState;
+import org.apache.flink.api.common.state.ValueStateDescriptor;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.memory.ByteArrayInputStreamWithPos;
+import org.apache.flink.core.memory.ByteArrayOutputStreamWithPos;
+import org.apache.flink.core.memory.DataInputViewStreamWrapper;
+import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
+import org.apache.flink.fnexecution.v1.FlinkFnApi;
+import org.apache.flink.python.PythonFunctionRunner;
+import org.apache.flink.runtime.state.VoidNamespace;
+import org.apache.flink.runtime.state.VoidNamespaceSerializer;
+import org.apache.flink.streaming.api.SimpleTimerService;
+import org.apache.flink.streaming.api.TimerService;
+import org.apache.flink.streaming.api.operators.InternalTimer;
+import org.apache.flink.streaming.api.operators.Triggerable;
+import 
org.apache.flink.streaming.api.operators.python.AbstractOneInputPythonFunctionOperator;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.table.data.GenericRowData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.UpdatableRowData;
+import org.apache.flink.table.functions.python.PythonEnv;
+import org.apache.flink.table.functions.python.PythonFunctionInfo;
+import org.apache.flink.table.planner.plan.utils.KeySelectorUtil;
+import org.apache.flink.table.runtime.functions.CleanupState;
+import org.apache.flink.table.runtime.keyselector.RowDataKeySelector;
+import 
org.apache.flink.table.runtime.operators.python.utils.PythonOperatorUtils;
+import 
org.apache.flink.table.runtime.operators.python.utils.StreamRecordRowDataWrappingCollector;
+import 
org.apache.flink.table.runtime.runners.python.beam.BeamTableStatefulPythonFunctionRunner;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.runtime.typeutils.PythonTypeUtils;
+import org.apache.flink.table.types.logical.BigIntType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.logical.TinyIntType;
+import org.apache.flink.util.Preconditions;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import static 
org.apache.flink.table.runtime.typeutils.PythonTypeUtils.toProtoType;
+
+/**
+ *  The Python AggregateFunction operator for the blink planner.
+ */
+public class RowDataPythonStreamGroupAggregateOperator
+       extends AbstractOneInputPythonFunctionOperator<RowData, RowData>
+       implements Triggerable<RowData, VoidNamespace>, CleanupState {
+
+       @VisibleForTesting
+       protected static final String 
FLINK_AGGREGATE_FUNCTION_INPUT_SCHEMA_CODER_URN =
+               "flink:coder:schema:aggregate_function_input:v1";
+
+       @VisibleForTesting
+       protected static final String 
FLINK_AGGREGATE_FUNCTION_OUTPUT_SCHEMA_CODER_URN =
+               "flink:coder:schema:aggregate_function_output:v1";
+
+       @VisibleForTesting
+       protected static final String STREAM_GROUP_AGGREGATE_URN = 
"flink:transform:stream_group_aggregate:v1";
+
+       @VisibleForTesting
+       protected static final byte NORMAL_RECORD = 0;
+
+       private static final byte TRIGGER_TIMER = 1;
+
+       private static final long serialVersionUID = 1L;
+
+       private static final int DEFAULT_STATE_CACHE_SIZE = 1000;
+
+       /**
+        * The input logical type.
+        */
+       protected final RowType inputType;
+
+       /**
+        * The output logical type.
+        */
+       protected final RowType outputType;
+
+       /**
+        * The options used to configure the Python worker process.
+        */
+       private final Map<String, String> jobOptions;
+
+       private final PythonFunctionInfo[] aggregateFunctions;
+
+       /**
+        * The array of the key indexes.
+        */
+       private final int[] grouping;
+
+       /**
+        * The index of a count aggregate used to calculate the number of 
accumulated rows.
+        */
+       private final int indexOfCountStar;
+
+       /**
+        * Generate retract messages if true.
+        */
+       private final boolean generateUpdateBefore;
+
+       /**
+        * The minimum time in milliseconds until state which was not updated 
will be retained.
+        */
+       final long minRetentionTime;
+
+       /**
+        * The maximum time in milliseconds until state which was not updated 
will be retained.
+        */
+       final long maxRetentionTime;
+
+       /**
+        * The maximum NUMBER of the states cached in Python side.
+        */
+       private int stateCacheSize = DEFAULT_STATE_CACHE_SIZE;
+
+       /**
+        * Indicates whether state cleaning is enabled. Can be calculated from 
the `minRetentionTime`.
+        */
+       protected final boolean stateCleaningEnabled;
+
+       private transient Object keyForTimerService;
+
+       /**
+        * The user-defined function input logical type.
+        */
+       protected transient RowType userDefinedFunctionInputType;
+
+       /**
+        * The TypeSerializer for udf execution results.
+        */
+       protected transient TypeSerializer<RowData> udfOutputTypeSerializer;
+
+       /**
+        * The TypeSerializer for udf input elements.
+        */
+       protected transient TypeSerializer<RowData> udfInputTypeSerializer;
+
+       /**
+        * Reusable InputStream used to holding the execution results to be 
deserialized.
+        */
+       private transient ByteArrayInputStreamWithPos bais;
+
+       /**
+        * InputStream Wrapper.
+        */
+       private transient DataInputViewStreamWrapper baisWrapper;
+
+       /**
+        * Reusable OutputStream used to holding the serialized input elements.
+        */
+       private transient ByteArrayOutputStreamWithPos baos;
+
+       /**
+        * OutputStream Wrapper.
+        */
+       private transient DataOutputViewStreamWrapper baosWrapper;
+
+       private transient TimerService timerService;
+
+       // holds the latest registered cleanup timer
+       private transient ValueState<Long> cleanupTimeState;
+
+       private transient UpdatableRowData reuseRowData;
+       private transient UpdatableRowData reuseTimerRowData;
+
+       /**
+        * The collector used to collect records.
+        */
+       private transient StreamRecordRowDataWrappingCollector rowDataWrapper;
+
+       public RowDataPythonStreamGroupAggregateOperator(
+                       Configuration config,
+                       RowType inputType,
+                       RowType outputType,
+                       PythonFunctionInfo[] aggregateFunctions,
+                       int[] grouping,
+                       int indexOfCountStar,
+                       boolean generateUpdateBefore,
+                       long minRetentionTime,
+                       long maxRetentionTime) {
+               super(config);
+               this.inputType = Preconditions.checkNotNull(inputType);
+               this.outputType = Preconditions.checkNotNull(outputType);
+               this.aggregateFunctions = aggregateFunctions;
+               this.jobOptions = buildJobOptions(config);
+               this.grouping = grouping;
+               this.indexOfCountStar = indexOfCountStar;
+               this.generateUpdateBefore = generateUpdateBefore;
+               this.minRetentionTime = minRetentionTime;
+               this.maxRetentionTime = maxRetentionTime;
+               this.stateCleaningEnabled = minRetentionTime > 1;
+       }
+
+       /**
+        * As the beam state gRPC service will access the KeyedStateBackend in 
parallel with this operator, we must
+        * override this method to prevent changing the current key of the 
KeyedStateBackend while the beam service
+        * is handling requests.
+        */
+       @Override
+       public void setCurrentKey(Object key) {
+               keyForTimerService = key;
+       }
+
+       @Override
+       public Object getCurrentKey() {
+               return keyForTimerService;
+       }
+
+       @Override
+       public void open() throws Exception {
+               List<RowType.RowField> fields = new ArrayList<>();
+               fields.add(new RowType.RowField("record_type", new 
TinyIntType()));
+               fields.add(new RowType.RowField("row", inputType));
+               fields.add(new RowType.RowField("timestamp", new BigIntType()));
+               fields.add(new RowType.RowField("key", getKeyType()));
+               userDefinedFunctionInputType = new RowType(fields);
+               udfInputTypeSerializer = 
PythonTypeUtils.toBlinkTypeSerializer(userDefinedFunctionInputType);
+               udfOutputTypeSerializer = 
PythonTypeUtils.toBlinkTypeSerializer(outputType);
+               bais = new ByteArrayInputStreamWithPos();
+               baisWrapper = new DataInputViewStreamWrapper(bais);
+               baos = new ByteArrayOutputStreamWithPos();
+               baosWrapper = new DataOutputViewStreamWrapper(baos);
+               timerService = new SimpleTimerService(
+                       getInternalTimerService("state-clean-timer", 
VoidNamespaceSerializer.INSTANCE, this));
+               reuseRowData = new 
UpdatableRowData(GenericRowData.of(NORMAL_RECORD, null, null, null), 4);
+               reuseTimerRowData = new 
UpdatableRowData(GenericRowData.of(TRIGGER_TIMER, null, null, null), 4);
+               rowDataWrapper = new 
StreamRecordRowDataWrappingCollector(output);
+               initCleanupTimeState();
+               super.open();
+       }
+
+       @Override
+       public PythonFunctionRunner createPythonFunctionRunner() throws 
Exception {
+               return new BeamTableStatefulPythonFunctionRunner(
+                       getRuntimeContext().getTaskName(),
+                       createPythonEnvironmentManager(),
+                       userDefinedFunctionInputType,
+                       outputType,
+                       STREAM_GROUP_AGGREGATE_URN,
+                       getUserDefinedFunctionsProto(),
+                       FLINK_AGGREGATE_FUNCTION_INPUT_SCHEMA_CODER_URN,
+                       FLINK_AGGREGATE_FUNCTION_OUTPUT_SCHEMA_CODER_URN,
+                       jobOptions,
+                       getFlinkMetricContainer(),
+                       getKeyedStateBackend(),
+                       getKeySerializer());
+       }
+
+       @Override
+       public void processElement(StreamRecord<RowData> element) throws 
Exception {
+               RowData value = element.getValue();
+               processElementInternal(value);
+               elementCount++;
+               checkInvokeFinishBundleByCount();
+               emitResults();
+       }
+
+       @Override
+       public void emitResult(Tuple2<byte[], Integer> resultTuple) throws 
Exception {
+               byte[] rawUdfResult = resultTuple.f0;
+               int length = resultTuple.f1;
+               bais.setBuffer(rawUdfResult, 0, length);
+               RowData udfResult = 
udfOutputTypeSerializer.deserialize(baisWrapper);
+               rowDataWrapper.collect(udfResult);
+       }
+
+       /**
+        * Invoked when an event-time timer fires.
+        */
+       @Override
+       public void onEventTime(InternalTimer<RowData, VoidNamespace> timer) {
+               // ignore
+       }
+
+       /**
+        * Invoked when a processing-time timer fires.
+        */
+       @Override
+       public void onProcessingTime(InternalTimer<RowData, VoidNamespace> 
timer) throws Exception {
+               if (stateCleaningEnabled) {
+                       RowData key = timer.getKey();
+                       long timestamp = timer.getTimestamp();
+                       reuseTimerRowData.setLong(2, timestamp);
+                       reuseTimerRowData.setField(3, key);
+                       udfInputTypeSerializer.serialize(reuseTimerRowData, 
baosWrapper);
+                       pythonFunctionRunner.process(baos.toByteArray());
+                       baos.reset();
+                       elementCount++;
+               }
+       }
+
+       @Override
+       public PythonEnv getPythonEnv() {
+               return aggregateFunctions[0].getPythonFunction().getPythonEnv();
+       }
+
+       private void registerProcessingCleanupTimer(long currentTime) throws 
Exception {
+               if (stateCleaningEnabled) {
+                       synchronized (getKeyedStateBackend()) {
+                               
getKeyedStateBackend().setCurrentKey(getCurrentKey());
+                               registerProcessingCleanupTimer(
+                                       cleanupTimeState,
+                                       currentTime,
+                                       minRetentionTime,
+                                       maxRetentionTime,
+                                       timerService
+                               );
+                       }
+               }
+       }
+
+       private void initCleanupTimeState() {
+               if (stateCleaningEnabled) {
+                       ValueStateDescriptor<Long> inputCntDescriptor =
+                               new 
ValueStateDescriptor<>("PythonGroupAggregateCleanupTime", Types.LONG);
+                       cleanupTimeState = 
getRuntimeContext().getState(inputCntDescriptor);
+               }
+       }
+
+       private Map<String, String> buildJobOptions(Configuration config) {
+               Map<String, String> jobOptions = new HashMap<>();
+               if (config.containsKey("table.exec.timezone")) {
+                       jobOptions.put("table.exec.timezone", 
config.getString("table.exec.timezone", null));
+               }
+               if (config.containsKey("python.state.cache.size")) {

Review comment:
       How users could know about this config? Should we add it in PythonConfig?




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

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Reply via email to