You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@ignite.apache.org by GitBox <gi...@apache.org> on 2021/06/03 12:01:47 UTC

[GitHub] [ignite] tledkov-gridgain opened a new pull request #9148: IGNITE-14681 Calcite engine. Extend return type of sum() aggregate function

tledkov-gridgain opened a new pull request #9148:
URL: https://github.com/apache/ignite/pull/9148


   Thank you for submitting the pull request to the Apache Ignite.
   
   In order to streamline the review of the contribution 
   we ask you to ensure the following steps have been taken:
   
   ### The Contribution Checklist
   - [ ] There is a single JIRA ticket related to the pull request. 
   - [ ] The web-link to the pull request is attached to the JIRA ticket.
   - [ ] The JIRA ticket has the _Patch Available_ state.
   - [ ] The pull request body describes changes that have been made. 
   The description explains _WHAT_ and _WHY_ was made instead of _HOW_.
   - [ ] The pull request title is treated as the final commit message. 
   The following pattern must be used: `IGNITE-XXXX Change summary` where `XXXX` - number of JIRA issue.
   - [ ] A reviewer has been mentioned through the JIRA comments 
   (see [the Maintainers list](https://cwiki.apache.org/confluence/display/IGNITE/How+to+Contribute#HowtoContribute-ReviewProcessandMaintainers)) 
   - [ ] The pull request has been checked by the Teamcity Bot and 
   the `green visa` attached to the JIRA ticket (see [TC.Bot: Check PR](https://mtcga.gridgain.com/prs.html))
   
   ### Notes
   - [How to Contribute](https://cwiki.apache.org/confluence/display/IGNITE/How+to+Contribute)
   - [Coding abbreviation rules](https://cwiki.apache.org/confluence/display/IGNITE/Abbreviation+Rules)
   - [Coding Guidelines](https://cwiki.apache.org/confluence/display/IGNITE/Coding+Guidelines)
   - [Apache Ignite Teamcity Bot](https://cwiki.apache.org/confluence/display/IGNITE/Apache+Ignite+Teamcity+Bot)
   
   If you need any help, please email dev@ignite.apache.org or ask anу advice on http://asf.slack.com _#ignite_ channel.
   


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

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



[GitHub] [ignite] zstan commented on a change in pull request #9148: IGNITE-14681 Calcite engine. Extend return type of sum() aggregate function

Posted by GitBox <gi...@apache.org>.
zstan commented on a change in pull request #9148:
URL: https://github.com/apache/ignite/pull/9148#discussion_r702701146



##########
File path: modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/BaseAggregateTest.java
##########
@@ -512,6 +513,98 @@ public void sumOnDifferentRowsCount() throws IgniteCheckedException {
         }
     }
 
+    /** */
+    @Test
+    public void sumIntegerOverflow() {
+        ExecutionContext<Object[]> ctx = executionContext(F.first(nodes()), UUID.randomUUID(), 0);
+        IgniteTypeFactory tf = ctx.getTypeFactory();
+        RelDataType rowType = TypeUtils.createRowType(tf, int.class, int.class);
+        ScanNode<Object[]> scan = new ScanNode<>(ctx, rowType, Arrays.asList(
+            row(0, Integer.MAX_VALUE / 2),
+            row(0, Integer.MAX_VALUE / 2 + 11)
+        ));
+
+        AggregateCall call = AggregateCall.create(
+            SqlStdOperatorTable.SUM,
+            false,
+            false,
+            false,
+            ImmutableIntList.of(1),
+            -1,
+            RelCollations.EMPTY,
+            tf.createJavaType(BigDecimal.class),
+            null);
+
+        ImmutableList<ImmutableBitSet> grpSets = ImmutableList.of(ImmutableBitSet.of(0));
+
+        RelDataType aggRowType = TypeUtils.createRowType(tf, int.class);
+
+        SingleNode<Object[]> aggChain = createAggregateNodesChain(
+            ctx,
+            grpSets,
+            call,
+            rowType,
+            aggRowType,
+            rowFactory(),
+            scan
+        );
+
+        RootNode<Object[]> root = new RootNode<>(ctx, aggRowType);
+        root.register(aggChain);
+
+        assertTrue(root.hasNext());
+
+        Assert.assertArrayEquals(row(0, new BigDecimal(Integer.MAX_VALUE).add(new BigDecimal(10))), root.next());

Review comment:
       it`s a bit confused why 
   
   >             row(0, Integer.MAX_VALUE / 2),
   and
   >            row(0, Integer.MAX_VALUE / 2 + 11)
   gives 
   > new BigDecimal(Integer.MAX_VALUE).add(new BigDecimal(10) ))
   may be more clear to rewrite test with  Integer.MIN_VALUE ?  i.e. :
               > row(0, Integer.MIN_VALUE / 2),
               > row(0, Integer.MIN_VALUE / 2 - 11)
   
   > Assert.assertArrayEquals(row(0, new BigDecimal(Integer.MIN_VALUE).add(new BigDecimal(-11))), root.next());
   but it`s up to you of course.




-- 
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: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] alex-plekhanov commented on a change in pull request #9148: IGNITE-14681 Calcite engine. Extend return type of sum() aggregate function

Posted by GitBox <gi...@apache.org>.
alex-plekhanov commented on a change in pull request #9148:
URL: https://github.com/apache/ignite/pull/9148#discussion_r702695844



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/agg/Accumulators.java
##########
@@ -93,32 +92,38 @@
     /** */
     private static Supplier<Accumulator> sumFactory(AggregateCall call) {
         switch (call.type.getSqlTypeName()) {
+            case INTEGER:
+            case BIGINT:
+            case DECIMAL:
+                return () -> new Sum(new DecimalSumEmptyIsZero());
+
             case DOUBLE:
             case REAL:
             case FLOAT:
-                return DoubleSum.FACTORY;
-            case DECIMAL:
-                return DecimalSum.FACTORY;
-            case INTEGER:
-                return IntSum.FACTORY;
-            case BIGINT:
+                return () -> new Sum(new DoubleSumEmptyIsZero());
+
+            case TINYINT:
+            case SMALLINT:
             default:
-                return LongSum.FACTORY;
+                return () -> new Sum(new LongSumEmptyIsZero());
         }
     }
 
     /** */
     private static Supplier<Accumulator> sumEmptyIsZeroFactory(AggregateCall call) {
         switch (call.type.getSqlTypeName()) {
+            case INTEGER:

Review comment:
       Do we really need DECIMAL for INTEGER sums? Perhaps BIGINT will be enough (postgree approach). 

##########
File path: modules/calcite/src/test/java/org/apache/ignite/testsuites/ScriptTestSuite.java
##########
@@ -36,6 +35,5 @@
  */
 @RunWith(ScriptTestRunner.class)
 @ScriptRunnerTestsEnvironment(scriptsRoot = "src/test/sql")
-@Ignore

Review comment:
       Unignoring this suite will cause Travis checks to fail, let's keep it ignored for a while, it doesn't prevent to run this suite manually.

##########
File path: modules/calcite/src/test/sql/aggregate/aggregates/test_aggr_string.test
##########
@@ -2,12 +2,6 @@
 # description: Test aggregations on strings
 # group: [aggregates]
 
-query TTTTI

Review comment:
       Why this test was removed? Looks like it works 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: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] zstan commented on a change in pull request #9148: IGNITE-14681 Calcite engine. Extend return type of sum() aggregate function

Posted by GitBox <gi...@apache.org>.
zstan commented on a change in pull request #9148:
URL: https://github.com/apache/ignite/pull/9148#discussion_r702707565



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/patch/AggregateExpandDistinctAggregatesRule.java
##########
@@ -0,0 +1,957 @@
+/*
+ * 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.ignite.internal.processors.query.calcite.rule.patch;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.NavigableSet;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
+import org.apache.calcite.plan.Contexts;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelRule;
+import org.apache.calcite.rel.RelCollations;
+import org.apache.calcite.rel.core.Aggregate;
+import org.apache.calcite.rel.core.Aggregate.Group;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.core.RelFactories;
+import org.apache.calcite.rel.logical.LogicalAggregate;
+import org.apache.calcite.rel.rules.TransformationRule;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.SqlAggFunction;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.fun.SqlSumEmptyIsZeroAggFunction;
+import org.apache.calcite.tools.RelBuilder;
+import org.apache.calcite.tools.RelBuilderFactory;
+import org.apache.calcite.util.ImmutableBeans;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.ImmutableIntList;
+import org.apache.calcite.util.Optionality;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Util;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+import static java.util.Objects.requireNonNull;
+
+/**
+ * Must be replaced by @link{org.apache.calcite.rules.AggregateExpandDistinctAggregatesRule} after upgrade the calcite

Review comment:
       we need appropriate calcite ticket here, isn`t it ?




-- 
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: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] alex-plekhanov commented on pull request #9148: IGNITE-14681 Calcite engine. Extend return type of sum() aggregate function

Posted by GitBox <gi...@apache.org>.
alex-plekhanov commented on pull request #9148:
URL: https://github.com/apache/ignite/pull/9148#issuecomment-931642837


   There are some check-style violations, please fix it.


-- 
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: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] tledkov-gridgain commented on a change in pull request #9148: IGNITE-14681 Calcite engine. Extend return type of sum() aggregate function

Posted by GitBox <gi...@apache.org>.
tledkov-gridgain commented on a change in pull request #9148:
URL: https://github.com/apache/ignite/pull/9148#discussion_r720097531



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/type/IgniteTypeSystem.java
##########
@@ -38,4 +40,36 @@
     @Override public int getMaxNumericPrecision() {
         return Short.MAX_VALUE;
     }
+
+    /** {@inheritDoc} */
+    @Override public RelDataType deriveSumType(RelDataTypeFactory typeFactory, RelDataType argumentType) {
+        RelDataType sumType;
+
+        switch (argumentType.getSqlTypeName()) {
+            case INTEGER:
+            case TINYINT:
+            case SMALLINT:
+                sumType = typeFactory.createJavaType(Long.class);

Review comment:
       Fixed by add separate branch for the `BasicSqlType`




-- 
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: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] tledkov-gridgain commented on a change in pull request #9148: IGNITE-14681 Calcite engine. Extend return type of sum() aggregate function

Posted by GitBox <gi...@apache.org>.
tledkov-gridgain commented on a change in pull request #9148:
URL: https://github.com/apache/ignite/pull/9148#discussion_r702955872



##########
File path: modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/BaseAggregateTest.java
##########
@@ -512,6 +513,98 @@ public void sumOnDifferentRowsCount() throws IgniteCheckedException {
         }
     }
 
+    /** */
+    @Test
+    public void sumIntegerOverflow() {
+        ExecutionContext<Object[]> ctx = executionContext(F.first(nodes()), UUID.randomUUID(), 0);
+        IgniteTypeFactory tf = ctx.getTypeFactory();
+        RelDataType rowType = TypeUtils.createRowType(tf, int.class, int.class);
+        ScanNode<Object[]> scan = new ScanNode<>(ctx, rowType, Arrays.asList(
+            row(0, Integer.MAX_VALUE / 2),
+            row(0, Integer.MAX_VALUE / 2 + 11)
+        ));
+
+        AggregateCall call = AggregateCall.create(
+            SqlStdOperatorTable.SUM,
+            false,
+            false,
+            false,
+            ImmutableIntList.of(1),
+            -1,
+            RelCollations.EMPTY,
+            tf.createJavaType(BigDecimal.class),
+            null);
+
+        ImmutableList<ImmutableBitSet> grpSets = ImmutableList.of(ImmutableBitSet.of(0));
+
+        RelDataType aggRowType = TypeUtils.createRowType(tf, int.class);
+
+        SingleNode<Object[]> aggChain = createAggregateNodesChain(
+            ctx,
+            grpSets,
+            call,
+            rowType,
+            aggRowType,
+            rowFactory(),
+            scan
+        );
+
+        RootNode<Object[]> root = new RootNode<>(ctx, aggRowType);
+        root.register(aggChain);
+
+        assertTrue(root.hasNext());
+
+        Assert.assertArrayEquals(row(0, new BigDecimal(Integer.MAX_VALUE).add(new BigDecimal(10))), root.next());

Review comment:
       Fixed with plain sum of values. Also changed the type of the sum of INTEGER from DECIMAL to BIGINT (Long)




-- 
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: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] alex-plekhanov commented on a change in pull request #9148: IGNITE-14681 Calcite engine. Extend return type of sum() aggregate function

Posted by GitBox <gi...@apache.org>.
alex-plekhanov commented on a change in pull request #9148:
URL: https://github.com/apache/ignite/pull/9148#discussion_r719738565



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/type/IgniteTypeSystem.java
##########
@@ -38,4 +40,36 @@
     @Override public int getMaxNumericPrecision() {
         return Short.MAX_VALUE;
     }
+
+    /** {@inheritDoc} */
+    @Override public RelDataType deriveSumType(RelDataTypeFactory typeFactory, RelDataType argumentType) {
+        RelDataType sumType;
+
+        switch (argumentType.getSqlTypeName()) {
+            case INTEGER:
+            case TINYINT:
+            case SMALLINT:
+                sumType = typeFactory.createJavaType(Long.class);

Review comment:
       Is there any reason to create java types but not SQL types? The overridden method creates SQL types.




-- 
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: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] zstan commented on a change in pull request #9148: IGNITE-14681 Calcite engine. Extend return type of sum() aggregate function

Posted by GitBox <gi...@apache.org>.
zstan commented on a change in pull request #9148:
URL: https://github.com/apache/ignite/pull/9148#discussion_r702704382



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/rule/patch/AggregateExpandDistinctAggregatesRule.java
##########
@@ -0,0 +1,957 @@
+/*
+ * 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.ignite.internal.processors.query.calcite.rule.patch;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.NavigableSet;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
+import org.apache.calcite.plan.Contexts;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelRule;
+import org.apache.calcite.rel.RelCollations;
+import org.apache.calcite.rel.core.Aggregate;
+import org.apache.calcite.rel.core.Aggregate.Group;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.core.RelFactories;
+import org.apache.calcite.rel.logical.LogicalAggregate;
+import org.apache.calcite.rel.rules.TransformationRule;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.SqlAggFunction;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.fun.SqlSumEmptyIsZeroAggFunction;
+import org.apache.calcite.tools.RelBuilder;
+import org.apache.calcite.tools.RelBuilderFactory;
+import org.apache.calcite.util.ImmutableBeans;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.ImmutableIntList;
+import org.apache.calcite.util.Optionality;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Util;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+import static java.util.Objects.requireNonNull;
+
+/**
+ * Must be replaced by @link{org.apache.calcite.rules.AggregateExpandDistinctAggregatesRule} after upgrade the calcite
+ * version.
+ *
+ * TODO: https://issues.apache.org/jira/browse/IGNITE-15426
+ */
+public final class AggregateExpandDistinctAggregatesRule
+    extends RelRule<AggregateExpandDistinctAggregatesRule.Config> implements TransformationRule {
+    /** Creates an AggregateExpandDistinctAggregatesRule. */
+    AggregateExpandDistinctAggregatesRule(Config config) {
+        super(config);
+    }
+
+    @Deprecated public AggregateExpandDistinctAggregatesRule(

Review comment:
       do we need all these Deprecated stuff ?




-- 
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: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] zstan commented on a change in pull request #9148: IGNITE-14681 Calcite engine. Extend return type of sum() aggregate function

Posted by GitBox <gi...@apache.org>.
zstan commented on a change in pull request #9148:
URL: https://github.com/apache/ignite/pull/9148#discussion_r702701146



##########
File path: modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/BaseAggregateTest.java
##########
@@ -512,6 +513,98 @@ public void sumOnDifferentRowsCount() throws IgniteCheckedException {
         }
     }
 
+    /** */
+    @Test
+    public void sumIntegerOverflow() {
+        ExecutionContext<Object[]> ctx = executionContext(F.first(nodes()), UUID.randomUUID(), 0);
+        IgniteTypeFactory tf = ctx.getTypeFactory();
+        RelDataType rowType = TypeUtils.createRowType(tf, int.class, int.class);
+        ScanNode<Object[]> scan = new ScanNode<>(ctx, rowType, Arrays.asList(
+            row(0, Integer.MAX_VALUE / 2),
+            row(0, Integer.MAX_VALUE / 2 + 11)
+        ));
+
+        AggregateCall call = AggregateCall.create(
+            SqlStdOperatorTable.SUM,
+            false,
+            false,
+            false,
+            ImmutableIntList.of(1),
+            -1,
+            RelCollations.EMPTY,
+            tf.createJavaType(BigDecimal.class),
+            null);
+
+        ImmutableList<ImmutableBitSet> grpSets = ImmutableList.of(ImmutableBitSet.of(0));
+
+        RelDataType aggRowType = TypeUtils.createRowType(tf, int.class);
+
+        SingleNode<Object[]> aggChain = createAggregateNodesChain(
+            ctx,
+            grpSets,
+            call,
+            rowType,
+            aggRowType,
+            rowFactory(),
+            scan
+        );
+
+        RootNode<Object[]> root = new RootNode<>(ctx, aggRowType);
+        root.register(aggChain);
+
+        assertTrue(root.hasNext());
+
+        Assert.assertArrayEquals(row(0, new BigDecimal(Integer.MAX_VALUE).add(new BigDecimal(10))), root.next());

Review comment:
       it`s a bit confused why 
   
   >             row(0, Integer.MAX_VALUE / 2),
   and
   >            row(0, Integer.MAX_VALUE / 2 + 11)
   gives 
   > new BigDecimal(Integer.MAX_VALUE).add(new BigDecimal(10) ))
   
   may be more clear to rewrite test with  Integer.MIN_VALUE ?  i.e. :
   > row(0, Integer.MIN_VALUE / 2),
   > row(0, Integer.MIN_VALUE / 2 - 11)
   
   > Assert.assertArrayEquals(row(0, new BigDecimal(Integer.MIN_VALUE).add(new BigDecimal(-11))), root.next());
   but it`s up to you of course.




-- 
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: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] tledkov-gridgain commented on a change in pull request #9148: IGNITE-14681 Calcite engine. Extend return type of sum() aggregate function

Posted by GitBox <gi...@apache.org>.
tledkov-gridgain commented on a change in pull request #9148:
URL: https://github.com/apache/ignite/pull/9148#discussion_r702955872



##########
File path: modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/BaseAggregateTest.java
##########
@@ -512,6 +513,98 @@ public void sumOnDifferentRowsCount() throws IgniteCheckedException {
         }
     }
 
+    /** */
+    @Test
+    public void sumIntegerOverflow() {
+        ExecutionContext<Object[]> ctx = executionContext(F.first(nodes()), UUID.randomUUID(), 0);
+        IgniteTypeFactory tf = ctx.getTypeFactory();
+        RelDataType rowType = TypeUtils.createRowType(tf, int.class, int.class);
+        ScanNode<Object[]> scan = new ScanNode<>(ctx, rowType, Arrays.asList(
+            row(0, Integer.MAX_VALUE / 2),
+            row(0, Integer.MAX_VALUE / 2 + 11)
+        ));
+
+        AggregateCall call = AggregateCall.create(
+            SqlStdOperatorTable.SUM,
+            false,
+            false,
+            false,
+            ImmutableIntList.of(1),
+            -1,
+            RelCollations.EMPTY,
+            tf.createJavaType(BigDecimal.class),
+            null);
+
+        ImmutableList<ImmutableBitSet> grpSets = ImmutableList.of(ImmutableBitSet.of(0));
+
+        RelDataType aggRowType = TypeUtils.createRowType(tf, int.class);
+
+        SingleNode<Object[]> aggChain = createAggregateNodesChain(
+            ctx,
+            grpSets,
+            call,
+            rowType,
+            aggRowType,
+            rowFactory(),
+            scan
+        );
+
+        RootNode<Object[]> root = new RootNode<>(ctx, aggRowType);
+        root.register(aggChain);
+
+        assertTrue(root.hasNext());
+
+        Assert.assertArrayEquals(row(0, new BigDecimal(Integer.MAX_VALUE).add(new BigDecimal(10))), root.next());

Review comment:
       Fixed with plain sum of long. Also changed the type of the sum of INTEGER from DECIMAL to BIGINT (Long)




-- 
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: notifications-unsubscribe@ignite.apache.org

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



[GitHub] [ignite] tledkov-gridgain merged pull request #9148: IGNITE-14681 Calcite engine. Extend return type of sum() aggregate function

Posted by GitBox <gi...@apache.org>.
tledkov-gridgain merged pull request #9148:
URL: https://github.com/apache/ignite/pull/9148


   


-- 
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: notifications-unsubscribe@ignite.apache.org

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