luoyuxia commented on code in PR #21601:
URL: https://github.com/apache/flink/pull/21601#discussion_r1070225062


##########
flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/functions/hive/HiveAverageAggFunction.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.functions.hive;
+
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.UnresolvedCallExpression;
+import org.apache.flink.table.expressions.UnresolvedReferenceExpression;
+import org.apache.flink.table.expressions.ValueLiteralExpression;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.inference.CallContext;
+
+import java.math.BigDecimal;
+
+import static 
org.apache.flink.table.expressions.ApiExpressionUtils.unresolvedRef;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.cast;
+import static org.apache.flink.table.planner.expressions.ExpressionBuilder.div;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.equalTo;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.hiveAggDecimalPlus;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.ifThenElse;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.isNull;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.literal;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.nullOf;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.plus;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.tryCast;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.typeLiteral;
+import static org.apache.flink.table.types.logical.DecimalType.MAX_PRECISION;
+import static org.apache.flink.table.types.logical.LogicalTypeRoot.DECIMAL;
+import static 
org.apache.flink.table.types.logical.utils.LogicalTypeChecks.getPrecision;
+import static 
org.apache.flink.table.types.logical.utils.LogicalTypeChecks.getScale;
+
+/** built-in hive average aggregate function. */
+public class HiveAverageAggFunction extends HiveDeclarativeAggregateFunction {
+
+    private final UnresolvedReferenceExpression sum = unresolvedRef("sum");
+    private final UnresolvedReferenceExpression count = unresolvedRef("count");
+    private DataType resultType;
+    private DataType sumResultType;
+
+    @Override
+    public int operandCount() {
+        return 1;
+    }
+
+    @Override
+    public UnresolvedReferenceExpression[] aggBufferAttributes() {
+        return new UnresolvedReferenceExpression[] {sum, count};
+    }
+
+    @Override
+    public DataType[] getAggBufferTypes() {
+        return new DataType[] {getSumResultType(), DataTypes.BIGINT()};
+    }
+
+    @Override
+    public DataType getResultType() {
+        return resultType;
+    }
+
+    @Override
+    public Expression[] initialValuesExpressions() {
+        return new Expression[] {/* sum = */ sumInitialValue(), /* count = */ 
literal(0L)};
+    }
+
+    @Override
+    public Expression[] accumulateExpressions() {
+        // cast the operand to sum needed type
+        Expression tryCastOperand = tryCast(operand(0), 
typeLiteral(getSumResultType()));
+        return new Expression[] {
+            /* sum = */ ifThenElse(isNull(tryCastOperand), sum, 
adjustedPlus(sum, tryCastOperand)),
+            /* count = */ ifThenElse(isNull(tryCastOperand), count, 
plus(count, literal(1L))),
+        };
+    }
+
+    @Override
+    public Expression[] retractExpressions() {
+        throw new TableException("Avg aggregate function does not support 
retraction.");
+    }
+
+    @Override
+    public Expression[] mergeExpressions() {
+        return new Expression[] {
+            /* sum = */ adjustedPlus(sum, mergeOperand(sum)),
+            /* count = */ plus(count, mergeOperand(count))
+        };
+    }
+
+    @Override
+    public Expression getValueExpression() {
+        // If all input are nulls, count will be 0 and we will get null after 
the division.
+        Expression ifTrue = nullOf(getResultType());
+        Expression ifFalse = cast(div(sum, count), 
typeLiteral(getResultType()));
+        return ifThenElse(equalTo(count, literal(0L)), ifTrue, ifFalse);
+    }
+
+    @Override
+    public void setArguments(CallContext callContext) {
+        if (resultType == null) {
+            DataType argsType = callContext.getArgumentDataTypes().get(0);
+            resultType = initResultType(argsType);
+            sumResultType = initSumResultType(argsType);
+        }
+    }
+
+    private DataType initResultType(DataType argsType) {
+        switch (argsType.getLogicalType().getTypeRoot()) {
+            case TINYINT:
+            case SMALLINT:
+            case INTEGER:
+            case BIGINT:
+            case FLOAT:
+            case DOUBLE:
+            case CHAR:
+            case VARCHAR:
+                return DataTypes.DOUBLE();
+            case DECIMAL:
+                // The avg result type has 4 more integer digits and 4 more 
decimal digits,
+                // following spark and hive
+                int precision =
+                        Math.min(MAX_PRECISION, 
getPrecision(argsType.getLogicalType()) + 4);
+                int scale = Math.min(38, getScale(argsType.getLogicalType()) + 
4);
+                return DataTypes.DECIMAL(precision, scale);
+            default:
+                throw new TableException(
+                        String.format(
+                                "Avg aggregate function does not support type: 
'%s'. Please re-check the data type.",
+                                argsType.getLogicalType().getTypeRoot()));
+        }
+    }
+
+    private DataType initSumResultType(DataType argsType) {
+        switch (argsType.getLogicalType().getTypeRoot()) {
+            case TINYINT:
+            case SMALLINT:
+            case INTEGER:
+            case BIGINT:
+            case FLOAT:
+            case DOUBLE:
+            case CHAR:
+            case VARCHAR:
+                return DataTypes.DOUBLE();
+            case DECIMAL:
+                // The intermediate sum field has 10 more integer digits with 
the same scale.
+                int precision =
+                        Math.min(MAX_PRECISION, 
getPrecision(argsType.getLogicalType()) + 10);
+                return DataTypes.DECIMAL(precision, 
getScale(argsType.getLogicalType()));
+            default:
+                throw new TableException(

Review Comment:
   Dito



##########
flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/functions/hive/HiveAverageAggFunction.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.functions.hive;
+
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.UnresolvedCallExpression;
+import org.apache.flink.table.expressions.UnresolvedReferenceExpression;
+import org.apache.flink.table.expressions.ValueLiteralExpression;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.inference.CallContext;
+
+import java.math.BigDecimal;
+
+import static 
org.apache.flink.table.expressions.ApiExpressionUtils.unresolvedRef;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.cast;
+import static org.apache.flink.table.planner.expressions.ExpressionBuilder.div;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.equalTo;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.hiveAggDecimalPlus;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.ifThenElse;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.isNull;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.literal;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.nullOf;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.plus;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.tryCast;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.typeLiteral;
+import static org.apache.flink.table.types.logical.DecimalType.MAX_PRECISION;
+import static org.apache.flink.table.types.logical.LogicalTypeRoot.DECIMAL;
+import static 
org.apache.flink.table.types.logical.utils.LogicalTypeChecks.getPrecision;
+import static 
org.apache.flink.table.types.logical.utils.LogicalTypeChecks.getScale;
+
+/** built-in hive average aggregate function. */
+public class HiveAverageAggFunction extends HiveDeclarativeAggregateFunction {
+
+    private final UnresolvedReferenceExpression sum = unresolvedRef("sum");
+    private final UnresolvedReferenceExpression count = unresolvedRef("count");
+    private DataType resultType;
+    private DataType sumResultType;
+
+    @Override
+    public int operandCount() {
+        return 1;
+    }
+
+    @Override
+    public UnresolvedReferenceExpression[] aggBufferAttributes() {
+        return new UnresolvedReferenceExpression[] {sum, count};
+    }
+
+    @Override
+    public DataType[] getAggBufferTypes() {
+        return new DataType[] {getSumResultType(), DataTypes.BIGINT()};
+    }
+
+    @Override
+    public DataType getResultType() {
+        return resultType;
+    }
+
+    @Override
+    public Expression[] initialValuesExpressions() {
+        return new Expression[] {/* sum = */ sumInitialValue(), /* count = */ 
literal(0L)};
+    }
+
+    @Override
+    public Expression[] accumulateExpressions() {
+        // cast the operand to sum needed type
+        Expression tryCastOperand = tryCast(operand(0), 
typeLiteral(getSumResultType()));
+        return new Expression[] {
+            /* sum = */ ifThenElse(isNull(tryCastOperand), sum, 
adjustedPlus(sum, tryCastOperand)),
+            /* count = */ ifThenElse(isNull(tryCastOperand), count, 
plus(count, literal(1L))),
+        };
+    }
+
+    @Override
+    public Expression[] retractExpressions() {
+        throw new TableException("Avg aggregate function does not support 
retraction.");
+    }
+
+    @Override
+    public Expression[] mergeExpressions() {
+        return new Expression[] {
+            /* sum = */ adjustedPlus(sum, mergeOperand(sum)),
+            /* count = */ plus(count, mergeOperand(count))
+        };
+    }
+
+    @Override
+    public Expression getValueExpression() {
+        // If all input are nulls, count will be 0 and we will get null after 
the division.
+        Expression ifTrue = nullOf(getResultType());
+        Expression ifFalse = cast(div(sum, count), 
typeLiteral(getResultType()));
+        return ifThenElse(equalTo(count, literal(0L)), ifTrue, ifFalse);
+    }
+
+    @Override
+    public void setArguments(CallContext callContext) {
+        if (resultType == null) {
+            DataType argsType = callContext.getArgumentDataTypes().get(0);
+            resultType = initResultType(argsType);
+            sumResultType = initSumResultType(argsType);
+        }
+    }
+
+    private DataType initResultType(DataType argsType) {
+        switch (argsType.getLogicalType().getTypeRoot()) {
+            case TINYINT:
+            case SMALLINT:
+            case INTEGER:
+            case BIGINT:
+            case FLOAT:
+            case DOUBLE:
+            case CHAR:
+            case VARCHAR:
+                return DataTypes.DOUBLE();
+            case DECIMAL:
+                // The avg result type has 4 more integer digits and 4 more 
decimal digits,
+                // following spark and hive
+                int precision =
+                        Math.min(MAX_PRECISION, 
getPrecision(argsType.getLogicalType()) + 4);
+                int scale = Math.min(38, getScale(argsType.getLogicalType()) + 
4);

Review Comment:
   for scale, Hive's implementation is 
   `scale = Math.min(scale + 4, HiveDecimal.MAX_SCALE - intPart);`
   Are they equal?
   Also, may be better to make `38` as a static field. 



##########
flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/functions/hive/HiveAverageAggFunction.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.functions.hive;
+
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.UnresolvedCallExpression;
+import org.apache.flink.table.expressions.UnresolvedReferenceExpression;
+import org.apache.flink.table.expressions.ValueLiteralExpression;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.inference.CallContext;
+
+import java.math.BigDecimal;
+
+import static 
org.apache.flink.table.expressions.ApiExpressionUtils.unresolvedRef;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.cast;
+import static org.apache.flink.table.planner.expressions.ExpressionBuilder.div;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.equalTo;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.hiveAggDecimalPlus;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.ifThenElse;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.isNull;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.literal;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.nullOf;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.plus;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.tryCast;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.typeLiteral;
+import static org.apache.flink.table.types.logical.DecimalType.MAX_PRECISION;
+import static org.apache.flink.table.types.logical.LogicalTypeRoot.DECIMAL;
+import static 
org.apache.flink.table.types.logical.utils.LogicalTypeChecks.getPrecision;
+import static 
org.apache.flink.table.types.logical.utils.LogicalTypeChecks.getScale;
+
+/** built-in hive average aggregate function. */
+public class HiveAverageAggFunction extends HiveDeclarativeAggregateFunction {
+
+    private final UnresolvedReferenceExpression sum = unresolvedRef("sum");
+    private final UnresolvedReferenceExpression count = unresolvedRef("count");
+    private DataType resultType;
+    private DataType sumResultType;
+
+    @Override
+    public int operandCount() {
+        return 1;
+    }
+
+    @Override
+    public UnresolvedReferenceExpression[] aggBufferAttributes() {
+        return new UnresolvedReferenceExpression[] {sum, count};
+    }
+
+    @Override
+    public DataType[] getAggBufferTypes() {
+        return new DataType[] {getSumResultType(), DataTypes.BIGINT()};
+    }
+
+    @Override
+    public DataType getResultType() {
+        return resultType;
+    }
+
+    @Override
+    public Expression[] initialValuesExpressions() {
+        return new Expression[] {/* sum = */ sumInitialValue(), /* count = */ 
literal(0L)};
+    }
+
+    @Override
+    public Expression[] accumulateExpressions() {
+        // cast the operand to sum needed type
+        Expression tryCastOperand = tryCast(operand(0), 
typeLiteral(getSumResultType()));
+        return new Expression[] {
+            /* sum = */ ifThenElse(isNull(tryCastOperand), sum, 
adjustedPlus(sum, tryCastOperand)),

Review Comment:
   IIUC, for decimal, if `tryCastOperand` is null, the `sum` should be null in 
[Hive's 
implementation.](https://github.com/apache/hive/blob/7b3ecf617a6d46f48a3b6f77e0339fd4ad95a420/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFAverage.java#L315)



##########
flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/functions/hive/HiveAverageAggFunction.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.functions.hive;
+
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.UnresolvedCallExpression;
+import org.apache.flink.table.expressions.UnresolvedReferenceExpression;
+import org.apache.flink.table.expressions.ValueLiteralExpression;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.inference.CallContext;
+
+import java.math.BigDecimal;
+
+import static 
org.apache.flink.table.expressions.ApiExpressionUtils.unresolvedRef;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.cast;
+import static org.apache.flink.table.planner.expressions.ExpressionBuilder.div;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.equalTo;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.hiveAggDecimalPlus;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.ifThenElse;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.isNull;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.literal;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.nullOf;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.plus;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.tryCast;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.typeLiteral;
+import static org.apache.flink.table.types.logical.DecimalType.MAX_PRECISION;
+import static org.apache.flink.table.types.logical.LogicalTypeRoot.DECIMAL;
+import static 
org.apache.flink.table.types.logical.utils.LogicalTypeChecks.getPrecision;
+import static 
org.apache.flink.table.types.logical.utils.LogicalTypeChecks.getScale;
+
+/** built-in hive average aggregate function. */
+public class HiveAverageAggFunction extends HiveDeclarativeAggregateFunction {
+
+    private final UnresolvedReferenceExpression sum = unresolvedRef("sum");
+    private final UnresolvedReferenceExpression count = unresolvedRef("count");
+    private DataType resultType;
+    private DataType sumResultType;
+
+    @Override
+    public int operandCount() {
+        return 1;
+    }
+
+    @Override
+    public UnresolvedReferenceExpression[] aggBufferAttributes() {
+        return new UnresolvedReferenceExpression[] {sum, count};
+    }
+
+    @Override
+    public DataType[] getAggBufferTypes() {
+        return new DataType[] {getSumResultType(), DataTypes.BIGINT()};
+    }
+
+    @Override
+    public DataType getResultType() {
+        return resultType;
+    }
+
+    @Override
+    public Expression[] initialValuesExpressions() {
+        return new Expression[] {/* sum = */ sumInitialValue(), /* count = */ 
literal(0L)};
+    }
+
+    @Override
+    public Expression[] accumulateExpressions() {
+        // cast the operand to sum needed type
+        Expression tryCastOperand = tryCast(operand(0), 
typeLiteral(getSumResultType()));
+        return new Expression[] {
+            /* sum = */ ifThenElse(isNull(tryCastOperand), sum, 
adjustedPlus(sum, tryCastOperand)),
+            /* count = */ ifThenElse(isNull(tryCastOperand), count, 
plus(count, literal(1L))),
+        };
+    }
+
+    @Override
+    public Expression[] retractExpressions() {
+        throw new TableException("Avg aggregate function does not support 
retraction.");
+    }
+
+    @Override
+    public Expression[] mergeExpressions() {
+        return new Expression[] {
+            /* sum = */ adjustedPlus(sum, mergeOperand(sum)),
+            /* count = */ plus(count, mergeOperand(count))
+        };
+    }
+
+    @Override
+    public Expression getValueExpression() {
+        // If all input are nulls, count will be 0 and we will get null after 
the division.
+        Expression ifTrue = nullOf(getResultType());
+        Expression ifFalse = cast(div(sum, count), 
typeLiteral(getResultType()));
+        return ifThenElse(equalTo(count, literal(0L)), ifTrue, ifFalse);
+    }
+
+    @Override
+    public void setArguments(CallContext callContext) {
+        if (resultType == null) {
+            DataType argsType = callContext.getArgumentDataTypes().get(0);
+            resultType = initResultType(argsType);
+            sumResultType = initSumResultType(argsType);
+        }
+    }
+
+    private DataType initResultType(DataType argsType) {
+        switch (argsType.getLogicalType().getTypeRoot()) {
+            case TINYINT:
+            case SMALLINT:
+            case INTEGER:
+            case BIGINT:
+            case FLOAT:
+            case DOUBLE:
+            case CHAR:
+            case VARCHAR:
+                return DataTypes.DOUBLE();
+            case DECIMAL:
+                // The avg result type has 4 more integer digits and 4 more 
decimal digits,
+                // following spark and hive
+                int precision =
+                        Math.min(MAX_PRECISION, 
getPrecision(argsType.getLogicalType()) + 4);
+                int scale = Math.min(38, getScale(argsType.getLogicalType()) + 
4);
+                return DataTypes.DECIMAL(precision, scale);
+            default:
+                throw new TableException(

Review Comment:
   Follow the hive way to throw exception message?
   Here's  hive's exception message:
   ```
   throw new UDFArgumentTypeException(0,
             "Only numeric or string type arguments are accepted but "
                 + parameters[0].getTypeName() + " is passed.");
   ```



##########
flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/functions/hive/HiveAverageAggFunction.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.functions.hive;
+
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.UnresolvedCallExpression;
+import org.apache.flink.table.expressions.UnresolvedReferenceExpression;
+import org.apache.flink.table.expressions.ValueLiteralExpression;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.inference.CallContext;
+
+import java.math.BigDecimal;
+
+import static 
org.apache.flink.table.expressions.ApiExpressionUtils.unresolvedRef;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.cast;
+import static org.apache.flink.table.planner.expressions.ExpressionBuilder.div;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.equalTo;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.hiveAggDecimalPlus;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.ifThenElse;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.isNull;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.literal;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.nullOf;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.plus;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.tryCast;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.typeLiteral;
+import static org.apache.flink.table.types.logical.DecimalType.MAX_PRECISION;
+import static org.apache.flink.table.types.logical.LogicalTypeRoot.DECIMAL;
+import static 
org.apache.flink.table.types.logical.utils.LogicalTypeChecks.getPrecision;
+import static 
org.apache.flink.table.types.logical.utils.LogicalTypeChecks.getScale;
+
+/** built-in hive average aggregate function. */
+public class HiveAverageAggFunction extends HiveDeclarativeAggregateFunction {
+
+    private final UnresolvedReferenceExpression sum = unresolvedRef("sum");
+    private final UnresolvedReferenceExpression count = unresolvedRef("count");
+    private DataType resultType;
+    private DataType sumResultType;
+
+    @Override
+    public int operandCount() {
+        return 1;
+    }
+
+    @Override
+    public UnresolvedReferenceExpression[] aggBufferAttributes() {
+        return new UnresolvedReferenceExpression[] {sum, count};
+    }
+
+    @Override
+    public DataType[] getAggBufferTypes() {
+        return new DataType[] {getSumResultType(), DataTypes.BIGINT()};
+    }
+
+    @Override
+    public DataType getResultType() {
+        return resultType;
+    }
+
+    @Override
+    public Expression[] initialValuesExpressions() {
+        return new Expression[] {/* sum = */ sumInitialValue(), /* count = */ 
literal(0L)};
+    }
+
+    @Override
+    public Expression[] accumulateExpressions() {
+        // cast the operand to sum needed type
+        Expression tryCastOperand = tryCast(operand(0), 
typeLiteral(getSumResultType()));
+        return new Expression[] {
+            /* sum = */ ifThenElse(isNull(tryCastOperand), sum, 
adjustedPlus(sum, tryCastOperand)),
+            /* count = */ ifThenElse(isNull(tryCastOperand), count, 
plus(count, literal(1L))),
+        };
+    }
+
+    @Override
+    public Expression[] retractExpressions() {
+        throw new TableException("Avg aggregate function does not support 
retraction.");
+    }
+
+    @Override
+    public Expression[] mergeExpressions() {
+        return new Expression[] {
+            /* sum = */ adjustedPlus(sum, mergeOperand(sum)),
+            /* count = */ plus(count, mergeOperand(count))
+        };
+    }
+
+    @Override
+    public Expression getValueExpression() {
+        // If all input are nulls, count will be 0 and we will get null after 
the division.
+        Expression ifTrue = nullOf(getResultType());
+        Expression ifFalse = cast(div(sum, count), 
typeLiteral(getResultType()));
+        return ifThenElse(equalTo(count, literal(0L)), ifTrue, ifFalse);
+    }
+
+    @Override
+    public void setArguments(CallContext callContext) {
+        if (resultType == null) {
+            DataType argsType = callContext.getArgumentDataTypes().get(0);
+            resultType = initResultType(argsType);
+            sumResultType = initSumResultType(argsType);
+        }
+    }
+
+    private DataType initResultType(DataType argsType) {
+        switch (argsType.getLogicalType().getTypeRoot()) {
+            case TINYINT:
+            case SMALLINT:
+            case INTEGER:
+            case BIGINT:
+            case FLOAT:
+            case DOUBLE:
+            case CHAR:
+            case VARCHAR:
+                return DataTypes.DOUBLE();
+            case DECIMAL:
+                // The avg result type has 4 more integer digits and 4 more 
decimal digits,
+                // following spark and hive
+                int precision =
+                        Math.min(MAX_PRECISION, 
getPrecision(argsType.getLogicalType()) + 4);
+                int scale = Math.min(38, getScale(argsType.getLogicalType()) + 
4);
+                return DataTypes.DECIMAL(precision, scale);
+            default:
+                throw new TableException(
+                        String.format(
+                                "Avg aggregate function does not support type: 
'%s'. Please re-check the data type.",
+                                argsType.getLogicalType().getTypeRoot()));
+        }
+    }
+
+    private DataType initSumResultType(DataType argsType) {
+        switch (argsType.getLogicalType().getTypeRoot()) {
+            case TINYINT:
+            case SMALLINT:
+            case INTEGER:
+            case BIGINT:
+            case FLOAT:
+            case DOUBLE:
+            case CHAR:
+            case VARCHAR:
+                return DataTypes.DOUBLE();
+            case DECIMAL:
+                // The intermediate sum field has 10 more integer digits with 
the same scale.
+                int precision =

Review Comment:
   for precision, Hive's implementation is 
   ```
   int intPart = precision - scale;
   intPart = Math.min(intPart + 10, HiveDecimal.MAX_PRECISION - scale);
   precision = intPart + scale;
   ```
   Are they equal?



##########
flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/functions/hive/HiveAverageAggFunction.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.functions.hive;
+
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.UnresolvedCallExpression;
+import org.apache.flink.table.expressions.UnresolvedReferenceExpression;
+import org.apache.flink.table.expressions.ValueLiteralExpression;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.inference.CallContext;
+
+import java.math.BigDecimal;
+
+import static 
org.apache.flink.table.expressions.ApiExpressionUtils.unresolvedRef;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.cast;
+import static org.apache.flink.table.planner.expressions.ExpressionBuilder.div;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.equalTo;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.hiveAggDecimalPlus;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.ifThenElse;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.isNull;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.literal;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.nullOf;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.plus;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.tryCast;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.typeLiteral;
+import static org.apache.flink.table.types.logical.DecimalType.MAX_PRECISION;
+import static org.apache.flink.table.types.logical.LogicalTypeRoot.DECIMAL;
+import static 
org.apache.flink.table.types.logical.utils.LogicalTypeChecks.getPrecision;
+import static 
org.apache.flink.table.types.logical.utils.LogicalTypeChecks.getScale;
+
+/** built-in hive average aggregate function. */
+public class HiveAverageAggFunction extends HiveDeclarativeAggregateFunction {
+
+    private final UnresolvedReferenceExpression sum = unresolvedRef("sum");
+    private final UnresolvedReferenceExpression count = unresolvedRef("count");
+    private DataType resultType;
+    private DataType sumResultType;
+
+    @Override
+    public int operandCount() {
+        return 1;
+    }
+
+    @Override
+    public UnresolvedReferenceExpression[] aggBufferAttributes() {
+        return new UnresolvedReferenceExpression[] {sum, count};
+    }
+
+    @Override
+    public DataType[] getAggBufferTypes() {
+        return new DataType[] {getSumResultType(), DataTypes.BIGINT()};
+    }
+
+    @Override
+    public DataType getResultType() {
+        return resultType;
+    }
+
+    @Override
+    public Expression[] initialValuesExpressions() {
+        return new Expression[] {/* sum = */ sumInitialValue(), /* count = */ 
literal(0L)};
+    }
+
+    @Override
+    public Expression[] accumulateExpressions() {
+        // cast the operand to sum needed type
+        Expression tryCastOperand = tryCast(operand(0), 
typeLiteral(getSumResultType()));
+        return new Expression[] {
+            /* sum = */ ifThenElse(isNull(tryCastOperand), sum, 
adjustedPlus(sum, tryCastOperand)),
+            /* count = */ ifThenElse(isNull(tryCastOperand), count, 
plus(count, literal(1L))),
+        };
+    }
+
+    @Override
+    public Expression[] retractExpressions() {
+        throw new TableException("Avg aggregate function does not support 
retraction.");
+    }
+
+    @Override
+    public Expression[] mergeExpressions() {
+        return new Expression[] {
+            /* sum = */ adjustedPlus(sum, mergeOperand(sum)),
+            /* count = */ plus(count, mergeOperand(count))
+        };
+    }
+
+    @Override
+    public Expression getValueExpression() {
+        // If all input are nulls, count will be 0 and we will get null after 
the division.
+        Expression ifTrue = nullOf(getResultType());
+        Expression ifFalse = cast(div(sum, count), 
typeLiteral(getResultType()));
+        return ifThenElse(equalTo(count, literal(0L)), ifTrue, ifFalse);
+    }
+
+    @Override
+    public void setArguments(CallContext callContext) {
+        if (resultType == null) {
+            DataType argsType = callContext.getArgumentDataTypes().get(0);
+            resultType = initResultType(argsType);
+            sumResultType = initSumResultType(argsType);
+        }
+    }
+
+    private DataType initResultType(DataType argsType) {
+        switch (argsType.getLogicalType().getTypeRoot()) {
+            case TINYINT:
+            case SMALLINT:
+            case INTEGER:
+            case BIGINT:
+            case FLOAT:
+            case DOUBLE:
+            case CHAR:
+            case VARCHAR:
+                return DataTypes.DOUBLE();
+            case DECIMAL:
+                // The avg result type has 4 more integer digits and 4 more 
decimal digits,
+                // following spark and hive
+                int precision =
+                        Math.min(MAX_PRECISION, 
getPrecision(argsType.getLogicalType()) + 4);
+                int scale = Math.min(38, getScale(argsType.getLogicalType()) + 
4);
+                return DataTypes.DECIMAL(precision, scale);
+            default:

Review Comment:
   Hive supports average for timestamp, may need to remind use to fall back.



##########
flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/functions/hive/HiveAverageAggFunction.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.functions.hive;
+
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.UnresolvedCallExpression;
+import org.apache.flink.table.expressions.UnresolvedReferenceExpression;
+import org.apache.flink.table.expressions.ValueLiteralExpression;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.inference.CallContext;
+
+import java.math.BigDecimal;
+
+import static 
org.apache.flink.table.expressions.ApiExpressionUtils.unresolvedRef;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.cast;
+import static org.apache.flink.table.planner.expressions.ExpressionBuilder.div;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.equalTo;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.hiveAggDecimalPlus;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.ifThenElse;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.isNull;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.literal;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.nullOf;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.plus;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.tryCast;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.typeLiteral;
+import static org.apache.flink.table.types.logical.DecimalType.MAX_PRECISION;
+import static org.apache.flink.table.types.logical.LogicalTypeRoot.DECIMAL;
+import static 
org.apache.flink.table.types.logical.utils.LogicalTypeChecks.getPrecision;
+import static 
org.apache.flink.table.types.logical.utils.LogicalTypeChecks.getScale;
+
+/** built-in hive average aggregate function. */
+public class HiveAverageAggFunction extends HiveDeclarativeAggregateFunction {
+
+    private final UnresolvedReferenceExpression sum = unresolvedRef("sum");
+    private final UnresolvedReferenceExpression count = unresolvedRef("count");
+    private DataType resultType;
+    private DataType sumResultType;
+
+    @Override
+    public int operandCount() {
+        return 1;
+    }
+
+    @Override
+    public UnresolvedReferenceExpression[] aggBufferAttributes() {
+        return new UnresolvedReferenceExpression[] {sum, count};
+    }
+
+    @Override
+    public DataType[] getAggBufferTypes() {
+        return new DataType[] {getSumResultType(), DataTypes.BIGINT()};
+    }
+
+    @Override
+    public DataType getResultType() {
+        return resultType;
+    }
+
+    @Override
+    public Expression[] initialValuesExpressions() {
+        return new Expression[] {/* sum = */ sumInitialValue(), /* count = */ 
literal(0L)};
+    }
+
+    @Override
+    public Expression[] accumulateExpressions() {
+        // cast the operand to sum needed type
+        Expression tryCastOperand = tryCast(operand(0), 
typeLiteral(getSumResultType()));
+        return new Expression[] {
+            /* sum = */ ifThenElse(isNull(tryCastOperand), sum, 
adjustedPlus(sum, tryCastOperand)),
+            /* count = */ ifThenElse(isNull(tryCastOperand), count, 
plus(count, literal(1L))),
+        };
+    }
+
+    @Override
+    public Expression[] retractExpressions() {
+        throw new TableException("Avg aggregate function does not support 
retraction.");
+    }
+
+    @Override
+    public Expression[] mergeExpressions() {
+        return new Expression[] {
+            /* sum = */ adjustedPlus(sum, mergeOperand(sum)),
+            /* count = */ plus(count, mergeOperand(count))
+        };
+    }
+
+    @Override
+    public Expression getValueExpression() {
+        // If all input are nulls, count will be 0 and we will get null after 
the division.
+        Expression ifTrue = nullOf(getResultType());
+        Expression ifFalse = cast(div(sum, count), 
typeLiteral(getResultType()));
+        return ifThenElse(equalTo(count, literal(0L)), ifTrue, ifFalse);
+    }
+
+    @Override
+    public void setArguments(CallContext callContext) {
+        if (resultType == null) {
+            DataType argsType = callContext.getArgumentDataTypes().get(0);
+            resultType = initResultType(argsType);
+            sumResultType = initSumResultType(argsType);
+        }
+    }
+
+    private DataType initResultType(DataType argsType) {
+        switch (argsType.getLogicalType().getTypeRoot()) {
+            case TINYINT:
+            case SMALLINT:
+            case INTEGER:
+            case BIGINT:
+            case FLOAT:
+            case DOUBLE:
+            case CHAR:
+            case VARCHAR:
+                return DataTypes.DOUBLE();
+            case DECIMAL:
+                // The avg result type has 4 more integer digits and 4 more 
decimal digits,
+                // following spark and hive
+                int precision =
+                        Math.min(MAX_PRECISION, 
getPrecision(argsType.getLogicalType()) + 4);
+                int scale = Math.min(38, getScale(argsType.getLogicalType()) + 
4);
+                return DataTypes.DECIMAL(precision, scale);
+            default:
+                throw new TableException(
+                        String.format(
+                                "Avg aggregate function does not support type: 
'%s'. Please re-check the data type.",
+                                argsType.getLogicalType().getTypeRoot()));
+        }
+    }
+
+    private DataType initSumResultType(DataType argsType) {
+        switch (argsType.getLogicalType().getTypeRoot()) {
+            case TINYINT:
+            case SMALLINT:
+            case INTEGER:
+            case BIGINT:
+            case FLOAT:
+            case DOUBLE:
+            case CHAR:
+            case VARCHAR:
+                return DataTypes.DOUBLE();
+            case DECIMAL:
+                // The intermediate sum field has 10 more integer digits with 
the same scale.
+                int precision =
+                        Math.min(MAX_PRECISION, 
getPrecision(argsType.getLogicalType()) + 10);
+                return DataTypes.DECIMAL(precision, 
getScale(argsType.getLogicalType()));
+            default:
+                throw new TableException(
+                        String.format(
+                                "Avg aggregate function does not support type: 
'%s'. Please re-check the data type.",
+                                argsType.getLogicalType().getTypeRoot()));
+        }
+    }
+
+    private DataType getSumResultType() {
+        return sumResultType;
+    }
+
+    private UnresolvedCallExpression adjustedPlus(Expression arg1, Expression 
arg2) {
+        if (getSumResultType().getLogicalType().is(DECIMAL)) {
+            return hiveAggDecimalPlus(arg1, arg2);
+        } else {
+            return plus(arg1, arg2);
+        }
+    }
+
+    private ValueLiteralExpression sumInitialValue() {
+        if (getSumResultType().getLogicalType().is(DECIMAL)) {
+            return literal(BigDecimal.ZERO, getSumResultType().notNull());

Review Comment:
   Should be nullable.



##########
flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/functions/hive/HiveAverageAggFunction.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.functions.hive;
+
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.UnresolvedCallExpression;
+import org.apache.flink.table.expressions.UnresolvedReferenceExpression;
+import org.apache.flink.table.expressions.ValueLiteralExpression;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.inference.CallContext;
+
+import java.math.BigDecimal;
+
+import static 
org.apache.flink.table.expressions.ApiExpressionUtils.unresolvedRef;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.cast;
+import static org.apache.flink.table.planner.expressions.ExpressionBuilder.div;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.equalTo;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.hiveAggDecimalPlus;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.ifThenElse;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.isNull;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.literal;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.nullOf;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.plus;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.tryCast;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.typeLiteral;
+import static org.apache.flink.table.types.logical.DecimalType.MAX_PRECISION;
+import static org.apache.flink.table.types.logical.LogicalTypeRoot.DECIMAL;
+import static 
org.apache.flink.table.types.logical.utils.LogicalTypeChecks.getPrecision;
+import static 
org.apache.flink.table.types.logical.utils.LogicalTypeChecks.getScale;
+
+/** built-in hive average aggregate function. */
+public class HiveAverageAggFunction extends HiveDeclarativeAggregateFunction {
+
+    private final UnresolvedReferenceExpression sum = unresolvedRef("sum");
+    private final UnresolvedReferenceExpression count = unresolvedRef("count");
+    private DataType resultType;
+    private DataType sumResultType;
+
+    @Override
+    public int operandCount() {
+        return 1;
+    }
+
+    @Override
+    public UnresolvedReferenceExpression[] aggBufferAttributes() {
+        return new UnresolvedReferenceExpression[] {sum, count};
+    }
+
+    @Override
+    public DataType[] getAggBufferTypes() {
+        return new DataType[] {getSumResultType(), DataTypes.BIGINT()};
+    }
+
+    @Override
+    public DataType getResultType() {
+        return resultType;
+    }
+
+    @Override
+    public Expression[] initialValuesExpressions() {
+        return new Expression[] {/* sum = */ sumInitialValue(), /* count = */ 
literal(0L)};
+    }
+
+    @Override
+    public Expression[] accumulateExpressions() {
+        // cast the operand to sum needed type
+        Expression tryCastOperand = tryCast(operand(0), 
typeLiteral(getSumResultType()));
+        return new Expression[] {
+            /* sum = */ ifThenElse(isNull(tryCastOperand), sum, 
adjustedPlus(sum, tryCastOperand)),
+            /* count = */ ifThenElse(isNull(tryCastOperand), count, 
plus(count, literal(1L))),
+        };
+    }
+
+    @Override
+    public Expression[] retractExpressions() {
+        throw new TableException("Avg aggregate function does not support 
retraction.");
+    }
+
+    @Override
+    public Expression[] mergeExpressions() {
+        return new Expression[] {
+            /* sum = */ adjustedPlus(sum, mergeOperand(sum)),
+            /* count = */ plus(count, mergeOperand(count))
+        };
+    }
+
+    @Override
+    public Expression getValueExpression() {
+        // If all input are nulls, count will be 0 and we will get null after 
the division.
+        Expression ifTrue = nullOf(getResultType());

Review Comment:
   the name `ifTrue` and `ifFalse` is confusing,  if no better name, I think we 
may just inline them
   `
   ifThenElse(equalTo(count, literal(0L)),
    nullOf(getResultType()), cast(div(sum, count), 
typeLiteral(getResultType())));
   `



##########
flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveDialectQueryITCase.java:
##########
@@ -1062,6 +1062,63 @@ public void testSumAggWithGroupKey() throws Exception {
         tableEnv.executeSql("drop table test_sum_group");
     }
 
+    @Test
+    public void testAvgAggFunctionPlan() {
+        // test explain
+        String actualPlan = explainSql("select x, avg(y) from foo group by x");
+        
assertThat(actualPlan).isEqualTo(readFromResource("/explain/testAvgAggFunctionPlan.out"));
+    }
+
+    @Test
+    public void testAvgAggFunction() throws Exception {
+        tableEnv.executeSql(
+                "create table test_avg(a int, x string, y string, z int, f 
bigint, d decimal(20, 5), e double)");
+        tableEnv.executeSql(
+                        "insert into test_avg values (1, NULL, '2', 1, 2, 
2.22, 2.3), "
+                                + "(1, NULL, 'b', 2, NULL, 3.33, 3.4), "
+                                + "(2, NULL, '4', 1, 2, 4.55, 4.5), "
+                                + "(2, NULL, NULL, 4, 3, 5.66, 5.2)")
+                .await();
+
+        // test avg all element is null
+        List<Row> result =
+                CollectionUtil.iteratorToList(
+                        tableEnv.executeSql("select avg(x) from 
test_avg").collect());
+        assertThat(result.toString()).isEqualTo("[+I[null]]");
+
+        // test avg that some string elements can't convert to double
+        List<Row> result2 =
+                CollectionUtil.iteratorToList(
+                        tableEnv.executeSql("select avg(y) from 
test_avg").collect());
+        assertThat(result2.toString()).isEqualTo("[+I[3.0]]");

Review Comment:
   should be `[+I[2.0]]`



##########
flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveDialectQueryITCase.java:
##########
@@ -1062,6 +1062,63 @@ public void testSumAggWithGroupKey() throws Exception {
         tableEnv.executeSql("drop table test_sum_group");
     }
 
+    @Test
+    public void testAvgAggFunctionPlan() {
+        // test explain
+        String actualPlan = explainSql("select x, avg(y) from foo group by x");
+        
assertThat(actualPlan).isEqualTo(readFromResource("/explain/testAvgAggFunctionPlan.out"));
+    }
+
+    @Test
+    public void testAvgAggFunction() throws Exception {
+        tableEnv.executeSql(
+                "create table test_avg(a int, x string, y string, z int, f 
bigint, d decimal(20, 5), e double)");
+        tableEnv.executeSql(
+                        "insert into test_avg values (1, NULL, '2', 1, 2, 
2.22, 2.3), "
+                                + "(1, NULL, 'b', 2, NULL, 3.33, 3.4), "
+                                + "(2, NULL, '4', 1, 2, 4.55, 4.5), "
+                                + "(2, NULL, NULL, 4, 3, 5.66, 5.2)")
+                .await();
+
+        // test avg all element is null
+        List<Row> result =
+                CollectionUtil.iteratorToList(
+                        tableEnv.executeSql("select avg(x) from 
test_avg").collect());
+        assertThat(result.toString()).isEqualTo("[+I[null]]");
+
+        // test avg that some string elements can't convert to double
+        List<Row> result2 =
+                CollectionUtil.iteratorToList(
+                        tableEnv.executeSql("select avg(y) from 
test_avg").collect());
+        assertThat(result2.toString()).isEqualTo("[+I[3.0]]");
+
+        // test avg bigint with null element
+        List<Row> result3 =
+                CollectionUtil.iteratorToList(
+                        tableEnv.executeSql("select avg(f) from 
test_avg").collect());
+        assertThat(result3.toString()).isEqualTo("[+I[2.3333333333333335]]");
+
+        // test avg decimal
+        List<Row> result4 =

Review Comment:
   add same case for decimal with 
   - all element is null
   - some string elements can't convert to decimal
   



##########
flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/functions/hive/HiveAverageAggFunction.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.functions.hive;
+
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.UnresolvedCallExpression;
+import org.apache.flink.table.expressions.UnresolvedReferenceExpression;
+import org.apache.flink.table.expressions.ValueLiteralExpression;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.inference.CallContext;
+
+import java.math.BigDecimal;
+
+import static 
org.apache.flink.table.expressions.ApiExpressionUtils.unresolvedRef;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.cast;
+import static org.apache.flink.table.planner.expressions.ExpressionBuilder.div;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.equalTo;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.hiveAggDecimalPlus;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.ifThenElse;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.isNull;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.literal;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.nullOf;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.plus;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.tryCast;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.typeLiteral;
+import static org.apache.flink.table.types.logical.DecimalType.MAX_PRECISION;
+import static org.apache.flink.table.types.logical.LogicalTypeRoot.DECIMAL;
+import static 
org.apache.flink.table.types.logical.utils.LogicalTypeChecks.getPrecision;
+import static 
org.apache.flink.table.types.logical.utils.LogicalTypeChecks.getScale;
+
+/** built-in hive average aggregate function. */
+public class HiveAverageAggFunction extends HiveDeclarativeAggregateFunction {
+
+    private final UnresolvedReferenceExpression sum = unresolvedRef("sum");
+    private final UnresolvedReferenceExpression count = unresolvedRef("count");
+    private DataType resultType;
+    private DataType sumResultType;
+
+    @Override
+    public int operandCount() {
+        return 1;
+    }
+
+    @Override
+    public UnresolvedReferenceExpression[] aggBufferAttributes() {
+        return new UnresolvedReferenceExpression[] {sum, count};
+    }
+
+    @Override
+    public DataType[] getAggBufferTypes() {
+        return new DataType[] {getSumResultType(), DataTypes.BIGINT()};
+    }
+
+    @Override
+    public DataType getResultType() {
+        return resultType;
+    }
+
+    @Override
+    public Expression[] initialValuesExpressions() {
+        return new Expression[] {/* sum = */ sumInitialValue(), /* count = */ 
literal(0L)};
+    }
+
+    @Override
+    public Expression[] accumulateExpressions() {
+        // cast the operand to sum needed type
+        Expression tryCastOperand = tryCast(operand(0), 
typeLiteral(getSumResultType()));
+        return new Expression[] {
+            /* sum = */ ifThenElse(isNull(tryCastOperand), sum, 
adjustedPlus(sum, tryCastOperand)),
+            /* count = */ ifThenElse(isNull(tryCastOperand), count, 
plus(count, literal(1L))),

Review Comment:
   `count` should always plus `1` no matter the casted value is null or not.



##########
flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/functions/hive/HiveAverageAggFunction.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * 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.functions.hive;
+
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.expressions.UnresolvedCallExpression;
+import org.apache.flink.table.expressions.UnresolvedReferenceExpression;
+import org.apache.flink.table.expressions.ValueLiteralExpression;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.inference.CallContext;
+
+import java.math.BigDecimal;
+
+import static 
org.apache.flink.table.expressions.ApiExpressionUtils.unresolvedRef;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.cast;
+import static org.apache.flink.table.planner.expressions.ExpressionBuilder.div;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.equalTo;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.hiveAggDecimalPlus;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.ifThenElse;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.isNull;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.literal;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.nullOf;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.plus;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.tryCast;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.typeLiteral;
+import static org.apache.flink.table.types.logical.DecimalType.MAX_PRECISION;
+import static org.apache.flink.table.types.logical.LogicalTypeRoot.DECIMAL;
+import static 
org.apache.flink.table.types.logical.utils.LogicalTypeChecks.getPrecision;
+import static 
org.apache.flink.table.types.logical.utils.LogicalTypeChecks.getScale;
+
+/** built-in hive average aggregate function. */
+public class HiveAverageAggFunction extends HiveDeclarativeAggregateFunction {
+
+    private final UnresolvedReferenceExpression sum = unresolvedRef("sum");
+    private final UnresolvedReferenceExpression count = unresolvedRef("count");
+    private DataType resultType;
+    private DataType sumResultType;
+
+    @Override
+    public int operandCount() {
+        return 1;
+    }
+
+    @Override
+    public UnresolvedReferenceExpression[] aggBufferAttributes() {
+        return new UnresolvedReferenceExpression[] {sum, count};
+    }
+
+    @Override
+    public DataType[] getAggBufferTypes() {
+        return new DataType[] {getSumResultType(), DataTypes.BIGINT()};
+    }
+
+    @Override
+    public DataType getResultType() {
+        return resultType;
+    }
+
+    @Override
+    public Expression[] initialValuesExpressions() {
+        return new Expression[] {/* sum = */ sumInitialValue(), /* count = */ 
literal(0L)};
+    }
+
+    @Override
+    public Expression[] accumulateExpressions() {
+        // cast the operand to sum needed type
+        Expression tryCastOperand = tryCast(operand(0), 
typeLiteral(getSumResultType()));

Review Comment:
   a question about it, is the it logic `tryCast` is same with 
`PrimitiveObjectInspectorUtils.getDouble` and 
`PrimitiveObjectInspectorUtils.getHiveDecimal`?



##########
flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveDialectQueryITCase.java:
##########
@@ -1062,6 +1062,63 @@ public void testSumAggWithGroupKey() throws Exception {
         tableEnv.executeSql("drop table test_sum_group");
     }
 
+    @Test
+    public void testAvgAggFunctionPlan() {
+        // test explain
+        String actualPlan = explainSql("select x, avg(y) from foo group by x");
+        
assertThat(actualPlan).isEqualTo(readFromResource("/explain/testAvgAggFunctionPlan.out"));
+    }
+
+    @Test
+    public void testAvgAggFunction() throws Exception {
+        tableEnv.executeSql(
+                "create table test_avg(a int, x string, y string, z int, f 
bigint, d decimal(20, 5), e double)");
+        tableEnv.executeSql(
+                        "insert into test_avg values (1, NULL, '2', 1, 2, 
2.22, 2.3), "
+                                + "(1, NULL, 'b', 2, NULL, 3.33, 3.4), "
+                                + "(2, NULL, '4', 1, 2, 4.55, 4.5), "
+                                + "(2, NULL, NULL, 4, 3, 5.66, 5.2)")
+                .await();
+
+        // test avg all element is null
+        List<Row> result =
+                CollectionUtil.iteratorToList(
+                        tableEnv.executeSql("select avg(x) from 
test_avg").collect());
+        assertThat(result.toString()).isEqualTo("[+I[null]]");
+
+        // test avg that some string elements can't convert to double
+        List<Row> result2 =
+                CollectionUtil.iteratorToList(
+                        tableEnv.executeSql("select avg(y) from 
test_avg").collect());
+        assertThat(result2.toString()).isEqualTo("[+I[3.0]]");
+
+        // test avg bigint with null element
+        List<Row> result3 =
+                CollectionUtil.iteratorToList(
+                        tableEnv.executeSql("select avg(f) from 
test_avg").collect());
+        assertThat(result3.toString()).isEqualTo("[+I[2.3333333333333335]]");
+
+        // test avg decimal
+        List<Row> result4 =
+                CollectionUtil.iteratorToList(
+                        tableEnv.executeSql("select avg(d) from 
test_avg").collect());
+        assertThat(result4.toString()).isEqualTo("[+I[3.940000000]]");

Review Comment:
   Hive's output is `3.9400000000000`. 
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org

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

Reply via email to