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/04/09 09:56:42 UTC

[GitHub] [ignite] korlov42 opened a new pull request #8994: IGNITE-13547 CREATE TABLE support

korlov42 opened a new pull request #8994:
URL: https://github.com/apache/ignite/pull/8994


   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] joooger commented on a change in pull request #8994: IGNITE-13547 CREATE TABLE support

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



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionServiceImpl.java
##########
@@ -596,6 +643,212 @@ private QueryPlan prepareDml(SqlNode sqlNode, PlanningContext ctx) throws Valida
         return new MultiStepDmlPlan(template, queryFieldsMetadata(ctx, igniteRel.getRowType(), null));
     }
 
+    /** */
+    private QueryPlan prepareDdl(SqlNode sqlNode, PlanningContext ctx) {
+        assert sqlNode instanceof SqlDdl : sqlNode == null ? "null" : sqlNode.getClass().getName();
+
+        SqlDdl ddlNode = (SqlDdl)sqlNode;
+
+        if (ddlNode instanceof IgniteSqlCreateTable) {
+            IgniteSqlCreateTable createTblNode = (IgniteSqlCreateTable)ddlNode;
+
+            SqlIdentifier tblFullName = createTblNode.name();
+
+            CreateTableCommand createTblCmd = new CreateTableCommand();
+
+            createTblCmd.schemaName("PUBLIC"); // TODO: get from the context
+            createTblCmd.tableName(Util.last(tblFullName.names));
+            createTblCmd.ifNotExists(createTblNode.ifNotExists());
+            createTblCmd.templateName(QueryUtils.TEMPLATE_PARTITIONED);
+
+            if (createTblNode.createOptionList() != null) {
+                for (SqlNode optNode : createTblNode.createOptionList().getList()) {
+                    IgniteSqlCreateTableOption opt = (IgniteSqlCreateTableOption)optNode;
+
+                    switch (opt.key()) {
+                        case TEMPLATE:
+                            if (!(opt.value() instanceof SqlIdentifier))
+                                throw new IgniteSQLException("Unexpected value for param " + TEMPLATE + " [" +
+                                    "expected an identifier, but was " + opt.value() + "; " +
+                                    "querySql=\"" + ctx.query() + "\"]", IgniteQueryErrorCode.PARSING);
+
+                            createTblCmd.templateName(((SqlIdentifier)opt.value()).names.get(0));
+                            break;
+
+                        case BACKUPS:
+                            if (!(opt.value() instanceof SqlNumericLiteral)
+                                || !((SqlNumericLiteral)opt.value()).isInteger()
+                                || ((SqlLiteral)opt.value()).intValue(true) < 0
+                            )
+                                throw new IgniteSQLException("Unexpected value for param " + BACKUPS + " [" +
+                                    "expected a non-negative integer, but was " + opt.value() + "; " +
+                                    "querySql=\"" + ctx.query() + "\"]", IgniteQueryErrorCode.PARSING);
+
+                            createTblCmd.backups(((SqlLiteral)opt.value()).intValue(true));
+                            break;
+
+                        case AFFINITY_KEY:
+                            if (!(opt.value() instanceof SqlIdentifier))
+                                throw new IgniteSQLException("Unexpected value for param " + AFFINITY_KEY + " [" +
+                                    "expected an identifier, but was " + opt.value() + "; " +
+                                    "querySql=\"" + ctx.query() + "\"]", IgniteQueryErrorCode.PARSING);
+
+                            createTblCmd.affinityKey(((SqlIdentifier)opt.value()).names.get(0));
+                            break;
+
+                        case ATOMICITY: {
+                            CacheAtomicityMode mode = null;
+
+                            if (opt.value() instanceof SqlIdentifier) {
+                                mode = Arrays.stream(CacheAtomicityMode.values())
+                                    .filter(m -> m.name().equalsIgnoreCase(opt.value().toString()))
+                                    .findFirst()
+                                    .orElse(null);
+                            }
+
+                            if (mode == null)
+                                throw new IgniteSQLException("Unexpected value for param " + ATOMICITY
+                                    + " [expected values are " + Arrays.toString(CacheAtomicityMode.values())
+                                    + ", but was " + opt.value() + "; " +
+                                    "querySql=\"" + ctx.query() + "\"]", IgniteQueryErrorCode.PARSING);
+
+                            createTblCmd.atomicityMode(mode);
+                            break;
+                        }
+                        case CACHE_GROUP:
+                            if (!(opt.value() instanceof SqlIdentifier))
+                                throw new IgniteSQLException("Unexpected value for param " + CACHE_GROUP + " [" +
+                                    "expected an identifier, but was " + opt.value() + "; " +
+                                    "querySql=\"" + ctx.query() + "\"]", IgniteQueryErrorCode.PARSING);
+
+                            createTblCmd.cacheGroup(((SqlIdentifier)opt.value()).names.get(0));
+                            break;
+
+                        case CACHE_NAME:
+                            if (!(opt.value() instanceof SqlIdentifier))
+                                throw new IgniteSQLException("Unexpected value for param " + CACHE_NAME + " [" +
+                                    "expected an identifier, but was " + opt.value() + "; " +
+                                    "querySql=\"" + ctx.query() + "\"]", IgniteQueryErrorCode.PARSING);
+
+                            createTblCmd.cacheName(((SqlIdentifier)opt.value()).names.get(0));
+                            break;
+
+                        case DATA_REGION:
+                            if (!(opt.value() instanceof SqlIdentifier))
+                                throw new IgniteSQLException("Unexpected value for param " + DATA_REGION + " [" +
+                                    "expected an identifier, but was " + opt.value() + "; " +
+                                    "querySql=\"" + ctx.query() + "\"]", IgniteQueryErrorCode.PARSING);
+
+                            createTblCmd.dataRegionName(((SqlIdentifier)opt.value()).names.get(0));
+                            break;
+
+                        case KEY_TYPE:
+                            if (!(opt.value() instanceof SqlIdentifier))
+                                throw new IgniteSQLException("Unexpected value for param " + KEY_TYPE + " [" +
+                                    "expected an identifier, but was " + opt.value() + "; " +
+                                    "querySql=\"" + ctx.query() + "\"]", IgniteQueryErrorCode.PARSING);
+
+                            createTblCmd.keyTypeName(((SqlIdentifier)opt.value()).names.get(0));
+                            break;
+
+                        case VALUE_TYPE:
+                            if (!(opt.value() instanceof SqlIdentifier))
+                                throw new IgniteSQLException("Unexpected value for param " + VALUE_TYPE + " [" +
+                                    "expected an identifier, but was " + opt.value() + "; " +
+                                    "querySql=\"" + ctx.query() + "\"]", IgniteQueryErrorCode.PARSING);
+
+                            createTblCmd.valueTypeName(((SqlIdentifier)opt.value()).names.get(0));
+                            break;
+
+                        case WRITE_SYNCHRONIZATION_MODE: {
+                            CacheWriteSynchronizationMode mode = null;
+
+                            if (opt.value() instanceof SqlIdentifier) {
+                                mode = Arrays.stream(CacheWriteSynchronizationMode.values())
+                                    .filter(m -> m.name().equalsIgnoreCase(opt.value().toString()))
+                                    .findFirst()
+                                    .orElse(null);
+                            }
+
+                            if (mode == null)
+                                throw new IgniteSQLException("Unexpected value for param " + WRITE_SYNCHRONIZATION_MODE
+                                    + " [expected values are " + Arrays.toString(CacheWriteSynchronizationMode.values())
+                                    + ", but was " + opt.value() + "; " +
+                                    "querySql=\"" + ctx.query() + "\"]", IgniteQueryErrorCode.PARSING);
+
+                            createTblCmd.writeSynchronizationMode(mode);
+                            break;
+                        }
+                        case ENCRYPTED:
+                            if (!(opt.value() instanceof SqlLiteral) && ((SqlLiteral)opt.value()).getTypeName() != BOOLEAN)
+                                throw new IgniteSQLException("Unexpected value for param " + ENCRYPTED + " [" +
+                                    "expected a boolean, but was " + opt.value() + "; " +
+                                    "querySql=\"" + ctx.query() + "\"]", IgniteQueryErrorCode.PARSING);
+
+                            createTblCmd.encrypted(((SqlLiteral)opt.value()).booleanValue());
+                            break;
+
+                        default:
+                            throw new IllegalStateException("Unsupported option " + opt.key());
+                    }
+                }
+            }
+
+            List<SqlColumnDeclaration> colDeclarations = createTblNode.columnList().getList().stream()
+                .filter(SqlColumnDeclaration.class::isInstance)
+                .map(SqlColumnDeclaration.class::cast)
+                .collect(Collectors.toList());
+
+            IgnitePlanner planner = ctx.planner();
+
+            List<ColumnDefinition> cols = new ArrayList<>();
+
+            for (SqlColumnDeclaration col : colDeclarations) {
+                String name = Util.last(col.name.names);

Review comment:
       we are going to support a compound identifier? As I remember 'No'. If so, maybe better to check that we have a simple name?




-- 
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] joooger commented on a change in pull request #8994: IGNITE-13547 CREATE TABLE support

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



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionServiceImpl.java
##########
@@ -596,6 +643,212 @@ private QueryPlan prepareDml(SqlNode sqlNode, PlanningContext ctx) throws Valida
         return new MultiStepDmlPlan(template, queryFieldsMetadata(ctx, igniteRel.getRowType(), null));
     }
 
+    /** */
+    private QueryPlan prepareDdl(SqlNode sqlNode, PlanningContext ctx) {
+        assert sqlNode instanceof SqlDdl : sqlNode == null ? "null" : sqlNode.getClass().getName();
+
+        SqlDdl ddlNode = (SqlDdl)sqlNode;
+
+        if (ddlNode instanceof IgniteSqlCreateTable) {
+            IgniteSqlCreateTable createTblNode = (IgniteSqlCreateTable)ddlNode;
+
+            SqlIdentifier tblFullName = createTblNode.name();
+
+            CreateTableCommand createTblCmd = new CreateTableCommand();
+
+            createTblCmd.schemaName("PUBLIC"); // TODO: get from the context
+            createTblCmd.tableName(Util.last(tblFullName.names));
+            createTblCmd.ifNotExists(createTblNode.ifNotExists());
+            createTblCmd.templateName(QueryUtils.TEMPLATE_PARTITIONED);
+
+            if (createTblNode.createOptionList() != null) {
+                for (SqlNode optNode : createTblNode.createOptionList().getList()) {
+                    IgniteSqlCreateTableOption opt = (IgniteSqlCreateTableOption)optNode;
+
+                    switch (opt.key()) {
+                        case TEMPLATE:
+                            if (!(opt.value() instanceof SqlIdentifier))
+                                throw new IgniteSQLException("Unexpected value for param " + TEMPLATE + " [" +
+                                    "expected an identifier, but was " + opt.value() + "; " +
+                                    "querySql=\"" + ctx.query() + "\"]", IgniteQueryErrorCode.PARSING);
+
+                            createTblCmd.templateName(((SqlIdentifier)opt.value()).names.get(0));
+                            break;
+
+                        case BACKUPS:
+                            if (!(opt.value() instanceof SqlNumericLiteral)
+                                || !((SqlNumericLiteral)opt.value()).isInteger()
+                                || ((SqlLiteral)opt.value()).intValue(true) < 0
+                            )
+                                throw new IgniteSQLException("Unexpected value for param " + BACKUPS + " [" +
+                                    "expected a non-negative integer, but was " + opt.value() + "; " +
+                                    "querySql=\"" + ctx.query() + "\"]", IgniteQueryErrorCode.PARSING);

Review comment:
       we have a few places for throwing parsing exception, maybe better to extract a separate method for it and pass only exception text and context?




-- 
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 #8994: IGNITE-13547 CREATE TABLE support

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



##########
File path: modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/sql/SqlDdlParserTest.java
##########
@@ -0,0 +1,299 @@
+/*
+ * 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.sql;
+
+import java.util.List;
+import java.util.Objects;
+import java.util.function.Predicate;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlNumericLiteral;
+import org.apache.calcite.sql.ddl.SqlColumnDeclaration;
+import org.apache.calcite.sql.ddl.SqlKeyConstraint;
+import org.apache.calcite.sql.parser.SqlParseException;
+import org.apache.calcite.sql.parser.SqlParser;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.hamcrest.CustomMatcher;
+import org.hamcrest.Matcher;
+import org.junit.Test;
+
+import static java.util.Collections.singleton;
+import static org.hamcrest.CoreMatchers.hasItem;
+import static org.hamcrest.CoreMatchers.instanceOf;
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Test suite to verify parsing of the DDL command.
+ */
+public class SqlDdlParserTest extends GridCommonAbstractTest {
+    /**
+     * Very simple case where only table name and a few columns are presented.
+     */
+    @Test
+    public void createTableSimpleCase() throws SqlParseException {
+        String query = "create table my_table(id int, val varchar)";
+
+        SqlNode node = parse(query);
+
+        assertThat(node, instanceOf(IgniteSqlCreateTable.class));
+
+        IgniteSqlCreateTable createTable = (IgniteSqlCreateTable)node;
+
+        assertThat(createTable.name().names, is(ImmutableList.of("MY_TABLE")));
+        assertThat(createTable.ifNotExists, is(false));
+        assertThat(createTable.columnList(), hasItem(columnWithName("ID")));
+        assertThat(createTable.columnList(), hasItem(columnWithName("VAL")));
+    }
+
+    /**
+     * Parsing of CREATE TABLE statement with quoted identifiers.
+     */
+    @Test
+    public void createTableQuotedIdentifiers() throws SqlParseException {
+        String query = "create table \"My_Table\"(\"Id\" int, \"Val\" varchar)";
+
+        SqlNode node = parse(query);
+
+        assertThat(node, instanceOf(IgniteSqlCreateTable.class));
+
+        IgniteSqlCreateTable createTable = (IgniteSqlCreateTable)node;
+
+        assertThat(createTable.name().names, is(ImmutableList.of("My_Table")));
+        assertThat(createTable.ifNotExists, is(false));
+        assertThat(createTable.columnList(), hasItem(columnWithName("Id")));
+        assertThat(createTable.columnList(), hasItem(columnWithName("Val")));
+    }
+
+    /**
+     * Parsing of CREATE TABLE statement with IF NOT EXISTS.
+     */
+    @Test
+    public void createTableIfNotExists() throws SqlParseException {
+        String query = "create table if not exists my_table(id int, val varchar)";
+
+        SqlNode node = parse(query);
+
+        assertThat(node, instanceOf(IgniteSqlCreateTable.class));
+
+        IgniteSqlCreateTable createTable = (IgniteSqlCreateTable)node;
+
+        assertThat(createTable.name().names, is(ImmutableList.of("MY_TABLE")));
+        assertThat(createTable.ifNotExists, is(true));
+        assertThat(createTable.columnList(), hasItem(columnWithName("ID")));
+        assertThat(createTable.columnList(), hasItem(columnWithName("VAL")));
+    }
+
+    /**
+     * Parsing of CREATE TABLE with specified PK constraint where constraint
+     * is a shortcut within a column definition.
+     */
+    @Test
+    public void createTableWithPkCase1() throws SqlParseException {
+        String query = "create table my_table(id int primary key, val varchar)";
+
+        SqlNode node = parse(query);
+
+        assertThat(node, instanceOf(IgniteSqlCreateTable.class));
+
+        IgniteSqlCreateTable createTable = (IgniteSqlCreateTable)node;
+
+        assertThat(createTable.name().names, is(ImmutableList.of("MY_TABLE")));
+        assertThat(createTable.ifNotExists, is(false));
+        assertThat(createTable.columnList(), hasItem(ofTypeMatching(
+            "PK constraint with name \"ID\"", SqlKeyConstraint.class,
+            constraint -> hasItem(ofTypeMatching("identifier \"ID\"", SqlIdentifier.class, id -> "ID".equals(id.names.get(0))))
+                .matches(constraint.getOperandList().get(1))
+                && constraint.getOperandList().get(0) == null
+                && constraint.isA(singleton(SqlKind.PRIMARY_KEY)))));
+    }
+
+    /**
+     * Parsing of CREATE TABLE with specified PK constraint where constraint
+     * is set explicitly and has no name.
+     */
+    @Test
+    public void createTableWithPkCase2() throws SqlParseException {
+        String query = "create table my_table(id int, val varchar, primary key(id))";
+
+        SqlNode node = parse(query);
+
+        assertThat(node, instanceOf(IgniteSqlCreateTable.class));
+
+        IgniteSqlCreateTable createTable = (IgniteSqlCreateTable)node;
+
+        assertThat(createTable.name().names, is(ImmutableList.of("MY_TABLE")));
+        assertThat(createTable.ifNotExists, is(false));
+        assertThat(createTable.columnList(), hasItem(ofTypeMatching(
+            "PK constraint without name containing column \"ID\"", SqlKeyConstraint.class,
+            constraint -> hasItem(ofTypeMatching("identifier \"ID\"", SqlIdentifier.class, id -> "ID".equals(id.names.get(0))))
+                .matches(constraint.getOperandList().get(1))
+                && constraint.getOperandList().get(0) == null
+                && constraint.isA(singleton(SqlKind.PRIMARY_KEY)))));
+    }
+
+    /**
+     * Parsing of CREATE TABLE with specified PK constraint where constraint
+     * is set explicitly and has a name.
+     */
+    @Test
+    public void createTableWithPkCase3() throws SqlParseException {
+        String query = "create table my_table(id int, val varchar, constraint pk_key primary key(id))";

Review comment:
       can u also append test for multiple pk in constraint ? i.e:
   primary key(id1, id2)




-- 
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 #8994: IGNITE-13547 CREATE TABLE support

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



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/sql/IgniteSqlCreateTable.java
##########
@@ -0,0 +1,117 @@
+/*
+ * 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.sql;
+
+import java.util.List;
+import java.util.Objects;
+
+import org.apache.calcite.sql.SqlCreate;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlNodeList;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlSpecialOperator;
+import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.util.ImmutableNullableList;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+/**
+ * Parse tree for {@code CREATE TABLE} statement with Ignite specific features.
+ */
+public class IgniteSqlCreateTable extends SqlCreate {
+    /** */
+    private final SqlIdentifier name;
+
+    /** */
+    private final @Nullable SqlNodeList columnList;
+
+    /** */
+    private final @Nullable SqlNodeList createOptionList;
+
+    /** */
+    private static final SqlOperator OPERATOR =
+        new SqlSpecialOperator("CREATE TABLE", SqlKind.CREATE_TABLE);
+
+    /** Creates a SqlCreateTable. */
+    protected IgniteSqlCreateTable(SqlParserPos pos, boolean ifNotExists,
+        SqlIdentifier name, @Nullable SqlNodeList columnList, @Nullable SqlNodeList createOptionList) {
+        super(OPERATOR, pos, false, ifNotExists);
+        this.name = Objects.requireNonNull(name, "name");
+        this.columnList = columnList; // may be null

Review comment:
       why do we need such comments ? already declaired in annotations.




-- 
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] joooger commented on a change in pull request #8994: IGNITE-13547 CREATE TABLE support

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



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionServiceImpl.java
##########
@@ -596,6 +643,212 @@ private QueryPlan prepareDml(SqlNode sqlNode, PlanningContext ctx) throws Valida
         return new MultiStepDmlPlan(template, queryFieldsMetadata(ctx, igniteRel.getRowType(), null));
     }
 
+    /** */
+    private QueryPlan prepareDdl(SqlNode sqlNode, PlanningContext ctx) {
+        assert sqlNode instanceof SqlDdl : sqlNode == null ? "null" : sqlNode.getClass().getName();
+
+        SqlDdl ddlNode = (SqlDdl)sqlNode;
+
+        if (ddlNode instanceof IgniteSqlCreateTable) {
+            IgniteSqlCreateTable createTblNode = (IgniteSqlCreateTable)ddlNode;
+
+            SqlIdentifier tblFullName = createTblNode.name();
+
+            CreateTableCommand createTblCmd = new CreateTableCommand();
+
+            createTblCmd.schemaName("PUBLIC"); // TODO: get from the context

Review comment:
       agree, my bad. In this case we should check that tablename is simple




-- 
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] korlov42 commented on a change in pull request #8994: IGNITE-13547 CREATE TABLE support

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



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionServiceImpl.java
##########
@@ -596,6 +643,212 @@ private QueryPlan prepareDml(SqlNode sqlNode, PlanningContext ctx) throws Valida
         return new MultiStepDmlPlan(template, queryFieldsMetadata(ctx, igniteRel.getRowType(), null));
     }
 
+    /** */
+    private QueryPlan prepareDdl(SqlNode sqlNode, PlanningContext ctx) {
+        assert sqlNode instanceof SqlDdl : sqlNode == null ? "null" : sqlNode.getClass().getName();
+
+        SqlDdl ddlNode = (SqlDdl)sqlNode;
+
+        if (ddlNode instanceof IgniteSqlCreateTable) {
+            IgniteSqlCreateTable createTblNode = (IgniteSqlCreateTable)ddlNode;
+
+            SqlIdentifier tblFullName = createTblNode.name();
+
+            CreateTableCommand createTblCmd = new CreateTableCommand();
+
+            createTblCmd.schemaName("PUBLIC"); // TODO: get from the context

Review comment:
       fixed anyway




-- 
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] korlov42 commented on a change in pull request #8994: IGNITE-13547 CREATE TABLE support

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



##########
File path: modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/sql/SqlDdlParserTest.java
##########
@@ -0,0 +1,299 @@
+/*
+ * 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.sql;
+
+import java.util.List;
+import java.util.Objects;
+import java.util.function.Predicate;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlNumericLiteral;
+import org.apache.calcite.sql.ddl.SqlColumnDeclaration;
+import org.apache.calcite.sql.ddl.SqlKeyConstraint;
+import org.apache.calcite.sql.parser.SqlParseException;
+import org.apache.calcite.sql.parser.SqlParser;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.hamcrest.CustomMatcher;
+import org.hamcrest.Matcher;
+import org.junit.Test;
+
+import static java.util.Collections.singleton;
+import static org.hamcrest.CoreMatchers.hasItem;
+import static org.hamcrest.CoreMatchers.instanceOf;
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Test suite to verify parsing of the DDL command.
+ */
+public class SqlDdlParserTest extends GridCommonAbstractTest {
+    /**
+     * Very simple case where only table name and a few columns are presented.
+     */
+    @Test
+    public void createTableSimpleCase() throws SqlParseException {
+        String query = "create table my_table(id int, val varchar)";
+
+        SqlNode node = parse(query);
+
+        assertThat(node, instanceOf(IgniteSqlCreateTable.class));
+
+        IgniteSqlCreateTable createTable = (IgniteSqlCreateTable)node;
+
+        assertThat(createTable.name().names, is(ImmutableList.of("MY_TABLE")));
+        assertThat(createTable.ifNotExists, is(false));
+        assertThat(createTable.columnList(), hasItem(columnWithName("ID")));
+        assertThat(createTable.columnList(), hasItem(columnWithName("VAL")));
+    }
+
+    /**
+     * Parsing of CREATE TABLE statement with quoted identifiers.
+     */
+    @Test
+    public void createTableQuotedIdentifiers() throws SqlParseException {
+        String query = "create table \"My_Table\"(\"Id\" int, \"Val\" varchar)";
+
+        SqlNode node = parse(query);
+
+        assertThat(node, instanceOf(IgniteSqlCreateTable.class));
+
+        IgniteSqlCreateTable createTable = (IgniteSqlCreateTable)node;
+
+        assertThat(createTable.name().names, is(ImmutableList.of("My_Table")));
+        assertThat(createTable.ifNotExists, is(false));
+        assertThat(createTable.columnList(), hasItem(columnWithName("Id")));
+        assertThat(createTable.columnList(), hasItem(columnWithName("Val")));
+    }
+
+    /**
+     * Parsing of CREATE TABLE statement with IF NOT EXISTS.
+     */
+    @Test
+    public void createTableIfNotExists() throws SqlParseException {
+        String query = "create table if not exists my_table(id int, val varchar)";
+
+        SqlNode node = parse(query);
+
+        assertThat(node, instanceOf(IgniteSqlCreateTable.class));
+
+        IgniteSqlCreateTable createTable = (IgniteSqlCreateTable)node;
+
+        assertThat(createTable.name().names, is(ImmutableList.of("MY_TABLE")));
+        assertThat(createTable.ifNotExists, is(true));
+        assertThat(createTable.columnList(), hasItem(columnWithName("ID")));
+        assertThat(createTable.columnList(), hasItem(columnWithName("VAL")));
+    }
+
+    /**
+     * Parsing of CREATE TABLE with specified PK constraint where constraint
+     * is a shortcut within a column definition.
+     */
+    @Test
+    public void createTableWithPkCase1() throws SqlParseException {
+        String query = "create table my_table(id int primary key, val varchar)";
+
+        SqlNode node = parse(query);
+
+        assertThat(node, instanceOf(IgniteSqlCreateTable.class));
+
+        IgniteSqlCreateTable createTable = (IgniteSqlCreateTable)node;
+
+        assertThat(createTable.name().names, is(ImmutableList.of("MY_TABLE")));
+        assertThat(createTable.ifNotExists, is(false));
+        assertThat(createTable.columnList(), hasItem(ofTypeMatching(
+            "PK constraint with name \"ID\"", SqlKeyConstraint.class,
+            constraint -> hasItem(ofTypeMatching("identifier \"ID\"", SqlIdentifier.class, id -> "ID".equals(id.names.get(0))))
+                .matches(constraint.getOperandList().get(1))
+                && constraint.getOperandList().get(0) == null
+                && constraint.isA(singleton(SqlKind.PRIMARY_KEY)))));
+    }
+
+    /**
+     * Parsing of CREATE TABLE with specified PK constraint where constraint
+     * is set explicitly and has no name.
+     */
+    @Test
+    public void createTableWithPkCase2() throws SqlParseException {
+        String query = "create table my_table(id int, val varchar, primary key(id))";
+
+        SqlNode node = parse(query);
+
+        assertThat(node, instanceOf(IgniteSqlCreateTable.class));
+
+        IgniteSqlCreateTable createTable = (IgniteSqlCreateTable)node;
+
+        assertThat(createTable.name().names, is(ImmutableList.of("MY_TABLE")));
+        assertThat(createTable.ifNotExists, is(false));
+        assertThat(createTable.columnList(), hasItem(ofTypeMatching(
+            "PK constraint without name containing column \"ID\"", SqlKeyConstraint.class,
+            constraint -> hasItem(ofTypeMatching("identifier \"ID\"", SqlIdentifier.class, id -> "ID".equals(id.names.get(0))))
+                .matches(constraint.getOperandList().get(1))
+                && constraint.getOperandList().get(0) == null
+                && constraint.isA(singleton(SqlKind.PRIMARY_KEY)))));
+    }
+
+    /**
+     * Parsing of CREATE TABLE with specified PK constraint where constraint
+     * is set explicitly and has a name.
+     */
+    @Test
+    public void createTableWithPkCase3() throws SqlParseException {
+        String query = "create table my_table(id int, val varchar, constraint pk_key primary key(id))";

Review comment:
       done




-- 
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 #8994: IGNITE-13547 CREATE TABLE support

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



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/TableDescriptorImpl.java
##########
@@ -123,8 +124,23 @@ public TableDescriptorImpl(GridCacheContextInfo<?, ?> cacheInfo, GridQueryTypeDe
         // A _key/_val field is virtual in case there is an alias or a property(es) mapped to the _key/_val field.
         BitSet virtualFields = new BitSet();
 
-        descriptors.add(
-            new KeyValDescriptor(QueryUtils.KEY_FIELD_NAME, typeDesc.keyClass(), true, QueryUtils.KEY_COL));
+        if (typeDesc.implicitPk()) {
+            // pk is not set, thus we need to provide default value for autogenerated key
+            descriptors.add(
+                new KeyValDescriptor(QueryUtils.KEY_FIELD_NAME, typeDesc.keyClass(), true, QueryUtils.KEY_COL) {
+                    @Override public Object defaultValue() {

Review comment:
       possibly we need to overwrite this method too ? 
   @Override public boolean hasDefaultValue() {
               return false;
           }




-- 
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] korlov42 commented on a change in pull request #8994: IGNITE-13547 CREATE TABLE support

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



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionServiceImpl.java
##########
@@ -596,6 +643,212 @@ private QueryPlan prepareDml(SqlNode sqlNode, PlanningContext ctx) throws Valida
         return new MultiStepDmlPlan(template, queryFieldsMetadata(ctx, igniteRel.getRowType(), null));
     }
 
+    /** */
+    private QueryPlan prepareDdl(SqlNode sqlNode, PlanningContext ctx) {
+        assert sqlNode instanceof SqlDdl : sqlNode == null ? "null" : sqlNode.getClass().getName();
+
+        SqlDdl ddlNode = (SqlDdl)sqlNode;
+
+        if (ddlNode instanceof IgniteSqlCreateTable) {
+            IgniteSqlCreateTable createTblNode = (IgniteSqlCreateTable)ddlNode;
+
+            SqlIdentifier tblFullName = createTblNode.name();
+
+            CreateTableCommand createTblCmd = new CreateTableCommand();
+
+            createTblCmd.schemaName("PUBLIC"); // TODO: get from the context

Review comment:
       > should check that tablename is simple
   
   currently it is handled by grammar. Are you OK if I just add an assertion here that would verify the grammar is not changed (names collection has exactly one element)

##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionServiceImpl.java
##########
@@ -596,6 +643,212 @@ private QueryPlan prepareDml(SqlNode sqlNode, PlanningContext ctx) throws Valida
         return new MultiStepDmlPlan(template, queryFieldsMetadata(ctx, igniteRel.getRowType(), null));
     }
 
+    /** */
+    private QueryPlan prepareDdl(SqlNode sqlNode, PlanningContext ctx) {
+        assert sqlNode instanceof SqlDdl : sqlNode == null ? "null" : sqlNode.getClass().getName();
+
+        SqlDdl ddlNode = (SqlDdl)sqlNode;
+
+        if (ddlNode instanceof IgniteSqlCreateTable) {
+            IgniteSqlCreateTable createTblNode = (IgniteSqlCreateTable)ddlNode;
+
+            SqlIdentifier tblFullName = createTblNode.name();
+
+            CreateTableCommand createTblCmd = new CreateTableCommand();
+
+            createTblCmd.schemaName("PUBLIC"); // TODO: get from the context

Review comment:
       > should check that tablename is simple
   
   currently it is handled by grammar. Are you OK if I just add an assertion here that would verify the grammar is not changed (names collection has exactly one element)?




-- 
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] korlov42 commented on a change in pull request #8994: IGNITE-13547 CREATE TABLE support

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



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/sql/IgniteSqlCreateTable.java
##########
@@ -0,0 +1,117 @@
+/*
+ * 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.sql;
+
+import java.util.List;
+import java.util.Objects;
+
+import org.apache.calcite.sql.SqlCreate;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlNodeList;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlSpecialOperator;
+import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.util.ImmutableNullableList;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+/**
+ * Parse tree for {@code CREATE TABLE} statement with Ignite specific features.
+ */
+public class IgniteSqlCreateTable extends SqlCreate {
+    /** */
+    private final SqlIdentifier name;
+
+    /** */
+    private final @Nullable SqlNodeList columnList;
+
+    /** */
+    private final @Nullable SqlNodeList createOptionList;
+
+    /** */
+    private static final SqlOperator OPERATOR =
+        new SqlSpecialOperator("CREATE TABLE", SqlKind.CREATE_TABLE);
+
+    /** Creates a SqlCreateTable. */
+    protected IgniteSqlCreateTable(SqlParserPos pos, boolean ifNotExists,
+        SqlIdentifier name, @Nullable SqlNodeList columnList, @Nullable SqlNodeList createOptionList) {
+        super(OPERATOR, pos, false, ifNotExists);
+        this.name = Objects.requireNonNull(name, "name");
+        this.columnList = columnList; // may be null

Review comment:
       because it was copy pasted from calcite. Removed this




-- 
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] joooger commented on a change in pull request #8994: IGNITE-13547 CREATE TABLE support

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



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionServiceImpl.java
##########
@@ -596,6 +643,212 @@ private QueryPlan prepareDml(SqlNode sqlNode, PlanningContext ctx) throws Valida
         return new MultiStepDmlPlan(template, queryFieldsMetadata(ctx, igniteRel.getRowType(), null));
     }
 
+    /** */
+    private QueryPlan prepareDdl(SqlNode sqlNode, PlanningContext ctx) {
+        assert sqlNode instanceof SqlDdl : sqlNode == null ? "null" : sqlNode.getClass().getName();
+
+        SqlDdl ddlNode = (SqlDdl)sqlNode;
+
+        if (ddlNode instanceof IgniteSqlCreateTable) {
+            IgniteSqlCreateTable createTblNode = (IgniteSqlCreateTable)ddlNode;
+
+            SqlIdentifier tblFullName = createTblNode.name();
+
+            CreateTableCommand createTblCmd = new CreateTableCommand();
+
+            createTblCmd.schemaName("PUBLIC"); // TODO: get from the context

Review comment:
       Seems not the only context could be a source for schemaName, but schemanName could also present in CREATE TABLE clause:
   CREATE TABLE [schemaname.]tableName ...
   As I understand for the second case we could extract it from` tblFullName.names`




-- 
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] korlov42 commented on a change in pull request #8994: IGNITE-13547 CREATE TABLE support

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



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionServiceImpl.java
##########
@@ -536,27 +554,21 @@ private QueryPlan prepareSingle(SqlNode sqlNode, PlanningContext ctx) throws Val
 
         ctx.planner().reset();
 
-        switch (sqlNode.getKind()) {
-            case SELECT:
-            case ORDER_BY:
-            case WITH:
-            case VALUES:
-            case UNION:
-                return prepareQuery(sqlNode, ctx);
+       if (sqlNode.getKind().belongsTo(SqlKind.QUERY))

Review comment:
       hm, it's better to return the old logic




-- 
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] korlov42 commented on a change in pull request #8994: IGNITE-13547 CREATE TABLE support

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



##########
File path: modules/calcite/src/main/codegen/config.fmpp
##########
@@ -0,0 +1,637 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more

Review comment:
       this is mainly was copy-pasted from calcite-babel module, hence it may not be paid too much attention




-- 
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] korlov42 commented on a change in pull request #8994: IGNITE-13547 CREATE TABLE support

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



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionServiceImpl.java
##########
@@ -596,6 +643,212 @@ private QueryPlan prepareDml(SqlNode sqlNode, PlanningContext ctx) throws Valida
         return new MultiStepDmlPlan(template, queryFieldsMetadata(ctx, igniteRel.getRowType(), null));
     }
 
+    /** */
+    private QueryPlan prepareDdl(SqlNode sqlNode, PlanningContext ctx) {
+        assert sqlNode instanceof SqlDdl : sqlNode == null ? "null" : sqlNode.getClass().getName();
+
+        SqlDdl ddlNode = (SqlDdl)sqlNode;
+
+        if (ddlNode instanceof IgniteSqlCreateTable) {
+            IgniteSqlCreateTable createTblNode = (IgniteSqlCreateTable)ddlNode;
+
+            SqlIdentifier tblFullName = createTblNode.name();
+
+            CreateTableCommand createTblCmd = new CreateTableCommand();
+
+            createTblCmd.schemaName("PUBLIC"); // TODO: get from the context
+            createTblCmd.tableName(Util.last(tblFullName.names));
+            createTblCmd.ifNotExists(createTblNode.ifNotExists());
+            createTblCmd.templateName(QueryUtils.TEMPLATE_PARTITIONED);
+
+            if (createTblNode.createOptionList() != null) {
+                for (SqlNode optNode : createTblNode.createOptionList().getList()) {
+                    IgniteSqlCreateTableOption opt = (IgniteSqlCreateTableOption)optNode;
+
+                    switch (opt.key()) {
+                        case TEMPLATE:
+                            if (!(opt.value() instanceof SqlIdentifier))
+                                throw new IgniteSQLException("Unexpected value for param " + TEMPLATE + " [" +
+                                    "expected an identifier, but was " + opt.value() + "; " +
+                                    "querySql=\"" + ctx.query() + "\"]", IgniteQueryErrorCode.PARSING);
+
+                            createTblCmd.templateName(((SqlIdentifier)opt.value()).names.get(0));
+                            break;
+
+                        case BACKUPS:
+                            if (!(opt.value() instanceof SqlNumericLiteral)
+                                || !((SqlNumericLiteral)opt.value()).isInteger()
+                                || ((SqlLiteral)opt.value()).intValue(true) < 0
+                            )
+                                throw new IgniteSQLException("Unexpected value for param " + BACKUPS + " [" +
+                                    "expected a non-negative integer, but was " + opt.value() + "; " +
+                                    "querySql=\"" + ctx.query() + "\"]", IgniteQueryErrorCode.PARSING);
+
+                            createTblCmd.backups(((SqlLiteral)opt.value()).intValue(true));
+                            break;
+
+                        case AFFINITY_KEY:
+                            if (!(opt.value() instanceof SqlIdentifier))
+                                throw new IgniteSQLException("Unexpected value for param " + AFFINITY_KEY + " [" +
+                                    "expected an identifier, but was " + opt.value() + "; " +
+                                    "querySql=\"" + ctx.query() + "\"]", IgniteQueryErrorCode.PARSING);
+
+                            createTblCmd.affinityKey(((SqlIdentifier)opt.value()).names.get(0));
+                            break;
+
+                        case ATOMICITY: {
+                            CacheAtomicityMode mode = null;
+
+                            if (opt.value() instanceof SqlIdentifier) {
+                                mode = Arrays.stream(CacheAtomicityMode.values())
+                                    .filter(m -> m.name().equalsIgnoreCase(opt.value().toString()))
+                                    .findFirst()
+                                    .orElse(null);
+                            }
+
+                            if (mode == null)
+                                throw new IgniteSQLException("Unexpected value for param " + ATOMICITY
+                                    + " [expected values are " + Arrays.toString(CacheAtomicityMode.values())
+                                    + ", but was " + opt.value() + "; " +
+                                    "querySql=\"" + ctx.query() + "\"]", IgniteQueryErrorCode.PARSING);
+
+                            createTblCmd.atomicityMode(mode);
+                            break;
+                        }
+                        case CACHE_GROUP:
+                            if (!(opt.value() instanceof SqlIdentifier))
+                                throw new IgniteSQLException("Unexpected value for param " + CACHE_GROUP + " [" +
+                                    "expected an identifier, but was " + opt.value() + "; " +
+                                    "querySql=\"" + ctx.query() + "\"]", IgniteQueryErrorCode.PARSING);
+
+                            createTblCmd.cacheGroup(((SqlIdentifier)opt.value()).names.get(0));
+                            break;
+
+                        case CACHE_NAME:
+                            if (!(opt.value() instanceof SqlIdentifier))
+                                throw new IgniteSQLException("Unexpected value for param " + CACHE_NAME + " [" +
+                                    "expected an identifier, but was " + opt.value() + "; " +
+                                    "querySql=\"" + ctx.query() + "\"]", IgniteQueryErrorCode.PARSING);
+
+                            createTblCmd.cacheName(((SqlIdentifier)opt.value()).names.get(0));
+                            break;
+
+                        case DATA_REGION:
+                            if (!(opt.value() instanceof SqlIdentifier))
+                                throw new IgniteSQLException("Unexpected value for param " + DATA_REGION + " [" +
+                                    "expected an identifier, but was " + opt.value() + "; " +
+                                    "querySql=\"" + ctx.query() + "\"]", IgniteQueryErrorCode.PARSING);
+
+                            createTblCmd.dataRegionName(((SqlIdentifier)opt.value()).names.get(0));
+                            break;
+
+                        case KEY_TYPE:
+                            if (!(opt.value() instanceof SqlIdentifier))
+                                throw new IgniteSQLException("Unexpected value for param " + KEY_TYPE + " [" +
+                                    "expected an identifier, but was " + opt.value() + "; " +
+                                    "querySql=\"" + ctx.query() + "\"]", IgniteQueryErrorCode.PARSING);
+
+                            createTblCmd.keyTypeName(((SqlIdentifier)opt.value()).names.get(0));
+                            break;
+
+                        case VALUE_TYPE:
+                            if (!(opt.value() instanceof SqlIdentifier))
+                                throw new IgniteSQLException("Unexpected value for param " + VALUE_TYPE + " [" +
+                                    "expected an identifier, but was " + opt.value() + "; " +
+                                    "querySql=\"" + ctx.query() + "\"]", IgniteQueryErrorCode.PARSING);
+
+                            createTblCmd.valueTypeName(((SqlIdentifier)opt.value()).names.get(0));
+                            break;
+
+                        case WRITE_SYNCHRONIZATION_MODE: {
+                            CacheWriteSynchronizationMode mode = null;
+
+                            if (opt.value() instanceof SqlIdentifier) {
+                                mode = Arrays.stream(CacheWriteSynchronizationMode.values())
+                                    .filter(m -> m.name().equalsIgnoreCase(opt.value().toString()))
+                                    .findFirst()
+                                    .orElse(null);
+                            }
+
+                            if (mode == null)
+                                throw new IgniteSQLException("Unexpected value for param " + WRITE_SYNCHRONIZATION_MODE
+                                    + " [expected values are " + Arrays.toString(CacheWriteSynchronizationMode.values())
+                                    + ", but was " + opt.value() + "; " +
+                                    "querySql=\"" + ctx.query() + "\"]", IgniteQueryErrorCode.PARSING);
+
+                            createTblCmd.writeSynchronizationMode(mode);
+                            break;
+                        }
+                        case ENCRYPTED:
+                            if (!(opt.value() instanceof SqlLiteral) && ((SqlLiteral)opt.value()).getTypeName() != BOOLEAN)
+                                throw new IgniteSQLException("Unexpected value for param " + ENCRYPTED + " [" +
+                                    "expected a boolean, but was " + opt.value() + "; " +
+                                    "querySql=\"" + ctx.query() + "\"]", IgniteQueryErrorCode.PARSING);
+
+                            createTblCmd.encrypted(((SqlLiteral)opt.value()).booleanValue());
+                            break;
+
+                        default:
+                            throw new IllegalStateException("Unsupported option " + opt.key());
+                    }
+                }
+            }
+
+            List<SqlColumnDeclaration> colDeclarations = createTblNode.columnList().getList().stream()
+                .filter(SqlColumnDeclaration.class::isInstance)
+                .map(SqlColumnDeclaration.class::cast)
+                .collect(Collectors.toList());
+
+            IgnitePlanner planner = ctx.planner();
+
+            List<ColumnDefinition> cols = new ArrayList<>();
+
+            for (SqlColumnDeclaration col : colDeclarations) {
+                String name = Util.last(col.name.names);

Review comment:
       fixed




-- 
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] korlov42 commented on a change in pull request #8994: IGNITE-13547 CREATE TABLE support

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



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionServiceImpl.java
##########
@@ -596,6 +643,212 @@ private QueryPlan prepareDml(SqlNode sqlNode, PlanningContext ctx) throws Valida
         return new MultiStepDmlPlan(template, queryFieldsMetadata(ctx, igniteRel.getRowType(), null));
     }
 
+    /** */
+    private QueryPlan prepareDdl(SqlNode sqlNode, PlanningContext ctx) {
+        assert sqlNode instanceof SqlDdl : sqlNode == null ? "null" : sqlNode.getClass().getName();
+
+        SqlDdl ddlNode = (SqlDdl)sqlNode;
+
+        if (ddlNode instanceof IgniteSqlCreateTable) {
+            IgniteSqlCreateTable createTblNode = (IgniteSqlCreateTable)ddlNode;
+
+            SqlIdentifier tblFullName = createTblNode.name();
+
+            CreateTableCommand createTblCmd = new CreateTableCommand();
+
+            createTblCmd.schemaName("PUBLIC"); // TODO: get from the context
+            createTblCmd.tableName(Util.last(tblFullName.names));
+            createTblCmd.ifNotExists(createTblNode.ifNotExists());
+            createTblCmd.templateName(QueryUtils.TEMPLATE_PARTITIONED);
+
+            if (createTblNode.createOptionList() != null) {
+                for (SqlNode optNode : createTblNode.createOptionList().getList()) {
+                    IgniteSqlCreateTableOption opt = (IgniteSqlCreateTableOption)optNode;
+
+                    switch (opt.key()) {
+                        case TEMPLATE:
+                            if (!(opt.value() instanceof SqlIdentifier))
+                                throw new IgniteSQLException("Unexpected value for param " + TEMPLATE + " [" +
+                                    "expected an identifier, but was " + opt.value() + "; " +
+                                    "querySql=\"" + ctx.query() + "\"]", IgniteQueryErrorCode.PARSING);
+
+                            createTblCmd.templateName(((SqlIdentifier)opt.value()).names.get(0));
+                            break;
+
+                        case BACKUPS:
+                            if (!(opt.value() instanceof SqlNumericLiteral)
+                                || !((SqlNumericLiteral)opt.value()).isInteger()
+                                || ((SqlLiteral)opt.value()).intValue(true) < 0
+                            )
+                                throw new IgniteSQLException("Unexpected value for param " + BACKUPS + " [" +
+                                    "expected a non-negative integer, but was " + opt.value() + "; " +
+                                    "querySql=\"" + ctx.query() + "\"]", IgniteQueryErrorCode.PARSING);

Review comment:
       fixed




-- 
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 #8994: IGNITE-13547 CREATE TABLE support

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



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionServiceImpl.java
##########
@@ -536,27 +554,21 @@ private QueryPlan prepareSingle(SqlNode sqlNode, PlanningContext ctx) throws Val
 
         ctx.planner().reset();
 
-        switch (sqlNode.getKind()) {
-            case SELECT:
-            case ORDER_BY:
-            case WITH:
-            case VALUES:
-            case UNION:
-                return prepareQuery(sqlNode, ctx);
+       if (sqlNode.getKind().belongsTo(SqlKind.QUERY))

Review comment:
       this Set is not equal with previous, is it ok here?




-- 
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] korlov42 commented on a change in pull request #8994: IGNITE-13547 CREATE TABLE support

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



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionServiceImpl.java
##########
@@ -596,6 +643,212 @@ private QueryPlan prepareDml(SqlNode sqlNode, PlanningContext ctx) throws Valida
         return new MultiStepDmlPlan(template, queryFieldsMetadata(ctx, igniteRel.getRowType(), null));
     }
 
+    /** */
+    private QueryPlan prepareDdl(SqlNode sqlNode, PlanningContext ctx) {
+        assert sqlNode instanceof SqlDdl : sqlNode == null ? "null" : sqlNode.getClass().getName();
+
+        SqlDdl ddlNode = (SqlDdl)sqlNode;
+
+        if (ddlNode instanceof IgniteSqlCreateTable) {
+            IgniteSqlCreateTable createTblNode = (IgniteSqlCreateTable)ddlNode;
+
+            SqlIdentifier tblFullName = createTblNode.name();
+
+            CreateTableCommand createTblCmd = new CreateTableCommand();
+
+            createTblCmd.schemaName("PUBLIC"); // TODO: get from the context

Review comment:
       according to [this](https://ignite.apache.org/docs/latest/sql-reference/ddl#create-table) there is no such option like `schemaName`. A schema could be specified as part of the cache's template only, I suppose




-- 
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] dspavlov closed pull request #8994: IGNITE-13547 CREATE TABLE support

Posted by GitBox <gi...@apache.org>.
dspavlov closed pull request #8994:
URL: https://github.com/apache/ignite/pull/8994


   


-- 
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] dspavlov commented on pull request #8994: IGNITE-13547 CREATE TABLE support

Posted by GitBox <gi...@apache.org>.
dspavlov commented on pull request #8994:
URL: https://github.com/apache/ignite/pull/8994#issuecomment-894335527


   This PR is associated with a resolved ticket. Closing the PR, please reopen it if you still need 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] korlov42 commented on a change in pull request #8994: IGNITE-13547 CREATE TABLE support

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



##########
File path: modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionServiceImpl.java
##########
@@ -596,6 +643,212 @@ private QueryPlan prepareDml(SqlNode sqlNode, PlanningContext ctx) throws Valida
         return new MultiStepDmlPlan(template, queryFieldsMetadata(ctx, igniteRel.getRowType(), null));
     }
 
+    /** */
+    private QueryPlan prepareDdl(SqlNode sqlNode, PlanningContext ctx) {
+        assert sqlNode instanceof SqlDdl : sqlNode == null ? "null" : sqlNode.getClass().getName();
+
+        SqlDdl ddlNode = (SqlDdl)sqlNode;
+
+        if (ddlNode instanceof IgniteSqlCreateTable) {
+            IgniteSqlCreateTable createTblNode = (IgniteSqlCreateTable)ddlNode;
+
+            SqlIdentifier tblFullName = createTblNode.name();
+
+            CreateTableCommand createTblCmd = new CreateTableCommand();
+
+            createTblCmd.schemaName("PUBLIC"); // TODO: get from the context

Review comment:
       regarding the ` [schemaName.]`... It appear that you were right. There is an ability in the old query engine to specify a schema name, so i've added this




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