You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ko...@apache.org on 2022/08/01 09:59:14 UTC

[ignite-3] branch main updated: IGNITE-17429 Sql. Extend sql grammar to support CREATE/DROP INDEX statements (#966)

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

korlov pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/ignite-3.git


The following commit(s) were added to refs/heads/main by this push:
     new 4b2d2346c IGNITE-17429 Sql. Extend sql grammar to support CREATE/DROP INDEX statements (#966)
4b2d2346c is described below

commit 4b2d2346c63a0b0ecbaf9999bdfea4fb4049ea90
Author: korlov42 <ko...@gridgain.com>
AuthorDate: Mon Aug 1 12:59:08 2022 +0300

    IGNITE-17429 Sql. Extend sql grammar to support CREATE/DROP INDEX statements (#966)
---
 modules/sql-engine/src/main/codegen/config.fmpp    |   5 +
 .../src/main/codegen/includes/parserImpls.ftl      |  58 +++++-
 .../prepare/ddl/DdlSqlToCommandConverter.java      |   2 +-
 .../sql/engine/sql/IgniteSqlCreateIndex.java       |  15 +-
 .../sql/engine/sql/IgniteSqlDropIndex.java         |   2 +-
 .../sql/engine/sql/IgniteSqlIndexType.java         |  32 ++++
 .../internal/sql/engine/sql/SqlDdlParserTest.java  | 205 +++++++++++++++++++++
 7 files changed, 309 insertions(+), 10 deletions(-)

diff --git a/modules/sql-engine/src/main/codegen/config.fmpp b/modules/sql-engine/src/main/codegen/config.fmpp
index 6fa9a9b1a..658ec120d 100644
--- a/modules/sql-engine/src/main/codegen/config.fmpp
+++ b/modules/sql-engine/src/main/codegen/config.fmpp
@@ -38,6 +38,7 @@ data: {
       "org.apache.ignite.internal.sql.engine.sql.IgniteSqlCreateTableOption",
       "org.apache.ignite.internal.sql.engine.sql.IgniteSqlDropIndex",
       "org.apache.ignite.internal.sql.engine.sql.IgniteSqlIntervalTypeNameSpec",
+      "org.apache.ignite.internal.sql.engine.sql.IgniteSqlIndexType",
       "org.apache.calcite.sql.ddl.SqlDdlNodes",
     ]
 
@@ -50,6 +51,8 @@ data: {
       "INDEX"
       "ENGINE"
 #     "KEY_TYPE" // already presented in Calcite
+      "TREE"
+      "HASH"
     ]
 
     # List of non-reserved keywords to add;
@@ -249,6 +252,7 @@ data: {
       "GROUPS"
 #     "HANDLER" # not a keyword in Calcite
 #     "HAVING"
+      "HASH"
       "HOLD"
       "HOUR"
       "IDENTITY"
@@ -496,6 +500,7 @@ data: {
       "TRANSLATE_REGEX"
       "TRANSLATION"
       "TREAT"
+      "TREE"
       "TRIGGER"
       "TRIM"
       "TRIM_ARRAY"
diff --git a/modules/sql-engine/src/main/codegen/includes/parserImpls.ftl b/modules/sql-engine/src/main/codegen/includes/parserImpls.ftl
index 9e68a1321..f3ef9c143 100644
--- a/modules/sql-engine/src/main/codegen/includes/parserImpls.ftl
+++ b/modules/sql-engine/src/main/codegen/includes/parserImpls.ftl
@@ -190,7 +190,7 @@ SqlCreate SqlCreateTable(Span s, boolean replace) :
     }
 }
 
-SqlNode IndexedColumn() :
+SqlNode ColumnNameWithSortDirection() :
 {
     final Span s;
     SqlNode col;
@@ -203,12 +203,21 @@ SqlNode IndexedColumn() :
             col = SqlStdOperatorTable.DESC.createCall(getPos(), col);
         }
     )?
+    (
+        <NULLS> <FIRST> {
+            col = SqlStdOperatorTable.NULLS_FIRST.createCall(getPos(), col);
+        }
+    |
+        <NULLS> <LAST> {
+            col = SqlStdOperatorTable.NULLS_LAST.createCall(getPos(), col);
+        }
+    )?
     {
         return col;
     }
 }
 
-SqlNodeList IndexedColumnList() :
+SqlNodeList ColumnNameWithSortDirectionList() :
 {
     final Span s;
     final List<SqlNode> list = new ArrayList<SqlNode>();
@@ -216,9 +225,26 @@ SqlNodeList IndexedColumnList() :
 }
 {
     <LPAREN> { s = span(); }
-    col = IndexedColumn() { list.add(col); }
+    col = ColumnNameWithSortDirection() { list.add(col); }
     (
-        <COMMA> col = IndexedColumn() { list.add(col); }
+        <COMMA> col = ColumnNameWithSortDirection() { list.add(col); }
+    )*
+    <RPAREN> {
+        return new SqlNodeList(list, s.end(this));
+    }
+}
+
+SqlNodeList ColumnNameList() :
+{
+    final Span s;
+    final List<SqlNode> list = new ArrayList<SqlNode>();
+    SqlNode col = null;
+}
+{
+    <LPAREN> { s = span(); }
+    col = SimpleIdentifier() { list.add(col); }
+    (
+        <COMMA> col = SimpleIdentifier() { list.add(col); }
     )*
     <RPAREN> {
         return new SqlNodeList(list, s.end(this));
@@ -231,6 +257,7 @@ SqlCreate SqlCreateIndex(Span s, boolean replace) :
     final SqlIdentifier idxId;
     final SqlIdentifier tblId;
     final SqlNodeList columnList;
+    IgniteSqlIndexType type = IgniteSqlIndexType.IMPLICIT_TREE;
 }
 {
     <INDEX>
@@ -238,9 +265,26 @@ SqlCreate SqlCreateIndex(Span s, boolean replace) :
     idxId = SimpleIdentifier()
     <ON>
     tblId = CompoundIdentifier()
-    columnList = IndexedColumnList()
-    {
-        return new IgniteSqlCreateIndex(s.end(this), ifNotExists, idxId, tblId, columnList);
+    (
+        columnList = ColumnNameWithSortDirectionList()
+    |
+        <USING> <TREE> {
+            s.add(this);
+
+            type = IgniteSqlIndexType.TREE;
+        }
+
+        columnList = ColumnNameWithSortDirectionList()
+    |
+        <USING> <HASH> {
+            s.add(this);
+
+            type = IgniteSqlIndexType.HASH;
+        }
+
+        columnList = ColumnNameList()
+    ) {
+        return new IgniteSqlCreateIndex(s.end(this), ifNotExists, idxId, tblId, type, columnList);
     }
 }
 
diff --git a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/prepare/ddl/DdlSqlToCommandConverter.java b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/prepare/ddl/DdlSqlToCommandConverter.java
index d8150ee33..22eac7adc 100644
--- a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/prepare/ddl/DdlSqlToCommandConverter.java
+++ b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/prepare/ddl/DdlSqlToCommandConverter.java
@@ -415,7 +415,7 @@ public class DdlSqlToCommandConverter {
     private DropIndexCommand convertDropIndex(IgniteSqlDropIndex sqlCmd) {
         DropIndexCommand dropCmd = new DropIndexCommand();
 
-        dropCmd.indexName(sqlCmd.idxName().getSimple());
+        dropCmd.indexName(sqlCmd.indexName().getSimple());
         dropCmd.ifExist(sqlCmd.ifExists());
 
         return dropCmd;
diff --git a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/sql/IgniteSqlCreateIndex.java b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/sql/IgniteSqlCreateIndex.java
index 1de9f3c0b..3aab0128f 100644
--- a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/sql/IgniteSqlCreateIndex.java
+++ b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/sql/IgniteSqlCreateIndex.java
@@ -41,6 +41,8 @@ public class IgniteSqlCreateIndex extends SqlCreate {
     /** Table name.  */
     private final SqlIdentifier tblName;
 
+    private final IgniteSqlIndexType type;
+
     /** Columns involved. */
     private final SqlNodeList columnList;
 
@@ -50,10 +52,11 @@ public class IgniteSqlCreateIndex extends SqlCreate {
 
     /** Creates a SqlCreateIndex. */
     public IgniteSqlCreateIndex(SqlParserPos pos, boolean ifNotExists, SqlIdentifier idxName, SqlIdentifier tblName,
-            SqlNodeList columnList) {
+            IgniteSqlIndexType type, SqlNodeList columnList) {
         super(OPERATOR, pos, false, ifNotExists);
         this.idxName = Objects.requireNonNull(idxName, "index name");
         this.tblName = Objects.requireNonNull(tblName, "table name");
+        this.type = Objects.requireNonNull(type, "type");
         this.columnList = columnList;
     }
 
@@ -78,6 +81,12 @@ public class IgniteSqlCreateIndex extends SqlCreate {
 
         tblName.unparse(writer, 0, 0);
 
+        if (type != IgniteSqlIndexType.IMPLICIT_TREE) {
+            writer.keyword("USING");
+
+            writer.keyword(type.name());
+        }
+
         SqlWriter.Frame frame = writer.startList("(", ")");
 
         for (SqlNode c : columnList) {
@@ -108,6 +117,10 @@ public class IgniteSqlCreateIndex extends SqlCreate {
         return tblName;
     }
 
+    public IgniteSqlIndexType type() {
+        return type;
+    }
+
     public SqlNodeList columnList() {
         return columnList;
     }
diff --git a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/sql/IgniteSqlDropIndex.java b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/sql/IgniteSqlDropIndex.java
index 875dd4f8f..9e5a95683 100644
--- a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/sql/IgniteSqlDropIndex.java
+++ b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/sql/IgniteSqlDropIndex.java
@@ -62,7 +62,7 @@ public class IgniteSqlDropIndex extends SqlDrop {
         indexName.unparse(writer, leftPrec, rightPrec);
     }
 
-    public SqlIdentifier idxName() {
+    public SqlIdentifier indexName() {
         return indexName;
     }
 
diff --git a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/sql/IgniteSqlIndexType.java b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/sql/IgniteSqlIndexType.java
new file mode 100644
index 000000000..a9290119f
--- /dev/null
+++ b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/sql/IgniteSqlIndexType.java
@@ -0,0 +1,32 @@
+/*
+ * 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.sql.engine.sql;
+
+/**
+ * Enumeration of supported index types.
+ */
+public enum IgniteSqlIndexType {
+    /** Sorted index. */
+    TREE,
+
+    /** Hash index. */
+    HASH,
+
+    /** The user have omitted USING clause, hence the type is set to {@link #TREE} implicitly. */
+    IMPLICIT_TREE
+}
diff --git a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/sql/SqlDdlParserTest.java b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/sql/SqlDdlParserTest.java
index 68115fc28..d3e1be0d0 100644
--- a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/sql/SqlDdlParserTest.java
+++ b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/sql/SqlDdlParserTest.java
@@ -23,14 +23,18 @@ import static org.hamcrest.CoreMatchers.hasItem;
 import static org.hamcrest.CoreMatchers.instanceOf;
 import static org.hamcrest.CoreMatchers.is;
 import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.containsString;
 import static org.hamcrest.Matchers.equalTo;
 import static org.hamcrest.Matchers.nullValue;
 import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
 
 import java.util.List;
 import java.util.Objects;
+import java.util.Set;
 import java.util.function.Predicate;
 import java.util.stream.Collectors;
+import org.apache.calcite.sql.SqlBasicCall;
 import org.apache.calcite.sql.SqlIdentifier;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.SqlLiteral;
@@ -331,6 +335,207 @@ public class SqlDdlParserTest {
         assertThatOptionPresent(createTable.createOptionList().getList(), "persistent", true);
     }
 
+    @Test
+    public void createIndexSimpleCase() throws SqlParseException {
+        var query = "create index my_index on my_table (col)";
+
+        SqlNode node = parse(query);
+
+        assertThat(node, instanceOf(IgniteSqlCreateIndex.class));
+
+        var createIndex = (IgniteSqlCreateIndex) node;
+
+        assertThat(createIndex.indexName().getSimple(), is("MY_INDEX"));
+        assertThat(createIndex.tableName().names, is(List.of("MY_TABLE")));
+        assertThat(createIndex.ifNotExists, is(false));
+        assertThat(createIndex.type(), is(IgniteSqlIndexType.IMPLICIT_TREE));
+        assertThat(createIndex.columnList(), hasItem(ofTypeMatching("col", SqlIdentifier.class,
+                id -> id.isSimple() && id.getSimple().equals("COL"))));
+    }
+
+    @Test
+    public void createIndexImplicitTypeExplicitDirection() throws SqlParseException {
+        var query = "create index my_index on my_table (col1 asc, col2 desc)";
+
+        SqlNode node = parse(query);
+
+        assertThat(node, instanceOf(IgniteSqlCreateIndex.class));
+
+        var createIndex = (IgniteSqlCreateIndex) node;
+
+        assertThat(createIndex.indexName().getSimple(), is("MY_INDEX"));
+        assertThat(createIndex.tableName().names, is(List.of("MY_TABLE")));
+        assertThat(createIndex.ifNotExists, is(false));
+        assertThat(createIndex.type(), is(IgniteSqlIndexType.IMPLICIT_TREE));
+        assertThat(createIndex.columnList(), hasItem(ofTypeMatching("col1 asc", SqlIdentifier.class,
+                id -> id.isSimple() && id.getSimple().equals("COL1"))));
+        assertThat(createIndex.columnList(), hasItem(ofTypeMatching("col2 desc", SqlBasicCall.class,
+                bc -> bc.isA(Set.of(SqlKind.DESCENDING))
+                        && bc.getOperandList().size() == 1
+                        && bc.getOperandList().get(0) instanceof SqlIdentifier
+                        && ((SqlIdentifier) bc.getOperandList().get(0)).isSimple()
+                        && ((SqlIdentifier) bc.getOperandList().get(0)).getSimple().equals("COL2"))));
+    }
+
+    @Test
+    public void createIndexExplicitTypeMixedDirection() throws SqlParseException {
+        var query = "create index my_index on my_table using tree (col1, col2 asc, col3 desc)";
+
+        SqlNode node = parse(query);
+
+        assertThat(node, instanceOf(IgniteSqlCreateIndex.class));
+
+        var createIndex = (IgniteSqlCreateIndex) node;
+
+        assertThat(createIndex.indexName().getSimple(), is("MY_INDEX"));
+        assertThat(createIndex.tableName().names, is(List.of("MY_TABLE")));
+        assertThat(createIndex.ifNotExists, is(false));
+        assertThat(createIndex.type(), is(IgniteSqlIndexType.TREE));
+        assertThat(createIndex.columnList(), hasItem(ofTypeMatching("col1", SqlIdentifier.class,
+                id -> id.isSimple() && id.getSimple().equals("COL1"))));
+        assertThat(createIndex.columnList(), hasItem(ofTypeMatching("col2 asc", SqlIdentifier.class,
+                id -> id.isSimple() && id.getSimple().equals("COL2"))));
+        assertThat(createIndex.columnList(), hasItem(ofTypeMatching("col3 desc", SqlBasicCall.class,
+                bc -> bc.isA(Set.of(SqlKind.DESCENDING))
+                        && bc.getOperandList().size() == 1
+                        && bc.getOperandList().get(0) instanceof SqlIdentifier
+                        && ((SqlIdentifier) bc.getOperandList().get(0)).isSimple()
+                        && ((SqlIdentifier) bc.getOperandList().get(0)).getSimple().equals("COL3"))));
+    }
+
+    @Test
+    public void createHashIndex() throws SqlParseException {
+        var query = "create index my_index on my_table using hash (col)";
+
+        SqlNode node = parse(query);
+
+        assertThat(node, instanceOf(IgniteSqlCreateIndex.class));
+
+        var createIndex = (IgniteSqlCreateIndex) node;
+
+        assertThat(createIndex.indexName().getSimple(), is("MY_INDEX"));
+        assertThat(createIndex.tableName().names, is(List.of("MY_TABLE")));
+        assertThat(createIndex.ifNotExists, is(false));
+        assertThat(createIndex.type(), is(IgniteSqlIndexType.HASH));
+        assertThat(createIndex.columnList(), hasItem(ofTypeMatching("col", SqlIdentifier.class,
+                id -> id.isSimple() && id.getSimple().equals("COL"))));
+    }
+
+    @Test
+    public void sortDirectionMustNotBeSpecifiedForHashIndex() {
+        var query = "create index my_index on my_table using hash (col1, col2 asc)";
+
+        var ex = assertThrows(SqlParseException.class, () -> parse(query));
+        assertThat(ex.getMessage(), containsString("Encountered \" \"ASC\""));
+    }
+
+    @Test
+    public void createIndexIfNotExists() throws SqlParseException {
+        var query = "create index if not exists my_index on my_table (col)";
+
+        SqlNode node = parse(query);
+
+        assertThat(node, instanceOf(IgniteSqlCreateIndex.class));
+
+        var createIndex = (IgniteSqlCreateIndex) node;
+
+        assertThat(createIndex.indexName().getSimple(), is("MY_INDEX"));
+        assertThat(createIndex.tableName().names, is(List.of("MY_TABLE")));
+        assertThat(createIndex.ifNotExists, is(true));
+    }
+
+    @Test
+    public void createIndexTableInParticularSchema() throws SqlParseException {
+        var query = "create index my_index on my_schema.my_table (col)";
+
+        SqlNode node = parse(query);
+
+        assertThat(node, instanceOf(IgniteSqlCreateIndex.class));
+
+        var createIndex = (IgniteSqlCreateIndex) node;
+
+        assertThat(createIndex.indexName().getSimple(), is("MY_INDEX"));
+        assertThat(createIndex.tableName().names, is(List.of("MY_SCHEMA", "MY_TABLE")));
+    }
+
+    @Test
+    public void createIndexExplicitNullDirection() throws SqlParseException {
+        var query = "create index my_index on my_table (col1 nulls first, col2 nulls last, col3 desc nulls first)";
+
+        SqlNode node = parse(query);
+
+        assertThat(node, instanceOf(IgniteSqlCreateIndex.class));
+
+        var createIndex = (IgniteSqlCreateIndex) node;
+
+        assertThat(createIndex.indexName().getSimple(), is("MY_INDEX"));
+        assertThat(createIndex.tableName().names, is(List.of("MY_TABLE")));
+        assertThat(createIndex.columnList(), hasItem(ofTypeMatching("col1 nulls first", SqlBasicCall.class,
+                bc -> bc.isA(Set.of(SqlKind.NULLS_FIRST))
+                        && bc.getOperandList().size() == 1
+                        && bc.getOperandList().get(0) instanceof SqlIdentifier
+                        && ((SqlIdentifier) bc.getOperandList().get(0)).isSimple()
+                        && ((SqlIdentifier) bc.getOperandList().get(0)).getSimple().equals("COL1"))));
+        assertThat(createIndex.columnList(), hasItem(ofTypeMatching("col2 nulls last", SqlBasicCall.class,
+                bc -> bc.isA(Set.of(SqlKind.NULLS_LAST))
+                        && bc.getOperandList().size() == 1
+                        && bc.getOperandList().get(0) instanceof SqlIdentifier
+                        && ((SqlIdentifier) bc.getOperandList().get(0)).isSimple()
+                        && ((SqlIdentifier) bc.getOperandList().get(0)).getSimple().equals("COL2"))));
+        assertThat(createIndex.columnList(), hasItem(ofTypeMatching("col3 desc nulls first", SqlBasicCall.class,
+                bc -> bc.isA(Set.of(SqlKind.NULLS_FIRST))
+                        && bc.getOperandList().size() == 1
+                        && bc.getOperandList().get(0) instanceof SqlBasicCall
+                        && bc.getOperandList().get(0).isA(Set.of(SqlKind.DESCENDING))
+                        && ((SqlBasicCall) bc.getOperandList().get(0)).getOperandList().size() == 1
+                        && ((SqlBasicCall) bc.getOperandList().get(0)).getOperandList().get(0) instanceof SqlIdentifier
+                        && ((SqlIdentifier) ((SqlBasicCall) bc.getOperandList().get(0)).getOperandList().get(0)).isSimple()
+                        && ((SqlIdentifier) ((SqlBasicCall) bc.getOperandList().get(0)).getOperandList().get(0))
+                                .getSimple().equals("COL3"))));
+    }
+
+    @Test
+    public void dropIndexSimpleCase() throws SqlParseException {
+        var query = "drop index my_index";
+
+        SqlNode node = parse(query);
+
+        assertThat(node, instanceOf(IgniteSqlDropIndex.class));
+
+        var dropIndex = (IgniteSqlDropIndex) node;
+
+        assertThat(dropIndex.ifExists(), is(false));
+        assertThat(dropIndex.indexName().names, is(List.of("MY_INDEX")));
+    }
+
+    @Test
+    public void dropIndexSchemaSpecified() throws SqlParseException {
+        var query = "drop index my_schema.my_index";
+
+        SqlNode node = parse(query);
+
+        assertThat(node, instanceOf(IgniteSqlDropIndex.class));
+
+        var dropIndex = (IgniteSqlDropIndex) node;
+
+        assertThat(dropIndex.ifExists(), is(false));
+        assertThat(dropIndex.indexName().names, is(List.of("MY_SCHEMA", "MY_INDEX")));
+    }
+
+    @Test
+    public void dropIndexIfExists() throws SqlParseException {
+        var query = "drop index if exists my_index";
+
+        SqlNode node = parse(query);
+
+        assertThat(node, instanceOf(IgniteSqlDropIndex.class));
+
+        var dropIndex = (IgniteSqlDropIndex) node;
+
+        assertThat(dropIndex.ifExists(), is(true));
+        assertThat(dropIndex.indexName().names, is(List.of("MY_INDEX")));
+    }
+
     private IgniteSqlCreateTable parseCreateTable(String stmt) throws SqlParseException {
         SqlNode node = parse(stmt);