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



##########
File path: 
flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecPythonAggregateRule.java
##########
@@ -0,0 +1,153 @@
+/*
+ * 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.planner.plan.rules.physical.batch;
+
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.functions.ImperativeAggregateFunction;
+import org.apache.flink.table.functions.UserDefinedFunction;
+import org.apache.flink.table.functions.python.PythonFunction;
+import org.apache.flink.table.functions.python.PythonFunctionKind;
+import org.apache.flink.table.planner.functions.utils.AggSqlFunction;
+import org.apache.flink.table.planner.plan.nodes.FlinkConventions;
+import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalAggregate;
+import 
org.apache.flink.table.planner.plan.nodes.physical.batch.BatchExecPythonGroupAggregate;
+import org.apache.flink.table.planner.plan.trait.FlinkRelDistribution;
+import org.apache.flink.table.planner.plan.utils.AggregateUtil;
+import org.apache.flink.table.planner.plan.utils.FlinkRelOptUtil;
+import org.apache.flink.table.types.DataType;
+
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelCollations;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.sql.SqlAggFunction;
+
+import java.util.LinkedList;
+import java.util.List;
+
+import scala.Tuple2;
+import scala.Tuple3;
+import scala.collection.Seq;
+
+/**
+ * The physical rule is responsible for convert {@link FlinkLogicalAggregate} 
to

Review comment:
       ```suggestion
    * The physical rule which is responsible for converting {@link 
FlinkLogicalAggregate} to
   ```

##########
File path: 
flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecPythonAggregateRule.java
##########
@@ -0,0 +1,153 @@
+/*
+ * 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.planner.plan.rules.physical.batch;
+
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.functions.ImperativeAggregateFunction;
+import org.apache.flink.table.functions.UserDefinedFunction;
+import org.apache.flink.table.functions.python.PythonFunction;
+import org.apache.flink.table.functions.python.PythonFunctionKind;
+import org.apache.flink.table.planner.functions.utils.AggSqlFunction;
+import org.apache.flink.table.planner.plan.nodes.FlinkConventions;
+import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalAggregate;
+import 
org.apache.flink.table.planner.plan.nodes.physical.batch.BatchExecPythonGroupAggregate;
+import org.apache.flink.table.planner.plan.trait.FlinkRelDistribution;
+import org.apache.flink.table.planner.plan.utils.AggregateUtil;
+import org.apache.flink.table.planner.plan.utils.FlinkRelOptUtil;
+import org.apache.flink.table.types.DataType;
+
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelCollations;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.sql.SqlAggFunction;
+
+import java.util.LinkedList;
+import java.util.List;
+
+import scala.Tuple2;
+import scala.Tuple3;
+import scala.collection.Seq;
+
+/**
+ * The physical rule is responsible for convert {@link FlinkLogicalAggregate} 
to
+ * {@link BatchExecPythonGroupAggregate}.
+ */
+public class BatchExecPythonAggregateRule extends ConverterRule {
+
+       public static final RelOptRule INSTANCE = new 
BatchExecPythonAggregateRule();
+
+       private BatchExecPythonAggregateRule() {
+               super(FlinkLogicalAggregate.class, FlinkConventions.LOGICAL(), 
FlinkConventions.BATCH_PHYSICAL(),
+                       "BatchExecPythonAggregateRule");
+       }
+
+       @Override
+       public boolean matches(RelOptRuleCall call) {
+               FlinkLogicalAggregate agg = call.rel(0);
+               List<AggregateCall> aggCalls = agg.getAggCallList();
+               boolean existPandasFunction = false;
+               boolean existGeneralPythonFunction = false;
+               boolean existJavaFunction = false;
+               for (AggregateCall aggCall : aggCalls) {
+                       SqlAggFunction aggregation = aggCall.getAggregation();
+                       if (aggregation instanceof AggSqlFunction) {
+                               ImperativeAggregateFunction<?, ?> func =
+                                       ((AggSqlFunction) 
aggregation).aggregateFunction();
+                               if (func instanceof PythonFunction) {
+                                       PythonFunction pythonFunction = 
(PythonFunction) func;
+                                       if 
(pythonFunction.getPythonFunctionKind() == PythonFunctionKind.PANDAS) {
+                                               existPandasFunction = true;
+                                       } else {
+                                               existGeneralPythonFunction = 
true;
+                                       }
+                               } else {
+                                       existJavaFunction = true;
+                               }
+                       }
+               }
+               if (existPandasFunction) {
+                       if (existGeneralPythonFunction) {
+                               throw new TableException("Pandas UDAF cannot be 
computed with General Python UDAF currently");

Review comment:
       ```suggestion
                                throw new TableException("Pandas UDAF and 
non-Pandas UDAF cannot be used together.");
   ```

##########
File path: 
flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecPythonAggregateRule.java
##########
@@ -0,0 +1,153 @@
+/*
+ * 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.planner.plan.rules.physical.batch;
+
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.functions.ImperativeAggregateFunction;
+import org.apache.flink.table.functions.UserDefinedFunction;
+import org.apache.flink.table.functions.python.PythonFunction;
+import org.apache.flink.table.functions.python.PythonFunctionKind;
+import org.apache.flink.table.planner.functions.utils.AggSqlFunction;
+import org.apache.flink.table.planner.plan.nodes.FlinkConventions;
+import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalAggregate;
+import 
org.apache.flink.table.planner.plan.nodes.physical.batch.BatchExecPythonGroupAggregate;
+import org.apache.flink.table.planner.plan.trait.FlinkRelDistribution;
+import org.apache.flink.table.planner.plan.utils.AggregateUtil;
+import org.apache.flink.table.planner.plan.utils.FlinkRelOptUtil;
+import org.apache.flink.table.types.DataType;
+
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelCollations;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.sql.SqlAggFunction;
+
+import java.util.LinkedList;
+import java.util.List;
+
+import scala.Tuple2;
+import scala.Tuple3;
+import scala.collection.Seq;
+
+/**
+ * The physical rule is responsible for convert {@link FlinkLogicalAggregate} 
to
+ * {@link BatchExecPythonGroupAggregate}.
+ */
+public class BatchExecPythonAggregateRule extends ConverterRule {
+
+       public static final RelOptRule INSTANCE = new 
BatchExecPythonAggregateRule();
+
+       private BatchExecPythonAggregateRule() {
+               super(FlinkLogicalAggregate.class, FlinkConventions.LOGICAL(), 
FlinkConventions.BATCH_PHYSICAL(),
+                       "BatchExecPythonAggregateRule");
+       }
+
+       @Override
+       public boolean matches(RelOptRuleCall call) {
+               FlinkLogicalAggregate agg = call.rel(0);
+               List<AggregateCall> aggCalls = agg.getAggCallList();
+               boolean existPandasFunction = false;
+               boolean existGeneralPythonFunction = false;
+               boolean existJavaFunction = false;
+               for (AggregateCall aggCall : aggCalls) {
+                       SqlAggFunction aggregation = aggCall.getAggregation();
+                       if (aggregation instanceof AggSqlFunction) {
+                               ImperativeAggregateFunction<?, ?> func =
+                                       ((AggSqlFunction) 
aggregation).aggregateFunction();
+                               if (func instanceof PythonFunction) {
+                                       PythonFunction pythonFunction = 
(PythonFunction) func;
+                                       if 
(pythonFunction.getPythonFunctionKind() == PythonFunctionKind.PANDAS) {
+                                               existPandasFunction = true;
+                                       } else {
+                                               existGeneralPythonFunction = 
true;
+                                       }
+                               } else {
+                                       existJavaFunction = true;
+                               }
+                       }
+               }
+               if (existPandasFunction) {
+                       if (existGeneralPythonFunction) {
+                               throw new TableException("Pandas UDAF cannot be 
computed with General Python UDAF currently");
+                       }
+                       if (existJavaFunction) {
+                               throw new TableException("Pandas UDAF cannot be 
computed with Java/Scala UDAF currently");
+                       }
+               }
+
+               return existPandasFunction || existGeneralPythonFunction;
+       }
+
+       @Override
+       public RelNode convert(RelNode relNode) {
+               FlinkLogicalAggregate agg = (FlinkLogicalAggregate) relNode;
+               RelNode input = agg.getInput();
+
+               int[] groupSet = agg.getGroupSet().toArray();
+               RelTraitSet traitSet = 
relNode.getTraitSet().replace(FlinkConventions.BATCH_PHYSICAL());
+
+               Tuple2<int[], Seq<AggregateCall>> auxGroupSetAndCallsTuple = 
AggregateUtil.checkAndSplitAggCalls(agg);
+               int[] auxGroupSet = auxGroupSetAndCallsTuple._1;
+               Seq<AggregateCall> aggCallsWithoutAuxGroupCalls = 
auxGroupSetAndCallsTuple._2;
+
+               Tuple3<int[][], DataType[][], UserDefinedFunction[]> 
aggBufferTypesAndFunctions =
+                       AggregateUtil.transformToBatchAggregateFunctions(
+                               aggCallsWithoutAuxGroupCalls, 
input.getRowType(), null);

Review comment:
       ```suggestion
                                aggCallsWithoutAuxGroupCalls, 
input.getRowType());
   ```

##########
File path: 
flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/table/AggregateTest.scala
##########
@@ -72,4 +73,16 @@ class AggregateTest extends TableTestBase {
 
     util.verifyPlan(resultTable)
   }
+
+  @Test

Review comment:
       Create a new class, e.g. PythonAggregateTest and move the test case 
there?

##########
File path: 
flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecSortAggRule.scala
##########
@@ -65,7 +67,13 @@ class BatchExecSortAggRule
 
   override def matches(call: RelOptRuleCall): Boolean = {
     val tableConfig = 
call.getPlanner.getContext.unwrap(classOf[FlinkContext]).getTableConfig
-    !isOperatorDisabled(tableConfig, OperatorType.SortAgg)
+    val agg: FlinkLogicalAggregate = call.rel(0)
+    !isOperatorDisabled(tableConfig, OperatorType.SortAgg) &&
+      !agg.getAggCallList.exists(x => {
+        val aggregation = x.getAggregation

Review comment:
       What about extracting this into a util method into PythonUtil?

##########
File path: 
flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecPythonAggregateRule.java
##########
@@ -0,0 +1,153 @@
+/*
+ * 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.planner.plan.rules.physical.batch;
+
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.functions.ImperativeAggregateFunction;
+import org.apache.flink.table.functions.UserDefinedFunction;
+import org.apache.flink.table.functions.python.PythonFunction;
+import org.apache.flink.table.functions.python.PythonFunctionKind;
+import org.apache.flink.table.planner.functions.utils.AggSqlFunction;
+import org.apache.flink.table.planner.plan.nodes.FlinkConventions;
+import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalAggregate;
+import 
org.apache.flink.table.planner.plan.nodes.physical.batch.BatchExecPythonGroupAggregate;
+import org.apache.flink.table.planner.plan.trait.FlinkRelDistribution;
+import org.apache.flink.table.planner.plan.utils.AggregateUtil;
+import org.apache.flink.table.planner.plan.utils.FlinkRelOptUtil;
+import org.apache.flink.table.types.DataType;
+
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelCollations;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.sql.SqlAggFunction;
+
+import java.util.LinkedList;
+import java.util.List;
+
+import scala.Tuple2;
+import scala.Tuple3;
+import scala.collection.Seq;
+
+/**
+ * The physical rule is responsible for convert {@link FlinkLogicalAggregate} 
to
+ * {@link BatchExecPythonGroupAggregate}.
+ */
+public class BatchExecPythonAggregateRule extends ConverterRule {
+
+       public static final RelOptRule INSTANCE = new 
BatchExecPythonAggregateRule();
+
+       private BatchExecPythonAggregateRule() {
+               super(FlinkLogicalAggregate.class, FlinkConventions.LOGICAL(), 
FlinkConventions.BATCH_PHYSICAL(),
+                       "BatchExecPythonAggregateRule");
+       }
+
+       @Override
+       public boolean matches(RelOptRuleCall call) {
+               FlinkLogicalAggregate agg = call.rel(0);
+               List<AggregateCall> aggCalls = agg.getAggCallList();
+               boolean existPandasFunction = false;
+               boolean existGeneralPythonFunction = false;
+               boolean existJavaFunction = false;
+               for (AggregateCall aggCall : aggCalls) {
+                       SqlAggFunction aggregation = aggCall.getAggregation();
+                       if (aggregation instanceof AggSqlFunction) {
+                               ImperativeAggregateFunction<?, ?> func =
+                                       ((AggSqlFunction) 
aggregation).aggregateFunction();
+                               if (func instanceof PythonFunction) {
+                                       PythonFunction pythonFunction = 
(PythonFunction) func;
+                                       if 
(pythonFunction.getPythonFunctionKind() == PythonFunctionKind.PANDAS) {
+                                               existPandasFunction = true;
+                                       } else {
+                                               existGeneralPythonFunction = 
true;
+                                       }
+                               } else {
+                                       existJavaFunction = true;
+                               }
+                       }
+               }
+               if (existPandasFunction) {
+                       if (existGeneralPythonFunction) {
+                               throw new TableException("Pandas UDAF cannot be 
computed with General Python UDAF currently");
+                       }
+                       if (existJavaFunction) {
+                               throw new TableException("Pandas UDAF cannot be 
computed with Java/Scala UDAF currently");

Review comment:
       ```suggestion
                                throw new TableException("Python UDAF and 
Java/Scala UDAF cannot be used together.");
   ```

##########
File path: 
flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchExecPythonGroupAggregate.scala
##########
@@ -0,0 +1,97 @@
+/*
+ * 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.planner.plan.nodes.physical.batch
+
+import org.apache.flink.api.dag.Transformation
+import org.apache.flink.runtime.operators.DamBehavior
+import org.apache.flink.table.data.RowData
+import org.apache.flink.table.functions.UserDefinedFunction
+import org.apache.flink.table.planner.delegation.BatchPlanner
+import org.apache.flink.table.planner.plan.nodes.exec.{BatchExecNode, ExecNode}
+import org.apache.calcite.plan.{RelOptCluster, RelTraitSet}
+import org.apache.calcite.rel.RelNode
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.AggregateCall
+import org.apache.calcite.tools.RelBuilder
+import java.util
+
+import org.apache.flink.table.api.TableException
+
+import scala.collection.JavaConversions._
+
+/**
+  * Batch physical RelNode for aggregate (Python user defined aggregate 
function).
+  */
+class BatchExecPythonGroupAggregate(
+    cluster: RelOptCluster,
+    relBuilder: RelBuilder,
+    traitSet: RelTraitSet,
+    inputRel: RelNode,
+    outputRowType: RelDataType,
+    inputRowType: RelDataType,
+    val aggInputRowType: RelDataType,
+    grouping: Array[Int],
+    auxGrouping: Array[Int],
+    aggCalls: Seq[AggregateCall],
+    aggFunctions: Array[UserDefinedFunction])
+  extends BatchExecGroupAggregateBase(
+    cluster,
+    relBuilder,
+    traitSet,
+    inputRel,
+    outputRowType,
+    inputRowType,
+    grouping,
+    auxGrouping,
+    aggCalls.zip(aggFunctions),
+    isMerge = false,
+    isFinal = true)
+  with BatchExecNode[RowData] {
+
+  override def getDamBehavior: DamBehavior = DamBehavior.FULL_DAM
+
+  override def getInputNodes: util.List[ExecNode[BatchPlanner, _]] =
+    List(getInput.asInstanceOf[ExecNode[BatchPlanner, _]])
+
+  override def replaceInputNode(
+      ordinalInParent: Int,
+      newInputNode: ExecNode[BatchPlanner, _]): Unit = {
+    replaceInput(ordinalInParent, newInputNode.asInstanceOf[RelNode])
+  }
+

Review comment:
       Should we also add the following methods: explainTerms, satisfyTraits?

##########
File path: 
flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecPythonAggregateRule.java
##########
@@ -0,0 +1,153 @@
+/*
+ * 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.planner.plan.rules.physical.batch;
+
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.functions.ImperativeAggregateFunction;
+import org.apache.flink.table.functions.UserDefinedFunction;
+import org.apache.flink.table.functions.python.PythonFunction;
+import org.apache.flink.table.functions.python.PythonFunctionKind;
+import org.apache.flink.table.planner.functions.utils.AggSqlFunction;
+import org.apache.flink.table.planner.plan.nodes.FlinkConventions;
+import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalAggregate;
+import 
org.apache.flink.table.planner.plan.nodes.physical.batch.BatchExecPythonGroupAggregate;
+import org.apache.flink.table.planner.plan.trait.FlinkRelDistribution;
+import org.apache.flink.table.planner.plan.utils.AggregateUtil;
+import org.apache.flink.table.planner.plan.utils.FlinkRelOptUtil;
+import org.apache.flink.table.types.DataType;
+
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelCollations;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.sql.SqlAggFunction;
+
+import java.util.LinkedList;
+import java.util.List;
+
+import scala.Tuple2;
+import scala.Tuple3;
+import scala.collection.Seq;
+
+/**
+ * The physical rule is responsible for convert {@link FlinkLogicalAggregate} 
to
+ * {@link BatchExecPythonGroupAggregate}.
+ */
+public class BatchExecPythonAggregateRule extends ConverterRule {
+
+       public static final RelOptRule INSTANCE = new 
BatchExecPythonAggregateRule();
+
+       private BatchExecPythonAggregateRule() {
+               super(FlinkLogicalAggregate.class, FlinkConventions.LOGICAL(), 
FlinkConventions.BATCH_PHYSICAL(),
+                       "BatchExecPythonAggregateRule");
+       }
+
+       @Override
+       public boolean matches(RelOptRuleCall call) {
+               FlinkLogicalAggregate agg = call.rel(0);
+               List<AggregateCall> aggCalls = agg.getAggCallList();
+               boolean existPandasFunction = false;
+               boolean existGeneralPythonFunction = false;
+               boolean existJavaFunction = false;
+               for (AggregateCall aggCall : aggCalls) {
+                       SqlAggFunction aggregation = aggCall.getAggregation();
+                       if (aggregation instanceof AggSqlFunction) {
+                               ImperativeAggregateFunction<?, ?> func =
+                                       ((AggSqlFunction) 
aggregation).aggregateFunction();
+                               if (func instanceof PythonFunction) {
+                                       PythonFunction pythonFunction = 
(PythonFunction) func;
+                                       if 
(pythonFunction.getPythonFunctionKind() == PythonFunctionKind.PANDAS) {
+                                               existPandasFunction = true;
+                                       } else {
+                                               existGeneralPythonFunction = 
true;
+                                       }
+                               } else {
+                                       existJavaFunction = true;
+                               }
+                       }
+               }
+               if (existPandasFunction) {

Review comment:
       What about refactor a bit as following?
   ```
   if (existPandasFunction || existGeneralPythonFunction) {
      if (existJavaFunction) {
         throw xxx;
      }
      if (existPandasFunction && existGeneralPythonFunction) {
       throw xxx;
      }
      return true;
   } else {
      return false;
   }
   ```

##########
File path: 
flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/physical/batch/BatchExecPythonAggregateRule.java
##########
@@ -0,0 +1,153 @@
+/*
+ * 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.planner.plan.rules.physical.batch;
+
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.functions.ImperativeAggregateFunction;
+import org.apache.flink.table.functions.UserDefinedFunction;
+import org.apache.flink.table.functions.python.PythonFunction;
+import org.apache.flink.table.functions.python.PythonFunctionKind;
+import org.apache.flink.table.planner.functions.utils.AggSqlFunction;
+import org.apache.flink.table.planner.plan.nodes.FlinkConventions;
+import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalAggregate;
+import 
org.apache.flink.table.planner.plan.nodes.physical.batch.BatchExecPythonGroupAggregate;
+import org.apache.flink.table.planner.plan.trait.FlinkRelDistribution;
+import org.apache.flink.table.planner.plan.utils.AggregateUtil;
+import org.apache.flink.table.planner.plan.utils.FlinkRelOptUtil;
+import org.apache.flink.table.types.DataType;
+
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelCollations;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.sql.SqlAggFunction;
+
+import java.util.LinkedList;
+import java.util.List;
+
+import scala.Tuple2;
+import scala.Tuple3;
+import scala.collection.Seq;
+
+/**
+ * The physical rule is responsible for convert {@link FlinkLogicalAggregate} 
to
+ * {@link BatchExecPythonGroupAggregate}.
+ */
+public class BatchExecPythonAggregateRule extends ConverterRule {
+
+       public static final RelOptRule INSTANCE = new 
BatchExecPythonAggregateRule();
+
+       private BatchExecPythonAggregateRule() {
+               super(FlinkLogicalAggregate.class, FlinkConventions.LOGICAL(), 
FlinkConventions.BATCH_PHYSICAL(),
+                       "BatchExecPythonAggregateRule");
+       }
+
+       @Override
+       public boolean matches(RelOptRuleCall call) {
+               FlinkLogicalAggregate agg = call.rel(0);
+               List<AggregateCall> aggCalls = agg.getAggCallList();
+               boolean existPandasFunction = false;
+               boolean existGeneralPythonFunction = false;
+               boolean existJavaFunction = false;
+               for (AggregateCall aggCall : aggCalls) {
+                       SqlAggFunction aggregation = aggCall.getAggregation();
+                       if (aggregation instanceof AggSqlFunction) {
+                               ImperativeAggregateFunction<?, ?> func =
+                                       ((AggSqlFunction) 
aggregation).aggregateFunction();
+                               if (func instanceof PythonFunction) {
+                                       PythonFunction pythonFunction = 
(PythonFunction) func;
+                                       if 
(pythonFunction.getPythonFunctionKind() == PythonFunctionKind.PANDAS) {
+                                               existPandasFunction = true;
+                                       } else {
+                                               existGeneralPythonFunction = 
true;
+                                       }
+                               } else {
+                                       existJavaFunction = true;
+                               }
+                       }
+               }
+               if (existPandasFunction) {
+                       if (existGeneralPythonFunction) {
+                               throw new TableException("Pandas UDAF cannot be 
computed with General Python UDAF currently");
+                       }
+                       if (existJavaFunction) {
+                               throw new TableException("Pandas UDAF cannot be 
computed with Java/Scala UDAF currently");
+                       }
+               }
+
+               return existPandasFunction || existGeneralPythonFunction;
+       }
+
+       @Override
+       public RelNode convert(RelNode relNode) {
+               FlinkLogicalAggregate agg = (FlinkLogicalAggregate) relNode;
+               RelNode input = agg.getInput();
+
+               int[] groupSet = agg.getGroupSet().toArray();
+               RelTraitSet traitSet = 
relNode.getTraitSet().replace(FlinkConventions.BATCH_PHYSICAL());
+
+               Tuple2<int[], Seq<AggregateCall>> auxGroupSetAndCallsTuple = 
AggregateUtil.checkAndSplitAggCalls(agg);
+               int[] auxGroupSet = auxGroupSetAndCallsTuple._1;
+               Seq<AggregateCall> aggCallsWithoutAuxGroupCalls = 
auxGroupSetAndCallsTuple._2;
+
+               Tuple3<int[][], DataType[][], UserDefinedFunction[]> 
aggBufferTypesAndFunctions =
+                       AggregateUtil.transformToBatchAggregateFunctions(
+                               aggCallsWithoutAuxGroupCalls, 
input.getRowType(), null);
+               UserDefinedFunction[] aggFunctions = 
aggBufferTypesAndFunctions._3();
+
+               RelTraitSet requiredTraitSet = input.getTraitSet()
+                       .replace(FlinkConventions.BATCH_PHYSICAL());
+               if (groupSet.length != 0) {
+                       FlinkRelDistribution requiredDistribution =
+                               FlinkRelDistribution.hash(groupSet, false);
+                       requiredTraitSet = 
requiredTraitSet.replace(requiredDistribution);
+                       RelCollation sortCollation = 
createRelCollation(groupSet);
+                       requiredTraitSet = 
requiredTraitSet.replace(sortCollation);
+               } else {
+                       requiredTraitSet = 
requiredTraitSet.replace(FlinkRelDistribution.SINGLETON());
+               }
+               RelNode convInput = RelOptRule.convert(input, requiredTraitSet);
+
+               return new BatchExecPythonGroupAggregate(
+                       relNode.getCluster(),
+                       null,

Review comment:
       Why set RelBuilder as null?




----------------------------------------------------------------
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:
[email protected]


Reply via email to