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

[ignite] branch sql-calcite updated: IGNITE-13548 Calcite integration. DROP TABLE support

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

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


The following commit(s) were added to refs/heads/sql-calcite by this push:
     new b839f0e  IGNITE-13548 Calcite integration. DROP TABLE support
b839f0e is described below

commit b839f0e9ae09e09124fb02999970dce33a56f8bc
Author: korlov42 <ko...@gridgain.com>
AuthorDate: Wed Apr 21 18:50:35 2021 +0300

    IGNITE-13548 Calcite integration. DROP TABLE support
---
 modules/calcite/src/main/codegen/config.fmpp       |  2 +
 .../src/main/codegen/includes/parserImpls.ftl      | 48 +++++++----
 .../java/org/apache/calcite/sql/IgniteSqlNode.java | 31 --------
 .../query/calcite/exec/ExecutionServiceImpl.java   |  1 +
 .../query/calcite/exec/ddl/DdlCommandHandler.java  | 26 ++++++
 .../prepare/ddl/DdlSqlToCommandConverter.java      | 92 ++++++++++++++--------
 .../calcite/prepare/ddl/DropTableCommand.java      | 74 +++++++++++++++++
 .../calcite/sql/IgniteSqlCreateTableOption.java    | 34 ++++++--
 ...ationTest.java => TableDdlIntegrationTest.java} | 79 ++++++++++++++++++-
 .../ignite/testsuites/IgniteCalciteTestSuite.java  |  4 +-
 10 files changed, 304 insertions(+), 87 deletions(-)

diff --git a/modules/calcite/src/main/codegen/config.fmpp b/modules/calcite/src/main/codegen/config.fmpp
index a7db7a4..635bf1f 100644
--- a/modules/calcite/src/main/codegen/config.fmpp
+++ b/modules/calcite/src/main/codegen/config.fmpp
@@ -28,6 +28,7 @@ data: {
     # Example: "org.apache.calcite.sql.*", "java.util.List".
     imports: [
       "org.apache.calcite.sql.SqlCreate",
+      "org.apache.calcite.sql.SqlDrop",
       "org.apache.calcite.sql.SqlLiteral",
       "org.apache.calcite.schema.ColumnStrategy",
       "org.apache.ignite.internal.processors.query.calcite.sql.IgniteSqlCreateTable",
@@ -589,6 +590,7 @@ data: {
     # Each must accept arguments "(SqlParserPos pos)".
     # Example: "SqlDropSchema".
     dropStatementParserMethods: [
+      "SqlDropTable"
     ]
 
     # List of methods for parsing extensions to "DROP" calls.
diff --git a/modules/calcite/src/main/codegen/includes/parserImpls.ftl b/modules/calcite/src/main/codegen/includes/parserImpls.ftl
index 6583b7d..7b43a6d 100644
--- a/modules/calcite/src/main/codegen/includes/parserImpls.ftl
+++ b/modules/calcite/src/main/codegen/includes/parserImpls.ftl
@@ -39,41 +39,41 @@ SqlNodeList CreateTableOptionList() :
     }
 }
 
-IgniteSqlCreateTableOptionEnum CreateTableOptionEnumOpt() :
+SqlLiteral CreateTableOptionKey() :
 {
 }
 {
-    <TEMPLATE> { return IgniteSqlCreateTableOptionEnum.TEMPLATE; }
+    <TEMPLATE> { return SqlLiteral.createSymbol(IgniteSqlCreateTableOptionEnum.TEMPLATE, getPos()); }
 |
-    <BACKUPS> { return IgniteSqlCreateTableOptionEnum.BACKUPS; }
+    <BACKUPS> { return SqlLiteral.createSymbol(IgniteSqlCreateTableOptionEnum.BACKUPS, getPos()); }
 |
-    <AFFINITY_KEY> { return IgniteSqlCreateTableOptionEnum.AFFINITY_KEY; }
+    <AFFINITY_KEY> { return SqlLiteral.createSymbol(IgniteSqlCreateTableOptionEnum.AFFINITY_KEY, getPos()); }
 |
-    <ATOMICITY> { return IgniteSqlCreateTableOptionEnum.ATOMICITY; }
+    <ATOMICITY> { return SqlLiteral.createSymbol(IgniteSqlCreateTableOptionEnum.ATOMICITY, getPos()); }
 |
-    <WRITE_SYNCHRONIZATION_MODE> { return IgniteSqlCreateTableOptionEnum.WRITE_SYNCHRONIZATION_MODE; }
+    <WRITE_SYNCHRONIZATION_MODE> { return SqlLiteral.createSymbol(IgniteSqlCreateTableOptionEnum.WRITE_SYNCHRONIZATION_MODE, getPos()); }
 |
-    <CACHE_GROUP> { return IgniteSqlCreateTableOptionEnum.CACHE_GROUP; }
+    <CACHE_GROUP> { return SqlLiteral.createSymbol(IgniteSqlCreateTableOptionEnum.CACHE_GROUP, getPos()); }
 |
-    <CACHE_NAME> { return IgniteSqlCreateTableOptionEnum.CACHE_NAME; }
+    <CACHE_NAME> { return SqlLiteral.createSymbol(IgniteSqlCreateTableOptionEnum.CACHE_NAME, getPos()); }
 |
-    <DATA_REGION> { return IgniteSqlCreateTableOptionEnum.DATA_REGION; }
+    <DATA_REGION> { return SqlLiteral.createSymbol(IgniteSqlCreateTableOptionEnum.DATA_REGION, getPos()); }
 |
-    <KEY_TYPE> { return IgniteSqlCreateTableOptionEnum.KEY_TYPE; }
+    <KEY_TYPE> { return SqlLiteral.createSymbol(IgniteSqlCreateTableOptionEnum.KEY_TYPE, getPos()); }
 |
-    <VALUE_TYPE> { return IgniteSqlCreateTableOptionEnum.VALUE_TYPE; }
+    <VALUE_TYPE> { return SqlLiteral.createSymbol(IgniteSqlCreateTableOptionEnum.VALUE_TYPE, getPos()); }
 |
-    <ENCRYPTED> { return IgniteSqlCreateTableOptionEnum.ENCRYPTED; }
+    <ENCRYPTED> { return SqlLiteral.createSymbol(IgniteSqlCreateTableOptionEnum.ENCRYPTED, getPos()); }
 }
 
 void CreateTableOption(List<SqlNode> list) :
 {
     final Span s;
-    final IgniteSqlCreateTableOptionEnum key;
+    final SqlLiteral key;
     final SqlNode val;
 }
 {
-    key = CreateTableOptionEnumOpt() { s = span(); }
+    key = CreateTableOptionKey() { s = span(); }
     <EQ>
     (
         val = Literal()
@@ -164,6 +164,26 @@ SqlCreate SqlCreateTable(Span s, boolean replace) :
     }
 }
 
+boolean IfExistsOpt() :
+{
+}
+{
+    <IF> <EXISTS> { return true; }
+|
+    { return false; }
+}
+
+SqlDrop SqlDropTable(Span s, boolean replace) :
+{
+    final boolean ifExists;
+    final SqlIdentifier id;
+}
+{
+    <TABLE> ifExists = IfExistsOpt() id = CompoundIdentifier() {
+        return SqlDdlNodes.dropTable(s.end(this), ifExists, id);
+    }
+}
+
 void InfixCast(List<Object> list, ExprContext exprContext, Span s) :
 {
     final SqlDataTypeSpec dt;
diff --git a/modules/calcite/src/main/java/org/apache/calcite/sql/IgniteSqlNode.java b/modules/calcite/src/main/java/org/apache/calcite/sql/IgniteSqlNode.java
deleted file mode 100644
index 5ce6fb0..0000000
--- a/modules/calcite/src/main/java/org/apache/calcite/sql/IgniteSqlNode.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/*
- * 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.calcite.sql;
-
-import org.apache.calcite.sql.parser.SqlParserPos;
-
-/** A {@link SqlNode} that exposes constructor for descendant classes. */
-public abstract class IgniteSqlNode extends SqlNode {
-    /**
-     * Creates a node.
-     *
-     * @param pos Parser position, must not be null.
-     */
-    protected IgniteSqlNode(SqlParserPos pos) {
-        super(pos);
-    }
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionServiceImpl.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionServiceImpl.java
index 449ddb5..b3b0b6c 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionServiceImpl.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionServiceImpl.java
@@ -574,6 +574,7 @@ public class ExecutionServiceImpl<Row> extends AbstractService implements Execut
                 return prepareExplain(sqlNode, ctx);
 
             case CREATE_TABLE:
+            case DROP_TABLE:
                 return prepareDdl(sqlNode, ctx);
 
             default:
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ddl/DdlCommandHandler.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ddl/DdlCommandHandler.java
index 529bcae..7303e84 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ddl/DdlCommandHandler.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ddl/DdlCommandHandler.java
@@ -26,6 +26,7 @@ import java.util.Set;
 import java.util.function.Supplier;
 
 import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.schema.Table;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.cache.QueryEntity;
 import org.apache.ignite.configuration.CacheConfiguration;
@@ -39,6 +40,8 @@ import org.apache.ignite.internal.processors.query.calcite.prepare.PlanningConte
 import org.apache.ignite.internal.processors.query.calcite.prepare.ddl.ColumnDefinition;
 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.DropTableCommand;
+import org.apache.ignite.internal.processors.query.calcite.schema.IgniteTable;
 import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
 import org.apache.ignite.internal.processors.query.schema.SchemaOperationException;
 import org.apache.ignite.internal.processors.security.IgniteSecurity;
@@ -77,6 +80,9 @@ public class DdlCommandHandler {
         if (cmd instanceof CreateTableCommand)
             handle0(pctx, (CreateTableCommand)cmd);
 
+        else if (cmd instanceof DropTableCommand)
+            handle0(pctx, (DropTableCommand)cmd);
+
         else {
             throw new IgniteSQLException("Unsupported DDL operation [" +
                 "cmdName=" + (cmd == null ? null : cmd.getClass().getSimpleName()) + "; " +
@@ -128,6 +134,26 @@ public class DdlCommandHandler {
     }
 
     /** */
+    private void handle0(PlanningContext pctx, DropTableCommand cmd) throws IgniteCheckedException {
+        isDdlOnSchemaSupported(cmd.schemaName());
+
+        Table tbl = schemaSupp.get().getSubSchema(cmd.schemaName()).getTable(cmd.tableName());
+
+        if (tbl == null) {
+            if (!cmd.ifExists())
+                throw new SchemaOperationException(SchemaOperationException.CODE_TABLE_NOT_FOUND, cmd.tableName());
+
+            return;
+        }
+
+        String cacheName = ((IgniteTable)tbl).descriptor().cacheInfo().name();
+
+        security.authorize(cacheName, SecurityPermission.CACHE_DESTROY);
+
+        qryProcessorSupp.get().dynamicTableDrop(cacheName, cmd.tableName(), cmd.ifExists());
+    }
+
+    /** */
     private QueryEntity toQueryEntity(CreateTableCommand cmd, PlanningContext pctx) {
         QueryEntity res = new QueryEntity();
 
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/ddl/DdlSqlToCommandConverter.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/ddl/DdlSqlToCommandConverter.java
index 4d63e76..8c0e0c3 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/ddl/DdlSqlToCommandConverter.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/ddl/DdlSqlToCommandConverter.java
@@ -37,6 +37,7 @@ import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.SqlNodeList;
 import org.apache.calcite.sql.SqlNumericLiteral;
 import org.apache.calcite.sql.ddl.SqlColumnDeclaration;
+import org.apache.calcite.sql.ddl.SqlDropTable;
 import org.apache.calcite.sql.ddl.SqlKeyConstraint;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheWriteSynchronizationMode;
@@ -119,6 +120,9 @@ public class DdlSqlToCommandConverter {
         if (ddlNode instanceof IgniteSqlCreateTable)
             return convertCreateTable((IgniteSqlCreateTable)ddlNode, ctx);
 
+        if (ddlNode instanceof SqlDropTable)
+            return convertDropTable((SqlDropTable)ddlNode, ctx);
+
         throw new IgniteSQLException("Unsupported operation [" +
             "sqlNodeKind=" + ddlNode.getKind() + "; " +
             "querySql=\"" + ctx.query() + "\"]", IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
@@ -131,40 +135,10 @@ public class DdlSqlToCommandConverter {
      * @param ctx Planning context.
      */
     private CreateTableCommand convertCreateTable(IgniteSqlCreateTable createTblNode, PlanningContext ctx) {
-        String schemaName, tableName;
-
-        if (createTblNode.name().isSimple()) {
-            schemaName = ctx.schemaName();
-            tableName = createTblNode.name().getSimple();
-        }
-        else {
-            SqlIdentifier schemaId = createTblNode.name().skipLast(1);
-
-            if (!schemaId.isSimple()) {
-                throw new IgniteSQLException("Unexpected value of schemaName [" +
-                    "expected a simple identifier, but was " + schemaId + "; " +
-                    "querySql=\"" + ctx.query() + "\"]", IgniteQueryErrorCode.PARSING);
-            }
-
-            schemaName = schemaId.getSimple();
-
-            SqlIdentifier tableId = createTblNode.name().getComponent(schemaId.names.size());
-
-            if (!tableId.isSimple()) {
-                throw new IgniteSQLException("Unexpected value of tableName [" +
-                    "expected a simple identifier, but was " + tableId + "; " +
-                    "querySql=\"" + ctx.query() + "\"]", IgniteQueryErrorCode.PARSING);
-            }
-
-            tableName = tableId.getSimple();
-        }
-
-        ensureSchemaExists(ctx, schemaName);
-
         CreateTableCommand createTblCmd = new CreateTableCommand();
 
-        createTblCmd.schemaName(schemaName);
-        createTblCmd.tableName(tableName);
+        createTblCmd.schemaName(deriveSchemaName(createTblNode.name(), ctx));
+        createTblCmd.tableName(deriveObjectName(createTblNode.name(), ctx, "tableName"));
         createTblCmd.ifNotExists(createTblNode.ifNotExists());
         createTblCmd.templateName(QueryUtils.TEMPLATE_PARTITIONED);
 
@@ -231,6 +205,60 @@ public class DdlSqlToCommandConverter {
         return createTblCmd;
     }
 
+    /**
+     * Converts a given DropTable AST to a DropTable command.
+     *
+     * @param dropTblNode Root node of the given AST.
+     * @param ctx Planning context.
+     */
+    private DropTableCommand convertDropTable(SqlDropTable dropTblNode, PlanningContext ctx) {
+        DropTableCommand dropTblCmd = new DropTableCommand();
+
+        dropTblCmd.schemaName(deriveSchemaName(dropTblNode.name, ctx));
+        dropTblCmd.tableName(deriveObjectName(dropTblNode.name, ctx, "tableName"));
+        dropTblCmd.ifExists(dropTblNode.ifExists);
+
+        return dropTblCmd;
+    }
+
+    /** Derives a schema name from the compound identifier. */
+    private String deriveSchemaName(SqlIdentifier id, PlanningContext ctx) {
+        String schemaName;
+        if (id.isSimple())
+            schemaName = ctx.schemaName();
+        else {
+            SqlIdentifier schemaId = id.skipLast(1);
+
+            if (!schemaId.isSimple()) {
+                throw new IgniteSQLException("Unexpected value of schemaName [" +
+                    "expected a simple identifier, but was " + schemaId + "; " +
+                    "querySql=\"" + ctx.query() + "\"]", IgniteQueryErrorCode.PARSING);
+            }
+
+            schemaName = schemaId.getSimple();
+        }
+
+        ensureSchemaExists(ctx, schemaName);
+
+        return schemaName;
+    }
+
+    /** Derives an object(a table, an index, etc) name from the compound identifier. */
+    private String deriveObjectName(SqlIdentifier id, PlanningContext ctx, String objDesc) {
+        if (id.isSimple())
+            return id.getSimple();
+
+        SqlIdentifier objId = id.getComponent(id.skipLast(1).names.size());
+
+        if (!objId.isSimple()) {
+            throw new IgniteSQLException("Unexpected value of " + objDesc + " [" +
+                "expected a simple identifier, but was " + objId + "; " +
+                "querySql=\"" + ctx.query() + "\"]", IgniteQueryErrorCode.PARSING);
+        }
+
+        return objId.getSimple();
+    }
+
     /** */
     private void ensureSchemaExists(PlanningContext ctx, String schemaName) {
         if (ctx.catalogReader().getRootSchema().getSubSchema(schemaName, true) == null)
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/ddl/DropTableCommand.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/ddl/DropTableCommand.java
new file mode 100644
index 0000000..638a46d
--- /dev/null
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/ddl/DropTableCommand.java
@@ -0,0 +1,74 @@
+/*
+ * 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;
+
+/**
+ * DROP TABLE statement.
+ */
+public class DropTableCommand implements DdlCommand {
+    /** Schema name. */
+    private String schemaName;
+
+    /** Table name. */
+    private String tblName;
+
+    /** Quietly ignore this command if table does not exist. */
+    private boolean ifExists;
+
+    /**
+     * @return Schema name.
+     */
+    public String schemaName() {
+        return schemaName;
+    }
+
+    /**
+     * @param schemaName Schema name.
+     */
+    public void schemaName(String schemaName) {
+        this.schemaName = schemaName;
+    }
+
+    /**
+     * @return Table name.
+     */
+    public String tableName() {
+        return tblName;
+    }
+
+    /**
+     * @param tblName Table name.
+     */
+    public void tableName(String tblName) {
+        this.tblName = tblName;
+    }
+
+    /**
+     * @return Quietly ignore this command if table does not exist.
+     */
+    public boolean ifExists() {
+        return ifExists;
+    }
+
+    /**
+     * @param ifExists Quietly ignore this command if table does not exist.
+     */
+    public void ifExists(boolean ifExists) {
+        this.ifExists = ifExists;
+    }
+}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/sql/IgniteSqlCreateTableOption.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/sql/IgniteSqlCreateTableOption.java
index 6191568..f4015ec 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/sql/IgniteSqlCreateTableOption.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/sql/IgniteSqlCreateTableOption.java
@@ -16,25 +16,37 @@
  */
 package org.apache.ignite.internal.processors.query.calcite.sql;
 
-import org.apache.calcite.sql.IgniteSqlNode;
+import java.util.List;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlLiteral;
 import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlSpecialOperator;
 import org.apache.calcite.sql.SqlWriter;
 import org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.calcite.sql.util.SqlVisitor;
 import org.apache.calcite.sql.validate.SqlValidator;
 import org.apache.calcite.sql.validate.SqlValidatorScope;
 import org.apache.calcite.util.Litmus;
+import org.jetbrains.annotations.NotNull;
 
 /** An AST node representing option to create table with. */
-public class IgniteSqlCreateTableOption extends IgniteSqlNode {
+public class IgniteSqlCreateTableOption extends SqlCall {
+    /** */
+    private static final SqlOperator OPERATOR =
+        new SqlSpecialOperator("TableOption", SqlKind.OTHER);
+
     /** Option key. */
-    private final IgniteSqlCreateTableOptionEnum key;
+    private final SqlLiteral key;
 
     /** Option value. */
     private final SqlNode value;
 
     /** Creates IgniteSqlCreateTableOption. */
-    public IgniteSqlCreateTableOption(IgniteSqlCreateTableOptionEnum key, SqlNode value, SqlParserPos pos) {
+    public IgniteSqlCreateTableOption(SqlLiteral key, SqlNode value, SqlParserPos pos) {
         super(pos);
 
         this.key = key;
@@ -42,13 +54,23 @@ public class IgniteSqlCreateTableOption extends IgniteSqlNode {
     }
 
     /** {@inheritDoc} */
+    @NotNull @Override public SqlOperator getOperator() {
+        return OPERATOR;
+    }
+
+    /** {@inheritDoc} */
+    @NotNull @Override public List<SqlNode> getOperandList() {
+        return ImmutableList.of(key, value);
+    }
+
+    /** {@inheritDoc} */
     @Override public SqlNode clone(SqlParserPos pos) {
         return new IgniteSqlCreateTableOption(key, value, pos);
     }
 
     /** {@inheritDoc} */
     @Override public void unparse(SqlWriter writer, int leftPrec, int rightPrec) {
-        writer.keyword(key.name());
+        key.unparse(writer, leftPrec, rightPrec);
         writer.keyword("=");
         value.unparse(writer, leftPrec, rightPrec);
     }
@@ -79,7 +101,7 @@ public class IgniteSqlCreateTableOption extends IgniteSqlNode {
      * @return Option's key.
      */
     public IgniteSqlCreateTableOptionEnum key() {
-        return key;
+        return key.getValueAs(IgniteSqlCreateTableOptionEnum.class);
     }
 
     /**
diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/CreateTableIntegrationTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/TableDdlIntegrationTest.java
similarity index 80%
rename from modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/CreateTableIntegrationTest.java
rename to modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/TableDdlIntegrationTest.java
index 80d61e0..67c1e2e 100644
--- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/CreateTableIntegrationTest.java
+++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/TableDdlIntegrationTest.java
@@ -52,10 +52,11 @@ import static org.apache.ignite.internal.util.IgniteUtils.map;
 import static org.apache.ignite.testframework.GridTestUtils.hasSize;
 import static org.hamcrest.CoreMatchers.equalTo;
 import static org.hamcrest.CoreMatchers.hasItem;
+import static org.hamcrest.CoreMatchers.nullValue;
 import static org.junit.Assert.assertThat;
 
 /** */
-public class CreateTableIntegrationTest extends GridCommonAbstractTest {
+public class TableDdlIntegrationTest extends GridCommonAbstractTest {
     /** */
     private static final String CLIENT_NODE_NAME = "client";
 
@@ -233,7 +234,7 @@ public class CreateTableIntegrationTest extends GridCommonAbstractTest {
     }
 
     /**
-     * Creates a table in a different schema.
+     * Creates a table in a custom schema.
      */
     @Test
     public void createTableCustomSchema() {
@@ -245,6 +246,80 @@ public class CreateTableIntegrationTest extends GridCommonAbstractTest {
         assertThat(executeSql("select * from my_schema.my_table"), hasSize(4));
     }
 
+    /**
+     * Drops a table created in a default schema.
+     */
+    @Test
+    public void dropTableDefaultSchema() {
+        Set<String> cachesBefore = new HashSet<>(client.cacheNames());
+
+        executeSql("create table my_table (id int primary key, val varchar)");
+
+        Set<String> cachesAfter = new HashSet<>(client.cacheNames());
+        cachesAfter.removeAll(cachesBefore);
+
+        assertThat(cachesAfter, hasSize(1));
+
+        String createdCacheName = cachesAfter.iterator().next();
+
+        executeSql("drop table my_table");
+
+        cachesAfter = new HashSet<>(client.cacheNames());
+        cachesAfter.removeAll(cachesBefore);
+
+        assertThat(cachesAfter, hasSize(0));
+
+        assertThat(client.cachex(createdCacheName), nullValue());
+    }
+
+    /**
+     * Drops a table created in a custom schema.
+     */
+    @Test
+    public void dropTableCustomSchema() {
+        Set<String> cachesBefore = new HashSet<>(client.cacheNames());
+
+        executeSql("create table my_schema.my_table (id int primary key, val varchar)");
+
+        Set<String> cachesAfter = new HashSet<>(client.cacheNames());
+        cachesAfter.removeAll(cachesBefore);
+
+        assertThat(cachesAfter, hasSize(1));
+
+        String createdCacheName = cachesAfter.iterator().next();
+
+        executeSql("drop table my_schema.my_table");
+
+        cachesAfter = new HashSet<>(client.cacheNames());
+        cachesAfter.removeAll(cachesBefore);
+
+        assertThat(cachesAfter, hasSize(0));
+
+        assertThat(client.cachex(createdCacheName), nullValue());
+    }
+
+    /**
+     * Drops a table several times with and without
+     * specifying IF EXISTS.
+     */
+    @Test
+    @SuppressWarnings("ThrowableNotThrown")
+    public void dropTableIfExists() {
+        executeSql("create table my_schema.my_table (id int primary key, val varchar)");
+
+        GridTestUtils.assertThrows(log,
+            () -> executeSql("drop table my_table"),
+            IgniteSQLException.class, "Table doesn't exist: MY_TABLE]");
+
+        executeSql("drop table my_schema.my_table");
+
+        GridTestUtils.assertThrows(log,
+            () -> executeSql("drop table my_schema.my_table"),
+            IgniteSQLException.class, "Table doesn't exist: MY_TABLE]");
+
+        executeSql("drop table if exists my_schema.my_table");
+    }
+
     /** */
     private List<List<?>> executeSql(String sql) {
         List<FieldsQueryCursor<List<?>>> cur = queryProcessor().query(null, "PUBLIC", sql);
diff --git a/modules/calcite/src/test/java/org/apache/ignite/testsuites/IgniteCalciteTestSuite.java b/modules/calcite/src/test/java/org/apache/ignite/testsuites/IgniteCalciteTestSuite.java
index 498178a..f9d8b0d 100644
--- a/modules/calcite/src/test/java/org/apache/ignite/testsuites/IgniteCalciteTestSuite.java
+++ b/modules/calcite/src/test/java/org/apache/ignite/testsuites/IgniteCalciteTestSuite.java
@@ -22,13 +22,13 @@ import org.apache.ignite.internal.processors.query.calcite.CalciteBasicSecondary
 import org.apache.ignite.internal.processors.query.calcite.CalciteErrorHandlilngIntegrationTest;
 import org.apache.ignite.internal.processors.query.calcite.CalciteQueryProcessorTest;
 import org.apache.ignite.internal.processors.query.calcite.CancelTest;
-import org.apache.ignite.internal.processors.query.calcite.CreateTableIntegrationTest;
 import org.apache.ignite.internal.processors.query.calcite.DateTimeTest;
 import org.apache.ignite.internal.processors.query.calcite.LimitOffsetTest;
 import org.apache.ignite.internal.processors.query.calcite.MetadataIntegrationTest;
 import org.apache.ignite.internal.processors.query.calcite.QueryCheckerTest;
 import org.apache.ignite.internal.processors.query.calcite.SortAggregateIntegrationTest;
 import org.apache.ignite.internal.processors.query.calcite.SqlFieldsQueryUsageTest;
+import org.apache.ignite.internal.processors.query.calcite.TableDdlIntegrationTest;
 import org.apache.ignite.internal.processors.query.calcite.exec.ClosableIteratorsHolderTest;
 import org.apache.ignite.internal.processors.query.calcite.exec.rel.ContinuousExecutionTest;
 import org.apache.ignite.internal.processors.query.calcite.jdbc.JdbcQueryTest;
@@ -63,7 +63,7 @@ import org.junit.runners.Suite;
     SortAggregateIntegrationTest.class,
 
     SqlDdlParserTest.class,
-    CreateTableIntegrationTest.class,
+    TableDdlIntegrationTest.class,
 })
 public class IgniteCalciteTestSuite {
 }