coderfender commented on code in PR #2600:
URL: https://github.com/apache/datafusion-comet/pull/2600#discussion_r2640708288


##########
spark/src/test/scala/org/apache/comet/exec/CometAggregateSuite.scala:
##########
@@ -1490,11 +1501,23 @@ class CometAggregateSuite extends CometTestBase with 
AdaptiveSparkPlanHelper {
     }
   }
 
+  test("ANSI support for try_sum - null test") {
+    Seq(true, false).foreach { ansiEnabled =>
+      withSQLConf(SQLConf.ANSI_ENABLED.key -> ansiEnabled.toString) {
+        withParquetTable(
+          Seq((null.asInstanceOf[java.lang.Long], "a"), 
(null.asInstanceOf[java.lang.Long], "b")),
+          "null_tbl") {
+          val res = sql("SELECT try_sum(_1) FROM null_tbl")
+          checkSparkAnswerAndOperator(res)
+          assert(res.collect() === Array(Row(null)))

Review Comment:
   @comphead  , Yeah , I found some inconsistencies while writing these tests 
as (partially) discussed here  : 
https://github.com/apache/datafusion-comet/issues/2692 . Also ,I wanted to be 
completely sure in terms of output given the core functionality of this PR 



##########
spark/src/test/scala/org/apache/comet/exec/CometAggregateSuite.scala:
##########
@@ -1508,11 +1531,28 @@ class CometAggregateSuite extends CometTestBase with 
AdaptiveSparkPlanHelper {
     }
   }
 
+  test("ANSI support for sum - null test (group by)") {
+    Seq(true, false).foreach { ansiEnabled =>
+      withSQLConf(SQLConf.ANSI_ENABLED.key -> ansiEnabled.toString) {
+        withParquetTable(
+          Seq(
+            (null.asInstanceOf[java.lang.Long], "a"),
+            (null.asInstanceOf[java.lang.Long], "a"),
+            (null.asInstanceOf[java.lang.Long], "b"),
+            (null.asInstanceOf[java.lang.Long], "b"),
+            (null.asInstanceOf[java.lang.Long], "b")),
+          "tbl") {
+          val res = sql("SELECT _2, sum(_1) FROM tbl group by 1")
+          checkSparkAnswerAndOperator(res)
+          assert(res.orderBy(col("_2")).collect() === Array(Row("a", null), 
Row("b", null)))

Review Comment:
   replied above 



##########
spark/src/test/scala/org/apache/comet/exec/CometAggregateSuite.scala:
##########
@@ -1602,35 +1768,68 @@ class CometAggregateSuite extends CometTestBase with 
AdaptiveSparkPlanHelper {
     }
   }
 
+  test("try_sum overflow - with GROUP BY") {
+    // Test Long overflow with GROUP BY - some groups overflow while some don't
+    withParquetTable(Seq((Long.MaxValue, 1), (100L, 1), (200L, 2), (300L, 2)), 
"tbl") {
+      val res = sql("SELECT _2, try_sum(_1) FROM tbl GROUP BY 
_2").repartition(2, col("_2"))

Review Comment:
   Sure let me add a comment here 



##########
spark/src/test/scala/org/apache/comet/exec/CometAggregateSuite.scala:
##########
@@ -1555,11 +1612,63 @@ class CometAggregateSuite extends CometTestBase with 
AdaptiveSparkPlanHelper {
     (1 to 50).flatMap(_ => Seq((maxDec38_0, 1)))
   }
 
+  test("ANSI support - SUM function") {
+    Seq(true, false).foreach { ansiEnabled =>
+      withSQLConf(SQLConf.ANSI_ENABLED.key -> ansiEnabled.toString) {
+        // Test long overflow
+        withParquetTable(Seq((Long.MaxValue, 1L), (100L, 1L)), "tbl") {
+          val res = sql("SELECT SUM(_1) FROM tbl")
+          if (ansiEnabled) {
+            checkSparkAnswerMaybeThrows(res) match {
+              case (Some(sparkExc), Some(cometExc)) =>
+                assert(sparkExc.getMessage.contains("ARITHMETIC_OVERFLOW"))

Review Comment:
   Sure . I thought that the `ansiEnabled` should give away why the overflow 
message is expected but let me add a comment to make things explicit 



##########
spark/src/main/scala/org/apache/comet/serde/aggregates.scala:
##########
@@ -213,17 +213,6 @@ object CometAverage extends 
CometAggregateExpressionSerde[Average] {
 
 object CometSum extends CometAggregateExpressionSerde[Sum] {
 
-  override def getSupportLevel(sum: Sum): SupportLevel = {
-    sum.evalMode match {
-      case EvalMode.ANSI if !sum.dataType.isInstanceOf[DecimalType] =>
-        Incompatible(Some("ANSI mode for non decimal inputs is not supported"))

Review Comment:
   @comphead  thank you . I generally follow up the main PR with a separate doc 
PR and I hope that is okay to do the same thing with this PR too ? 



##########
spark/src/test/scala/org/apache/comet/exec/CometAggregateSuite.scala:
##########
@@ -1529,11 +1569,28 @@ class CometAggregateSuite extends CometTestBase with 
AdaptiveSparkPlanHelper {
     }
   }
 
+  test("ANSI support for try_sum - null test (group by)") {
+    Seq(true, false).foreach { ansiEnabled =>
+      withSQLConf(SQLConf.ANSI_ENABLED.key -> ansiEnabled.toString) {
+        withParquetTable(
+          Seq(
+            (null.asInstanceOf[java.lang.Long], "a"),
+            (null.asInstanceOf[java.lang.Long], "a"),
+            (null.asInstanceOf[java.lang.Long], "b"),
+            (null.asInstanceOf[java.lang.Long], "b"),
+            (null.asInstanceOf[java.lang.Long], "b")),
+          "tbl") {
+          val res = sql("SELECT _2, try_sum(_1) FROM tbl group by 1")
+          checkSparkAnswerAndOperator(res)
+          assert(res.orderBy(col("_2")).collect() === Array(Row("a", null), 
Row("b", null)))

Review Comment:
   replied above 



##########
native/spark-expr/src/agg_funcs/sum_int.rs:
##########
@@ -0,0 +1,589 @@
+// 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.
+
+use crate::{arithmetic_overflow_error, EvalMode};
+use arrow::array::{
+    as_primitive_array, cast::AsArray, Array, ArrayRef, ArrowNativeTypeOp, 
ArrowPrimitiveType,
+    BooleanArray, Int64Array, PrimitiveArray,
+};
+use arrow::datatypes::{
+    ArrowNativeType, DataType, Field, FieldRef, Int16Type, Int32Type, 
Int64Type, Int8Type,
+};
+use datafusion::common::{DataFusionError, Result as DFResult, ScalarValue};
+use datafusion::logical_expr::function::{AccumulatorArgs, StateFieldsArgs};
+use datafusion::logical_expr::Volatility::Immutable;
+use datafusion::logical_expr::{
+    Accumulator, AggregateUDFImpl, EmitTo, GroupsAccumulator, ReversedUDAF, 
Signature,
+};
+use std::{any::Any, sync::Arc};
+
+#[derive(Debug, PartialEq, Eq, Hash)]
+pub struct SumInteger {
+    signature: Signature,
+    eval_mode: EvalMode,
+}
+
+impl SumInteger {
+    pub fn try_new(data_type: DataType, eval_mode: EvalMode) -> DFResult<Self> 
{
+        match data_type {
+            DataType::Int8 | DataType::Int16 | DataType::Int32 | 
DataType::Int64 => Ok(Self {
+                signature: Signature::user_defined(Immutable),
+                eval_mode,
+            }),
+            _ => Err(DataFusionError::Internal(
+                "Invalid data type for SumInteger".into(),
+            )),
+        }
+    }
+}
+
+impl AggregateUDFImpl for SumInteger {
+    fn as_any(&self) -> &dyn Any {
+        self
+    }
+
+    fn name(&self) -> &str {

Review Comment:
   It would be the same :)  . We even have the same name for the decimal sum 
(which is a custom implementation)  as well . 



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


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to