You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2022/12/27 04:46:36 UTC

[GitHub] [flink] beyond1920 commented on a diff in pull request #21401: [FLINK-29718][table] Supports hive sum function by native implementation

beyond1920 commented on code in PR #21401:
URL: https://github.com/apache/flink/pull/21401#discussion_r1057434644


##########
flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/functions/hive/HiveSumAggFunction.java:
##########
@@ -0,0 +1,167 @@
+/*
+ * 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 static org.apache.flink.table.expressions.ApiExpressionUtils.unresolvedRef;
+import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.UNIX_TIMESTAMP;
+import static org.apache.flink.table.planner.expressions.ExpressionBuilder.call;
+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.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.LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE;
+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 sum aggregate function. */
+public class HiveSumAggFunction extends HiveDeclarativeAggregateFunction {
+
+    private final UnresolvedReferenceExpression sum = unresolvedRef("sum");
+    private DataType argsType;
+    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() {
+        Expression operand;
+        // TimestampToNumericCastRule can't cast timestamp to numeric directly, so here use
+        // UNIX_TIMESTAMP(CAST(timestamp_col AS STRING)) instead
+        if (argsType.getLogicalType().is(TIMESTAMP_WITHOUT_TIME_ZONE)) {
+            operand = castTimestampToLong(operand(0));
+        } else {
+            operand = operand(0);
+        }
+        Expression tryCastOperand = tryCast(operand, typeLiteral(getResultType()));
+        return new Expression[] {
+            /* sum = */ ifThenElse(
+                    isNull(operand(0)),
+                    sum,
+                    ifThenElse(
+                            isNull(tryCastOperand),
+                            sum,
+                            ifThenElse(
+                                    isNull(sum),
+                                    tryCastOperand,
+                                    adjustedPlus(sum, tryCastOperand))))
+        };
+    }
+
+    @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) {
+            argsType = callContext.getArgumentDataTypes().get(0);
+            resultType = initResultType(argsType);
+        }
+    }
+
+    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:
+                int precision =
+                        Math.min(MAX_PRECISION, getPrecision(argsType.getLogicalType()) + 10);

Review Comment:
   Could you please explain why we need add '10' here?



##########
flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/functions/hive/HiveSumAggFunction.java:
##########
@@ -0,0 +1,167 @@
+/*
+ * 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 static org.apache.flink.table.expressions.ApiExpressionUtils.unresolvedRef;
+import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.UNIX_TIMESTAMP;
+import static org.apache.flink.table.planner.expressions.ExpressionBuilder.call;
+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.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.LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE;
+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 sum aggregate function. */
+public class HiveSumAggFunction extends HiveDeclarativeAggregateFunction {
+
+    private final UnresolvedReferenceExpression sum = unresolvedRef("sum");
+    private DataType argsType;
+    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() {
+        Expression operand;
+        // TimestampToNumericCastRule can't cast timestamp to numeric directly, so here use
+        // UNIX_TIMESTAMP(CAST(timestamp_col AS STRING)) instead
+        if (argsType.getLogicalType().is(TIMESTAMP_WITHOUT_TIME_ZONE)) {
+            operand = castTimestampToLong(operand(0));
+        } else {
+            operand = operand(0);
+        }
+        Expression tryCastOperand = tryCast(operand, typeLiteral(getResultType()));
+        return new Expression[] {
+            /* sum = */ ifThenElse(
+                    isNull(operand(0)),
+                    sum,
+                    ifThenElse(
+                            isNull(tryCastOperand),
+                            sum,
+                            ifThenElse(
+                                    isNull(sum),
+                                    tryCastOperand,
+                                    adjustedPlus(sum, tryCastOperand))))
+        };
+    }
+
+    @Override
+    public Expression[] retractExpressions() {

Review Comment:
   Why sum could not support retraction?



##########
flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/functions/hive/HiveSumAggFunction.java:
##########
@@ -0,0 +1,167 @@
+/*
+ * 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 static org.apache.flink.table.expressions.ApiExpressionUtils.unresolvedRef;
+import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.UNIX_TIMESTAMP;
+import static org.apache.flink.table.planner.expressions.ExpressionBuilder.call;
+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.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.LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE;
+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 sum aggregate function. */
+public class HiveSumAggFunction extends HiveDeclarativeAggregateFunction {
+
+    private final UnresolvedReferenceExpression sum = unresolvedRef("sum");
+    private DataType argsType;
+    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() {
+        Expression operand;
+        // TimestampToNumericCastRule can't cast timestamp to numeric directly, so here use
+        // UNIX_TIMESTAMP(CAST(timestamp_col AS STRING)) instead
+        if (argsType.getLogicalType().is(TIMESTAMP_WITHOUT_TIME_ZONE)) {

Review Comment:
   Does hive sum udaf even support input parameter of `TIMESTAMP_WITHOUT_TIME_ZONE` type?



-- 
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-unsubscribe@flink.apache.org

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