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


##########
flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/functions/hive/HiveMaxAggFunction.java:
##########
@@ -0,0 +1,124 @@
+/*
+ * 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.TableException;
+import org.apache.flink.table.expressions.Expression;
+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.LogicalType;
+import org.apache.flink.table.types.logical.LogicalTypeRoot;
+
+import static 
org.apache.flink.table.expressions.ApiExpressionUtils.unresolvedRef;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.greaterThan;
+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;
+
+/** built-in hive max aggregate function. */
+public class HiveMaxAggFunction extends HiveDeclarativeAggregateFunction {
+
+    private final UnresolvedReferenceExpression max = unresolvedRef("max");
+    private DataType resultType;
+
+    @Override
+    public int operandCount() {
+        return 1;
+    }
+
+    @Override
+    public UnresolvedReferenceExpression[] aggBufferAttributes() {
+        return new UnresolvedReferenceExpression[] {max};
+    }
+
+    @Override
+    public DataType[] getAggBufferTypes() {
+        return new DataType[] {getResultType()};
+    }
+
+    @Override
+    public DataType getResultType() {
+        return resultType;
+    }
+
+    @Override
+    public Expression[] initialValuesExpressions() {
+        return new Expression[] {/* max = */ nullOf(getResultType())};
+    }
+
+    @Override
+    public Expression[] accumulateExpressions() {
+        return new Expression[] {
+            /* max = */ ifThenElse(
+                    isNull(operand(0)),
+                    max,
+                    ifThenElse(
+                            isNull(max),
+                            operand(0),
+                            ifThenElse(greaterThan(operand(0), max), 
operand(0), max)))
+        };
+    }
+
+    @Override
+    public Expression[] retractExpressions() {
+        throw new TableException("Max aggregate function does not support 
retraction.");
+    }
+
+    @Override
+    public Expression[] mergeExpressions() {
+        return new Expression[] {
+            /* max = */ ifThenElse(
+                    isNull(mergeOperand(max)),
+                    max,
+                    ifThenElse(
+                            isNull(max),
+                            mergeOperand(max),
+                            ifThenElse(
+                                    greaterThan(mergeOperand(max), max), 
mergeOperand(max), max)))
+        };
+    }
+
+    @Override
+    public Expression getValueExpression() {
+        return max;
+    }
+
+    @Override
+    public void setArguments(CallContext callContext) {
+        if (resultType == null) {
+            // check argument type firstly

Review Comment:
   check the count of argument num?



##########
flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveDialectQueryITCase.java:
##########
@@ -1062,6 +1062,87 @@ public void testSumAggWithGroupKey() throws Exception {
         tableEnv.executeSql("drop table test_sum_group");
     }
 
+    @Test
+    public void testMaxAggFunctionPlan() {
+        // test explain
+        String actualPlan = explainSql("select x, max(y) from foo group by x");
+        
assertThat(actualPlan).isEqualTo(readFromResource("/explain/testMaxAggFunctionPlan.out"));
+    }
+
+    @Test
+    public void testMaxAggFunction() throws Exception {
+        tableEnv.executeSql(
+                "create table test_max(a int, b boolean, x string, y string, z 
int, d decimal(10,5), e float, f double, ts timestamp, dt date, bar binary)");
+        tableEnv.executeSql(
+                        "insert into test_max values (1, true, NULL, '2', 1, 
1.11, 1.2, 1.3, '2021-08-04 16:26:33.4','2021-08-04', 'data1'), "
+                                + "(1, false, NULL, 'b', 2, 2.22, 2.3, 2.4, 
'2021-08-06 16:26:33.4','2021-08-07', 'data2'), "
+                                + "(2, false, NULL, '4', 1, 3.33, 3.5, 3.6, 
'2021-08-08 16:26:33.4','2021-08-08', 'data3'), "
+                                + "(2, true, NULL, NULL, 4, 4.45, 4.7, 4.8, 
'2021-08-10 16:26:33.4','2021-08-01', 'data4')")
+                .await();
+
+        // test max with all elements are null
+        List<Row> result =
+                CollectionUtil.iteratorToList(
+                        tableEnv.executeSql("select max(x) from 
test_max").collect());
+        assertThat(result.toString()).isEqualTo("[+I[null]]");
+
+        // test max with some elements are null
+        List<Row> result2 =
+                CollectionUtil.iteratorToList(
+                        tableEnv.executeSql("select max(y) from 
test_max").collect());
+        assertThat(result2.toString()).isEqualTo("[+I[b]]");
+
+        // test max with some elements repeated
+        List<Row> result3 =
+                CollectionUtil.iteratorToList(
+                        tableEnv.executeSql("select max(z) from 
test_max").collect());
+        assertThat(result3.toString()).isEqualTo("[+I[4]]");
+
+        // test max with decimal type
+        List<Row> result4 =
+                CollectionUtil.iteratorToList(
+                        tableEnv.executeSql("select max(d) from 
test_max").collect());
+        assertThat(result4.toString()).isEqualTo("[+I[4.45000]]");
+
+        // test max with float type
+        List<Row> result5 =
+                CollectionUtil.iteratorToList(
+                        tableEnv.executeSql("select max(e) from 
test_max").collect());
+        assertThat(result5.toString()).isEqualTo("[+I[4.7]]");
+
+        // test max with double type
+        List<Row> result6 =
+                CollectionUtil.iteratorToList(
+                        tableEnv.executeSql("select max(f) from 
test_max").collect());
+        assertThat(result6.toString()).isEqualTo("[+I[4.8]]");
+
+        // test max with boolean type
+        List<Row> result7 =
+                CollectionUtil.iteratorToList(
+                        tableEnv.executeSql("select max(b) from 
test_max").collect());
+        assertThat(result7.toString()).isEqualTo("[+I[true]]");
+
+        // test max with timestamp type
+        List<Row> result8 =
+                CollectionUtil.iteratorToList(
+                        tableEnv.executeSql("select max(ts) from 
test_max").collect());
+        
assertThat(result8.toString()).isEqualTo("[+I[2021-08-10T16:26:33.400]]");

Review Comment:
   Please verify the assertion in Hive3 as there's some difference in timestamp 
type between Hive2 and Hive3.
   You can verify it in your local.



##########
flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/functions/hive/HiveMaxAggFunction.java:
##########
@@ -0,0 +1,124 @@
+/*
+ * 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.TableException;
+import org.apache.flink.table.expressions.Expression;
+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.LogicalType;
+import org.apache.flink.table.types.logical.LogicalTypeRoot;
+
+import static 
org.apache.flink.table.expressions.ApiExpressionUtils.unresolvedRef;
+import static 
org.apache.flink.table.planner.expressions.ExpressionBuilder.greaterThan;
+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;
+
+/** built-in hive max aggregate function. */
+public class HiveMaxAggFunction extends HiveDeclarativeAggregateFunction {
+
+    private final UnresolvedReferenceExpression max = unresolvedRef("max");
+    private DataType resultType;
+
+    @Override
+    public int operandCount() {
+        return 1;
+    }
+
+    @Override
+    public UnresolvedReferenceExpression[] aggBufferAttributes() {
+        return new UnresolvedReferenceExpression[] {max};
+    }
+
+    @Override
+    public DataType[] getAggBufferTypes() {
+        return new DataType[] {getResultType()};
+    }
+
+    @Override
+    public DataType getResultType() {
+        return resultType;
+    }
+
+    @Override
+    public Expression[] initialValuesExpressions() {
+        return new Expression[] {/* max = */ nullOf(getResultType())};
+    }
+
+    @Override
+    public Expression[] accumulateExpressions() {
+        return new Expression[] {
+            /* max = */ ifThenElse(
+                    isNull(operand(0)),
+                    max,
+                    ifThenElse(
+                            isNull(max),
+                            operand(0),
+                            ifThenElse(greaterThan(operand(0), max), 
operand(0), max)))
+        };
+    }
+
+    @Override
+    public Expression[] retractExpressions() {
+        throw new TableException("Max aggregate function does not support 
retraction.");
+    }
+
+    @Override
+    public Expression[] mergeExpressions() {
+        return new Expression[] {
+            /* max = */ ifThenElse(
+                    isNull(mergeOperand(max)),
+                    max,
+                    ifThenElse(
+                            isNull(max),
+                            mergeOperand(max),
+                            ifThenElse(
+                                    greaterThan(mergeOperand(max), max), 
mergeOperand(max), max)))
+        };
+    }
+
+    @Override
+    public Expression getValueExpression() {
+        return max;
+    }
+
+    @Override
+    public void setArguments(CallContext callContext) {
+        if (resultType == null) {
+            // check argument type firstly
+            
checkArgumentType(callContext.getArgumentDataTypes().get(0).getLogicalType());
+            resultType = callContext.getArgumentDataTypes().get(0);
+        }
+    }
+
+    private void checkArgumentType(LogicalType logicalType) {
+        // Flink doesn't support to compare nested type now, so here can't 
support it, see
+        // ScalarOperatorGens#generateComparison for more detail
+        if (logicalType.is(LogicalTypeRoot.ARRAY)
+                || logicalType.is(LogicalTypeRoot.MAP)
+                || logicalType.is(LogicalTypeRoot.ROW)) {
+            throw new TableException(
+                    String.format(
+                            "Hive native max aggregate function does not 
support type: '%s' now. Please re-check the data type.",

Review Comment:
   May need to remind user to fall back to hive's implementation.



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

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to