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


##########
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:
   They are not equivalent, the `PrimitiveObjectInspectorUtils.getDouble` will 
throw `NumberFormatException` when parse failed, but `tryCast` is equal with 
   ```
   try {
   
   } catch(NumberFortmatException e) {
   } 
   ```
   Hive avg udaf will catch the exception. Please see following code in hive 
`GenericUDAFAverage` for more detail
   ```
   try {
             // Skip the same value if avgDistinct is true
             if (isWindowingDistinct()) {
               ObjectInspectorObject obj = new ObjectInspectorObject(
                   ObjectInspectorUtils.copyToStandardObject(parameter, 
inputOI, ObjectInspectorCopyOption.JAVA),
                   copiedOI);
               if (averageAggregation.uniqueObjects.contains(obj)) {
                 return;
               }
               averageAggregation.uniqueObjects.add(obj);
             }
   
             doIterate(averageAggregation, inputOI, parameter);
           } catch (NumberFormatException e) {
             if (!warned) {
               warned = true;
               LOG.warn("Ignoring similar exceptions", e);
             }
           }
   ```



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