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/03/04 09:46:15 UTC

[ignite] branch sql-calcite updated: IGNITE-14157 SQL. Calcite: Return human-readable column names for expression results in select

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 89600f9  IGNITE-14157 SQL. Calcite: Return human-readable column names for expression results in select
89600f9 is described below

commit 89600f98cfd56cc7f7276cbae6fb51af14ddac74
Author: Yury Gerzhedovich <yg...@gridgain.com>
AuthorDate: Thu Mar 4 12:45:54 2021 +0300

    IGNITE-14157 SQL. Calcite: Return human-readable column names for expression results in select
---
 .../query/calcite/prepare/IgniteSqlValidator.java  | 39 ++++++++++-
 ...Test.java => AbstractBasicIntegrationTest.java} | 68 +++---------------
 .../query/calcite/AggregatesIntegrationTest.java   | 80 +---------------------
 .../query/calcite/MetadataIntegrationTest.java     | 71 +++++++++++++++++++
 .../processors/query/calcite/QueryChecker.java     | 20 ++++++
 .../ignite/testsuites/IgniteCalciteTestSuite.java  |  4 +-
 6 files changed, 146 insertions(+), 136 deletions(-)

diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/IgniteSqlValidator.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/IgniteSqlValidator.java
index 0770f4e..b696116 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/IgniteSqlValidator.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/IgniteSqlValidator.java
@@ -18,10 +18,11 @@
 package org.apache.ignite.internal.processors.query.calcite.prepare;
 
 import java.math.BigDecimal;
+import java.util.Collections;
+import java.util.EnumSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-
 import org.apache.calcite.plan.RelOptTable;
 import org.apache.calcite.prepare.CalciteCatalogReader;
 import org.apache.calcite.prepare.Prepare;
@@ -41,6 +42,7 @@ import org.apache.calcite.sql.SqlOperatorTable;
 import org.apache.calcite.sql.SqlSelect;
 import org.apache.calcite.sql.SqlUpdate;
 import org.apache.calcite.sql.SqlUtil;
+import org.apache.calcite.sql.dialect.CalciteSqlDialect;
 import org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.calcite.sql.validate.SelectScope;
 import org.apache.calcite.sql.validate.SqlValidator;
@@ -62,6 +64,26 @@ public class IgniteSqlValidator extends SqlValidatorImpl {
     /** Decimal of Integer.MAX_VALUE for fetch/offset bounding. */
     private static final BigDecimal DEC_INT_MAX = BigDecimal.valueOf(Integer.MAX_VALUE);
 
+    /** **/
+    private static final int MAX_LENGTH_OF_ALIASES = 256;
+
+    /** **/
+    private static final Set<SqlKind> HUMAN_READABLE_ALIASES_FOR;
+
+    static {
+        EnumSet<SqlKind> kinds = EnumSet.noneOf(SqlKind.class);
+
+        kinds.addAll(SqlKind.AGGREGATE);
+        kinds.addAll(SqlKind.BINARY_ARITHMETIC);
+        kinds.addAll(SqlKind.FUNCTION);
+
+        kinds.add(SqlKind.CEIL);
+        kinds.add(SqlKind.FLOOR);
+        kinds.add(SqlKind.LITERAL);
+
+        HUMAN_READABLE_ALIASES_FOR = Collections.unmodifiableSet(kinds);
+    }
+
     /** Dynamic parameters. */
     Object[] parameters;
 
@@ -187,6 +209,21 @@ public class IgniteSqlValidator extends SqlValidatorImpl {
     }
 
     /** {@inheritDoc} */
+    @Override public String deriveAlias(SqlNode node, int ordinal) {
+        if (node.isA(HUMAN_READABLE_ALIASES_FOR)) {
+            String alias = node.toSqlString(c -> c.withDialect(CalciteSqlDialect.DEFAULT)
+                .withQuoteAllIdentifiers(false)
+                .withAlwaysUseParentheses(false)
+                .withClauseStartsLine(false)
+            ).getSql();
+
+            return alias.substring(0, Math.min(alias.length(), MAX_LENGTH_OF_ALIASES));
+        }
+
+        return super.deriveAlias(node, ordinal);
+    }
+
+    /** {@inheritDoc} */
     @Override public void validateAggregateParams(SqlCall aggCall, SqlNode filter, SqlNodeList orderList, SqlValidatorScope scope) {
         validateAggregateFunction(aggCall, (SqlAggFunction) aggCall.getOperator());
 
diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/AggregatesIntegrationTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/AbstractBasicIntegrationTest.java
similarity index 59%
copy from modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/AggregatesIntegrationTest.java
copy to modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/AbstractBasicIntegrationTest.java
index 61fb358..741f0f3 100644
--- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/AggregatesIntegrationTest.java
+++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/AbstractBasicIntegrationTest.java
@@ -29,13 +29,12 @@ import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.G;
 import org.apache.ignite.testframework.junits.WithSystemProperty;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-import org.junit.Test;
 
 /**
  *
  */
 @WithSystemProperty(key = "calcite.debug", value = "false")
-public class AggregatesIntegrationTest extends GridCommonAbstractTest {
+public class AbstractBasicIntegrationTest extends GridCommonAbstractTest {
     /** */
     private static IgniteEx client;
 
@@ -60,8 +59,16 @@ public class AggregatesIntegrationTest extends GridCommonAbstractTest {
     }
 
     /** */
-    @Test
-    public void countOfNonNumericField() throws InterruptedException {
+    protected QueryChecker assertQuery(String qry) {
+        return new QueryChecker(qry) {
+            @Override protected QueryEngine getEngine() {
+                return Commons.lookupComponent(client.context(), QueryEngine.class);
+            }
+        };
+    }
+
+    /** */
+    protected void createAndPopulateTable() {
         IgniteCache<Integer, Employer> person = client.getOrCreateCache(new CacheConfiguration<Integer, Employer>()
             .setName("person")
             .setSqlSchema("PUBLIC")
@@ -75,59 +82,6 @@ public class AggregatesIntegrationTest extends GridCommonAbstractTest {
         person.put(idx++, new Employer("Ilya", 15d));
         person.put(idx++, new Employer("Roma", 10d));
         person.put(idx++, new Employer("Roma", 10d));
-
-        assertQuery("select count(name) from person").returns(4L).check();
-        assertQuery("select count(*) from person").returns(5L).check();
-        assertQuery("select count(1) from person").returns(5L).check();
-
-        assertQuery("select count(*) from person where salary < 0").returns(0L).check();
-        assertQuery("select count(*) from person where salary < 0 and salary > 0").returns(0L).check();
-
-        assertQuery("select count(case when name like 'R%' then 1 else null end) from person").returns(2L).check();
-        assertQuery("select count(case when name not like 'I%' then 1 else null end) from person").returns(2L).check();
-
-        assertQuery("select count(name) from person where salary > 10").returns(1L).check();
-        assertQuery("select count(*) from person where salary > 10").returns(2L).check();
-        assertQuery("select count(1) from person where salary > 10").returns(2L).check();
-
-        assertQuery("select salary, count(name) from person group by salary order by salary")
-            .returns(10d, 3L)
-            .returns(15d, 1L)
-            .check();
-
-        assertQuery("select salary, count(*) from person group by salary order by salary")
-            .returns(10d, 3L)
-            .returns(15d, 2L)
-            .check();
-
-        assertQuery("select salary, count(1) from person group by salary order by salary")
-            .returns(10d, 3L)
-            .returns(15d, 2L)
-            .check();
-
-        assertQuery("select salary, count(1), sum(1) from person group by salary order by salary")
-            .returns(10d, 3L, 3)
-            .returns(15d, 2L, 2)
-            .check();
-
-        assertQuery("select salary, name, count(1), sum(salary) from person group by salary, name order by salary")
-            .returns(10d, "Igor", 1L, 10d)
-            .returns(10d, "Roma", 2L, 20d)
-            .returns(15d, "Ilya", 1L, 15d)
-            .returns(15d, null, 1L, 15d)
-            .check();
-
-        assertQuery("select salary, count(name) from person group by salary having salary < 10 order by salary")
-            .check();
-    }
-
-    /** */
-    private QueryChecker assertQuery(String qry) {
-        return new QueryChecker(qry) {
-            @Override protected QueryEngine getEngine() {
-                return Commons.lookupComponent(client.context(), QueryEngine.class);
-            }
-        };
     }
 
     /** */
diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/AggregatesIntegrationTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/AggregatesIntegrationTest.java
index 61fb358..439c06e 100644
--- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/AggregatesIntegrationTest.java
+++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/AggregatesIntegrationTest.java
@@ -17,64 +17,16 @@
 
 package org.apache.ignite.internal.processors.query.calcite;
 
-import org.apache.ignite.Ignite;
-import org.apache.ignite.IgniteCache;
-import org.apache.ignite.cache.QueryEntity;
-import org.apache.ignite.cache.query.annotations.QuerySqlField;
-import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.internal.IgniteEx;
-import org.apache.ignite.internal.processors.query.QueryEngine;
-import org.apache.ignite.internal.processors.query.calcite.util.Commons;
-import org.apache.ignite.internal.util.typedef.F;
-import org.apache.ignite.internal.util.typedef.G;
-import org.apache.ignite.testframework.junits.WithSystemProperty;
-import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.junit.Test;
 
 /**
  *
  */
-@WithSystemProperty(key = "calcite.debug", value = "false")
-public class AggregatesIntegrationTest extends GridCommonAbstractTest {
-    /** */
-    private static IgniteEx client;
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        startGrids(3);
-
-        client = startClientGrid();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTest() {
-        for (Ignite ign : G.allGrids()) {
-            for (String cacheName : ign.cacheNames())
-                ign.destroyCache(cacheName);
-
-            CalciteQueryProcessor qryProc = (CalciteQueryProcessor)Commons.lookupComponent(
-                ((IgniteEx)ign).context(), QueryEngine.class);
-
-            qryProc.queryPlanCache().clear();
-        }
-    }
-
+public class AggregatesIntegrationTest extends AbstractBasicIntegrationTest {
     /** */
     @Test
-    public void countOfNonNumericField() throws InterruptedException {
-        IgniteCache<Integer, Employer> person = client.getOrCreateCache(new CacheConfiguration<Integer, Employer>()
-            .setName("person")
-            .setSqlSchema("PUBLIC")
-            .setQueryEntities(F.asList(new QueryEntity(Integer.class, Employer.class).setTableName("person")))
-            .setBackups(2)
-        );
-
-        int idx = 0;
-        person.put(idx++, new Employer("Igor", 10d));
-        person.put(idx++, new Employer(null, 15d));
-        person.put(idx++, new Employer("Ilya", 15d));
-        person.put(idx++, new Employer("Roma", 10d));
-        person.put(idx++, new Employer("Roma", 10d));
+    public void countOfNonNumericField() {
+        createAndPopulateTable();
 
         assertQuery("select count(name) from person").returns(4L).check();
         assertQuery("select count(*) from person").returns(5L).check();
@@ -120,30 +72,4 @@ public class AggregatesIntegrationTest extends GridCommonAbstractTest {
         assertQuery("select salary, count(name) from person group by salary having salary < 10 order by salary")
             .check();
     }
-
-    /** */
-    private QueryChecker assertQuery(String qry) {
-        return new QueryChecker(qry) {
-            @Override protected QueryEngine getEngine() {
-                return Commons.lookupComponent(client.context(), QueryEngine.class);
-            }
-        };
-    }
-
-    /** */
-    public static class Employer {
-        /** */
-        @QuerySqlField
-        public String name;
-
-        /** */
-        @QuerySqlField
-        public Double salary;
-
-        /** */
-        public Employer(String name, Double salary) {
-            this.name = name;
-            this.salary = salary;
-        }
-    }
 }
diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/MetadataIntegrationTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/MetadataIntegrationTest.java
new file mode 100644
index 0000000..26f4749
--- /dev/null
+++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/MetadataIntegrationTest.java
@@ -0,0 +1,71 @@
+/*
+ * 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;
+
+import org.junit.Test;
+
+import static java.util.stream.Collectors.joining;
+import static java.util.stream.Stream.generate;
+
+/**
+ *
+ */
+public class MetadataIntegrationTest extends AbstractBasicIntegrationTest {
+    /** */
+    @Test
+    public void trimColumnNames() {
+        createAndPopulateTable();
+
+        String X300 = generate(() -> "X").limit(300).collect(joining());
+        String X256 = "'" + X300.substring(0, 255);
+
+        assertQuery("select '" + X300 + "' from person").columnNames(X256).check();
+    }
+
+    /** */
+    @Test
+    public void columnNames() {
+        createAndPopulateTable();
+
+        assertQuery("select (select count(*) from person), (select avg(salary) from person) from person")
+            .columnNames("EXPR$0", "EXPR$1").check();
+        assertQuery("select (select count(*) from person) as subquery from person")
+            .columnNames("SUBQUERY").check();
+
+        assertQuery("select salary*2, salary/2, salary+2, salary-2, mod(salary, 2)  from person")
+            .columnNames("SALARY * 2", "SALARY / 2", "SALARY + 2", "SALARY - 2",  "MOD(SALARY, 2)").check();
+        assertQuery("select salary*2 as first, salary/2 as secOND from person").columnNames("FIRST", "SECOND").check();
+
+        assertQuery("select trim(name) tr_name from person").columnNames("TR_NAME").check();
+        assertQuery("select trim(name) from person").columnNames("TRIM(BOTH ' ' FROM NAME)").check();
+        assertQuery("select row(1), ceil(salary), floor(salary), position('text' IN salary) from person")
+            .columnNames("ROW(1)", "CEIL(SALARY)", "FLOOR(SALARY)", "POSITION('text' IN SALARY)").check();
+
+        assertQuery("select count(*) from person").columnNames("COUNT(*)").check();
+        assertQuery("select count(name) from person").columnNames("COUNT(NAME)").check();
+        assertQuery("select max(salary) from person").columnNames("MAX(SALARY)").check();
+        assertQuery("select min(salary) from person").columnNames("MIN(SALARY)").check();
+        assertQuery("select aVg(salary) from person").columnNames("AVG(SALARY)").check();
+        assertQuery("select sum(salary) from person").columnNames("SUM(SALARY)").check();
+
+        assertQuery("select salary, count(name) from person group by salary").columnNames("SALARY", "COUNT(NAME)").check();
+
+        assertQuery("select 1, -1, 'some string' from person").columnNames("1", "-1", "'some string'").check();
+    }
+}
diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/QueryChecker.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/QueryChecker.java
index e687054..56f880e 100644
--- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/QueryChecker.java
+++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/QueryChecker.java
@@ -26,6 +26,7 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.regex.Pattern;
 import java.util.stream.Collectors;
+import java.util.stream.IntStream;
 import org.apache.ignite.cache.query.FieldsQueryCursor;
 import org.apache.ignite.internal.processors.query.QueryEngine;
 import org.apache.ignite.internal.util.typedef.F;
@@ -34,6 +35,7 @@ import org.hamcrest.CoreMatchers;
 import org.hamcrest.Matcher;
 import org.hamcrest.core.SubstringMatcher;
 
+import static org.hamcrest.CoreMatchers.equalTo;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.fail;
@@ -222,6 +224,9 @@ public abstract class QueryChecker {
     private List<List<?>> expectedResult;
 
     /** */
+    private List<String> expectedColumnNames;
+
+    /** */
     private boolean ordered;
 
     /** */
@@ -265,8 +270,16 @@ public abstract class QueryChecker {
     }
 
     /** */
+    public QueryChecker columnNames(String... columns) {
+        expectedColumnNames = Arrays.asList(columns);
+
+        return this;
+    }
+
+    /** */
     public QueryChecker matches(Matcher<String>... planMatcher) {
         Collections.addAll(planMatchers, planMatcher);
+
         return this;
     }
 
@@ -303,6 +316,13 @@ public abstract class QueryChecker {
 
         FieldsQueryCursor<List<?>> cur = cursors.get(0);
 
+        if (expectedColumnNames != null) {
+            List<String> colNames = IntStream.range(0, cur.getColumnsCount())
+                .mapToObj(cur::getFieldName).collect(Collectors.toList());
+
+            assertThat("Column names don't match", colNames, equalTo(expectedColumnNames));
+        }
+
         List<List<?>> res = cur.getAll();
 
         if (expectedResult != null) {
diff --git a/modules/calcite/src/test/java/org/apache/ignite/testsuites/IgniteCalciteTestSuite.java b/modules/calcite/src/test/java/org/apache/ignite/testsuites/IgniteCalciteTestSuite.java
index 0b83ac8..088e15e 100644
--- a/modules/calcite/src/test/java/org/apache/ignite/testsuites/IgniteCalciteTestSuite.java
+++ b/modules/calcite/src/test/java/org/apache/ignite/testsuites/IgniteCalciteTestSuite.java
@@ -22,6 +22,7 @@ import org.apache.ignite.internal.processors.query.calcite.CalciteBasicSecondary
 import org.apache.ignite.internal.processors.query.calcite.CalciteErrorHandlilngIntegrationTest;
 import org.apache.ignite.internal.processors.query.calcite.CalciteQueryProcessorTest;
 import org.apache.ignite.internal.processors.query.calcite.CancelTest;
+import org.apache.ignite.internal.processors.query.calcite.MetadataIntegrationTest;
 import org.apache.ignite.internal.processors.query.calcite.DateTimeTest;
 import org.apache.ignite.internal.processors.query.calcite.LimitOffsetTest;
 import org.apache.ignite.internal.processors.query.calcite.QueryCheckerTest;
@@ -54,7 +55,8 @@ import org.junit.runners.Suite;
     DateTimeTest.class,
     LimitOffsetTest.class,
     SqlFieldsQueryUsageTest.class,
-    AggregatesIntegrationTest.class
+    AggregatesIntegrationTest.class,
+    MetadataIntegrationTest.class
 })
 public class IgniteCalciteTestSuite {
 }