korlov42 commented on a change in pull request #484:
URL: https://github.com/apache/ignite-3/pull/484#discussion_r763922882



##########
File path: 
modules/schema/src/main/java/org/apache/ignite/internal/schema/definition/builder/PrimaryKeyDefinitionBuilderImpl.java
##########
@@ -72,14 +92,14 @@ public PrimaryKeyDefinition build() {
             throw new IllegalStateException("Primary key column(s) must be 
configured.");
         }
 
-        Set<String> cols = Set.of(columns);
+        Set<String> cols = Set.copyOf(columns);
 
         Set<String> affCols;
 
-        if (ArrayUtils.nullOrEmpty(affinityColumns)) {
+        if (CollectionUtils.nullOrEmpty(affinityColumns)) {
             affCols = cols;
         } else {
-            affCols = Set.of(affinityColumns);
+            affCols = new HashSet<>(affinityColumns);

Review comment:
       `copyOf`?

##########
File path: 
modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ddl/DdlCommandHandler.java
##########
@@ -0,0 +1,281 @@
+/*
+ * 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.exec.ddl;
+
+import static 
org.apache.ignite.internal.schema.configuration.SchemaConfigurationConverter.convert;
+import static org.apache.ignite.internal.util.CollectionUtils.nullOrEmpty;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.ColumnView;
+import org.apache.ignite.configuration.schemas.table.PrimaryKeyView;
+import org.apache.ignite.configuration.schemas.table.TableChange;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.PlanningContext;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.ddl.AbstractDdlCommand;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.ddl.AlterTableAddCommand;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.ddl.AlterTableDropCommand;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.ddl.CreateIndexCommand;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.ddl.CreateTableCommand;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.ddl.DdlCommand;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.ddl.DropIndexCommand;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.ddl.DropTableCommand;
+import org.apache.ignite.internal.schema.definition.TableDefinitionImpl;
+import org.apache.ignite.internal.table.distributed.TableManager;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.apache.ignite.internal.util.Pair;
+import org.apache.ignite.lang.ColumnAlreadyExistsException;
+import org.apache.ignite.lang.ColumnNotFoundException;
+import org.apache.ignite.lang.IgniteException;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+import org.apache.ignite.lang.IndexAlreadyExistsException;
+import org.apache.ignite.lang.LoggerMessageHelper;
+import org.apache.ignite.schema.SchemaBuilders;
+import org.apache.ignite.schema.definition.ColumnDefinition;
+import org.apache.ignite.schema.definition.builder.PrimaryKeyDefinitionBuilder;
+import 
org.apache.ignite.schema.definition.builder.SortedIndexDefinitionBuilder;
+import 
org.apache.ignite.schema.definition.builder.SortedIndexDefinitionBuilder.SortedIndexColumnBuilder;
+
+/** DDL commands handler. */
+public class DdlCommandHandler {
+    private final TableManager tableManager;
+
+    public DdlCommandHandler(TableManager tblManager) {
+        tableManager = tblManager;
+    }
+
+    /** Handles ddl commands. */
+    public void handle(DdlCommand cmd, PlanningContext pctx) throws 
IgniteInternalCheckedException {
+        validateCommand(cmd);
+
+        if (cmd instanceof CreateTableCommand) {
+            handleCreateTable((CreateTableCommand) cmd);
+        } else if (cmd instanceof DropTableCommand) {
+            handleDropTable((DropTableCommand) cmd);
+        } else if (cmd instanceof AlterTableAddCommand) {
+            handleAlterAddColumn((AlterTableAddCommand) cmd);
+        } else if (cmd instanceof AlterTableDropCommand) {
+            handleAlterDropColumn((AlterTableDropCommand) cmd);
+        } else if (cmd instanceof CreateIndexCommand) {
+            handleCreateIndex((CreateIndexCommand) cmd);
+        } else if (cmd instanceof DropIndexCommand) {
+            handleDropIndex((DropIndexCommand) cmd);
+        } else {
+            throw new IgniteInternalCheckedException("Unsupported DDL 
operation ["
+                    + "cmdName=" + (cmd == null ? null : 
cmd.getClass().getSimpleName()) + "; "
+                    + "querySql=\"" + pctx.query() + "\"]");
+        }
+    }
+
+    /** Validate command. */
+    private void validateCommand(DdlCommand cmd) {
+        if (cmd instanceof AbstractDdlCommand) {
+            AbstractDdlCommand cmd0 = (AbstractDdlCommand) cmd;
+
+            if (IgniteUtils.nullOrEmpty(cmd0.tableName())) {
+                throw new IllegalArgumentException("Table name is undefined.");
+            }
+        }
+    }
+
+    /** Handles create table command. */
+    private void handleCreateTable(CreateTableCommand cmd) {
+        PrimaryKeyDefinitionBuilder pkeyDef = SchemaBuilders.primaryKey();
+        pkeyDef.withColumns(cmd.primaryKeyColumns());
+        pkeyDef.withAffinityColumns(cmd.affColumns());
+
+        Consumer<TableChange> tblChanger = tblCh -> {
+            TableChange conv = 
convert(SchemaBuilders.tableBuilder(cmd.schemaName(), cmd.tableName())
+                    .columns(cmd.columns())
+                    .withPrimaryKey(pkeyDef.build()).build(), tblCh);
+
+            if (cmd.partitions() != null) {
+                conv.changePartitions(cmd.partitions());
+            }
+
+            if (cmd.replicas() != null) {
+                conv.changeReplicas(cmd.replicas());
+            }
+        };
+
+        String fullName = TableDefinitionImpl.canonicalName(cmd.schemaName(), 
cmd.tableName());
+
+        if (cmd.ifTableExists()) {
+            tableManager.createTableIfNotExists(fullName, tblChanger);
+        } else {
+            tableManager.createTable(fullName, tblChanger);
+        }
+    }
+
+    /** Handles drop table command. */
+    private void handleDropTable(DropTableCommand cmd) {
+        String fullName = TableDefinitionImpl.canonicalName(cmd.schemaName(), 
cmd.tableName());
+
+        // if (!cmd.ifTableExists()) todo will be implemented after 
IGNITE-15926
+
+        tableManager.dropTable(fullName);
+    }
+
+    /** Handles add column command. */
+    private void handleAlterAddColumn(AlterTableAddCommand cmd) {
+        if (nullOrEmpty(cmd.columns())) {
+            return;
+        }
+
+        String fullName = TableDefinitionImpl.canonicalName(cmd.schemaName(), 
cmd.tableName());
+
+        addColumnInternal(fullName, cmd.columns(), cmd.ifColumnNotExists());
+    }
+
+    /** Handles drop column command. */
+    private void handleAlterDropColumn(AlterTableDropCommand cmd) {
+        if (nullOrEmpty(cmd.columns())) {
+            return;
+        }
+
+        String fullName = TableDefinitionImpl.canonicalName(cmd.schemaName(), 
cmd.tableName());
+
+        dropColumnInternal(fullName, cmd.columns(), cmd.ifColumnExists());
+    }
+
+    /** Handles create index command. */
+    private void handleCreateIndex(CreateIndexCommand cmd) {
+        // Only sorted idx for now.
+        SortedIndexDefinitionBuilder idx = 
SchemaBuilders.sortedIndex(cmd.indexName());
+
+        for (Pair<String, Boolean> idxInfo : cmd.columns()) {
+            SortedIndexColumnBuilder idx0 = 
idx.addIndexColumn(idxInfo.getFirst());
+
+            if (idxInfo.getSecond()) {
+                idx0.desc();
+            }
+
+            idx0.done();
+        }
+
+        String fullName = TableDefinitionImpl.canonicalName(cmd.schemaName(), 
cmd.tableName());
+
+        tableManager.alterTable(fullName, chng -> chng.changeIndices(idxes -> {
+            if (idxes.get(cmd.indexName()) != null) {
+                throw new IndexAlreadyExistsException(cmd.indexName());
+            }
+
+            idxes.create(cmd.indexName(), tableIndexChange -> 
convert(idx.build(), tableIndexChange));
+        }));
+    }
+
+    /** Handles create index command. */
+    private void handleDropIndex(DropIndexCommand cmd) {
+        String fullName = TableDefinitionImpl.canonicalName(cmd.schemaName(), 
cmd.tableName());
+
+        tableManager.alterTable(fullName, chng -> chng.changeIndices(idxes -> {
+            if (idxes.get(cmd.indexName()) == null) {
+                if (!cmd.ifExist()) {
+                    throw new IndexAlreadyExistsException(cmd.indexName());
+                }
+            }
+
+            idxes.delete(cmd.indexName());
+        }));
+    }
+
+    /**
+     * Adds a column according to the column definition.
+     *
+     * @param fullName Table with schema name.
+     * @param colsDef  Columns defenitions.
+     * @param colNotExist Flag indicates exceptionally behavior in case of 
already existing column.
+     */
+    private void addColumnInternal(String fullName, Set<ColumnDefinition> 
colsDef, boolean colNotExist) {
+        tableManager.alterTable(
+                fullName,
+                chng -> chng.changeColumns(cols -> {
+                    Map<String, String> colNamesToOrders = 
columnOrdersToNames(chng.columns());
+
+                    Set<ColumnDefinition> colsDef0;
+
+                    if (!colNotExist) {
+                        colsDef.stream().filter(k -> 
colNamesToOrders.containsKey(k.name())).findAny()
+                                .ifPresent(c -> {
+                                    throw new 
ColumnAlreadyExistsException(c.name());
+                                });
+
+                        colsDef0 = colsDef;
+                    } else {
+                        colsDef0 = colsDef.stream().filter(k -> 
!colNamesToOrders.containsKey(k.name())).collect(Collectors.toSet());
+                    }
+
+                    for (ColumnDefinition colDef : colsDef0) {
+                        cols.create(colDef.name(), colChg -> convert(colDef, 
colChg));
+                    }
+                }));
+    }
+
+    /**
+     * Adds a column according to the column definition.

Review comment:
       wrong javadoc

##########
File path: 
modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/ddl/DdlSqlToCommandConverter.java
##########
@@ -209,6 +210,68 @@ private CreateTableCommand 
convertCreateTable(IgniteSqlCreateTable createTblNode
         return createTblCmd;
     }
 
+    /**
+     * Converts a given IgniteSqlAlterTableAddColumn AST to a 
AlterTableAddCommand.
+     *
+     * @param alterTblNode Root node of the given AST.
+     * @param ctx          Planning context.
+     */
+    private AlterTableAddCommand 
convertAlterTableAdd(IgniteSqlAlterTableAddColumn alterTblNode, PlanningContext 
ctx) {
+        AlterTableAddCommand alterTblCmd = new AlterTableAddCommand();
+
+        alterTblCmd.schemaName(deriveSchemaName(alterTblNode.name(), ctx));
+        alterTblCmd.tableName(deriveObjectName(alterTblNode.name(), ctx, 
"table name"));
+        alterTblCmd.ifTableExists(alterTblNode.ifExists());
+        alterTblCmd.ifColumnNotExists(alterTblNode.ifNotExistsColumn());
+
+        IgniteTypeFactory typeFactory = ctx.typeFactory();
+
+        Set<ColumnDefinition> cols = new 
HashSet<>(alterTblNode.columns().size());
+
+        for (SqlNode colNode : alterTblNode.columns()) {
+            assert colNode instanceof SqlColumnDeclaration : 
colNode.getClass();
+
+            SqlColumnDeclaration col = (SqlColumnDeclaration) colNode;
+
+            assert col.name.isSimple();
+            assert col.expression == null : "Unexpected column default value" 
+ col.expression;
+
+            String name = col.name.getSimple();
+            RelDataType type = ctx.planner().convert(col.dataType);
+
+            ColumnDefinitionBuilder col0 = SchemaBuilders.column(name, 
typeFactory.columnType(type))

Review comment:
       at this step we should create 
`org.apache.ignite.internal.processors.query.calcite.prepare.ddl.ColumnDefinition`

##########
File path: 
modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/ddl/AbstractDdlCommand.java
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.prepare.ddl;
+
+/**
+ * ALTER TABLE ... ADD/DROP COLUMN statement.
+ */
+public abstract class AbstractDdlCommand implements DdlCommand {

Review comment:
       CREATE USER is a DDL command and doesn't have a table reference

##########
File path: 
modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ddl/DdlCommandHandler.java
##########
@@ -0,0 +1,281 @@
+/*
+ * 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.exec.ddl;
+
+import static 
org.apache.ignite.internal.schema.configuration.SchemaConfigurationConverter.convert;
+import static org.apache.ignite.internal.util.CollectionUtils.nullOrEmpty;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.ColumnView;
+import org.apache.ignite.configuration.schemas.table.PrimaryKeyView;
+import org.apache.ignite.configuration.schemas.table.TableChange;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.PlanningContext;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.ddl.AbstractDdlCommand;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.ddl.AlterTableAddCommand;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.ddl.AlterTableDropCommand;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.ddl.CreateIndexCommand;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.ddl.CreateTableCommand;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.ddl.DdlCommand;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.ddl.DropIndexCommand;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.ddl.DropTableCommand;
+import org.apache.ignite.internal.schema.definition.TableDefinitionImpl;
+import org.apache.ignite.internal.table.distributed.TableManager;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.apache.ignite.internal.util.Pair;
+import org.apache.ignite.lang.ColumnAlreadyExistsException;
+import org.apache.ignite.lang.ColumnNotFoundException;
+import org.apache.ignite.lang.IgniteException;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+import org.apache.ignite.lang.IndexAlreadyExistsException;
+import org.apache.ignite.lang.LoggerMessageHelper;
+import org.apache.ignite.schema.SchemaBuilders;
+import org.apache.ignite.schema.definition.ColumnDefinition;
+import org.apache.ignite.schema.definition.builder.PrimaryKeyDefinitionBuilder;
+import 
org.apache.ignite.schema.definition.builder.SortedIndexDefinitionBuilder;
+import 
org.apache.ignite.schema.definition.builder.SortedIndexDefinitionBuilder.SortedIndexColumnBuilder;
+
+/** DDL commands handler. */
+public class DdlCommandHandler {
+    private final TableManager tableManager;
+
+    public DdlCommandHandler(TableManager tblManager) {
+        tableManager = tblManager;
+    }
+
+    /** Handles ddl commands. */
+    public void handle(DdlCommand cmd, PlanningContext pctx) throws 
IgniteInternalCheckedException {
+        validateCommand(cmd);
+
+        if (cmd instanceof CreateTableCommand) {
+            handleCreateTable((CreateTableCommand) cmd);
+        } else if (cmd instanceof DropTableCommand) {
+            handleDropTable((DropTableCommand) cmd);
+        } else if (cmd instanceof AlterTableAddCommand) {
+            handleAlterAddColumn((AlterTableAddCommand) cmd);
+        } else if (cmd instanceof AlterTableDropCommand) {
+            handleAlterDropColumn((AlterTableDropCommand) cmd);
+        } else if (cmd instanceof CreateIndexCommand) {
+            handleCreateIndex((CreateIndexCommand) cmd);
+        } else if (cmd instanceof DropIndexCommand) {
+            handleDropIndex((DropIndexCommand) cmd);
+        } else {
+            throw new IgniteInternalCheckedException("Unsupported DDL 
operation ["
+                    + "cmdName=" + (cmd == null ? null : 
cmd.getClass().getSimpleName()) + "; "
+                    + "querySql=\"" + pctx.query() + "\"]");
+        }
+    }
+
+    /** Validate command. */
+    private void validateCommand(DdlCommand cmd) {
+        if (cmd instanceof AbstractDdlCommand) {
+            AbstractDdlCommand cmd0 = (AbstractDdlCommand) cmd;
+
+            if (IgniteUtils.nullOrEmpty(cmd0.tableName())) {
+                throw new IllegalArgumentException("Table name is undefined.");
+            }
+        }
+    }
+
+    /** Handles create table command. */
+    private void handleCreateTable(CreateTableCommand cmd) {
+        PrimaryKeyDefinitionBuilder pkeyDef = SchemaBuilders.primaryKey();
+        pkeyDef.withColumns(cmd.primaryKeyColumns());
+        pkeyDef.withAffinityColumns(cmd.affColumns());
+
+        Consumer<TableChange> tblChanger = tblCh -> {
+            TableChange conv = 
convert(SchemaBuilders.tableBuilder(cmd.schemaName(), cmd.tableName())
+                    .columns(cmd.columns())
+                    .withPrimaryKey(pkeyDef.build()).build(), tblCh);
+
+            if (cmd.partitions() != null) {
+                conv.changePartitions(cmd.partitions());
+            }
+
+            if (cmd.replicas() != null) {
+                conv.changeReplicas(cmd.replicas());
+            }
+        };
+
+        String fullName = TableDefinitionImpl.canonicalName(cmd.schemaName(), 
cmd.tableName());
+
+        if (cmd.ifTableExists()) {
+            tableManager.createTableIfNotExists(fullName, tblChanger);
+        } else {
+            tableManager.createTable(fullName, tblChanger);
+        }
+    }
+
+    /** Handles drop table command. */
+    private void handleDropTable(DropTableCommand cmd) {
+        String fullName = TableDefinitionImpl.canonicalName(cmd.schemaName(), 
cmd.tableName());
+
+        // if (!cmd.ifTableExists()) todo will be implemented after 
IGNITE-15926
+
+        tableManager.dropTable(fullName);
+    }
+
+    /** Handles add column command. */
+    private void handleAlterAddColumn(AlterTableAddCommand cmd) {
+        if (nullOrEmpty(cmd.columns())) {
+            return;
+        }
+
+        String fullName = TableDefinitionImpl.canonicalName(cmd.schemaName(), 
cmd.tableName());
+
+        addColumnInternal(fullName, cmd.columns(), cmd.ifColumnNotExists());
+    }
+
+    /** Handles drop column command. */
+    private void handleAlterDropColumn(AlterTableDropCommand cmd) {
+        if (nullOrEmpty(cmd.columns())) {
+            return;
+        }
+
+        String fullName = TableDefinitionImpl.canonicalName(cmd.schemaName(), 
cmd.tableName());
+
+        dropColumnInternal(fullName, cmd.columns(), cmd.ifColumnExists());
+    }
+
+    /** Handles create index command. */
+    private void handleCreateIndex(CreateIndexCommand cmd) {
+        // Only sorted idx for now.
+        SortedIndexDefinitionBuilder idx = 
SchemaBuilders.sortedIndex(cmd.indexName());
+
+        for (Pair<String, Boolean> idxInfo : cmd.columns()) {
+            SortedIndexColumnBuilder idx0 = 
idx.addIndexColumn(idxInfo.getFirst());
+
+            if (idxInfo.getSecond()) {
+                idx0.desc();
+            }
+
+            idx0.done();
+        }
+
+        String fullName = TableDefinitionImpl.canonicalName(cmd.schemaName(), 
cmd.tableName());
+
+        tableManager.alterTable(fullName, chng -> chng.changeIndices(idxes -> {
+            if (idxes.get(cmd.indexName()) != null) {
+                throw new IndexAlreadyExistsException(cmd.indexName());
+            }
+
+            idxes.create(cmd.indexName(), tableIndexChange -> 
convert(idx.build(), tableIndexChange));
+        }));
+    }
+
+    /** Handles create index command. */
+    private void handleDropIndex(DropIndexCommand cmd) {
+        String fullName = TableDefinitionImpl.canonicalName(cmd.schemaName(), 
cmd.tableName());
+
+        tableManager.alterTable(fullName, chng -> chng.changeIndices(idxes -> {
+            if (idxes.get(cmd.indexName()) == null) {
+                if (!cmd.ifExist()) {
+                    throw new IndexAlreadyExistsException(cmd.indexName());
+                }
+            }
+
+            idxes.delete(cmd.indexName());
+        }));
+    }
+
+    /**
+     * Adds a column according to the column definition.
+     *
+     * @param fullName Table with schema name.
+     * @param colsDef  Columns defenitions.
+     * @param colNotExist Flag indicates exceptionally behavior in case of 
already existing column.
+     */
+    private void addColumnInternal(String fullName, Set<ColumnDefinition> 
colsDef, boolean colNotExist) {
+        tableManager.alterTable(
+                fullName,
+                chng -> chng.changeColumns(cols -> {
+                    Map<String, String> colNamesToOrders = 
columnOrdersToNames(chng.columns());
+
+                    Set<ColumnDefinition> colsDef0;

Review comment:
       we need to preserve column order

##########
File path: 
modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ddl/DdlCommandHandler.java
##########
@@ -0,0 +1,281 @@
+/*
+ * 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.exec.ddl;
+
+import static 
org.apache.ignite.internal.schema.configuration.SchemaConfigurationConverter.convert;
+import static org.apache.ignite.internal.util.CollectionUtils.nullOrEmpty;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.ColumnView;
+import org.apache.ignite.configuration.schemas.table.PrimaryKeyView;
+import org.apache.ignite.configuration.schemas.table.TableChange;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.PlanningContext;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.ddl.AbstractDdlCommand;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.ddl.AlterTableAddCommand;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.ddl.AlterTableDropCommand;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.ddl.CreateIndexCommand;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.ddl.CreateTableCommand;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.ddl.DdlCommand;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.ddl.DropIndexCommand;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.ddl.DropTableCommand;
+import org.apache.ignite.internal.schema.definition.TableDefinitionImpl;
+import org.apache.ignite.internal.table.distributed.TableManager;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.apache.ignite.internal.util.Pair;
+import org.apache.ignite.lang.ColumnAlreadyExistsException;
+import org.apache.ignite.lang.ColumnNotFoundException;
+import org.apache.ignite.lang.IgniteException;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+import org.apache.ignite.lang.IndexAlreadyExistsException;
+import org.apache.ignite.lang.LoggerMessageHelper;
+import org.apache.ignite.schema.SchemaBuilders;
+import org.apache.ignite.schema.definition.ColumnDefinition;
+import org.apache.ignite.schema.definition.builder.PrimaryKeyDefinitionBuilder;
+import 
org.apache.ignite.schema.definition.builder.SortedIndexDefinitionBuilder;
+import 
org.apache.ignite.schema.definition.builder.SortedIndexDefinitionBuilder.SortedIndexColumnBuilder;
+
+/** DDL commands handler. */
+public class DdlCommandHandler {
+    private final TableManager tableManager;
+
+    public DdlCommandHandler(TableManager tblManager) {
+        tableManager = tblManager;
+    }
+
+    /** Handles ddl commands. */
+    public void handle(DdlCommand cmd, PlanningContext pctx) throws 
IgniteInternalCheckedException {
+        validateCommand(cmd);
+
+        if (cmd instanceof CreateTableCommand) {
+            handleCreateTable((CreateTableCommand) cmd);
+        } else if (cmd instanceof DropTableCommand) {
+            handleDropTable((DropTableCommand) cmd);
+        } else if (cmd instanceof AlterTableAddCommand) {
+            handleAlterAddColumn((AlterTableAddCommand) cmd);
+        } else if (cmd instanceof AlterTableDropCommand) {
+            handleAlterDropColumn((AlterTableDropCommand) cmd);
+        } else if (cmd instanceof CreateIndexCommand) {
+            handleCreateIndex((CreateIndexCommand) cmd);
+        } else if (cmd instanceof DropIndexCommand) {
+            handleDropIndex((DropIndexCommand) cmd);
+        } else {
+            throw new IgniteInternalCheckedException("Unsupported DDL 
operation ["
+                    + "cmdName=" + (cmd == null ? null : 
cmd.getClass().getSimpleName()) + "; "
+                    + "querySql=\"" + pctx.query() + "\"]");
+        }
+    }
+
+    /** Validate command. */
+    private void validateCommand(DdlCommand cmd) {
+        if (cmd instanceof AbstractDdlCommand) {
+            AbstractDdlCommand cmd0 = (AbstractDdlCommand) cmd;
+
+            if (IgniteUtils.nullOrEmpty(cmd0.tableName())) {
+                throw new IllegalArgumentException("Table name is undefined.");
+            }
+        }
+    }
+
+    /** Handles create table command. */
+    private void handleCreateTable(CreateTableCommand cmd) {
+        PrimaryKeyDefinitionBuilder pkeyDef = SchemaBuilders.primaryKey();
+        pkeyDef.withColumns(cmd.primaryKeyColumns());
+        pkeyDef.withAffinityColumns(cmd.affColumns());
+
+        Consumer<TableChange> tblChanger = tblCh -> {
+            TableChange conv = 
convert(SchemaBuilders.tableBuilder(cmd.schemaName(), cmd.tableName())
+                    .columns(cmd.columns())
+                    .withPrimaryKey(pkeyDef.build()).build(), tblCh);
+
+            if (cmd.partitions() != null) {
+                conv.changePartitions(cmd.partitions());
+            }
+
+            if (cmd.replicas() != null) {
+                conv.changeReplicas(cmd.replicas());
+            }
+        };
+
+        String fullName = TableDefinitionImpl.canonicalName(cmd.schemaName(), 
cmd.tableName());
+
+        if (cmd.ifTableExists()) {
+            tableManager.createTableIfNotExists(fullName, tblChanger);
+        } else {
+            tableManager.createTable(fullName, tblChanger);
+        }
+    }
+
+    /** Handles drop table command. */
+    private void handleDropTable(DropTableCommand cmd) {
+        String fullName = TableDefinitionImpl.canonicalName(cmd.schemaName(), 
cmd.tableName());
+
+        // if (!cmd.ifTableExists()) todo will be implemented after 
IGNITE-15926
+
+        tableManager.dropTable(fullName);
+    }
+
+    /** Handles add column command. */
+    private void handleAlterAddColumn(AlterTableAddCommand cmd) {
+        if (nullOrEmpty(cmd.columns())) {
+            return;
+        }
+
+        String fullName = TableDefinitionImpl.canonicalName(cmd.schemaName(), 
cmd.tableName());
+
+        addColumnInternal(fullName, cmd.columns(), cmd.ifColumnNotExists());
+    }
+
+    /** Handles drop column command. */
+    private void handleAlterDropColumn(AlterTableDropCommand cmd) {
+        if (nullOrEmpty(cmd.columns())) {
+            return;
+        }
+
+        String fullName = TableDefinitionImpl.canonicalName(cmd.schemaName(), 
cmd.tableName());
+
+        dropColumnInternal(fullName, cmd.columns(), cmd.ifColumnExists());
+    }
+
+    /** Handles create index command. */
+    private void handleCreateIndex(CreateIndexCommand cmd) {
+        // Only sorted idx for now.
+        SortedIndexDefinitionBuilder idx = 
SchemaBuilders.sortedIndex(cmd.indexName());
+
+        for (Pair<String, Boolean> idxInfo : cmd.columns()) {
+            SortedIndexColumnBuilder idx0 = 
idx.addIndexColumn(idxInfo.getFirst());
+
+            if (idxInfo.getSecond()) {
+                idx0.desc();
+            }
+
+            idx0.done();
+        }
+
+        String fullName = TableDefinitionImpl.canonicalName(cmd.schemaName(), 
cmd.tableName());
+
+        tableManager.alterTable(fullName, chng -> chng.changeIndices(idxes -> {
+            if (idxes.get(cmd.indexName()) != null) {
+                throw new IndexAlreadyExistsException(cmd.indexName());
+            }
+
+            idxes.create(cmd.indexName(), tableIndexChange -> 
convert(idx.build(), tableIndexChange));
+        }));
+    }
+
+    /** Handles create index command. */
+    private void handleDropIndex(DropIndexCommand cmd) {
+        String fullName = TableDefinitionImpl.canonicalName(cmd.schemaName(), 
cmd.tableName());
+
+        tableManager.alterTable(fullName, chng -> chng.changeIndices(idxes -> {
+            if (idxes.get(cmd.indexName()) == null) {
+                if (!cmd.ifExist()) {
+                    throw new IndexAlreadyExistsException(cmd.indexName());
+                }
+            }
+
+            idxes.delete(cmd.indexName());
+        }));
+    }
+
+    /**
+     * Adds a column according to the column definition.
+     *
+     * @param fullName Table with schema name.
+     * @param colsDef  Columns defenitions.
+     * @param colNotExist Flag indicates exceptionally behavior in case of 
already existing column.
+     */
+    private void addColumnInternal(String fullName, Set<ColumnDefinition> 
colsDef, boolean colNotExist) {
+        tableManager.alterTable(
+                fullName,
+                chng -> chng.changeColumns(cols -> {
+                    Map<String, String> colNamesToOrders = 
columnOrdersToNames(chng.columns());
+
+                    Set<ColumnDefinition> colsDef0;
+
+                    if (!colNotExist) {
+                        colsDef.stream().filter(k -> 
colNamesToOrders.containsKey(k.name())).findAny()
+                                .ifPresent(c -> {
+                                    throw new 
ColumnAlreadyExistsException(c.name());
+                                });
+
+                        colsDef0 = colsDef;
+                    } else {
+                        colsDef0 = colsDef.stream().filter(k -> 
!colNamesToOrders.containsKey(k.name())).collect(Collectors.toSet());
+                    }
+
+                    for (ColumnDefinition colDef : colsDef0) {
+                        cols.create(colDef.name(), colChg -> convert(colDef, 
colChg));
+                    }
+                }));
+    }
+
+    /**
+     * Adds a column according to the column definition.
+     *
+     * @param fullName Table with schema name.
+     * @param colNames  Columns defenitions.
+     * @param colNotExist Flag indicates exceptionally behavior in case of 
already existing column.
+     */
+    private void dropColumnInternal(String fullName, Set<String> colNames, 
boolean colExist) {
+        tableManager.alterTable(

Review comment:
       does this call respect `IF EXISTS` flag? I mean in the case `ALTER TABLE 
IF EXISTS ...`

##########
File path: modules/calcite/src/main/codegen/includes/parserImpls.ftl
##########
@@ -184,6 +220,23 @@ SqlDrop SqlDropTable(Span s, boolean replace) :
     }
 }
 
+SqlDrop SqlDropIndex(Span s, boolean replace) :
+{
+    final boolean ifExists;
+    final SqlIdentifier idxId;
+    final SqlIdentifier tblId;
+}
+{
+    <INDEX> 
+    ifExists = IfExistsOpt()
+    idxId = SimpleIdentifier()
+    <ON>

Review comment:
       ON clause is not the part of DROP command

##########
File path: 
modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ddl/DdlCommandHandler.java
##########
@@ -0,0 +1,281 @@
+/*
+ * 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.exec.ddl;
+
+import static 
org.apache.ignite.internal.schema.configuration.SchemaConfigurationConverter.convert;
+import static org.apache.ignite.internal.util.CollectionUtils.nullOrEmpty;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.ColumnView;
+import org.apache.ignite.configuration.schemas.table.PrimaryKeyView;
+import org.apache.ignite.configuration.schemas.table.TableChange;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.PlanningContext;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.ddl.AbstractDdlCommand;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.ddl.AlterTableAddCommand;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.ddl.AlterTableDropCommand;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.ddl.CreateIndexCommand;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.ddl.CreateTableCommand;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.ddl.DdlCommand;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.ddl.DropIndexCommand;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.ddl.DropTableCommand;
+import org.apache.ignite.internal.schema.definition.TableDefinitionImpl;
+import org.apache.ignite.internal.table.distributed.TableManager;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.apache.ignite.internal.util.Pair;
+import org.apache.ignite.lang.ColumnAlreadyExistsException;
+import org.apache.ignite.lang.ColumnNotFoundException;
+import org.apache.ignite.lang.IgniteException;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+import org.apache.ignite.lang.IndexAlreadyExistsException;
+import org.apache.ignite.lang.LoggerMessageHelper;
+import org.apache.ignite.schema.SchemaBuilders;
+import org.apache.ignite.schema.definition.ColumnDefinition;
+import org.apache.ignite.schema.definition.builder.PrimaryKeyDefinitionBuilder;
+import 
org.apache.ignite.schema.definition.builder.SortedIndexDefinitionBuilder;
+import 
org.apache.ignite.schema.definition.builder.SortedIndexDefinitionBuilder.SortedIndexColumnBuilder;
+
+/** DDL commands handler. */
+public class DdlCommandHandler {
+    private final TableManager tableManager;
+
+    public DdlCommandHandler(TableManager tblManager) {
+        tableManager = tblManager;
+    }
+
+    /** Handles ddl commands. */
+    public void handle(DdlCommand cmd, PlanningContext pctx) throws 
IgniteInternalCheckedException {
+        validateCommand(cmd);
+
+        if (cmd instanceof CreateTableCommand) {
+            handleCreateTable((CreateTableCommand) cmd);
+        } else if (cmd instanceof DropTableCommand) {
+            handleDropTable((DropTableCommand) cmd);
+        } else if (cmd instanceof AlterTableAddCommand) {
+            handleAlterAddColumn((AlterTableAddCommand) cmd);
+        } else if (cmd instanceof AlterTableDropCommand) {
+            handleAlterDropColumn((AlterTableDropCommand) cmd);
+        } else if (cmd instanceof CreateIndexCommand) {
+            handleCreateIndex((CreateIndexCommand) cmd);
+        } else if (cmd instanceof DropIndexCommand) {
+            handleDropIndex((DropIndexCommand) cmd);
+        } else {
+            throw new IgniteInternalCheckedException("Unsupported DDL 
operation ["
+                    + "cmdName=" + (cmd == null ? null : 
cmd.getClass().getSimpleName()) + "; "
+                    + "querySql=\"" + pctx.query() + "\"]");
+        }
+    }
+
+    /** Validate command. */
+    private void validateCommand(DdlCommand cmd) {
+        if (cmd instanceof AbstractDdlCommand) {
+            AbstractDdlCommand cmd0 = (AbstractDdlCommand) cmd;
+
+            if (IgniteUtils.nullOrEmpty(cmd0.tableName())) {
+                throw new IllegalArgumentException("Table name is undefined.");
+            }
+        }
+    }
+
+    /** Handles create table command. */
+    private void handleCreateTable(CreateTableCommand cmd) {
+        PrimaryKeyDefinitionBuilder pkeyDef = SchemaBuilders.primaryKey();
+        pkeyDef.withColumns(cmd.primaryKeyColumns());
+        pkeyDef.withAffinityColumns(cmd.affColumns());
+
+        Consumer<TableChange> tblChanger = tblCh -> {
+            TableChange conv = 
convert(SchemaBuilders.tableBuilder(cmd.schemaName(), cmd.tableName())
+                    .columns(cmd.columns())
+                    .withPrimaryKey(pkeyDef.build()).build(), tblCh);
+
+            if (cmd.partitions() != null) {
+                conv.changePartitions(cmd.partitions());
+            }
+
+            if (cmd.replicas() != null) {
+                conv.changeReplicas(cmd.replicas());
+            }
+        };
+
+        String fullName = TableDefinitionImpl.canonicalName(cmd.schemaName(), 
cmd.tableName());
+
+        if (cmd.ifTableExists()) {
+            tableManager.createTableIfNotExists(fullName, tblChanger);
+        } else {
+            tableManager.createTable(fullName, tblChanger);
+        }
+    }
+
+    /** Handles drop table command. */
+    private void handleDropTable(DropTableCommand cmd) {
+        String fullName = TableDefinitionImpl.canonicalName(cmd.schemaName(), 
cmd.tableName());
+
+        // if (!cmd.ifTableExists()) todo will be implemented after 
IGNITE-15926
+
+        tableManager.dropTable(fullName);
+    }
+
+    /** Handles add column command. */
+    private void handleAlterAddColumn(AlterTableAddCommand cmd) {
+        if (nullOrEmpty(cmd.columns())) {
+            return;
+        }
+
+        String fullName = TableDefinitionImpl.canonicalName(cmd.schemaName(), 
cmd.tableName());
+
+        addColumnInternal(fullName, cmd.columns(), cmd.ifColumnNotExists());
+    }
+
+    /** Handles drop column command. */
+    private void handleAlterDropColumn(AlterTableDropCommand cmd) {
+        if (nullOrEmpty(cmd.columns())) {
+            return;
+        }
+
+        String fullName = TableDefinitionImpl.canonicalName(cmd.schemaName(), 
cmd.tableName());
+
+        dropColumnInternal(fullName, cmd.columns(), cmd.ifColumnExists());
+    }
+
+    /** Handles create index command. */
+    private void handleCreateIndex(CreateIndexCommand cmd) {
+        // Only sorted idx for now.
+        SortedIndexDefinitionBuilder idx = 
SchemaBuilders.sortedIndex(cmd.indexName());
+
+        for (Pair<String, Boolean> idxInfo : cmd.columns()) {
+            SortedIndexColumnBuilder idx0 = 
idx.addIndexColumn(idxInfo.getFirst());
+
+            if (idxInfo.getSecond()) {
+                idx0.desc();
+            }
+
+            idx0.done();
+        }
+
+        String fullName = TableDefinitionImpl.canonicalName(cmd.schemaName(), 
cmd.tableName());
+
+        tableManager.alterTable(fullName, chng -> chng.changeIndices(idxes -> {
+            if (idxes.get(cmd.indexName()) != null) {
+                throw new IndexAlreadyExistsException(cmd.indexName());
+            }
+
+            idxes.create(cmd.indexName(), tableIndexChange -> 
convert(idx.build(), tableIndexChange));
+        }));
+    }
+
+    /** Handles create index command. */

Review comment:
       wrong javadoc

##########
File path: 
modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/ddl/AlterTableAddCommand.java
##########
@@ -0,0 +1,49 @@
+/*
+ * 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.prepare.ddl;
+
+import java.util.Set;
+import org.apache.ignite.schema.definition.ColumnDefinition;
+
+/**
+ * ALTER TABLE ... ADD COLUMN statement.
+ */
+@SuppressWarnings("AssignmentOrReturnOfFieldWithMutableType")
+public class AlterTableAddCommand extends AbstractTableDdlCommand {
+    /** Quietly ignore this command if column already exists. */
+    private boolean ifColumnNotExists;
+
+    /** Columns. */
+    private Set<ColumnDefinition> cols;
+
+    public Set<ColumnDefinition> columns() {
+        return cols;
+    }
+
+    public void columns(Set<ColumnDefinition> cols) {

Review comment:
       We have to preserve order of columns to be added

##########
File path: 
modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ddl/DdlCommandHandler.java
##########
@@ -0,0 +1,281 @@
+/*
+ * 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.exec.ddl;
+
+import static 
org.apache.ignite.internal.schema.configuration.SchemaConfigurationConverter.convert;
+import static org.apache.ignite.internal.util.CollectionUtils.nullOrEmpty;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.ColumnView;
+import org.apache.ignite.configuration.schemas.table.PrimaryKeyView;
+import org.apache.ignite.configuration.schemas.table.TableChange;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.PlanningContext;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.ddl.AbstractDdlCommand;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.ddl.AlterTableAddCommand;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.ddl.AlterTableDropCommand;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.ddl.CreateIndexCommand;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.ddl.CreateTableCommand;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.ddl.DdlCommand;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.ddl.DropIndexCommand;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.ddl.DropTableCommand;
+import org.apache.ignite.internal.schema.definition.TableDefinitionImpl;
+import org.apache.ignite.internal.table.distributed.TableManager;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.apache.ignite.internal.util.Pair;
+import org.apache.ignite.lang.ColumnAlreadyExistsException;
+import org.apache.ignite.lang.ColumnNotFoundException;
+import org.apache.ignite.lang.IgniteException;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+import org.apache.ignite.lang.IndexAlreadyExistsException;
+import org.apache.ignite.lang.LoggerMessageHelper;
+import org.apache.ignite.schema.SchemaBuilders;
+import org.apache.ignite.schema.definition.ColumnDefinition;
+import org.apache.ignite.schema.definition.builder.PrimaryKeyDefinitionBuilder;
+import 
org.apache.ignite.schema.definition.builder.SortedIndexDefinitionBuilder;
+import 
org.apache.ignite.schema.definition.builder.SortedIndexDefinitionBuilder.SortedIndexColumnBuilder;
+
+/** DDL commands handler. */
+public class DdlCommandHandler {
+    private final TableManager tableManager;
+
+    public DdlCommandHandler(TableManager tblManager) {
+        tableManager = tblManager;
+    }
+
+    /** Handles ddl commands. */
+    public void handle(DdlCommand cmd, PlanningContext pctx) throws 
IgniteInternalCheckedException {
+        validateCommand(cmd);
+
+        if (cmd instanceof CreateTableCommand) {
+            handleCreateTable((CreateTableCommand) cmd);
+        } else if (cmd instanceof DropTableCommand) {
+            handleDropTable((DropTableCommand) cmd);
+        } else if (cmd instanceof AlterTableAddCommand) {
+            handleAlterAddColumn((AlterTableAddCommand) cmd);
+        } else if (cmd instanceof AlterTableDropCommand) {
+            handleAlterDropColumn((AlterTableDropCommand) cmd);
+        } else if (cmd instanceof CreateIndexCommand) {
+            handleCreateIndex((CreateIndexCommand) cmd);
+        } else if (cmd instanceof DropIndexCommand) {
+            handleDropIndex((DropIndexCommand) cmd);
+        } else {
+            throw new IgniteInternalCheckedException("Unsupported DDL 
operation ["
+                    + "cmdName=" + (cmd == null ? null : 
cmd.getClass().getSimpleName()) + "; "
+                    + "querySql=\"" + pctx.query() + "\"]");
+        }
+    }
+
+    /** Validate command. */
+    private void validateCommand(DdlCommand cmd) {
+        if (cmd instanceof AbstractDdlCommand) {
+            AbstractDdlCommand cmd0 = (AbstractDdlCommand) cmd;
+
+            if (IgniteUtils.nullOrEmpty(cmd0.tableName())) {
+                throw new IllegalArgumentException("Table name is undefined.");
+            }
+        }
+    }
+
+    /** Handles create table command. */
+    private void handleCreateTable(CreateTableCommand cmd) {
+        PrimaryKeyDefinitionBuilder pkeyDef = SchemaBuilders.primaryKey();
+        pkeyDef.withColumns(cmd.primaryKeyColumns());
+        pkeyDef.withAffinityColumns(cmd.affColumns());
+
+        Consumer<TableChange> tblChanger = tblCh -> {
+            TableChange conv = 
convert(SchemaBuilders.tableBuilder(cmd.schemaName(), cmd.tableName())
+                    .columns(cmd.columns())
+                    .withPrimaryKey(pkeyDef.build()).build(), tblCh);
+
+            if (cmd.partitions() != null) {
+                conv.changePartitions(cmd.partitions());
+            }
+
+            if (cmd.replicas() != null) {
+                conv.changeReplicas(cmd.replicas());
+            }
+        };
+
+        String fullName = TableDefinitionImpl.canonicalName(cmd.schemaName(), 
cmd.tableName());
+
+        if (cmd.ifTableExists()) {
+            tableManager.createTableIfNotExists(fullName, tblChanger);
+        } else {
+            tableManager.createTable(fullName, tblChanger);
+        }
+    }
+
+    /** Handles drop table command. */
+    private void handleDropTable(DropTableCommand cmd) {
+        String fullName = TableDefinitionImpl.canonicalName(cmd.schemaName(), 
cmd.tableName());
+
+        // if (!cmd.ifTableExists()) todo will be implemented after 
IGNITE-15926
+
+        tableManager.dropTable(fullName);
+    }
+
+    /** Handles add column command. */
+    private void handleAlterAddColumn(AlterTableAddCommand cmd) {
+        if (nullOrEmpty(cmd.columns())) {
+            return;
+        }
+
+        String fullName = TableDefinitionImpl.canonicalName(cmd.schemaName(), 
cmd.tableName());
+
+        addColumnInternal(fullName, cmd.columns(), cmd.ifColumnNotExists());
+    }
+
+    /** Handles drop column command. */
+    private void handleAlterDropColumn(AlterTableDropCommand cmd) {
+        if (nullOrEmpty(cmd.columns())) {
+            return;
+        }
+
+        String fullName = TableDefinitionImpl.canonicalName(cmd.schemaName(), 
cmd.tableName());
+
+        dropColumnInternal(fullName, cmd.columns(), cmd.ifColumnExists());
+    }
+
+    /** Handles create index command. */
+    private void handleCreateIndex(CreateIndexCommand cmd) {
+        // Only sorted idx for now.
+        SortedIndexDefinitionBuilder idx = 
SchemaBuilders.sortedIndex(cmd.indexName());
+
+        for (Pair<String, Boolean> idxInfo : cmd.columns()) {
+            SortedIndexColumnBuilder idx0 = 
idx.addIndexColumn(idxInfo.getFirst());
+
+            if (idxInfo.getSecond()) {
+                idx0.desc();
+            }
+
+            idx0.done();
+        }
+
+        String fullName = TableDefinitionImpl.canonicalName(cmd.schemaName(), 
cmd.tableName());
+
+        tableManager.alterTable(fullName, chng -> chng.changeIndices(idxes -> {
+            if (idxes.get(cmd.indexName()) != null) {
+                throw new IndexAlreadyExistsException(cmd.indexName());

Review comment:
       why do we disrespect IF NOT EXISTS flag?

##########
File path: 
modules/api/src/main/java/org/apache/ignite/configuration/schemas/table/TableConfigurationSchema.java
##########
@@ -43,7 +43,8 @@
     @Min(0)
     @Max(65000)
     @Value(hasDefault = true)
-    public int partitions = 1024;
+    // todo: https://issues.apache.org/jira/browse/IGNITE-16065, with prewious 
default it was imposible to start multi nodes cluster.

Review comment:
       ```suggestion
       // todo: https://issues.apache.org/jira/browse/IGNITE-16065, with 
previous default it was impossible to start multi nodes cluster.
   ```

##########
File path: 
modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/ddl/DdlSqlToCommandConverter.java
##########
@@ -209,6 +210,68 @@ private CreateTableCommand 
convertCreateTable(IgniteSqlCreateTable createTblNode
         return createTblCmd;
     }
 
+    /**
+     * Converts a given IgniteSqlAlterTableAddColumn AST to a 
AlterTableAddCommand.
+     *
+     * @param alterTblNode Root node of the given AST.
+     * @param ctx          Planning context.
+     */
+    private AlterTableAddCommand 
convertAlterTableAdd(IgniteSqlAlterTableAddColumn alterTblNode, PlanningContext 
ctx) {
+        AlterTableAddCommand alterTblCmd = new AlterTableAddCommand();
+
+        alterTblCmd.schemaName(deriveSchemaName(alterTblNode.name(), ctx));
+        alterTblCmd.tableName(deriveObjectName(alterTblNode.name(), ctx, 
"table name"));
+        alterTblCmd.ifTableExists(alterTblNode.ifExists());
+        alterTblCmd.ifColumnNotExists(alterTblNode.ifNotExistsColumn());
+
+        IgniteTypeFactory typeFactory = ctx.typeFactory();
+
+        Set<ColumnDefinition> cols = new 
HashSet<>(alterTblNode.columns().size());

Review comment:
       order should be preserved

##########
File path: 
modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/type/IgniteTypeFactory.java
##########
@@ -136,8 +140,35 @@ public Type getJavaClass(RelDataType type) {
     }
 
     /**
-     * Get result type by field data type.
-     * TODO Documentation https://issues.apache.org/jira/browse/IGNITE-15859
+     * Gets ColumnType type for given class.
+     *
+     * @param relType Rel type.
+     * @return ColumnType type or null.
+     */
+    public ColumnType columnType(RelDataType relType) {
+        assert relType != null;
+
+        Type javaType = getResultClass(relType);
+
+        if (javaType == byte[].class) {
+            return relType.getPrecision() == PRECISION_NOT_SPECIFIED ? 
ColumnType.blobOf() :
+                ColumnType.blobOf(relType.getPrecision());
+        } else if (javaType == String.class) {
+            return relType.getPrecision() == PRECISION_NOT_SPECIFIED ? 
ColumnType.string() :
+                ColumnType.stringOf(relType.getPrecision());
+        } else if (javaType == BigInteger.class) {
+            return relType.getPrecision() == PRECISION_NOT_SPECIFIED ? 
ColumnType.numberOf() :
+                ColumnType.numberOf(relType.getPrecision());
+        } else if (javaType == BigDecimal.class) {
+            return relType.getPrecision() == PRECISION_NOT_SPECIFIED ? 
ColumnType.decimalOf() :
+                ColumnType.decimalOf(relType.getPrecision(), 
relType.getScale());

Review comment:
       `DECIMAL(10)` is a valid syntaxis

##########
File path: 
modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ddl/DdlCommandHandler.java
##########
@@ -0,0 +1,281 @@
+/*
+ * 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.exec.ddl;
+
+import static 
org.apache.ignite.internal.schema.configuration.SchemaConfigurationConverter.convert;
+import static org.apache.ignite.internal.util.CollectionUtils.nullOrEmpty;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.ColumnView;
+import org.apache.ignite.configuration.schemas.table.PrimaryKeyView;
+import org.apache.ignite.configuration.schemas.table.TableChange;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.PlanningContext;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.ddl.AbstractDdlCommand;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.ddl.AlterTableAddCommand;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.ddl.AlterTableDropCommand;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.ddl.CreateIndexCommand;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.ddl.CreateTableCommand;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.ddl.DdlCommand;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.ddl.DropIndexCommand;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.ddl.DropTableCommand;
+import org.apache.ignite.internal.schema.definition.TableDefinitionImpl;
+import org.apache.ignite.internal.table.distributed.TableManager;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.apache.ignite.internal.util.Pair;
+import org.apache.ignite.lang.ColumnAlreadyExistsException;
+import org.apache.ignite.lang.ColumnNotFoundException;
+import org.apache.ignite.lang.IgniteException;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+import org.apache.ignite.lang.IndexAlreadyExistsException;
+import org.apache.ignite.lang.LoggerMessageHelper;
+import org.apache.ignite.schema.SchemaBuilders;
+import org.apache.ignite.schema.definition.ColumnDefinition;
+import org.apache.ignite.schema.definition.builder.PrimaryKeyDefinitionBuilder;
+import 
org.apache.ignite.schema.definition.builder.SortedIndexDefinitionBuilder;
+import 
org.apache.ignite.schema.definition.builder.SortedIndexDefinitionBuilder.SortedIndexColumnBuilder;
+
+/** DDL commands handler. */
+public class DdlCommandHandler {
+    private final TableManager tableManager;
+
+    public DdlCommandHandler(TableManager tblManager) {
+        tableManager = tblManager;
+    }
+
+    /** Handles ddl commands. */
+    public void handle(DdlCommand cmd, PlanningContext pctx) throws 
IgniteInternalCheckedException {
+        validateCommand(cmd);
+
+        if (cmd instanceof CreateTableCommand) {
+            handleCreateTable((CreateTableCommand) cmd);
+        } else if (cmd instanceof DropTableCommand) {
+            handleDropTable((DropTableCommand) cmd);
+        } else if (cmd instanceof AlterTableAddCommand) {
+            handleAlterAddColumn((AlterTableAddCommand) cmd);
+        } else if (cmd instanceof AlterTableDropCommand) {
+            handleAlterDropColumn((AlterTableDropCommand) cmd);
+        } else if (cmd instanceof CreateIndexCommand) {
+            handleCreateIndex((CreateIndexCommand) cmd);
+        } else if (cmd instanceof DropIndexCommand) {
+            handleDropIndex((DropIndexCommand) cmd);
+        } else {
+            throw new IgniteInternalCheckedException("Unsupported DDL 
operation ["
+                    + "cmdName=" + (cmd == null ? null : 
cmd.getClass().getSimpleName()) + "; "
+                    + "querySql=\"" + pctx.query() + "\"]");
+        }
+    }
+
+    /** Validate command. */
+    private void validateCommand(DdlCommand cmd) {
+        if (cmd instanceof AbstractDdlCommand) {
+            AbstractDdlCommand cmd0 = (AbstractDdlCommand) cmd;
+
+            if (IgniteUtils.nullOrEmpty(cmd0.tableName())) {
+                throw new IllegalArgumentException("Table name is undefined.");
+            }
+        }
+    }
+
+    /** Handles create table command. */
+    private void handleCreateTable(CreateTableCommand cmd) {
+        PrimaryKeyDefinitionBuilder pkeyDef = SchemaBuilders.primaryKey();
+        pkeyDef.withColumns(cmd.primaryKeyColumns());
+        pkeyDef.withAffinityColumns(cmd.affColumns());
+
+        Consumer<TableChange> tblChanger = tblCh -> {
+            TableChange conv = 
convert(SchemaBuilders.tableBuilder(cmd.schemaName(), cmd.tableName())
+                    .columns(cmd.columns())
+                    .withPrimaryKey(pkeyDef.build()).build(), tblCh);
+
+            if (cmd.partitions() != null) {
+                conv.changePartitions(cmd.partitions());
+            }
+
+            if (cmd.replicas() != null) {
+                conv.changeReplicas(cmd.replicas());
+            }
+        };
+
+        String fullName = TableDefinitionImpl.canonicalName(cmd.schemaName(), 
cmd.tableName());
+
+        if (cmd.ifTableExists()) {
+            tableManager.createTableIfNotExists(fullName, tblChanger);
+        } else {
+            tableManager.createTable(fullName, tblChanger);
+        }
+    }
+
+    /** Handles drop table command. */
+    private void handleDropTable(DropTableCommand cmd) {
+        String fullName = TableDefinitionImpl.canonicalName(cmd.schemaName(), 
cmd.tableName());
+
+        // if (!cmd.ifTableExists()) todo will be implemented after 
IGNITE-15926
+
+        tableManager.dropTable(fullName);
+    }
+
+    /** Handles add column command. */
+    private void handleAlterAddColumn(AlterTableAddCommand cmd) {
+        if (nullOrEmpty(cmd.columns())) {
+            return;
+        }
+
+        String fullName = TableDefinitionImpl.canonicalName(cmd.schemaName(), 
cmd.tableName());
+
+        addColumnInternal(fullName, cmd.columns(), cmd.ifColumnNotExists());
+    }
+
+    /** Handles drop column command. */
+    private void handleAlterDropColumn(AlterTableDropCommand cmd) {
+        if (nullOrEmpty(cmd.columns())) {
+            return;
+        }
+
+        String fullName = TableDefinitionImpl.canonicalName(cmd.schemaName(), 
cmd.tableName());
+
+        dropColumnInternal(fullName, cmd.columns(), cmd.ifColumnExists());
+    }
+
+    /** Handles create index command. */
+    private void handleCreateIndex(CreateIndexCommand cmd) {
+        // Only sorted idx for now.
+        SortedIndexDefinitionBuilder idx = 
SchemaBuilders.sortedIndex(cmd.indexName());
+
+        for (Pair<String, Boolean> idxInfo : cmd.columns()) {
+            SortedIndexColumnBuilder idx0 = 
idx.addIndexColumn(idxInfo.getFirst());
+
+            if (idxInfo.getSecond()) {
+                idx0.desc();
+            }
+
+            idx0.done();
+        }
+
+        String fullName = TableDefinitionImpl.canonicalName(cmd.schemaName(), 
cmd.tableName());
+
+        tableManager.alterTable(fullName, chng -> chng.changeIndices(idxes -> {
+            if (idxes.get(cmd.indexName()) != null) {
+                throw new IndexAlreadyExistsException(cmd.indexName());
+            }
+
+            idxes.create(cmd.indexName(), tableIndexChange -> 
convert(idx.build(), tableIndexChange));
+        }));
+    }
+
+    /** Handles create index command. */
+    private void handleDropIndex(DropIndexCommand cmd) {
+        String fullName = TableDefinitionImpl.canonicalName(cmd.schemaName(), 
cmd.tableName());
+
+        tableManager.alterTable(fullName, chng -> chng.changeIndices(idxes -> {
+            if (idxes.get(cmd.indexName()) == null) {
+                if (!cmd.ifExist()) {
+                    throw new IndexAlreadyExistsException(cmd.indexName());
+                }
+            }
+
+            idxes.delete(cmd.indexName());
+        }));
+    }
+
+    /**
+     * Adds a column according to the column definition.
+     *
+     * @param fullName Table with schema name.
+     * @param colsDef  Columns defenitions.
+     * @param colNotExist Flag indicates exceptionally behavior in case of 
already existing column.
+     */
+    private void addColumnInternal(String fullName, Set<ColumnDefinition> 
colsDef, boolean colNotExist) {
+        tableManager.alterTable(
+                fullName,
+                chng -> chng.changeColumns(cols -> {
+                    Map<String, String> colNamesToOrders = 
columnOrdersToNames(chng.columns());
+
+                    Set<ColumnDefinition> colsDef0;
+
+                    if (!colNotExist) {
+                        colsDef.stream().filter(k -> 
colNamesToOrders.containsKey(k.name())).findAny()
+                                .ifPresent(c -> {
+                                    throw new 
ColumnAlreadyExistsException(c.name());
+                                });
+
+                        colsDef0 = colsDef;
+                    } else {
+                        colsDef0 = colsDef.stream().filter(k -> 
!colNamesToOrders.containsKey(k.name())).collect(Collectors.toSet());
+                    }
+
+                    for (ColumnDefinition colDef : colsDef0) {
+                        cols.create(colDef.name(), colChg -> convert(colDef, 
colChg));
+                    }
+                }));
+    }
+
+    /**
+     * Adds a column according to the column definition.
+     *
+     * @param fullName Table with schema name.
+     * @param colNames  Columns defenitions.
+     * @param colNotExist Flag indicates exceptionally behavior in case of 
already existing column.

Review comment:
       wrong param name

##########
File path: 
modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ddl/DdlCommandHandler.java
##########
@@ -0,0 +1,281 @@
+/*
+ * 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.exec.ddl;
+
+import static 
org.apache.ignite.internal.schema.configuration.SchemaConfigurationConverter.convert;
+import static org.apache.ignite.internal.util.CollectionUtils.nullOrEmpty;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.ColumnView;
+import org.apache.ignite.configuration.schemas.table.PrimaryKeyView;
+import org.apache.ignite.configuration.schemas.table.TableChange;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.PlanningContext;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.ddl.AbstractDdlCommand;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.ddl.AlterTableAddCommand;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.ddl.AlterTableDropCommand;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.ddl.CreateIndexCommand;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.ddl.CreateTableCommand;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.ddl.DdlCommand;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.ddl.DropIndexCommand;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.ddl.DropTableCommand;
+import org.apache.ignite.internal.schema.definition.TableDefinitionImpl;
+import org.apache.ignite.internal.table.distributed.TableManager;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.apache.ignite.internal.util.Pair;
+import org.apache.ignite.lang.ColumnAlreadyExistsException;
+import org.apache.ignite.lang.ColumnNotFoundException;
+import org.apache.ignite.lang.IgniteException;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+import org.apache.ignite.lang.IndexAlreadyExistsException;
+import org.apache.ignite.lang.LoggerMessageHelper;
+import org.apache.ignite.schema.SchemaBuilders;
+import org.apache.ignite.schema.definition.ColumnDefinition;
+import org.apache.ignite.schema.definition.builder.PrimaryKeyDefinitionBuilder;
+import 
org.apache.ignite.schema.definition.builder.SortedIndexDefinitionBuilder;
+import 
org.apache.ignite.schema.definition.builder.SortedIndexDefinitionBuilder.SortedIndexColumnBuilder;
+
+/** DDL commands handler. */
+public class DdlCommandHandler {
+    private final TableManager tableManager;
+
+    public DdlCommandHandler(TableManager tblManager) {
+        tableManager = tblManager;
+    }
+
+    /** Handles ddl commands. */
+    public void handle(DdlCommand cmd, PlanningContext pctx) throws 
IgniteInternalCheckedException {
+        validateCommand(cmd);
+
+        if (cmd instanceof CreateTableCommand) {
+            handleCreateTable((CreateTableCommand) cmd);
+        } else if (cmd instanceof DropTableCommand) {
+            handleDropTable((DropTableCommand) cmd);
+        } else if (cmd instanceof AlterTableAddCommand) {
+            handleAlterAddColumn((AlterTableAddCommand) cmd);
+        } else if (cmd instanceof AlterTableDropCommand) {
+            handleAlterDropColumn((AlterTableDropCommand) cmd);
+        } else if (cmd instanceof CreateIndexCommand) {
+            handleCreateIndex((CreateIndexCommand) cmd);
+        } else if (cmd instanceof DropIndexCommand) {
+            handleDropIndex((DropIndexCommand) cmd);
+        } else {
+            throw new IgniteInternalCheckedException("Unsupported DDL 
operation ["
+                    + "cmdName=" + (cmd == null ? null : 
cmd.getClass().getSimpleName()) + "; "
+                    + "querySql=\"" + pctx.query() + "\"]");
+        }
+    }
+
+    /** Validate command. */
+    private void validateCommand(DdlCommand cmd) {
+        if (cmd instanceof AbstractDdlCommand) {
+            AbstractDdlCommand cmd0 = (AbstractDdlCommand) cmd;
+
+            if (IgniteUtils.nullOrEmpty(cmd0.tableName())) {
+                throw new IllegalArgumentException("Table name is undefined.");
+            }
+        }
+    }
+
+    /** Handles create table command. */
+    private void handleCreateTable(CreateTableCommand cmd) {
+        PrimaryKeyDefinitionBuilder pkeyDef = SchemaBuilders.primaryKey();
+        pkeyDef.withColumns(cmd.primaryKeyColumns());
+        pkeyDef.withAffinityColumns(cmd.affColumns());
+
+        Consumer<TableChange> tblChanger = tblCh -> {
+            TableChange conv = 
convert(SchemaBuilders.tableBuilder(cmd.schemaName(), cmd.tableName())
+                    .columns(cmd.columns())
+                    .withPrimaryKey(pkeyDef.build()).build(), tblCh);
+
+            if (cmd.partitions() != null) {
+                conv.changePartitions(cmd.partitions());
+            }
+
+            if (cmd.replicas() != null) {
+                conv.changeReplicas(cmd.replicas());
+            }
+        };
+
+        String fullName = TableDefinitionImpl.canonicalName(cmd.schemaName(), 
cmd.tableName());
+
+        if (cmd.ifTableExists()) {
+            tableManager.createTableIfNotExists(fullName, tblChanger);
+        } else {
+            tableManager.createTable(fullName, tblChanger);
+        }
+    }
+
+    /** Handles drop table command. */
+    private void handleDropTable(DropTableCommand cmd) {
+        String fullName = TableDefinitionImpl.canonicalName(cmd.schemaName(), 
cmd.tableName());
+
+        // if (!cmd.ifTableExists()) todo will be implemented after 
IGNITE-15926
+
+        tableManager.dropTable(fullName);
+    }
+
+    /** Handles add column command. */
+    private void handleAlterAddColumn(AlterTableAddCommand cmd) {
+        if (nullOrEmpty(cmd.columns())) {
+            return;
+        }
+
+        String fullName = TableDefinitionImpl.canonicalName(cmd.schemaName(), 
cmd.tableName());
+
+        addColumnInternal(fullName, cmd.columns(), cmd.ifColumnNotExists());
+    }
+
+    /** Handles drop column command. */
+    private void handleAlterDropColumn(AlterTableDropCommand cmd) {
+        if (nullOrEmpty(cmd.columns())) {
+            return;
+        }
+
+        String fullName = TableDefinitionImpl.canonicalName(cmd.schemaName(), 
cmd.tableName());
+
+        dropColumnInternal(fullName, cmd.columns(), cmd.ifColumnExists());
+    }
+
+    /** Handles create index command. */
+    private void handleCreateIndex(CreateIndexCommand cmd) {
+        // Only sorted idx for now.
+        SortedIndexDefinitionBuilder idx = 
SchemaBuilders.sortedIndex(cmd.indexName());
+
+        for (Pair<String, Boolean> idxInfo : cmd.columns()) {
+            SortedIndexColumnBuilder idx0 = 
idx.addIndexColumn(idxInfo.getFirst());
+
+            if (idxInfo.getSecond()) {
+                idx0.desc();
+            }
+
+            idx0.done();
+        }
+
+        String fullName = TableDefinitionImpl.canonicalName(cmd.schemaName(), 
cmd.tableName());
+
+        tableManager.alterTable(fullName, chng -> chng.changeIndices(idxes -> {
+            if (idxes.get(cmd.indexName()) != null) {
+                throw new IndexAlreadyExistsException(cmd.indexName());
+            }
+
+            idxes.create(cmd.indexName(), tableIndexChange -> 
convert(idx.build(), tableIndexChange));
+        }));
+    }
+
+    /** Handles create index command. */
+    private void handleDropIndex(DropIndexCommand cmd) {
+        String fullName = TableDefinitionImpl.canonicalName(cmd.schemaName(), 
cmd.tableName());
+
+        tableManager.alterTable(fullName, chng -> chng.changeIndices(idxes -> {
+            if (idxes.get(cmd.indexName()) == null) {
+                if (!cmd.ifExist()) {
+                    throw new IndexAlreadyExistsException(cmd.indexName());
+                }
+            }
+
+            idxes.delete(cmd.indexName());
+        }));
+    }
+
+    /**
+     * Adds a column according to the column definition.
+     *
+     * @param fullName Table with schema name.
+     * @param colsDef  Columns defenitions.
+     * @param colNotExist Flag indicates exceptionally behavior in case of 
already existing column.
+     */
+    private void addColumnInternal(String fullName, Set<ColumnDefinition> 
colsDef, boolean colNotExist) {
+        tableManager.alterTable(
+                fullName,
+                chng -> chng.changeColumns(cols -> {
+                    Map<String, String> colNamesToOrders = 
columnOrdersToNames(chng.columns());
+
+                    Set<ColumnDefinition> colsDef0;
+
+                    if (!colNotExist) {
+                        colsDef.stream().filter(k -> 
colNamesToOrders.containsKey(k.name())).findAny()
+                                .ifPresent(c -> {
+                                    throw new 
ColumnAlreadyExistsException(c.name());
+                                });
+
+                        colsDef0 = colsDef;
+                    } else {
+                        colsDef0 = colsDef.stream().filter(k -> 
!colNamesToOrders.containsKey(k.name())).collect(Collectors.toSet());
+                    }
+
+                    for (ColumnDefinition colDef : colsDef0) {
+                        cols.create(colDef.name(), colChg -> convert(colDef, 
colChg));
+                    }
+                }));
+    }
+
+    /**
+     * Adds a column according to the column definition.
+     *
+     * @param fullName Table with schema name.
+     * @param colNames  Columns defenitions.
+     * @param colNotExist Flag indicates exceptionally behavior in case of 
already existing column.
+     */
+    private void dropColumnInternal(String fullName, Set<String> colNames, 
boolean colExist) {
+        tableManager.alterTable(
+                fullName,
+                chng -> chng.changeColumns(cols -> {
+                    PrimaryKeyView priKey = chng.primaryKey();
+
+                    Map<String, String> colNamesToOrders = 
columnOrdersToNames(chng.columns());
+
+                    Set<String> colNames0 = new HashSet<>();
+
+                    for (String colName : colNames) {
+                        if (!colNamesToOrders.containsKey(colName)) {
+                            if (!colExist) {
+                                throw new ColumnNotFoundException(colName);
+                            }
+                        } else {
+                            colNames0.add(colName);
+                        }
+
+                        for (String priColName : priKey.columns()) {

Review comment:
       it would be better to put key columns in a set

##########
File path: 
modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/ddl/DdlSqlToCommandConverter.java
##########
@@ -157,25 +153,30 @@ private CreateTableCommand 
convertCreateTable(IgniteSqlCreateTable createTblNode
                 .collect(Collectors.toList());
 
         IgnitePlanner planner = ctx.planner();
+        IgniteTypeFactory typeFactory = ctx.typeFactory();
 
-        List<ColumnDefinition> cols = new ArrayList<>();
+        List<ColumnDefinition> cols = new ArrayList<>(colDeclarations.size());
 
         for (SqlColumnDeclaration col : colDeclarations) {
             if (!col.name.isSimple()) {
                 throw new IgniteException("Unexpected value of columnName ["
                         + "expected a simple identifier, but was " + col.name 
+ "; "
-                        + "querySql=\"" + ctx.query() + "\"]"/*, 
IgniteQueryErrorCode.PARSING*/);
+                        + "querySql=\"" + ctx.query() + "\"]");
             }
 
             String name = col.name.getSimple();
-            RelDataType type = planner.convert(col.dataType);
+            RelDataType relType = planner.convert(col.dataType);
 
             Object dflt = null;
             if (col.expression != null) {
                 dflt = ((SqlLiteral) col.expression).getValue();
             }
 
-            cols.add(new ColumnDefinition(name, type, dflt));
+            ColumnDefinitionBuilder col0 = SchemaBuilders.column(name, 
typeFactory.columnType(relType))

Review comment:
       please return back `ColumnDefinition`

##########
File path: 
modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ddl/DdlCommandHandler.java
##########
@@ -0,0 +1,281 @@
+/*
+ * 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.exec.ddl;
+
+import static 
org.apache.ignite.internal.schema.configuration.SchemaConfigurationConverter.convert;
+import static org.apache.ignite.internal.util.CollectionUtils.nullOrEmpty;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.ColumnView;
+import org.apache.ignite.configuration.schemas.table.PrimaryKeyView;
+import org.apache.ignite.configuration.schemas.table.TableChange;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.PlanningContext;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.ddl.AbstractDdlCommand;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.ddl.AlterTableAddCommand;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.ddl.AlterTableDropCommand;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.ddl.CreateIndexCommand;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.ddl.CreateTableCommand;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.ddl.DdlCommand;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.ddl.DropIndexCommand;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.ddl.DropTableCommand;
+import org.apache.ignite.internal.schema.definition.TableDefinitionImpl;
+import org.apache.ignite.internal.table.distributed.TableManager;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.apache.ignite.internal.util.Pair;
+import org.apache.ignite.lang.ColumnAlreadyExistsException;
+import org.apache.ignite.lang.ColumnNotFoundException;
+import org.apache.ignite.lang.IgniteException;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+import org.apache.ignite.lang.IndexAlreadyExistsException;
+import org.apache.ignite.lang.LoggerMessageHelper;
+import org.apache.ignite.schema.SchemaBuilders;
+import org.apache.ignite.schema.definition.ColumnDefinition;
+import org.apache.ignite.schema.definition.builder.PrimaryKeyDefinitionBuilder;
+import 
org.apache.ignite.schema.definition.builder.SortedIndexDefinitionBuilder;
+import 
org.apache.ignite.schema.definition.builder.SortedIndexDefinitionBuilder.SortedIndexColumnBuilder;
+
+/** DDL commands handler. */
+public class DdlCommandHandler {
+    private final TableManager tableManager;
+
+    public DdlCommandHandler(TableManager tblManager) {
+        tableManager = tblManager;
+    }
+
+    /** Handles ddl commands. */
+    public void handle(DdlCommand cmd, PlanningContext pctx) throws 
IgniteInternalCheckedException {
+        validateCommand(cmd);
+
+        if (cmd instanceof CreateTableCommand) {
+            handleCreateTable((CreateTableCommand) cmd);
+        } else if (cmd instanceof DropTableCommand) {
+            handleDropTable((DropTableCommand) cmd);
+        } else if (cmd instanceof AlterTableAddCommand) {
+            handleAlterAddColumn((AlterTableAddCommand) cmd);
+        } else if (cmd instanceof AlterTableDropCommand) {
+            handleAlterDropColumn((AlterTableDropCommand) cmd);
+        } else if (cmd instanceof CreateIndexCommand) {
+            handleCreateIndex((CreateIndexCommand) cmd);
+        } else if (cmd instanceof DropIndexCommand) {
+            handleDropIndex((DropIndexCommand) cmd);
+        } else {
+            throw new IgniteInternalCheckedException("Unsupported DDL 
operation ["
+                    + "cmdName=" + (cmd == null ? null : 
cmd.getClass().getSimpleName()) + "; "
+                    + "querySql=\"" + pctx.query() + "\"]");
+        }
+    }
+
+    /** Validate command. */
+    private void validateCommand(DdlCommand cmd) {
+        if (cmd instanceof AbstractDdlCommand) {
+            AbstractDdlCommand cmd0 = (AbstractDdlCommand) cmd;
+
+            if (IgniteUtils.nullOrEmpty(cmd0.tableName())) {
+                throw new IllegalArgumentException("Table name is undefined.");
+            }
+        }
+    }
+
+    /** Handles create table command. */
+    private void handleCreateTable(CreateTableCommand cmd) {
+        PrimaryKeyDefinitionBuilder pkeyDef = SchemaBuilders.primaryKey();
+        pkeyDef.withColumns(cmd.primaryKeyColumns());
+        pkeyDef.withAffinityColumns(cmd.affColumns());
+
+        Consumer<TableChange> tblChanger = tblCh -> {
+            TableChange conv = 
convert(SchemaBuilders.tableBuilder(cmd.schemaName(), cmd.tableName())
+                    .columns(cmd.columns())
+                    .withPrimaryKey(pkeyDef.build()).build(), tblCh);
+
+            if (cmd.partitions() != null) {
+                conv.changePartitions(cmd.partitions());
+            }
+
+            if (cmd.replicas() != null) {
+                conv.changeReplicas(cmd.replicas());
+            }
+        };
+
+        String fullName = TableDefinitionImpl.canonicalName(cmd.schemaName(), 
cmd.tableName());
+
+        if (cmd.ifTableExists()) {
+            tableManager.createTableIfNotExists(fullName, tblChanger);

Review comment:
       `if table exists then create table if not exists` -- this one looks very 
strange. Let's rename flag to match sql statement (`IF NOT EXISTS`)

##########
File path: 
modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ddl/DdlCommandHandler.java
##########
@@ -0,0 +1,281 @@
+/*
+ * 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.exec.ddl;
+
+import static 
org.apache.ignite.internal.schema.configuration.SchemaConfigurationConverter.convert;
+import static org.apache.ignite.internal.util.CollectionUtils.nullOrEmpty;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.ColumnView;
+import org.apache.ignite.configuration.schemas.table.PrimaryKeyView;
+import org.apache.ignite.configuration.schemas.table.TableChange;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.PlanningContext;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.ddl.AbstractDdlCommand;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.ddl.AlterTableAddCommand;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.ddl.AlterTableDropCommand;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.ddl.CreateIndexCommand;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.ddl.CreateTableCommand;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.ddl.DdlCommand;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.ddl.DropIndexCommand;
+import 
org.apache.ignite.internal.processors.query.calcite.prepare.ddl.DropTableCommand;
+import org.apache.ignite.internal.schema.definition.TableDefinitionImpl;
+import org.apache.ignite.internal.table.distributed.TableManager;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.apache.ignite.internal.util.Pair;
+import org.apache.ignite.lang.ColumnAlreadyExistsException;
+import org.apache.ignite.lang.ColumnNotFoundException;
+import org.apache.ignite.lang.IgniteException;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+import org.apache.ignite.lang.IndexAlreadyExistsException;
+import org.apache.ignite.lang.LoggerMessageHelper;
+import org.apache.ignite.schema.SchemaBuilders;
+import org.apache.ignite.schema.definition.ColumnDefinition;
+import org.apache.ignite.schema.definition.builder.PrimaryKeyDefinitionBuilder;
+import 
org.apache.ignite.schema.definition.builder.SortedIndexDefinitionBuilder;
+import 
org.apache.ignite.schema.definition.builder.SortedIndexDefinitionBuilder.SortedIndexColumnBuilder;
+
+/** DDL commands handler. */
+public class DdlCommandHandler {
+    private final TableManager tableManager;
+
+    public DdlCommandHandler(TableManager tblManager) {
+        tableManager = tblManager;
+    }
+
+    /** Handles ddl commands. */
+    public void handle(DdlCommand cmd, PlanningContext pctx) throws 
IgniteInternalCheckedException {
+        validateCommand(cmd);
+
+        if (cmd instanceof CreateTableCommand) {
+            handleCreateTable((CreateTableCommand) cmd);
+        } else if (cmd instanceof DropTableCommand) {
+            handleDropTable((DropTableCommand) cmd);
+        } else if (cmd instanceof AlterTableAddCommand) {
+            handleAlterAddColumn((AlterTableAddCommand) cmd);
+        } else if (cmd instanceof AlterTableDropCommand) {
+            handleAlterDropColumn((AlterTableDropCommand) cmd);
+        } else if (cmd instanceof CreateIndexCommand) {
+            handleCreateIndex((CreateIndexCommand) cmd);
+        } else if (cmd instanceof DropIndexCommand) {
+            handleDropIndex((DropIndexCommand) cmd);
+        } else {
+            throw new IgniteInternalCheckedException("Unsupported DDL 
operation ["
+                    + "cmdName=" + (cmd == null ? null : 
cmd.getClass().getSimpleName()) + "; "
+                    + "querySql=\"" + pctx.query() + "\"]");
+        }
+    }
+
+    /** Validate command. */
+    private void validateCommand(DdlCommand cmd) {
+        if (cmd instanceof AbstractDdlCommand) {
+            AbstractDdlCommand cmd0 = (AbstractDdlCommand) cmd;
+
+            if (IgniteUtils.nullOrEmpty(cmd0.tableName())) {
+                throw new IllegalArgumentException("Table name is undefined.");
+            }
+        }
+    }
+
+    /** Handles create table command. */
+    private void handleCreateTable(CreateTableCommand cmd) {
+        PrimaryKeyDefinitionBuilder pkeyDef = SchemaBuilders.primaryKey();
+        pkeyDef.withColumns(cmd.primaryKeyColumns());
+        pkeyDef.withAffinityColumns(cmd.affColumns());
+
+        Consumer<TableChange> tblChanger = tblCh -> {
+            TableChange conv = 
convert(SchemaBuilders.tableBuilder(cmd.schemaName(), cmd.tableName())
+                    .columns(cmd.columns())
+                    .withPrimaryKey(pkeyDef.build()).build(), tblCh);
+
+            if (cmd.partitions() != null) {
+                conv.changePartitions(cmd.partitions());
+            }
+
+            if (cmd.replicas() != null) {
+                conv.changeReplicas(cmd.replicas());
+            }
+        };
+
+        String fullName = TableDefinitionImpl.canonicalName(cmd.schemaName(), 
cmd.tableName());
+
+        if (cmd.ifTableExists()) {
+            tableManager.createTableIfNotExists(fullName, tblChanger);
+        } else {
+            tableManager.createTable(fullName, tblChanger);
+        }
+    }
+
+    /** Handles drop table command. */
+    private void handleDropTable(DropTableCommand cmd) {
+        String fullName = TableDefinitionImpl.canonicalName(cmd.schemaName(), 
cmd.tableName());
+
+        // if (!cmd.ifTableExists()) todo will be implemented after 
IGNITE-15926
+
+        tableManager.dropTable(fullName);
+    }
+
+    /** Handles add column command. */
+    private void handleAlterAddColumn(AlterTableAddCommand cmd) {
+        if (nullOrEmpty(cmd.columns())) {
+            return;
+        }
+
+        String fullName = TableDefinitionImpl.canonicalName(cmd.schemaName(), 
cmd.tableName());
+
+        addColumnInternal(fullName, cmd.columns(), cmd.ifColumnNotExists());
+    }
+
+    /** Handles drop column command. */
+    private void handleAlterDropColumn(AlterTableDropCommand cmd) {
+        if (nullOrEmpty(cmd.columns())) {
+            return;
+        }
+
+        String fullName = TableDefinitionImpl.canonicalName(cmd.schemaName(), 
cmd.tableName());
+
+        dropColumnInternal(fullName, cmd.columns(), cmd.ifColumnExists());
+    }
+
+    /** Handles create index command. */
+    private void handleCreateIndex(CreateIndexCommand cmd) {
+        // Only sorted idx for now.
+        SortedIndexDefinitionBuilder idx = 
SchemaBuilders.sortedIndex(cmd.indexName());
+
+        for (Pair<String, Boolean> idxInfo : cmd.columns()) {
+            SortedIndexColumnBuilder idx0 = 
idx.addIndexColumn(idxInfo.getFirst());
+
+            if (idxInfo.getSecond()) {
+                idx0.desc();
+            }
+
+            idx0.done();
+        }
+
+        String fullName = TableDefinitionImpl.canonicalName(cmd.schemaName(), 
cmd.tableName());
+
+        tableManager.alterTable(fullName, chng -> chng.changeIndices(idxes -> {
+            if (idxes.get(cmd.indexName()) != null) {
+                throw new IndexAlreadyExistsException(cmd.indexName());
+            }
+
+            idxes.create(cmd.indexName(), tableIndexChange -> 
convert(idx.build(), tableIndexChange));
+        }));
+    }
+
+    /** Handles create index command. */
+    private void handleDropIndex(DropIndexCommand cmd) {
+        String fullName = TableDefinitionImpl.canonicalName(cmd.schemaName(), 
cmd.tableName());
+
+        tableManager.alterTable(fullName, chng -> chng.changeIndices(idxes -> {
+            if (idxes.get(cmd.indexName()) == null) {
+                if (!cmd.ifExist()) {
+                    throw new IndexAlreadyExistsException(cmd.indexName());

Review comment:
       it's strange to see `IndexAlreadyExistsException` in a DROP operation




-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to