lsyldliu commented on code in PR #21401: URL: https://github.com/apache/flink/pull/21401#discussion_r1057025152
########## flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/functions/hive/HiveSumAggFunction.java: ########## @@ -0,0 +1,145 @@ +/* + * 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.types.DataType; +import org.apache.flink.table.types.inference.CallContext; +import org.apache.flink.table.types.logical.DecimalType; +import org.apache.flink.table.types.logical.utils.LogicalTypeMerging; + +import static org.apache.flink.table.expressions.ApiExpressionUtils.unresolvedRef; +import static org.apache.flink.table.planner.expressions.ExpressionBuilder.aggDecimalPlus; +import static org.apache.flink.table.planner.expressions.ExpressionBuilder.cast; +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.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; + +/** built-in hive sum aggregate function. */ +public class HiveSumAggFunction extends HiveDeclarativeAggregateFunction { + + private final UnresolvedReferenceExpression sum = unresolvedRef("sum"); + private DataType resultType; + + @Override + public int operandCount() { + return 1; + } + + @Override + public UnresolvedReferenceExpression[] aggBufferAttributes() { + return new UnresolvedReferenceExpression[] {sum}; + } + + @Override + public DataType[] getAggBufferTypes() { + return new DataType[] {getResultType()}; + } + + @Override + public DataType getResultType() { + return resultType; + } + + @Override + public Expression[] initialValuesExpressions() { + return new Expression[] {/* sum = */ nullOf(getResultType())}; + } + + @Override + public Expression[] accumulateExpressions() { + return new Expression[] { + /* sum = */ ifThenElse( + isNull(operand(0)), + sum, + ifThenElse( + isNull(sum), + tryCast(operand(0), typeLiteral(getResultType())), + adjustedPlus(sum, tryCast(operand(0), typeLiteral(getResultType()))))) + }; + } + + @Override + public Expression[] retractExpressions() { + throw new TableException("Sum aggregate function does not support retraction."); + } + + @Override + public Expression[] mergeExpressions() { + return new Expression[] { + /* sum = */ ifThenElse( + isNull(mergeOperand(sum)), + sum, + ifThenElse( + isNull(sum), mergeOperand(sum), adjustedPlus(sum, mergeOperand(sum)))) + }; + } + + @Override + public Expression getValueExpression() { + return sum; + } + + @Override + public void setArguments(CallContext callContext) { + if (resultType == null) { + resultType = initResultType(callContext.getArgumentDataTypes().get(0)); + } + } + + private DataType initResultType(DataType argsType) { + switch (argsType.getLogicalType().getTypeRoot()) { + case TINYINT: + case SMALLINT: + case INTEGER: + case BIGINT: + return DataTypes.BIGINT(); + case TIMESTAMP_WITHOUT_TIME_ZONE: + case FLOAT: + case DOUBLE: + case CHAR: + case VARCHAR: + return DataTypes.DOUBLE(); + case DECIMAL: + DecimalType sumType = + (DecimalType) LogicalTypeMerging.findSumAggType(argsType.getLogicalType()); Review Comment: You are right, I have followed the hive/spark logic. -- 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