You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by tl...@apache.org on 2021/04/22 12:34:19 UTC

[ignite] branch sql-calcite updated: IGNITE-14546 Calcite engine. MIN/MAX aggregates doesn't support string types

This is an automated email from the ASF dual-hosted git repository.

tledkov pushed a commit to branch sql-calcite
in repository https://gitbox.apache.org/repos/asf/ignite.git


The following commit(s) were added to refs/heads/sql-calcite by this push:
     new f013182  IGNITE-14546 Calcite engine. MIN/MAX aggregates doesn't support string types
f013182 is described below

commit f01318295967cdd514e69ccfedd4914fdca7dadc
Author: zstan <st...@gmail.com>
AuthorDate: Thu Apr 22 15:33:54 2021 +0300

    IGNITE-14546 Calcite engine. MIN/MAX aggregates doesn't support string types
---
 .../query/calcite/exec/exp/agg/Accumulators.java   | 94 ++++++++++++++++++++++
 .../query/calcite/CalciteQueryProcessorTest.java   | 11 +++
 .../sql/aggregate/aggregates/test_aggr_string.test | 22 ++++-
 .../aggregates/test_aggr_string.test_ignore        | 13 +--
 4 files changed, 127 insertions(+), 13 deletions(-)

diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/agg/Accumulators.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/agg/Accumulators.java
index afb0ea3..f9a0511 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/agg/Accumulators.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/agg/Accumulators.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.query.calcite.exec.exp.agg;
 
 import java.math.BigDecimal;
 import java.math.MathContext;
+import java.util.Comparator;
 import java.util.List;
 import java.util.function.Supplier;
 import org.apache.calcite.rel.core.AggregateCall;
@@ -31,6 +32,7 @@ import static org.apache.calcite.sql.type.SqlTypeName.BIGINT;
 import static org.apache.calcite.sql.type.SqlTypeName.DECIMAL;
 import static org.apache.calcite.sql.type.SqlTypeName.DOUBLE;
 import static org.apache.calcite.sql.type.SqlTypeName.INTEGER;
+import static org.apache.calcite.sql.type.SqlTypeName.VARCHAR;
 
 /**
  *
@@ -118,6 +120,8 @@ public class Accumulators {
                 return DecimalMinMax.MIN_FACTORY;
             case INTEGER:
                 return IntMinMax.MIN_FACTORY;
+            case VARCHAR:
+                return VarCharMinMax.MIN_FACTORY;
             case BIGINT:
             default:
                 return LongMinMax.MIN_FACTORY;
@@ -135,6 +139,8 @@ public class Accumulators {
                 return DecimalMinMax.MAX_FACTORY;
             case INTEGER:
                 return IntMinMax.MAX_FACTORY;
+            case VARCHAR:
+                return VarCharMinMax.MAX_FACTORY;
             case BIGINT:
             default:
                 return LongMinMax.MAX_FACTORY;
@@ -726,6 +732,94 @@ public class Accumulators {
     }
 
     /** */
+    private static class VarCharMinMax implements Accumulator {
+        /** */
+        public static final Supplier<Accumulator> MIN_FACTORY = () -> new VarCharMinMax(true);
+
+        /** */
+        public static final Supplier<Accumulator> MAX_FACTORY = () -> new VarCharMinMax(false);
+
+        /** */
+        private final boolean min;
+
+        /** */
+        private CharSequence val;
+
+        /** */
+        private boolean empty = true;
+
+        /** */
+        private VarCharMinMax(boolean min) {
+            this.min = min;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void add(Object... args) {
+            CharSequence in = (CharSequence)args[0];
+
+            if (in == null)
+                return;
+
+            val = empty ? in : min ?
+                (CharSeqComparator.INSTANCE.compare(val, in) < 0 ? val : in) :
+                (CharSeqComparator.INSTANCE.compare(val, in) < 0 ? in : val);
+
+            empty = false;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void apply(Accumulator other) {
+            VarCharMinMax other0 = (VarCharMinMax)other;
+
+            if (other0.empty)
+                return;
+
+            val = empty ? other0.val : min ?
+                (CharSeqComparator.INSTANCE.compare(val, other0.val) < 0 ? val : other0.val) :
+                (CharSeqComparator.INSTANCE.compare(val, other0.val) < 0 ? other0.val : val);
+
+            empty = false;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object end() {
+            return empty ? null : val;
+        }
+
+        /** {@inheritDoc} */
+        @Override public List<RelDataType> argumentTypes(IgniteTypeFactory typeFactory) {
+            return F.asList(typeFactory.createTypeWithNullability(typeFactory.createSqlType(VARCHAR), true));
+        }
+
+        /** {@inheritDoc} */
+        @Override public RelDataType returnType(IgniteTypeFactory typeFactory) {
+            return typeFactory.createTypeWithNullability(typeFactory.createSqlType(VARCHAR), true);
+        }
+
+        /** */
+        @SuppressWarnings("ComparatorNotSerializable")
+        private static class CharSeqComparator implements Comparator<CharSequence> {
+            /** */
+            private static final CharSeqComparator INSTANCE = new CharSeqComparator();
+
+            /** */
+            @Override public int compare(CharSequence s1, CharSequence s2) {
+                int len = Math.min(s1.length(), s2.length());
+
+                // find the first difference and return
+                for (int i = 0; i < len; i += 1) {
+                    int cmp = Character.compare(s1.charAt(i), s2.charAt(i));
+                    if (cmp != 0)
+                        return cmp;
+                }
+
+                // if there are no differences, then the shorter seq is first
+                return Integer.compare(s1.length(), s2.length());
+            }
+        }
+    }
+
+    /** */
     private static class IntMinMax implements Accumulator {
         /** */
         public static final Supplier<Accumulator> MIN_FACTORY = () -> new IntMinMax(true);
diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessorTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessorTest.java
index a6299e3..42e53dc 100644
--- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessorTest.java
+++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessorTest.java
@@ -146,6 +146,17 @@ public class CalciteQueryProcessorTest extends GridCommonAbstractTest {
             .check();
     }
 
+    /** Tests varchar min\max aggregates. */
+    @Test
+    public void testVarCharMinMax() throws IgniteInterruptedCheckedException {
+        execute(client, "CREATE TABLE TEST(val VARCHAR primary key, val1 integer);");
+        execute(client, "INSERT INTO test VALUES ('б', 1), ('бб', 2), ('щ', 3), ('щщ', 4), ('Б', 4), ('ББ', 4), ('Я', 4);");
+        List<List<?>> rows = sql("SELECT MAX(val), MIN(val) FROM TEST");
+
+        assertEquals(1, rows.size());
+        assertEquals(Arrays.asList("щщ", "Б"), F.first(rows));
+    }
+
     /** */
     @Test
     public void testCountWithJoin() throws Exception {
diff --git a/modules/calcite/src/test/sql/aggregate/aggregates/test_aggr_string.test b/modules/calcite/src/test/sql/aggregate/aggregates/test_aggr_string.test
index 5c2e5d8..cfc4961 100644
--- a/modules/calcite/src/test/sql/aggregate/aggregates/test_aggr_string.test
+++ b/modules/calcite/src/test/sql/aggregate/aggregates/test_aggr_string.test
@@ -29,7 +29,27 @@ hello	11.000000
 world	12.000000
 NULL	11.000000
 
-# distinct aggregations ons tring
+# distinct aggregations on string
 statement ok
 INSERT INTO test VALUES (11, 'hello'), (12, 'world')
 
+# string min/max with long strings
+statement ok
+CREATE TABLE test_strings(s VARCHAR);
+INSERT INTO test_strings VALUES ('aaaaaaaahello'), ('bbbbbbbbbbbbbbbbbbbbhello'), ('ccccccccccccccchello'), ('aaaaaaaaaaaaaaaaaaaaaaaahello')
+
+query II
+SELECT MIN(s), MAX(s) FROM test_strings;
+----
+aaaaaaaaaaaaaaaaaaaaaaaahello	ccccccccccccccchello
+
+# string min/max with long strings 2
+statement ok
+CREATE TABLE test_strings2(s VARCHAR);
+INSERT INTO test_strings2 VALUES ('a'), ('aa'), ('A'), ('AA'), ('D')
+
+query II
+SELECT MAX(s), MIN(s) FROM test_strings2;
+----
+aa	A
+
diff --git a/modules/calcite/src/test/sql/aggregate/aggregates/test_aggr_string.test_ignore b/modules/calcite/src/test/sql/aggregate/aggregates/test_aggr_string.test_ignore
index 3f7ef2b..33b849c 100644
--- a/modules/calcite/src/test/sql/aggregate/aggregates/test_aggr_string.test_ignore
+++ b/modules/calcite/src/test/sql/aggregate/aggregates/test_aggr_string.test_ignore
@@ -3,7 +3,6 @@
 # group: [aggregates]
 # Ignore: https://issues.apache.org/jira/browse/IGNITE-14544
 # Ignore: https://issues.apache.org/jira/browse/IGNITE-14545
-# Ignore: https://issues.apache.org/jira/browse/IGNITE-14546
 
 query TTTTI
 SELECT NULL as a, NULL as b, NULL as c, NULL as d, 1 as id UNION SELECT 'Кирилл' as a, 'Müller' as b, '我是谁' as c, 'ASCII' as d, 2 as id ORDER BY 1
@@ -38,7 +37,7 @@ hello	11.000000
 world	12.000000
 NULL	11.000000
 
-# distinct aggregations ons tring
+# distinct aggregations on string
 statement ok
 INSERT INTO test VALUES (11, 'hello'), (12, 'world')
 
@@ -61,13 +60,3 @@ SELECT a, COUNT(*), COUNT(s), COUNT(DISTINCT s) FROM test WHERE s IS NOT NULL GR
 ----
 11	2	2	1
 12	2	2	1
-
-# string min/max with long strings
-statement ok
-CREATE TABLE test_strings(s VARCHAR);
-INSERT INTO test_strings VALUES ('aaaaaaaahello'), ('bbbbbbbbbbbbbbbbbbbbhello'), ('ccccccccccccccchello'), ('aaaaaaaaaaaaaaaaaaaaaaaahello');;
-
-query II
-SELECT MIN(s), MAX(s) FROM test_strings;
-----
-aaaaaaaaaaaaaaaaaaaaaaaahello	ccccccccccccccchello