You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by vo...@apache.org on 2019/04/01 13:36:16 UTC

[ignite] branch master updated: IGNITE-7664: SQL: Throw better error messages for unsupported SQL keywords and features. This closes #6138.

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

vozerov pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ignite.git


The following commit(s) were added to refs/heads/master by this push:
     new 70b9e0e  IGNITE-7664: SQL: Throw better error messages for unsupported SQL keywords and features. This closes #6138.
70b9e0e is described below

commit 70b9e0e83e36be04ff306cf809f024f736cbf7e8
Author: tledkov <tl...@gridgain.com>
AuthorDate: Mon Apr 1 16:36:02 2019 +0300

    IGNITE-7664: SQL: Throw better error messages for unsupported SQL keywords and features. This closes #6138.
---
 .../org/apache/ignite/internal/sql/SqlKeyword.java |  12 +
 .../org/apache/ignite/internal/sql/SqlParser.java  |  67 +++++
 .../apache/ignite/internal/sql/SqlParserUtils.java |   2 +-
 .../internal/sql/SqlStrictParseException.java      |  27 +-
 .../internal/processors/query/h2/QueryParser.java  |  25 +-
 .../processors/query/h2/sql/GridSqlKeyword.java    |   3 +-
 .../query/h2/sql/GridSqlQueryParser.java           | 199 +++++++++---
 .../cache/IgniteCacheUpdateSqlQuerySelfTest.java   |  33 --
 .../query/h2/sql/GridQueryParsingTest.java         |  47 +--
 .../query/h2/sql/SqlUnsupportedSelfTest.java       | 332 +++++++++++++++++++++
 .../IgniteBinaryCacheQueryTestSuite.java           |   2 +
 11 files changed, 639 insertions(+), 110 deletions(-)

diff --git a/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlKeyword.java b/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlKeyword.java
index 05cde79..c249a8f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlKeyword.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlKeyword.java
@@ -284,6 +284,18 @@ public class SqlKeyword {
     /** Keyword: PASSWORD. */
     public static final String PASSWORD = "PASSWORD";
 
+    /** Keyword: SHOW. */
+    public static final String SHOW = "SHOW";
+
+    /** Keyword: HELP. */
+    public static final String HELP = "HELP";
+
+    /** Keyword: GRANT. */
+    public static final String GRANT = "GRANT";
+
+    /** Keyword: REVOKE. */
+    public static final String REVOKE = "REVOKE";
+
     /** All keywords. */
     private static final HashSet<String> KEYWORDS;
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlParser.java b/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlParser.java
index aaf5614..c5a0371 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlParser.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlParser.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.sql;
 
+import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
 import org.apache.ignite.internal.sql.command.SqlAlterTableCommand;
 import org.apache.ignite.internal.sql.command.SqlAlterUserCommand;
 import org.apache.ignite.internal.sql.command.SqlBeginTransactionCommand;
@@ -37,11 +38,15 @@ import static org.apache.ignite.internal.sql.SqlKeyword.COMMIT;
 import static org.apache.ignite.internal.sql.SqlKeyword.COPY;
 import static org.apache.ignite.internal.sql.SqlKeyword.CREATE;
 import static org.apache.ignite.internal.sql.SqlKeyword.DROP;
+import static org.apache.ignite.internal.sql.SqlKeyword.GRANT;
 import static org.apache.ignite.internal.sql.SqlKeyword.HASH;
+import static org.apache.ignite.internal.sql.SqlKeyword.HELP;
 import static org.apache.ignite.internal.sql.SqlKeyword.INDEX;
 import static org.apache.ignite.internal.sql.SqlKeyword.PRIMARY;
+import static org.apache.ignite.internal.sql.SqlKeyword.REVOKE;
 import static org.apache.ignite.internal.sql.SqlKeyword.ROLLBACK;
 import static org.apache.ignite.internal.sql.SqlKeyword.SET;
+import static org.apache.ignite.internal.sql.SqlKeyword.SHOW;
 import static org.apache.ignite.internal.sql.SqlKeyword.SPATIAL;
 import static org.apache.ignite.internal.sql.SqlKeyword.START;
 import static org.apache.ignite.internal.sql.SqlKeyword.STREAMING;
@@ -175,6 +180,28 @@ public class SqlParser {
 
                         case ALTER:
                             cmd = processAlter();
+
+                            break;
+
+                        case HELP:
+                            cmd = processHelp();
+
+                            break;
+
+                        case SHOW:
+                            cmd = processShow();
+
+                            break;
+
+                        case GRANT:
+                            cmd = processGrant();
+
+                            break;
+
+                        case REVOKE:
+                            cmd = processRevoke();
+
+                            break;
                     }
 
                     if (cmd != null) {
@@ -371,6 +398,46 @@ public class SqlParser {
     }
 
     /**
+     * Process HELP keyword.
+     *
+     * @return Command.
+     */
+    private SqlCommand processHelp() {
+        throw new SqlStrictParseException("HELP command is not supported", IgniteQueryErrorCode.UNSUPPORTED_OPERATION,
+            SqlParserUtils.errorUnsupported(lex));
+    }
+
+    /**
+     * Process SHOW keyword.
+     *
+     * @return Command.
+     */
+    private SqlCommand processShow() {
+        throw new SqlStrictParseException("SHOW command is not supported", IgniteQueryErrorCode.UNSUPPORTED_OPERATION,
+            SqlParserUtils.errorUnsupported(lex));
+    }
+
+    /**
+     * Process GRANT keyword.
+     *
+     * @return Command.
+     */
+    private SqlCommand processGrant() {
+        throw new SqlStrictParseException("GRANT command is not supported", IgniteQueryErrorCode.UNSUPPORTED_OPERATION,
+            SqlParserUtils.errorUnsupported(lex));
+    }
+
+    /**
+     * Process REVOKE keyword.
+     *
+     * @return Command.
+     */
+    private SqlCommand processRevoke() {
+        throw new SqlStrictParseException("REVOKE command is not supported", IgniteQueryErrorCode.UNSUPPORTED_OPERATION,
+            SqlParserUtils.errorUnsupported(lex));
+    }
+
+    /**
      * Not yet parsed part of the sql query. Result is invalid if parsing error was thrown.
      */
     public String remainingSql() {
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlParserUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlParserUtils.java
index 074bffa..773fd43 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlParserUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlParserUtils.java
@@ -352,7 +352,7 @@ public class SqlParserUtils {
      * @return Error.
      */
     public static SqlParseException errorUnsupported(SqlLexerToken token) {
-        throw error0(token, IgniteQueryErrorCode.UNSUPPORTED_OPERATION,
+        return error0(token, IgniteQueryErrorCode.UNSUPPORTED_OPERATION,
             "Unsupported keyword: \"" + token.token() + "\"");
     }
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlStrictParseException.java b/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlStrictParseException.java
index 119b854..c9f965e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlStrictParseException.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlStrictParseException.java
@@ -17,6 +17,8 @@
 
 package org.apache.ignite.internal.sql;
 
+import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
+
 /**
  * Parse exception guarantees parse error without. Such error deliver to user
  * statement isn't passed to H2 parser.
@@ -25,11 +27,34 @@ public class SqlStrictParseException extends SqlParseException {
     /** */
     private static final long serialVersionUID = 0L;
 
+    /** SQL error code. */
+    private final int errCode;
+
     /**
      * Constructor.
      * @param e SQL parse exception.
      */
     public SqlStrictParseException(SqlParseException e) {
-        super(e);
+        this(e.getMessage(), IgniteQueryErrorCode.PARSING, e);
+    }
+
+
+    /**
+     * Constructor.
+     * @param e SQL parse exception.
+     * @param msg Error message.
+     * @param errCode SQL error code.
+     */
+    public SqlStrictParseException(String msg, int errCode, SqlParseException e) {
+        super(e.sql(), e.position(), e.code(), msg);
+
+        this.errCode = errCode;
+    }
+
+    /**
+     * @return SQL error code.
+     */
+    public int errorCode() {
+        return errCode;
     }
 }
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/QueryParser.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/QueryParser.java
index cded1fe..7f6a720 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/QueryParser.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/QueryParser.java
@@ -17,6 +17,13 @@
 
 package org.apache.ignite.internal.processors.query.h2;
 
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Pattern;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.IgniteSystemProperties;
@@ -63,14 +70,6 @@ import org.apache.ignite.internal.util.typedef.internal.U;
 import org.h2.command.Prepared;
 import org.jetbrains.annotations.Nullable;
 
-import java.sql.Connection;
-import java.sql.PreparedStatement;
-import java.sql.SQLException;
-import java.util.Arrays;
-import java.util.List;
-import java.util.Map;
-import java.util.regex.Pattern;
-
 /**
  * Parser module. Splits incoming request into a series of parsed results.
  */
@@ -80,7 +79,8 @@ public class QueryParser {
 
     /** A pattern for commands having internal implementation in Ignite. */
     private static final Pattern INTERNAL_CMD_RE = Pattern.compile(
-        "^(create|drop)\\s+index|^alter\\s+table|^copy|^set|^begin|^commit|^rollback|^(create|alter|drop)\\s+user",
+        "^(create|drop)\\s+index|^alter\\s+table|^copy|^set|^begin|^commit|^rollback|^(create|alter|drop)\\s+user" +
+            "|show|help|grant|revoke",
         Pattern.CASE_INSENSITIVE);
 
     /** Indexing. */
@@ -148,7 +148,7 @@ public class QueryParser {
                 cached.command()
             );
 
-        // Try parting as native command.
+        // Try parsing as native command.
         QueryParserResult parseRes = parseNative(schemaName, qry, remainingAllowed);
 
         // Otherwise parse with H2.
@@ -229,7 +229,7 @@ public class QueryParser {
             );
         }
         catch (SqlStrictParseException e) {
-            throw new IgniteSQLException(e.getMessage(), IgniteQueryErrorCode.PARSING, e);
+            throw new IgniteSQLException(e.getMessage(), e.errorCode(), e);
         }
         catch (Exception e) {
             // Cannot parse, return.
@@ -241,7 +241,8 @@ public class QueryParser {
 
             int code = IgniteQueryErrorCode.PARSING;
 
-            if (e instanceof SqlParseException)                code = ((SqlParseException)e).code();
+            if (e instanceof SqlParseException)
+                code = ((SqlParseException)e).code();
 
             throw new IgniteSQLException("Failed to parse DDL statement: " + sql + ": " + e.getMessage(),
                 code, e);
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlKeyword.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlKeyword.java
index 6c05928..40bdaf4 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlKeyword.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlKeyword.java
@@ -18,15 +18,14 @@
 package org.apache.ignite.internal.processors.query.h2.sql;
 
 import java.util.Collections;
-import org.h2.command.Parser;
 import org.h2.expression.ValueExpression;
 
 /** Keyword (like DEFAULT). */
 public final class GridSqlKeyword extends GridSqlElement {
     /**
      * Default update value - analogous to H2.
+     *
      * @see ValueExpression#getDefault()
-     * @see Parser#parseUpdate()
      */
     public static final GridSqlKeyword DEFAULT = new GridSqlKeyword("DEFAULT");
 
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java
index 132a5af..48ccb84 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java
@@ -19,7 +19,6 @@ package org.apache.ignite.internal.processors.query.h2.sql;
 
 import java.lang.reflect.Field;
 import java.sql.PreparedStatement;
-import java.sql.SQLException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
@@ -103,6 +102,7 @@ import org.h2.table.TableBase;
 import org.h2.table.TableFilter;
 import org.h2.table.TableView;
 import org.h2.value.DataType;
+import org.h2.value.Value;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
 
@@ -464,6 +464,10 @@ public class GridSqlQueryParser {
         getter(AlterTableAlterColumn.class, "addBefore");
 
     /** */
+    private static final Getter<AlterTableAlterColumn, Boolean> ALTER_COLUMN_FIRST =
+        getter(AlterTableAlterColumn.class, "addFirst");
+
+    /** */
     private static final Getter<AlterTableAlterColumn, String> ALTER_COLUMN_AFTER_COL =
         getter(AlterTableAlterColumn.class, "addAfter");
 
@@ -605,9 +609,10 @@ public class GridSqlQueryParser {
         boolean forUpdate = (!union && SELECT_IS_FOR_UPDATE.get((Select)p)) ||
             (union && UNION_IS_FOR_UPDATE.get((SelectUnion)p));
 
-        if (union && forUpdate)
+        if (union && forUpdate) {
             throw new IgniteSQLException("SELECT UNION FOR UPDATE is not supported.",
                 IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
+        }
 
         return forUpdate;
     }
@@ -711,6 +716,11 @@ public class GridSqlQueryParser {
             // Other stuff can be cached because we will have separate instances in
             // different table filters anyways. Thus the semantics will be correct.
             if (tbl instanceof TableView) {
+                if (((TableView)tbl).isRecursive()) {
+                    throw new IgniteSQLException("Recursive CTE ('WITH RECURSIVE (...)') is not supported.",
+                        IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
+                }
+
                 Query qry = VIEW_QUERY.get((TableView) tbl);
 
                 res = new GridSqlSubquery(parseQuery(qry));
@@ -791,26 +801,30 @@ public class GridSqlQueryParser {
 
         if (isForUpdate) {
             if (!(from instanceof GridSqlTable ||
-                (from instanceof GridSqlAlias && from.size() == 1 && from.child() instanceof GridSqlTable)))
+                (from instanceof GridSqlAlias && from.size() == 1 && from.child() instanceof GridSqlTable))) {
                 throw new IgniteSQLException("SELECT FOR UPDATE with joins is not supported.",
                     IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
+            }
 
             GridSqlTable gridTbl = from instanceof GridSqlTable ? (GridSqlTable)from :
                 ((GridSqlAlias)from).child();
 
             GridH2Table tbl = gridTbl.dataTable();
 
-            if (tbl == null)
+            if (tbl == null) {
                 throw new IgniteSQLException("SELECT FOR UPDATE query must involve Ignite table.",
                     IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
+            }
 
-            if (select.getLimit() != null || select.getOffset() != null)
+            if (select.getLimit() != null || select.getOffset() != null) {
                 throw new IgniteSQLException("LIMIT/OFFSET clauses are not supported for SELECT FOR UPDATE.",
                     IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
+            }
 
-            if (SELECT_IS_GROUP_QUERY.get(select))
+            if (SELECT_IS_GROUP_QUERY.get(select)) {
                 throw new IgniteSQLException("SELECT FOR UPDATE with aggregates and/or GROUP BY is not supported.",
                     IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
+            }
         }
 
         ArrayList<Expression> expressions = select.getExpressions();
@@ -895,9 +909,10 @@ public class GridSqlQueryParser {
 
             int colId = intoTbl.getColumn(colName).getColumnId();
 
-            if (!rowDesc.isKeyColumn(colId) && !F.eq(colName, rowDesc.type().affinityKey()))
+            if (!rowDesc.isKeyColumn(colId) && !F.eq(colName, rowDesc.type().affinityKey())) {
                 throw new IgniteSQLException("Invalid column name in KEYS clause of MERGE - it may include only " +
                     "key and/or affinity columns: " + colName, IgniteQueryErrorCode.PARSING);
+            }
 
             keys[i] = new GridSqlColumn(srcKeys[i], tbl, null, null, colName);
         }
@@ -910,9 +925,15 @@ public class GridSqlQueryParser {
             for (Expression[] srcRow : srcRows) {
                 GridSqlElement[] row = new GridSqlElement[srcRow.length];
 
-                for (int i = 0; i < srcRow.length; i++)
+                for (int i = 0; i < srcRow.length; i++) {
                     row[i] = parseExpression(srcRow[i], false);
 
+                    if (row[i] == null) {
+                        throw new IgniteSQLException("DEFAULT values are unsupported for MERGE.",
+                            IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
+                    }
+                }
+
                 rows.add(row);
             }
             res.rows(rows);
@@ -962,9 +983,15 @@ public class GridSqlQueryParser {
             for (Expression[] srcRow : srcRows) {
                 GridSqlElement[] row = new GridSqlElement[srcRow.length];
 
-                for (int i = 0; i < srcRow.length; i++)
+                for (int i = 0; i < srcRow.length; i++) {
                     row[i] = parseExpression(srcRow[i], false);
 
+                    if (row[i] == null) {
+                        throw new IgniteSQLException("DEFAULT values are unsupported for MERGE.",
+                            IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
+                    }
+                }
+
                 rows.add(row);
             }
             res.rows(rows);
@@ -1022,7 +1049,15 @@ public class GridSqlQueryParser {
             GridSqlColumn col = new GridSqlColumn(c, tbl, null, null, c.getName());
             col.resultType(fromColumn(c));
             cols.add(col);
-            set.put(col.columnName(), parseExpression(srcSet.get(c), true));
+
+            GridSqlElement setVal = parseExpression(srcSet.get(c), true);
+
+            if (containsDefaultKeyword(setVal)) {
+                throw new IgniteSQLException("DEFAULT values are unsupported for UPDATE.",
+                    IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
+            }
+
+            set.put(col.columnName(), setVal);
         }
 
         GridSqlElement where = parseExpression(UPDATE_WHERE.get(update), true);
@@ -1032,7 +1067,21 @@ public class GridSqlQueryParser {
         return res;
     }
 
+    /**
+     * @param val SQL expression.
+     * @return {@code true} if the expression contains DEFAULT keyword.
+     */
+    private boolean containsDefaultKeyword(GridSqlAst val) {
+        if (val == GridSqlKeyword.DEFAULT)
+            return true;
+
+        for (int i = 0; i < val.size(); ++i) {
+            if (containsDefaultKeyword(val.child(i)))
+                return true;
+        }
 
+        return false;
+    }
 
     /**
      * Parse {@code DROP INDEX} statement.
@@ -1058,9 +1107,10 @@ public class GridSqlQueryParser {
      */
     private GridSqlCreateIndex parseCreateIndex(CreateIndex createIdx) {
         if (CREATE_INDEX_HASH.get(createIdx) || CREATE_INDEX_PRIMARY_KEY.get(createIdx) ||
-            CREATE_INDEX_UNIQUE.get(createIdx))
+            CREATE_INDEX_UNIQUE.get(createIdx)) {
             throw new IgniteSQLException("Only SPATIAL modifier is supported for CREATE INDEX",
                 IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
+        }
 
         GridSqlCreateIndex res = new GridSqlCreateIndex();
 
@@ -1084,9 +1134,10 @@ public class GridSqlQueryParser {
         for (IndexColumn col : CREATE_INDEX_COLUMNS.get(createIdx)) {
             int sortType = INDEX_COLUMN_SORT_TYPE.get(col);
 
-            if ((sortType & SortOrder.NULLS_FIRST) != 0 || (sortType & SortOrder.NULLS_LAST) != 0)
+            if ((sortType & SortOrder.NULLS_FIRST) != 0 || (sortType & SortOrder.NULLS_LAST) != 0) {
                 throw new IgniteSQLException("NULLS FIRST and NULLS LAST modifiers are not supported for index columns",
                     IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
+            }
 
             flds.put(INDEX_COLUMN_NAME.get(col), (sortType & SortOrder.DESCENDING) == 0);
         }
@@ -1111,31 +1162,36 @@ public class GridSqlQueryParser {
 
         Query qry = CREATE_TABLE_QUERY.get(createTbl);
 
-        if (qry != null)
+        if (qry != null) {
             throw new IgniteSQLException("CREATE TABLE ... AS ... syntax is not supported",
                 IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
+        }
 
         List<DefineCommand> constraints = CREATE_TABLE_CONSTRAINTS.get(createTbl);
 
-        if (F.isEmpty(constraints))
+        if (F.isEmpty(constraints)) {
             throw new IgniteSQLException("No PRIMARY KEY defined for CREATE TABLE",
                 IgniteQueryErrorCode.PARSING);
+        }
 
-        if (constraints.size() > 1)
+        if (constraints.size() > 1) {
             throw new IgniteSQLException("Too many constraints - only PRIMARY KEY is supported for CREATE TABLE",
                 IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
+        }
 
         DefineCommand constraint = constraints.get(0);
 
-        if (!(constraint instanceof AlterTableAddConstraint))
+        if (!(constraint instanceof AlterTableAddConstraint)) {
             throw new IgniteSQLException("Unsupported type of constraint for CREATE TABLE - only PRIMARY KEY " +
                 "is supported", IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
+        }
 
         AlterTableAddConstraint alterTbl = (AlterTableAddConstraint)constraint;
 
-        if (alterTbl.getType() != Command.ALTER_TABLE_ADD_CONSTRAINT_PRIMARY_KEY)
+        if (alterTbl.getType() != Command.ALTER_TABLE_ADD_CONSTRAINT_PRIMARY_KEY) {
             throw new IgniteSQLException("Unsupported type of constraint for CREATE TABLE - only PRIMARY KEY " +
                 "is supported", IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
+        }
 
         Schema schema = SCHEMA_COMMAND_SCHEMA.get(createTbl);
 
@@ -1143,15 +1199,36 @@ public class GridSqlQueryParser {
 
         CreateTableData data = CREATE_TABLE_DATA.get(createTbl);
 
+        if (data.globalTemporary) {
+            throw new IgniteSQLException("GLOBAL TEMPORARY keyword is not supported",
+                IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
+        }
+
+        if (data.temporary) {
+            throw new IgniteSQLException("TEMPORARY keyword is not supported",
+                IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
+        }
+
+        if (data.isHidden) {
+            throw new IgniteSQLException("HIDDEN keyword is not supported",
+                IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
+        }
+
+        if (!data.persistIndexes) {
+            throw new IgniteSQLException("MEMORY and NOT PERSISTENT keywords are not supported",
+                IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
+        }
+
         LinkedHashMap<String, GridSqlColumn> cols = new LinkedHashMap<>(data.columns.size());
 
         for (Column col : data.columns)
             cols.put(col.getName(), parseColumn(col));
 
         if (cols.containsKey(QueryUtils.KEY_FIELD_NAME.toUpperCase()) ||
-            cols.containsKey(QueryUtils.VAL_FIELD_NAME.toUpperCase()))
+            cols.containsKey(QueryUtils.VAL_FIELD_NAME.toUpperCase())) {
             throw new IgniteSQLException("Direct specification of _KEY and _VAL columns is forbidden",
                 IgniteQueryErrorCode.PARSING);
+        }
 
         IndexColumn[] pkIdxCols = CREATE_TABLE_PK.get(createTbl);
 
@@ -1163,9 +1240,10 @@ public class GridSqlQueryParser {
         for (IndexColumn pkIdxCol : pkIdxCols) {
             GridSqlColumn gridCol = cols.get(pkIdxCol.columnName);
 
-            if (gridCol == null)
+            if (gridCol == null) {
                 throw new IgniteSQLException("PRIMARY KEY column is not defined: " + pkIdxCol.columnName,
                     IgniteQueryErrorCode.PARSING);
+            }
 
             pkCols.add(gridCol.columnName());
         }
@@ -1173,9 +1251,10 @@ public class GridSqlQueryParser {
         int keyColsNum = pkCols.size();
         int valColsNum = cols.size() - keyColsNum;
 
-        if (valColsNum == 0)
+        if (valColsNum == 0) {
             throw new IgniteSQLException("Table must have at least one non PRIMARY KEY column.",
                 IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
+        }
 
         res.columns(cols);
         res.primaryKeyColumns(pkCols);
@@ -1196,17 +1275,19 @@ public class GridSqlQueryParser {
             for (String p : extraParams) {
                 String[] parts = p.split(PARAM_NAME_VALUE_SEPARATOR);
 
-                if (parts.length > 2)
+                if (parts.length > 2) {
                     throw new IgniteSQLException("Invalid parameter (key[=value] expected): " + p,
                         IgniteQueryErrorCode.PARSING);
+                }
 
                 String name = parts[0].trim().toUpperCase();
 
                 String val = parts.length > 1 ? parts[1].trim() : null;
 
-                if (F.isEmpty(name))
+                if (F.isEmpty(name)) {
                     throw new IgniteSQLException("Invalid parameter (key[=value] expected): " + p,
                         IgniteQueryErrorCode.PARSING);
+                }
 
                 if (params.put(name, val) != null)
                     throw new IgniteSQLException("Duplicate parameter: " + p, IgniteQueryErrorCode.PARSING);
@@ -1254,9 +1335,10 @@ public class GridSqlQueryParser {
         else
             res.wrapValue(true); // By default value is always wrapped to allow for ALTER TABLE ADD COLUMN commands.
 
-        if (!F.isEmpty(res.valueTypeName()) && F.eq(res.keyTypeName(), res.valueTypeName()))
+        if (!F.isEmpty(res.valueTypeName()) && F.eq(res.keyTypeName(), res.valueTypeName())) {
             throw new IgniteSQLException("Key and value type names " +
                 "should be different for CREATE TABLE: " + res.valueTypeName(), IgniteQueryErrorCode.PARSING);
+        }
 
         if (res.affinityKey() == null) {
             LinkedHashSet<String> pkCols0 = res.primaryKeyColumns();
@@ -1334,13 +1416,17 @@ public class GridSqlQueryParser {
      * @return Grid column.
      */
     private static GridSqlColumn parseColumn(Column col) {
-        if (col.isAutoIncrement())
+        if (col.isAutoIncrement()) {
             throw new IgniteSQLException("AUTO_INCREMENT columns are not supported [colName=" + col.getName() + ']',
                 IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
+        }
 
-        if (COLUMN_IS_COMPUTED.get(col))
+        if (COLUMN_IS_COMPUTED.get(col)) {
             throw new IgniteSQLException("Computed columns are not supported [colName=" + col.getName() + ']',
                 IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
+        }
+
+        checkTypeSupported(col.getType(), "[colName=" + col.getName() + ']');
 
         if (col.getDefaultExpression() != null) {
             if (!col.getDefaultExpression().isConstant()) {
@@ -1364,13 +1450,15 @@ public class GridSqlQueryParser {
             throw new IgniteSQLException("SEQUENCE columns are not supported [colName=" + col.getName() + ']',
                 IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
 
-        if (col.getSelectivity() != Constants.SELECTIVITY_DEFAULT)
+        if (col.getSelectivity() != Constants.SELECTIVITY_DEFAULT) {
             throw new IgniteSQLException("SELECTIVITY column attribute is not supported [colName=" + col.getName() + ']',
                 IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
+        }
 
-        if (COLUMN_CHECK_CONSTRAINT.get(col) != null)
+        if (COLUMN_CHECK_CONSTRAINT.get(col) != null) {
             throw new IgniteSQLException("Column CHECK constraints are not supported [colName=" + col.getName() +
                 ']', IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
+        }
 
         GridSqlColumn gridCol = new GridSqlColumn(col, null, col.getName());
 
@@ -1389,9 +1477,14 @@ public class GridSqlQueryParser {
     private GridSqlStatement parseAddColumn(AlterTableAlterColumn addCol) {
         assert addCol.getType() == CommandInterface.ALTER_TABLE_ADD_COLUMN;
 
-        if (ALTER_COLUMN_BEFORE_COL.get(addCol) != null || ALTER_COLUMN_AFTER_COL.get(addCol) != null)
-            throw new IgniteSQLException("ALTER TABLE ADD COLUMN BEFORE/AFTER is not supported" ,
-                IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
+        if (ALTER_COLUMN_BEFORE_COL.get(addCol) != null )
+            throw new IgniteSQLException("BEFORE keyword is not supported", IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
+
+        if (ALTER_COLUMN_AFTER_COL.get(addCol) != null)
+            throw new IgniteSQLException("AFTER keyword is not supported", IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
+
+        if (ALTER_COLUMN_FIRST.get(addCol))
+            throw new IgniteSQLException("FIRST keyword is not supported", IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
 
         GridSqlAlterTableAddColumn res = new GridSqlAlterTableAddColumn();
 
@@ -1402,9 +1495,10 @@ public class GridSqlQueryParser {
         for (int i = 0; i < h2NewCols.size(); i++) {
             Column col = h2NewCols.get(i);
 
-            if (col.getDefaultExpression() != null)
+            if (col.getDefaultExpression() != null) {
                 throw new IgniteSQLException("ALTER TABLE ADD COLUMN with DEFAULT value is not supported " +
                     "[col=" + col.getName() + ']', IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
+            }
 
             gridNewCols[i] = parseColumn(h2NewCols.get(i));
         }
@@ -1479,9 +1573,10 @@ public class GridSqlQueryParser {
 
                 int backups = parseIntParam(PARAM_BACKUPS, val);
 
-                if (backups < 0)
+                if (backups < 0) {
                     throw new IgniteSQLException("\"" + PARAM_BACKUPS + "\" cannot be negative: " + backups,
                         IgniteQueryErrorCode.PARSING);
+                }
 
                 res.backups(backups);
 
@@ -1511,9 +1606,10 @@ public class GridSqlQueryParser {
                     atomicityMode = CacheAtomicityMode.ATOMIC;
                 else if (CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT.name().equalsIgnoreCase(val))
                     atomicityMode = CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT;
-                else
+                else {
                     throw new IgniteSQLException("Invalid value of \"" + PARAM_ATOMICITY + "\" parameter " +
                         "(should be either TRANSACTIONAL or ATOMIC): " + val, IgniteQueryErrorCode.PARSING);
+                }
 
                 res.atomicityMode(atomicityMode);
 
@@ -1556,9 +1652,10 @@ public class GridSqlQueryParser {
 
                 // Either strip column name off its quotes, or uppercase it.
                 if (val.startsWith("'")) {
-                    if (val.length() == 1 || !val.endsWith("'"))
+                    if (val.length() == 1 || !val.endsWith("'")) {
                         throw new IgniteSQLException("Affinity key column name does not have trailing quote: " + val,
                             IgniteQueryErrorCode.PARSING);
+                    }
 
                     val = val.substring(1, val.length() - 1);
 
@@ -1569,22 +1666,25 @@ public class GridSqlQueryParser {
                 else {
                     for (String colName : res.columns().keySet()) {
                         if (val.equalsIgnoreCase(colName)) {
-                            if (affColName != null)
+                            if (affColName != null) {
                                 throw new IgniteSQLException("Ambiguous affinity column name, use single quotes " +
                                     "for case sensitivity: " + val, IgniteQueryErrorCode.PARSING);
+                            }
 
                             affColName = colName;
                         }
                     }
                 }
 
-                if (affColName == null || !res.columns().containsKey(affColName))
+                if (affColName == null || !res.columns().containsKey(affColName)) {
                     throw new IgniteSQLException("Affinity key column with given name not found: " + val,
                         IgniteQueryErrorCode.PARSING);
+                }
 
-                if (!res.primaryKeyColumns().contains(affColName))
+                if (!res.primaryKeyColumns().contains(affColName)) {
                     throw new IgniteSQLException("Affinity key column must be one of key columns: " + affColName,
                         IgniteQueryErrorCode.PARSING);
+                }
 
                 res.affinityKey(affColName);
 
@@ -1601,9 +1701,10 @@ public class GridSqlQueryParser {
                     writeSyncMode = CacheWriteSynchronizationMode.FULL_SYNC;
                 else if (CacheWriteSynchronizationMode.PRIMARY_SYNC.name().equalsIgnoreCase(val))
                     writeSyncMode = CacheWriteSynchronizationMode.PRIMARY_SYNC;
-                else
+                else {
                     throw new IgniteSQLException("Invalid value of \"" + PARAM_WRITE_SYNC + "\" parameter " +
                         "(should be FULL_SYNC, FULL_ASYNC, or PRIMARY_SYNC): " + val, IgniteQueryErrorCode.PARSING);
+                }
 
                 res.writeSynchronizationMode(writeSyncMode);
 
@@ -1909,7 +2010,8 @@ public class GridSqlQueryParser {
         if (stmt instanceof AlterTableAlterColumn)
             return parseAlterColumn((AlterTableAlterColumn)stmt);
 
-        throw new CacheException("Unsupported SQL statement: " + stmt);
+        throw new IgniteSQLException("Unsupported statement: " + stmt,
+            IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
     }
 
     /**
@@ -2231,8 +2333,11 @@ public class GridSqlQueryParser {
                 }
             }
 
-            if (f.getFunctionType() == Function.CAST || f.getFunctionType() == Function.CONVERT)
+            if (f.getFunctionType() == Function.CAST || f.getFunctionType() == Function.CONVERT) {
+                checkTypeSupported(f.getType(), "[expSql=" + f.getSQL() + ']');
+
                 res.resultType(fromExpression(f));
+            }
 
             return res;
         }
@@ -2372,6 +2477,20 @@ public class GridSqlQueryParser {
     }
 
     /**
+     */
+    public static void checkTypeSupported(int type, String errMsg) {
+        if (type == Value.TIMESTAMP_TZ) {
+            throw new IgniteSQLException("TIMESTAMP WITH TIMEZONE type is not supported " + errMsg,
+                IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
+        }
+
+        if (type == Value.ENUM) {
+            throw new IgniteSQLException("ENUM type is not supported " + errMsg,
+                IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
+        }
+    }
+
+    /**
      * @param cls Class.
      * @param fldName Fld name.
      */
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheUpdateSqlQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheUpdateSqlQuerySelfTest.java
index 5599271..d17ca0b 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheUpdateSqlQuerySelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheUpdateSqlQuerySelfTest.java
@@ -182,39 +182,6 @@ public class IgniteCacheUpdateSqlQuerySelfTest extends IgniteCacheAbstractSqlDml
         }
     }
 
-
-    /**
-     *
-     */
-    @Test
-    public void testDefault() {
-        IgniteCache p = cache();
-
-        QueryCursor<List<?>> c = p.query(new SqlFieldsQuery(
-            "UPDATE Person p SET id = DEFAULT, firstName = ?, secondName = ? WHERE _key = ?"
-        ).setArgs( "Jo", "Woo", "FirstKey"));
-
-        c.iterator();
-
-        c = p.query(new SqlFieldsQuery("select _key, _val, * from Person order by _key, id"));
-
-        List<List<?>> leftovers = c.getAll();
-
-        assertEquals(4, leftovers.size());
-
-        assertEqualsCollections(asList("FirstKey", createPerson(0, "Jo", "Woo"), 0, "Jo", "Woo"),
-            leftovers.get(0));
-
-        assertEqualsCollections(asList("SecondKey", createPerson(2, "Joe", "Black"), 2, "Joe", "Black"),
-            leftovers.get(1));
-
-        assertEqualsCollections(asList("f0u4thk3y", createPerson(4, "Jane", "Silver"), 4, "Jane", "Silver"),
-            leftovers.get(2));
-
-        assertEqualsCollections(asList("k3", createPerson(3, "Sylvia", "Green"), 3, "Sylvia", "Green"),
-            leftovers.get(3));
-    }
-
     /** */
     @Test
     public void testTypeConversions() throws ParseException {
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java
index d76acab..47b636f 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java
@@ -383,24 +383,24 @@ public class GridQueryParsingTest extends AbstractIndexingCommonTest {
     public void testParseMerge() throws Exception {
         /* Plain rows w/functions, operators, defaults, and placeholders. */
         checkQuery("merge into Person(old, name) values(5, 'John')");
-        checkQuery("merge into Person(name) values(DEFAULT)");
-        checkQuery("merge into Person(name) values(DEFAULT), (null)");
-        checkQuery("merge into Person(name, parentName) values(DEFAULT, null), (?, ?)");
+        checkQuery("merge into Person(name) values(null)");
+        checkQuery("merge into Person(name) values(null), (null)");
+        checkQuery("merge into Person(name, parentName) values(null, null), (?, ?)");
         checkQuery("merge into Person(old, name) values(5, 'John',), (6, 'Jack')");
-        checkQuery("merge into Person(old, name) values(5 * 3, DEFAULT,)");
+        checkQuery("merge into Person(old, name) values(5 * 3, null,)");
         checkQuery("merge into Person(old, name) values(ABS(-8), 'Max')");
-        checkQuery("merge into Person(old, name) values(5, 'Jane'), (DEFAULT, DEFAULT), (6, 'Jill')");
-        checkQuery("merge into Person(old, name, parentName) values(8 * 7, DEFAULT, 'Unknown')");
+        checkQuery("merge into Person(old, name) values(5, 'Jane'), (null, null), (6, 'Jill')");
+        checkQuery("merge into Person(old, name, parentName) values(8 * 7, null, 'Unknown')");
         checkQuery("merge into Person(old, name, parentName) values" +
             "(2016 - 1828, CONCAT('Leo', 'Tolstoy'), CONCAT(?, 'Tolstoy'))," +
             "(?, 'AlexanderPushkin', null)," +
-            "(ABS(1821 - 2016), CONCAT('Fyodor', null, UPPER(CONCAT(SQRT(?), 'dostoevsky'))), DEFAULT)");
+            "(ABS(1821 - 2016), CONCAT('Fyodor', null, UPPER(CONCAT(SQRT(?), 'dostoevsky'))), null)");
         checkQuery("merge into Person(date, old, name, parentName, addrId) values " +
             "('20160112', 1233, 'Ivan Ivanov', 'Peter Ivanov', 123)");
         checkQuery("merge into Person(date, old, name, parentName, addrId) values " +
             "(CURRENT_DATE(), RAND(), ASCII('Hi'), INSERT('Leo Tolstoy', 4, 4, 'Max'), ASCII('HI'))");
         checkQuery("merge into Person(date, old, name, parentName, addrId) values " +
-            "(TRUNCATE(TIMESTAMP '2015-12-31 23:59:59'), POWER(3,12), NULL, DEFAULT, DEFAULT)");
+            "(TRUNCATE(TIMESTAMP '2015-12-31 23:59:59'), POWER(3,12), NULL, NULL, NULL)");
         checkQuery("merge into Person(old, name) select ASCII(parentName), INSERT(parentName, 4, 4, 'Max') from " +
             "Person where date='2011-03-12'");
 
@@ -430,27 +430,26 @@ public class GridQueryParsingTest extends AbstractIndexingCommonTest {
     public void testParseInsert() throws Exception {
         /* Plain rows w/functions, operators, defaults, and placeholders. */
         checkQuery("insert into Person(old, name) values(5, 'John')");
-        checkQuery("insert into Person(name) values(DEFAULT)");
-        checkQuery("insert into Person default values");
+        checkQuery("insert into Person(name) values(null)");
         checkQuery("insert into Person() values()");
-        checkQuery("insert into Person(name) values(DEFAULT), (null)");
-        checkQuery("insert into Person(name) values(DEFAULT),");
-        checkQuery("insert into Person(name, parentName) values(DEFAULT, null), (?, ?)");
+        checkQuery("insert into Person(name) values(null), (null)");
+        checkQuery("insert into Person(name) values(null),");
+        checkQuery("insert into Person(name, parentName) values(null, null), (?, ?)");
         checkQuery("insert into Person(old, name) values(5, 'John',), (6, 'Jack')");
-        checkQuery("insert into Person(old, name) values(5 * 3, DEFAULT,)");
+        checkQuery("insert into Person(old, name) values(5 * 3, null,)");
         checkQuery("insert into Person(old, name) values(ABS(-8), 'Max')");
-        checkQuery("insert into Person(old, name) values(5, 'Jane'), (DEFAULT, DEFAULT), (6, 'Jill')");
-        checkQuery("insert into Person(old, name, parentName) values(8 * 7, DEFAULT, 'Unknown')");
+        checkQuery("insert into Person(old, name) values(5, 'Jane'), (null, null), (6, 'Jill')");
+        checkQuery("insert into Person(old, name, parentName) values(8 * 7, null, 'Unknown')");
         checkQuery("insert into Person(old, name, parentName) values" +
             "(2016 - 1828, CONCAT('Leo', 'Tolstoy'), CONCAT(?, 'Tolstoy'))," +
             "(?, 'AlexanderPushkin', null)," +
-            "(ABS(1821 - 2016), CONCAT('Fyodor', null, UPPER(CONCAT(SQRT(?), 'dostoevsky'))), DEFAULT),");
+            "(ABS(1821 - 2016), CONCAT('Fyodor', null, UPPER(CONCAT(SQRT(?), 'dostoevsky'))), null),");
         checkQuery("insert into Person(date, old, name, parentName, addrId) values " +
             "('20160112', 1233, 'Ivan Ivanov', 'Peter Ivanov', 123)");
         checkQuery("insert into Person(date, old, name, parentName, addrId) values " +
             "(CURRENT_DATE(), RAND(), ASCII('Hi'), INSERT('Leo Tolstoy', 4, 4, 'Max'), ASCII('HI'))");
         checkQuery("insert into Person(date, old, name, parentName, addrId) values " +
-            "(TRUNCATE(TIMESTAMP '2015-12-31 23:59:59'), POWER(3,12), NULL, DEFAULT, DEFAULT)");
+            "(TRUNCATE(TIMESTAMP '2015-12-31 23:59:59'), POWER(3,12), NULL, NULL, NULL)");
         checkQuery("insert into Person SET old = 5, name = 'John'");
         checkQuery("insert into Person SET name = CONCAT('Fyodor', null, UPPER(CONCAT(SQRT(?), 'dostoevsky'))), " +
             "old = select (5, 6)");
@@ -487,7 +486,6 @@ public class GridQueryParsingTest extends AbstractIndexingCommonTest {
         checkQuery("update Person per set name='Peter', old = 5");
         checkQuery("update Person p set name='Peter' limit 20");
         checkQuery("update Person p set name='Peter', old = length('zzz') limit 20");
-        checkQuery("update Person p set name=DEFAULT, old = null limit ?");
         checkQuery("update Person p set name=? where old >= ? and old < ? limit ?");
         checkQuery("update Person p set name=(select a.Street from sch2.Address a where a.id=p.addrId), old = " +
             "(select 42) where old = sqrt(?)");
@@ -702,9 +700,16 @@ public class GridQueryParsingTest extends AbstractIndexingCommonTest {
         assertParseThrows("ALTER TABLE IF EXISTS SCH2.Person ADD if not exists (company varchar, city varchar)",
             DbException.class, null);
 
-        // Both BEFORE and AFTER keywords.
+        // Both BEFORE keyword.
         assertParseThrows("ALTER TABLE IF EXISTS SCH2.Person ADD if not exists company varchar before addrid",
-            IgniteSQLException.class, "ALTER TABLE ADD COLUMN BEFORE/AFTER is not supported");
+            IgniteSQLException.class, "BEFORE keyword is not supported");
+
+        // Both AFTER keyword.
+        assertParseThrows("ALTER TABLE IF EXISTS SCH2.Person ADD if not exists company varchar after addrid",
+            IgniteSQLException.class, "AFTER keyword is not supported");
+
+        assertParseThrows("ALTER TABLE IF EXISTS SCH2.Person ADD if not exists company varchar first",
+            IgniteSQLException.class, "FIRST keyword is not supported");
 
         // No such schema.
         assertParseThrows("ALTER TABLE SCH5.\"Person\" ADD (city varchar)", DbException.class, null);
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/SqlUnsupportedSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/SqlUnsupportedSelfTest.java
new file mode 100644
index 0000000..66233bb
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/SqlUnsupportedSelfTest.java
@@ -0,0 +1,332 @@
+/*
+ * 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.h2.sql;
+
+import java.util.List;
+import java.util.concurrent.Callable;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.processors.cache.index.AbstractIndexingCommonTest;
+import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
+import org.apache.ignite.internal.processors.query.IgniteSQLException;
+import org.apache.ignite.internal.util.typedef.X;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.junit.Test;
+
+/**
+ * Tests for unsupported SQL statements.
+ */
+public class SqlUnsupportedSelfTest extends AbstractIndexingCommonTest {
+    /** Local. */
+    private boolean local;
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        cleanPersistenceDir();
+
+        startGrid();
+        startGrid(1);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+
+        cleanPersistenceDir();
+    }
+
+    /**
+     * Test for unsupported SQL types.
+     */
+    @Test
+    public void testUnsupportedTypes() {
+        assertSqlUnsupported("CREATE TABLE test (id integer PRIMARY KEY, val TIMESTAMP WITH TIME ZONE)");
+        assertSqlUnsupported("CREATE TABLE test (id integer PRIMARY KEY, val ENUM ('A', 'B', 'C'))");
+
+        execSql("CREATE TABLE test (id integer PRIMARY KEY, val TIMESTAMP)");
+
+        assertSqlUnsupported("SELECT CAST (val as TIMESTAMP WITH TIME ZONE) FROM test ");
+
+        // H2 bug. Fixed at H2 version 1.4.198
+        // assertSqlUnsupported("SELECT CAST (id AS ENUM('A', 'B')) FROM test ");
+    }
+
+
+    /**
+     * Test for unsupported SQL statements in CREATE TABLE statement.
+     */
+    @Test
+    public void testUnsupportedCreateTable() {
+        assertSqlUnsupported("CREATE MEMORY TABLE unsupported_tbl0 (id integer primary key, val integer)",
+            "MEMORY and NOT PERSISTENT keywords are not supported");
+        assertSqlUnsupported("CREATE GLOBAL TEMPORARY TABLE unsupported_tbl1 (id integer primary key, val integer)",
+            "GLOBAL TEMPORARY keyword is not supported");
+        assertSqlUnsupported("CREATE LOCAL TEMPORARY TABLE unsupported_tbl2 (id integer primary key, val integer)",
+            "TEMPORARY keyword is not supported");
+        assertSqlUnsupported("CREATE TEMPORARY TABLE unsupported_tbl3 (id integer primary key, val integer)",
+            "TEMPORARY keyword is not supported");
+        assertSqlUnsupported("CREATE TABLE unsupported_tbl4 (id integer primary key, val integer) HIDDEN",
+            "HIDDEN keyword is not supported");
+    }
+
+    /**
+     * Test for unsupported SQL statements in CREATE TABLE statement.
+     */
+    @Test
+    public void testUnsupportedCreateIndex() {
+        execSql(
+            "CREATE TABLE test ( " +
+                "id integer PRIMARY KEY, " +
+                "val varchar DEFAULT 'test_val')");
+
+        assertSqlUnsupported("CREATE INDEX test_idx ON test (val NULLS FIRST)");
+        assertSqlUnsupported("CREATE INDEX test_idx ON test (val NULLS LAST)");
+        assertSqlUnsupported("CREATE UNIQUE INDEX test_idx ON test (val)");
+        assertSqlUnsupported("CREATE HASH INDEX test_idx ON test (val)");
+    }
+
+    /**
+     * Test for unsupported DEFAULT value at the INSERT/UPDATE/MERGE SQL statements.
+     */
+    @Test
+    public void testUnsupportedDefault() {
+        execSql(
+            "CREATE TABLE test ( " +
+                "id integer PRIMARY KEY, " +
+                "val varchar DEFAULT 'test_val')");
+
+        assertSqlUnsupported("INSERT INTO test (id, val) VALUES (0, DEFAULT)");
+        assertSqlUnsupported("MERGE INTO test (id, val) VALUES (0, DEFAULT)");
+        assertSqlUnsupported("UPDATE test SET val=DEFAULT");
+    }
+
+    /**
+     * Test for unsupported SQL statements in CREATE TABLE statement.
+     */
+    @Test
+    public void testUnsupportedAlterTableColumn() {
+        execSql(
+            "CREATE TABLE test ( " +
+                "id integer PRIMARY KEY, " +
+                "val varchar DEFAULT 'test_val')");
+
+        assertSqlUnsupported("ALTER TABLE test ALTER COLUMN val SELECTIVITY 1");
+        assertSqlUnsupported("ALTER TABLE test ALTER COLUMN val SET DEFAULT 'new val'");
+        assertSqlUnsupported("ALTER TABLE test ALTER COLUMN val DROP DEFAULT");
+        assertSqlUnsupported("ALTER TABLE test ALTER COLUMN val SET ON UPDATE 'new val'");
+        assertSqlUnsupported("ALTER TABLE test ALTER COLUMN val DROP ON UPDATE");
+        assertSqlUnsupported("ALTER TABLE test ALTER COLUMN val SET NULL");
+        assertSqlUnsupported("ALTER TABLE test ALTER COLUMN val SET NOT NULL");
+        assertSqlUnsupported("ALTER TABLE test ALTER COLUMN val SET VISIBLE");
+        assertSqlUnsupported("ALTER TABLE test ALTER COLUMN val SET INVISIBLE");
+
+        assertSqlUnsupported("ALTER TABLE test ADD COLUMN (q integer) FIRST",
+            "FIRST keyword is not supported");
+        assertSqlUnsupported("ALTER TABLE test ADD COLUMN (q integer) BEFORE val",
+            "BEFORE keyword is not supported");
+        assertSqlUnsupported("ALTER TABLE test ADD COLUMN (q integer) AFTER val",
+            "AFTER keyword is not supported");
+    }
+
+    /**
+     * Test for unsupported SQL statements in CREATE TABLE statement.
+     */
+    @Test
+    public void testUnsupportedCTE() {
+        // Simple CTE supports
+        execSql(
+            "WITH temp (A, B) AS (SELECT 1, 2) " +
+                "SELECT * FROM temp");
+
+        assertSqlUnsupported(
+            "WITH RECURSIVE temp (n, fact) AS " +
+            "(SELECT 0, 1 " +
+            "UNION ALL " +
+            "SELECT n+1, (n+1)*fact FROM temp WHERE n < 9) " +
+            "SELECT * FROM temp;");
+
+        execSql(
+            "CREATE TABLE test ( " +
+                "id integer primary key, " +
+                "parent integer DEFAULT 0, " +
+                "nm varchar)");
+
+        assertSqlUnsupported(
+            "WITH RECURSIVE tree (nm, id, level, pathstr) AS " +
+            "(SELECT nm, id, 0, CAST('' AS text) FROM test WHERE parent IS NULL " +
+            "UNION ALL " +
+            "SELECT test.nm, test.id, tree.level + 1, tree.pathstr + test.nm " +
+            "FROM TEST " +
+            "INNER JOIN tree ON tree.id = test.parent) " +
+            "SELECT id, space( level ) + nm AS nm FROM tree ORDER BY pathstr");
+    }
+
+    /**
+     * Test for unsupported SQL statements.
+     */
+    @Test
+    public void testUnsupportedSqlStatements() {
+        execSql(
+            "CREATE TABLE test ( " +
+                "id integer PRIMARY KEY, " +
+                "val varchar DEFAULT 'test_val')");
+
+        assertSqlUnsupported("CREATE SCHEMA my_schema");
+        assertSqlUnsupported("DROP SCHEMA my_schema");
+        assertSqlUnsupported("ALTER SCHEMA public RENAME TO private");
+
+        assertSqlUnsupported("ALTER TABLE test ADD CONSTRAINT c0 UNIQUE(val)");
+        assertSqlUnsupported("ALTER TABLE test RENAME CONSTRAINT c0 TO c1");
+        assertSqlUnsupported("ALTER TABLE test DROP CONSTRAINT c0");
+        assertSqlUnsupported("ALTER TABLE test RENAME TO new_test");
+
+        assertSqlUnsupported("ALTER TABLE test SET REFERENTIAL_INTEGRITY FALSE");
+
+        assertSqlUnsupported("ANALYZE TABLE test");
+
+        assertSqlUnsupported("ALTER INDEX idx0 RENAME TO idx1");
+
+        assertSqlUnsupported("CREATE VIEW test_view AS SELECT * FROM test WHERE id < 100");
+        assertSqlUnsupported("DROP VIEW test_view");
+
+        assertSqlUnsupported("CREATE SEQUENCE SEQ_0");
+        assertSqlUnsupported("ALTER SEQUENCE SEQ_ID RESTART WITH 1000");
+        assertSqlUnsupported("DROP SEQUENCE SEQ_0");
+
+        assertSqlUnsupported("CREATE TRIGGER trig_0 BEFORE INSERT ON TEST FOR EACH ROW CALL \"MyTrigger\"");
+        assertSqlUnsupported("DROP TRIGGER trig_0");
+
+        assertSqlUnsupported("CREATE ROLE newRole");
+        assertSqlUnsupported("DROP ROLE newRole");
+
+        assertSqlUnsupported("RUNSCRIPT FROM 'q.sql'");
+        assertSqlUnsupported("SCRIPT NODATA");
+
+        assertSqlUnsupported("BACKUP TO 'q.bak'");
+        assertSqlUnsupported("CALL 15*25");
+
+        assertSqlUnsupported("COMMENT ON TABLE test IS 'Table used for testing'");
+
+        assertSqlUnsupported("CREATE AGGREGATE testAgg FOR \"class_name\"");
+
+        assertSqlUnsupported("CREATE ALIAS my_sqrt FOR \"java.lang.Math.sqrt\"");
+        assertSqlUnsupported("DROP ALIAS my_sqrt");
+
+        assertSqlUnsupported("CREATE CONSTANT ONE VALUE 1");
+        assertSqlUnsupported("DROP CONSTANT ONE");
+
+        assertSqlUnsupported("CREATE DOMAIN EMAIL AS VARCHAR(255) CHECK (POSITION('@', VALUE) > 1)");
+        assertSqlUnsupported("DROP DOMAIN EMAIL");
+
+        assertSqlUnsupported("CREATE LINKED TABLE link('', '', '', '', '(SELECT * FROM test WHERE ID>0)');");
+
+        assertSqlUnsupported("DROP ALL OBJECTS");
+
+        assertSqlUnsupported("TRUNCATE TABLE test");
+
+        assertSqlUnsupported("COMMIT TRANSACTION t0");
+
+        assertSqlUnsupported("SAVEPOINT sp0");
+
+        // Any set command
+        assertSqlUnsupported("SET LOG 1");
+
+        assertSqlUnsupported("SHOW SCHEMAS");
+        assertSqlUnsupported("SHOW TABLES");
+        assertSqlUnsupported("SHOW COLUMNS FROM test");
+
+        assertSqlUnsupported("HELP SELECT");
+
+        assertSqlUnsupported("GRANT SELECT ON test TO PUBLIC");
+        assertSqlUnsupported("REVOKE SELECT ON test FROM PUBLIC");
+    }
+
+    /**
+     * @param ignite Ignite.
+     * @param sql Sql.
+     * @param args Args.
+     * @return Results.
+     */
+    @SuppressWarnings("unchecked")
+    private List<List<?>> execSql(Ignite ignite, String sql, Object... args) {
+        SqlFieldsQuery qry = new SqlFieldsQuery(sql).setLocal(local);
+
+        if (args != null && args.length > 0)
+            qry.setArgs(args);
+
+        return ((IgniteEx)ignite).context().query().querySqlFields(qry, false).getAll();
+    }
+
+    /**
+     * @param sql Sql.
+     * @param args Args.
+     * @return Query results.
+     */
+    private List<List<?>> execSql(String sql, Object... args) {
+        return execSql(grid(), sql, args);
+    }
+
+    /**
+     * @param sql Sql.
+     */
+    private void assertSqlUnsupported(final String sql) {
+        assertSqlUnsupported(sql, "");
+    }
+
+    /**
+     * @param sql Sql.
+     * @param msg Error message to check.
+     */
+    private void assertSqlUnsupported(final String sql, String msg) {
+        try {
+            local = false;
+            assertSqlUnsupported0(sql, msg);
+
+            local = true;
+            assertSqlUnsupported0(sql, msg);
+        }
+        finally {
+            local = false;
+        }
+    }
+
+    /**
+     * @param sql Sql.
+     * @param msg Error message match
+     */
+    private void assertSqlUnsupported0(final String sql, String msg) {
+        Throwable t = GridTestUtils.assertThrowsWithCause((Callable<Void>)() -> {
+            execSql(sql);
+
+            return null;
+        }, IgniteSQLException.class);
+
+        IgniteSQLException sqlE = X.cause(t, IgniteSQLException.class);
+
+        assert sqlE != null;
+
+        if (IgniteQueryErrorCode.UNSUPPORTED_OPERATION != sqlE.statusCode() || !sqlE.getMessage().contains(msg)) {
+            log.error("Unexpected exception", t);
+
+            fail("Unexpected exception. See above");
+        }
+    }
+}
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java
index 3a043a7..33b5de1 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java
@@ -219,6 +219,7 @@ import org.apache.ignite.internal.processors.query.h2.sql.GridQueryParsingTest;
 import org.apache.ignite.internal.processors.query.h2.sql.H2CompareBigQueryDistributedJoinsTest;
 import org.apache.ignite.internal.processors.query.h2.sql.H2CompareBigQueryTest;
 import org.apache.ignite.internal.processors.query.h2.sql.ParameterTypeInferenceTest;
+import org.apache.ignite.internal.processors.query.h2.sql.SqlUnsupportedSelfTest;
 import org.apache.ignite.internal.processors.query.h2.twostep.AndOperationExtractPartitionSelfTest;
 import org.apache.ignite.internal.processors.query.h2.twostep.BetweenOperationExtractPartitionSelfTest;
 import org.apache.ignite.internal.processors.query.h2.twostep.DmlSelectPartitionPruningSelfTest;
@@ -298,6 +299,7 @@ import org.junit.runners.Suite;
     GridQueryParsingTest.class,
     IgniteCacheSqlQueryErrorSelfTest.class,
     IgniteCacheSqlDmlErrorSelfTest.class,
+    SqlUnsupportedSelfTest.class,
 
     // Config.
     IgniteCacheDuplicateEntityConfigurationSelfTest.class,