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/02/10 09:38:51 UTC

[ignite] branch master updated: IGNITE-11278: SQL: Extracted parser logic from IgniteH2Indexing. This closes #6075.

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 5725a77  IGNITE-11278: SQL: Extracted parser logic from IgniteH2Indexing. This closes #6075.
5725a77 is described below

commit 5725a77b264f4f9ea9b5b16e8275c92b842e8568
Author: devozerov <pp...@gmail.com>
AuthorDate: Sun Feb 10 12:38:08 2019 +0300

    IGNITE-11278: SQL: Extracted parser logic from IgniteH2Indexing. This closes #6075.
---
 .../thin/JdbcThinBulkLoadAbstractSelfTest.java     |   4 +-
 .../jdbc/thin/JdbcThinLocalQueriesSelfTest.java    |   2 +-
 .../processors/query/h2/CommandProcessor.java      |   9 +
 .../processors/query/h2/ConnectionManager.java     |   2 +-
 .../processors/query/h2/IgniteH2Indexing.java      | 478 ++++-----------------
 .../processors/query/h2/ParsingResult.java         | 186 --------
 .../internal/processors/query/h2/QueryParser.java  | 456 ++++++++++++++++++++
 ...CachedQuery.java => QueryParserCacheEntry.java} |   6 +-
 ...achedQueryKey.java => QueryParserCacheKey.java} |   6 +-
 .../processors/query/h2/QueryParserResult.java     | 120 ++++++
 .../query/h2/QueryParserResultCommand.java         |  70 +++
 .../processors/query/h2/QueryParserResultDml.java  |  44 ++
 ...chedQuery.java => QueryParserResultSelect.java} |  50 ++-
 .../query/IgniteCachelessQueriesSelfTest.java      |   6 +-
 .../processors/query/SqlQueryHistorySelfTest.java  |  15 +-
 15 files changed, 844 insertions(+), 610 deletions(-)

diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinBulkLoadAbstractSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinBulkLoadAbstractSelfTest.java
index 7aea354..525d89a 100644
--- a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinBulkLoadAbstractSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinBulkLoadAbstractSelfTest.java
@@ -642,7 +642,7 @@ public abstract class JdbcThinBulkLoadAbstractSelfTest extends JdbcThinAbstractD
 
                 return null;
             }
-        }, SQLException.class, "The query isn't SELECT query");
+        }, SQLException.class, "Given statement type does not match that declared by JDBC driver");
     }
 
     /**
@@ -727,7 +727,7 @@ public abstract class JdbcThinBulkLoadAbstractSelfTest extends JdbcThinAbstractD
 
                 return null;
             }
-        }, SQLException.class, "The query isn't SELECT query");
+        }, SQLException.class, "Given statement type does not match that declared by JDBC driver");
     }
 
     /**
diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinLocalQueriesSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinLocalQueriesSelfTest.java
index 370b40f..ac81a71 100644
--- a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinLocalQueriesSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinLocalQueriesSelfTest.java
@@ -66,7 +66,7 @@ public class JdbcThinLocalQueriesSelfTest extends JdbcThinAbstractSelfTest {
 
             assertEqualsCollections(F.asList(2, "John", "Apple"), res.get(0));
 
-            Map twoStepCache = U.field(grid(0).context().query().getIndexing(), "twoStepCache");
+            Map twoStepCache = U.field((Object)U.field(grid(0).context().query().getIndexing(), "parser"), "cache");
 
             // No two step queries cached => local select.
             assertEquals(0, twoStepCache.size());
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/CommandProcessor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/CommandProcessor.java
index 8c52296..def2447 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/CommandProcessor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/CommandProcessor.java
@@ -98,6 +98,7 @@ import org.h2.command.ddl.CreateIndex;
 import org.h2.command.ddl.CreateTable;
 import org.h2.command.ddl.DropIndex;
 import org.h2.command.ddl.DropTable;
+import org.h2.command.dml.NoOperation;
 import org.h2.table.Column;
 import org.h2.value.DataType;
 import org.h2.value.Value;
@@ -797,6 +798,14 @@ public class CommandProcessor {
     }
 
     /**
+     * @param cmd Statement.
+     * @return Whether {@code cmd} is a no-op.
+     */
+    public static boolean isCommandNoOp(Prepared cmd) {
+        return cmd instanceof NoOperation;
+    }
+
+    /**
      * Helper function for obtaining type class name for H2.
      *
      * @param col Column.
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ConnectionManager.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ConnectionManager.java
index 9b00a63..8c1e89c 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ConnectionManager.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ConnectionManager.java
@@ -364,7 +364,7 @@ public class ConnectionManager {
     /**
      * Clear statement cache when cache is unregistered..
      */
-    public void onCacheUnregistered() {
+    public void onCacheDestroyed() {
         threadConns.values().forEach(set -> set.keySet().forEach(H2ConnectionWrapper::clearStatementCache));
     }
 
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
index f6867d8..c876d3c 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
@@ -34,7 +34,6 @@ import java.util.Map;
 import java.util.UUID;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.TimeUnit;
-import java.util.regex.Pattern;
 import javax.cache.CacheException;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteDataStreamer;
@@ -91,8 +90,6 @@ import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
 import org.apache.ignite.internal.processors.query.GridRunningQueryInfo;
 import org.apache.ignite.internal.processors.query.IgniteSQLException;
 import org.apache.ignite.internal.processors.query.QueryField;
-import org.apache.ignite.internal.processors.query.QueryHistoryMetrics;
-import org.apache.ignite.internal.processors.query.QueryHistoryMetricsKey;
 import org.apache.ignite.internal.processors.query.QueryIndexDescriptorImpl;
 import org.apache.ignite.internal.processors.query.QueryUtils;
 import org.apache.ignite.internal.processors.query.RunningQueryManager;
@@ -123,9 +120,7 @@ import org.apache.ignite.internal.processors.query.h2.opt.GridH2IndexBase;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table;
 import org.apache.ignite.internal.processors.query.h2.sql.GridSqlAlias;
 import org.apache.ignite.internal.processors.query.h2.sql.GridSqlAst;
-import org.apache.ignite.internal.processors.query.h2.sql.GridSqlQuery;
 import org.apache.ignite.internal.processors.query.h2.sql.GridSqlQueryParser;
-import org.apache.ignite.internal.processors.query.h2.sql.GridSqlQuerySplitter;
 import org.apache.ignite.internal.processors.query.h2.sql.GridSqlStatement;
 import org.apache.ignite.internal.processors.query.h2.sql.GridSqlTable;
 import org.apache.ignite.internal.processors.query.h2.twostep.GridMapQueryExecutor;
@@ -135,21 +130,9 @@ import org.apache.ignite.internal.processors.query.h2.twostep.msg.GridH2QueryReq
 import org.apache.ignite.internal.processors.query.schema.SchemaIndexCacheVisitor;
 import org.apache.ignite.internal.processors.query.schema.SchemaIndexCacheVisitorClosure;
 import org.apache.ignite.internal.processors.query.schema.SchemaIndexCacheVisitorImpl;
-import org.apache.ignite.internal.sql.SqlParseException;
-import org.apache.ignite.internal.sql.SqlParser;
-import org.apache.ignite.internal.sql.SqlStrictParseException;
-import org.apache.ignite.internal.sql.command.SqlAlterTableCommand;
-import org.apache.ignite.internal.sql.command.SqlAlterUserCommand;
-import org.apache.ignite.internal.sql.command.SqlBeginTransactionCommand;
-import org.apache.ignite.internal.sql.command.SqlBulkLoadCommand;
 import org.apache.ignite.internal.sql.command.SqlCommand;
 import org.apache.ignite.internal.sql.command.SqlCommitTransactionCommand;
-import org.apache.ignite.internal.sql.command.SqlCreateIndexCommand;
-import org.apache.ignite.internal.sql.command.SqlCreateUserCommand;
-import org.apache.ignite.internal.sql.command.SqlDropIndexCommand;
-import org.apache.ignite.internal.sql.command.SqlDropUserCommand;
 import org.apache.ignite.internal.sql.command.SqlRollbackTransactionCommand;
-import org.apache.ignite.internal.sql.command.SqlSetStreamingCommand;
 import org.apache.ignite.internal.util.GridBoundedConcurrentLinkedHashMap;
 import org.apache.ignite.internal.util.GridEmptyCloseableIterator;
 import org.apache.ignite.internal.util.GridSpinBusyLock;
@@ -179,7 +162,6 @@ import org.apache.ignite.spi.indexing.IndexingQueryFilterImpl;
 import org.h2.api.ErrorCode;
 import org.h2.api.JavaObjectSerializer;
 import org.h2.command.Prepared;
-import org.h2.command.dml.NoOperation;
 import org.h2.engine.Session;
 import org.h2.engine.SysProperties;
 import org.h2.table.IndexColumn;
@@ -207,11 +189,6 @@ import static org.apache.ignite.internal.processors.query.h2.twostep.msg.GridH2Q
  * For each table it will create indexes declared in {@link GridQueryTypeDescriptor#indexes()}.
  */
 public class IgniteH2Indexing implements GridQueryIndexing {
-    /** A pattern for commands having internal implementation in Ignite. */
-    public 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",
-        Pattern.CASE_INSENSITIVE);
-
     /*
      * Register IO for indexes.
      */
@@ -226,9 +203,6 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     public static final List<GridQueryFieldMetadata> UPDATE_RESULT_META =
         Collections.singletonList(new H2SqlFieldMetadata(null, null, "UPDATED", Long.class.getName(), -1, -1));
 
-    /** */
-    private static final int TWO_STEP_QRY_CACHE_SIZE = 1024;
-
     /** Default number of attempts to re-run DELETE and UPDATE queries in case of concurrent modifications of values. */
     private static final int DFLT_UPDATE_RERUN_ATTEMPTS = 4;
 
@@ -286,9 +260,8 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     /** Running query manager. */
     private RunningQueryManager runningQryMgr;
 
-    /** */
-    private volatile GridBoundedConcurrentLinkedHashMap<H2TwoStepCachedQueryKey, H2TwoStepCachedQuery> twoStepCache =
-        new GridBoundedConcurrentLinkedHashMap<>(TWO_STEP_QRY_CACHE_SIZE);
+    /** Parser. */
+    private QueryParser parser;
 
     /** */
     private final IgniteInClosure<? super IgniteInternalFuture<?>> logger = new IgniteInClosure<IgniteInternalFuture<?>>() {
@@ -1391,83 +1364,26 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     }
 
     /**
-     * Tries to parse sql query text using native parser. Only first (leading) sql command of the multi-statement is
-     * actually parsed.
-     *
-     * @param schemaName Schema name.
-     * @param qry which sql text to parse.
-     * @return Command or {@code null} if cannot parse this query.
-     */
-     @Nullable private ParsingResult parseNativeCommand(String schemaName, SqlFieldsQuery qry) {
-         String sql = qry.getSql();
-
-         // Heuristic check for fast return.
-         if (!INTERNAL_CMD_RE.matcher(sql.trim()).find())
-             return null;
-
-        try {
-            SqlParser parser = new SqlParser(schemaName, sql);
-
-            SqlCommand leadingCmd = parser.nextCommand();
-
-            assert leadingCmd != null : "Empty query. Parser met end of data";
-
-            if (!(leadingCmd instanceof SqlCreateIndexCommand
-                || leadingCmd instanceof SqlDropIndexCommand
-                || leadingCmd instanceof SqlBeginTransactionCommand
-                || leadingCmd instanceof SqlCommitTransactionCommand
-                || leadingCmd instanceof SqlRollbackTransactionCommand
-                || leadingCmd instanceof SqlBulkLoadCommand
-                || leadingCmd instanceof SqlAlterTableCommand
-                || leadingCmd instanceof SqlSetStreamingCommand
-                || leadingCmd instanceof SqlCreateUserCommand
-                || leadingCmd instanceof SqlAlterUserCommand
-                || leadingCmd instanceof SqlDropUserCommand)
-            )
-                return null;
-
-            SqlFieldsQuery newQry = cloneFieldsQuery(qry).setSql(parser.lastCommandSql());
-
-            return new ParsingResult(newQry, leadingCmd, null, parser.remainingSql());
-        }
-        catch (SqlStrictParseException e) {
-            throw new IgniteSQLException(e.getMessage(), IgniteQueryErrorCode.PARSING, e);
-        }
-        catch (Exception e) {
-            // Cannot parse, return.
-            if (log.isDebugEnabled())
-                log.debug("Failed to parse SQL with native parser [qry=" + sql + ", err=" + e + ']');
-
-            if (!IgniteSystemProperties.getBoolean(IgniteSystemProperties.IGNITE_SQL_PARSER_DISABLE_H2_FALLBACK))
-                return null;
-
-            int code = IgniteQueryErrorCode.PARSING;
-
-            if (e instanceof SqlParseException)
-                code = ((SqlParseException)e).code();
-
-            throw new IgniteSQLException("Failed to parse DDL statement: " + sql + ": " + e.getMessage(),
-                code, e);
-        }
-    }
-
-    /**
      * Execute command.
      *
      * @param schemaName Schema name.
      * @param qry Query.
      * @param cliCtx CLient context.
-     * @param cmdNative Command (native).
-     * @param cmdH2 Command (H2).
+     * @param cmd Command (native).
      * @return Result.
      */
     public FieldsQueryCursor<List<?>> executeCommand(
         String schemaName,
         SqlFieldsQuery qry,
         @Nullable SqlClientContext cliCtx,
-        SqlCommand cmdNative,
-        GridSqlStatement cmdH2
+        QueryParserResultCommand cmd
     ) {
+        if (cmd.noOp())
+            return H2Utils.zeroCursor();
+
+        SqlCommand cmdNative = cmd.commandNative();
+        GridSqlStatement cmdH2 = cmd.commandH2();
+
         if (qry.isLocal()) {
             throw new IgniteSQLException("DDL statements are not supported for LOCAL caches",
                 IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
@@ -1497,17 +1413,23 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     }
 
     /**
-     * Check expected statement type (when it is set by JDBC) and given statement type.
+     * Check whether command could be executed with the given cluster state.
      *
-     * @param qry Query.
-     * @param isQry {@code true} for select queries, otherwise (DML/DDL queries) {@code false}.
+     * @param parseRes Parsing result.
      */
-    private static void checkQueryType(SqlFieldsQuery qry, boolean isQry) {
-        Boolean qryFlag = qry instanceof SqlFieldsQueryEx ? ((SqlFieldsQueryEx) qry).isQuery() : null;
+    private void checkClusterState(QueryParserResult parseRes) {
+        if (!ctx.state().publicApiActiveState(true)) {
+            if (parseRes.isCommand()) {
+                SqlCommand cmd = parseRes.command().commandNative();
 
-        if (qryFlag != null && qryFlag != isQry)
-            throw new IgniteSQLException("Given statement type does not match that declared by JDBC driver",
-                IgniteQueryErrorCode.STMT_TYPE_MISMATCH);
+                if (cmd instanceof SqlCommitTransactionCommand || cmd instanceof SqlRollbackTransactionCommand)
+                    return;
+            }
+
+            throw new IgniteException("Can not perform the operation because the cluster is inactive. Note, " +
+                "that the cluster is considered inactive by default if Ignite Persistent Store is used to " +
+                "let all the nodes join the cluster. To activate the cluster call Ignite.active(true).");
+        }
     }
 
     /** {@inheritDoc} */
@@ -1520,33 +1442,21 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         try {
             List<FieldsQueryCursor<List<?>>> res = new ArrayList<>(1);
 
-            int firstArg = 0;
-
-            String remainingSql = qry.getSql();
-
-            while (remainingSql != null) {
-                SqlFieldsQuery remainingQry = remainingSql != qry.getSql() ? cloneFieldsQuery(qry).setSql(remainingSql) : qry;
+            SqlFieldsQuery remainingQry = qry;
 
-                ParsingResult parseRes = parse(schemaName, remainingQry, firstArg);
+            while (remainingQry != null) {
+                QueryParserResult parseRes = parser.parse(schemaName, remainingQry);
 
-                remainingSql = parseRes.remainingSql();
+                remainingQry = parseRes.remainingQuery();
 
-                if (remainingSql != null && failOnMultipleStmts)
+                if (remainingQry != null && failOnMultipleStmts)
                     throw new IgniteSQLException("Multiple statements queries are not supported");
 
-                SqlFieldsQuery newQry = parseRes.newQuery();
+                SqlFieldsQuery newQry = parseRes.query();
 
                 assert newQry.getSql() != null;
 
-                // Check if operation is performed on an active cluster.
-                SqlCommand nativeCmd = parseRes.commandNative();
-
-                if (!(nativeCmd instanceof SqlCommitTransactionCommand || nativeCmd instanceof SqlRollbackTransactionCommand)
-                    && !ctx.state().publicApiActiveState(true)) {
-                    throw new IgniteException("Can not perform the operation because the cluster is inactive. Note, " +
-                        "that the cluster is considered inactive by default if Ignite Persistent Store is used to " +
-                        "let all the nodes join the cluster. To activate the cluster call Ignite.active(true).");
-                }
+                checkClusterState(parseRes);
 
                 if (parseRes.isCommand()) {
                     // Execute command.
@@ -1554,24 +1464,18 @@ public class IgniteH2Indexing implements GridQueryIndexing {
                         schemaName,
                         newQry,
                         cliCtx,
-                        parseRes.commandNative(),
-                        parseRes.commandH2()
+                        parseRes.command()
                     );
 
                     res.add(cmdRes);
                 }
                 else {
                     // Execute query or DML.
-                    List<GridQueryFieldMetadata> meta = parseRes.meta();
-
-                    GridCacheTwoStepQuery twoStepQry = parseRes.twoStepQuery();
-
-                    List<? extends FieldsQueryCursor<List<?>>> qryRes = doRunPrepared(
+                    List<? extends FieldsQueryCursor<List<?>>> qryRes = executeSelectOrDml(
                         schemaName,
-                        parseRes.prepared(),
                         newQry,
-                        twoStepQry,
-                        meta,
+                        parseRes.select(),
+                        parseRes.dml(),
                         keepBinary,
                         startTx,
                         tracker,
@@ -1580,8 +1484,6 @@ public class IgniteH2Indexing implements GridQueryIndexing {
                     );
 
                     res.addAll(qryRes);
-
-                    firstArg += parseRes.parametersCount();
                 }
             }
 
@@ -1602,47 +1504,11 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     }
 
     /**
-     * Parses generic query. Tries : 1) Get plan of the distributed select from the cache; 2)parse query using native
-     * parser; 3)uses h2 parser if 1 and 2 were not succeed.
-     * Actually, only first (leading) sql statement is parsed (lazy parsing).
-     *
-     * @param schemaName schema name.
-     * @param qry query to parse.
-     * @param firstArg index of the first argument of the leading query. In other words - offset in the qry args array.
-     * @return Parsing result that contains Parsed leading query and remaining sql script.
-     */
-    private ParsingResult parse(String schemaName, SqlFieldsQuery qry, int firstArg) {
-        // First, let's check if we already have a two-step query for this statement...
-        H2TwoStepCachedQueryKey cachedQryKey = new H2TwoStepCachedQueryKey(
-            schemaName,
-            qry.getSql(),
-            qry.isCollocated(),
-            qry.isDistributedJoins(),
-            qry.isEnforceJoinOrder(),
-            qry.isLocal());
-
-        H2TwoStepCachedQuery cachedQry = twoStepCache.get(cachedQryKey);
-
-        if (cachedQry != null)
-            return new ParsingResult(null, qry, null, cachedQry.query(), cachedQryKey, cachedQry.meta());
-
-        // Try parting as native command.
-        ParsingResult parseRes = parseNativeCommand(schemaName, qry);
-
-        if (parseRes != null)
-            return parseRes;
-
-        // Parse with H2.
-        return parseAndSplit(schemaName, qry, firstArg);
-    }
-
-    /**
      * Execute an all-ready {@link SqlFieldsQuery}.
      * @param schemaName Schema name.
-     * @param prepared H2 command.
      * @param qry Fields query with flags.
-     * @param twoStepQry Two-step query if this query must be executed in a distributed way.
-     * @param meta Metadata for {@code twoStepQry}.
+     * @param select Select.
+     * @param dml DML.
      * @param keepBinary Whether binary objects must not be deserialized automatically.
      * @param startTx Start transaction flag.
      * @param mvccTracker MVCC tracker.
@@ -1650,16 +1516,26 @@ public class IgniteH2Indexing implements GridQueryIndexing {
      * @param registerAsNewQry {@code true} In case it's new query which should be registered as running query,
      * @return Query result.
      */
-    private List<? extends FieldsQueryCursor<List<?>>> doRunPrepared(String schemaName, Prepared prepared,
-        SqlFieldsQuery qry, GridCacheTwoStepQuery twoStepQry, List<GridQueryFieldMetadata> meta, boolean keepBinary,
-        boolean startTx, MvccQueryTracker mvccTracker, GridQueryCancel cancel, boolean registerAsNewQry) {
+    private List<? extends FieldsQueryCursor<List<?>>> executeSelectOrDml(
+        String schemaName,
+        SqlFieldsQuery qry,
+        @Nullable QueryParserResultSelect select,
+        @Nullable QueryParserResultDml dml,
+        boolean keepBinary,
+        boolean startTx,
+        MvccQueryTracker mvccTracker,
+        GridQueryCancel cancel,
+        boolean registerAsNewQry
+    ) {
         String sqlQry = qry.getSql();
 
         boolean loc = qry.isLocal();
 
         IndexingQueryFilter filter = (loc ? backupFilter(null, qry.getPartitions()) : null);
 
-        if (prepared != null && !prepared.isQuery()) {
+        if (dml != null) {
+            Prepared prepared = dml.prepared();
+
             Long qryId = registerRunningQuery(schemaName, cancel, sqlQry, loc, registerAsNewQry);
 
             boolean fail = false;
@@ -1690,9 +1566,6 @@ public class IgniteH2Indexing implements GridQueryIndexing {
                     }
                 }
 
-                if (prepared instanceof NoOperation)
-                    return Collections.singletonList(H2Utils.zeroCursor());
-
                 fail = true;
 
                 throw new IgniteSQLException("Unsupported DDL/DML operation: " + prepared.getClass().getName(),
@@ -1703,30 +1576,52 @@ public class IgniteH2Indexing implements GridQueryIndexing {
             }
         }
 
-        if (twoStepQry != null) {
-            if (log.isDebugEnabled())
-                log.debug("Parsed query: `" + sqlQry + "` into two step query: " + twoStepQry);
+        // Execute SQL.
+        assert select != null;
 
-            checkQueryType(qry, true);
+        GridCacheTwoStepQuery twoStepQry = select.twoStepQuery();
 
+        if (twoStepQry != null) {
+            // Distributed query.
             if (ctx.security().enabled())
                 checkSecurity(twoStepQry.cacheIds());
 
-            return Collections.singletonList(executeQuery(schemaName, qry, twoStepQry, meta, keepBinary,
-                startTx, mvccTracker, cancel, registerAsNewQry));
+            FieldsQueryCursor<List<?>> res = executeQuery(
+                schemaName,
+                qry,
+                twoStepQry,
+                select.twoStepQueryMeta(),
+                keepBinary,
+                startTx,
+                mvccTracker,
+                cancel,
+                registerAsNewQry
+            );
+
+            return Collections.singletonList(res);
         }
+        else {
+            // Local query.
+            Long qryId = registerRunningQuery(schemaName, cancel, sqlQry, loc, registerAsNewQry);
 
-        // We've encountered a local query, let's just run it.
-        Long qryId = registerRunningQuery(schemaName, cancel, sqlQry, loc, registerAsNewQry);
+            try {
+                FieldsQueryCursor<List<?>> res = executeQueryLocal(
+                    schemaName,
+                    qry,
+                    keepBinary,
+                    filter,
+                    cancel,
+                    qryId
+                );
 
-        try {
-            return Collections.singletonList(executeQueryLocal(schemaName, qry, keepBinary, filter, cancel, qryId));
-        }
-        catch (IgniteCheckedException e) {
-            runningQryMgr.unregister(qryId, true);
+                return Collections.singletonList(res);
+            }
+            catch (IgniteCheckedException e) {
+                runningQryMgr.unregister(qryId, true);
 
-            throw new IgniteSQLException("Failed to execute local statement [stmt=" + sqlQry +
-                ", params=" + Arrays.deepToString(qry.getArgs()) + "]", e);
+                throw new IgniteSQLException("Failed to execute local statement [stmt=" + sqlQry +
+                    ", params=" + Arrays.deepToString(qry.getArgs()) + "]", e);
+            }
         }
     }
 
@@ -1764,168 +1659,6 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     }
 
     /**
-     * Parse and split query if needed, cache either two-step query or statement.
-     * @param schemaName Schema name.
-     * @param qry Query.
-     * @param firstArg Position of the first argument of the following {@code Prepared}.
-     * @return Result: prepared statement, H2 command, two-step query (if needed),
-     *     metadata for two-step query (if needed), evaluated query local execution flag.
-     */
-    private ParsingResult parseAndSplit(String schemaName, SqlFieldsQuery qry, int firstArg) {
-        Connection c = connMgr.connectionForThread().connection(schemaName);
-
-        // For queries that are explicitly local, we rely on the flag specified in the query
-        // because this parsing result will be cached and used for queries directly.
-        // For other queries, we enforce join order at this stage to avoid premature optimizations
-        // (and therefore longer parsing) as long as there'll be more parsing at split stage.
-        boolean enforceJoinOrderOnParsing = (!qry.isLocal() || qry.isEnforceJoinOrder());
-
-        H2Utils.setupConnection(c, /*distributedJoins*/false, /*enforceJoinOrder*/enforceJoinOrderOnParsing);
-
-        boolean loc = qry.isLocal();
-
-        PreparedStatement stmt = prepareStatementAndCaches(c, qry.getSql());
-
-        if (loc && GridSqlQueryParser.checkMultipleStatements(stmt))
-            throw new IgniteSQLException("Multiple statements queries are not supported for local queries.",
-                IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
-
-        GridSqlQueryParser.PreparedWithRemaining prep = GridSqlQueryParser.preparedWithRemaining(stmt);
-
-        Prepared prepared = prep.prepared();
-
-        if (GridSqlQueryParser.isExplainUpdate(prepared))
-            throw new IgniteSQLException("Explains of update queries are not supported.",
-                IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
-
-        checkQueryType(qry, prepared.isQuery());
-
-        String remainingSql = prep.remainingSql();
-
-        int paramsCnt = prepared.getParameters().size();
-
-        Object[] argsOrig = qry.getArgs();
-        Object[] args = null;
-
-        if (!DmlUtils.isBatched(qry) && paramsCnt > 0) {
-            if (argsOrig == null || argsOrig.length < firstArg + paramsCnt) {
-                throw new IgniteException("Invalid number of query parameters. " +
-                    "Cannot find " + (argsOrig != null ? argsOrig.length + 1 - firstArg : 1) + " parameter.");
-            }
-
-            args = Arrays.copyOfRange(argsOrig, firstArg, firstArg + paramsCnt);
-        }
-
-        if (prepared.isQuery()) {
-            try {
-                H2Utils.bindParameters(stmt, F.asList(args));
-            }
-            catch (IgniteCheckedException e) {
-                U.closeQuiet(stmt);
-
-                throw new IgniteSQLException("Failed to bind parameters: [qry=" + prepared.getSQL() + ", params=" +
-                    Arrays.deepToString(args) + "]", IgniteQueryErrorCode.PARSING, e);
-            }
-
-            GridSqlQueryParser parser = null;
-
-            if (!loc) {
-                parser = new GridSqlQueryParser(false);
-
-                GridSqlStatement parsedStmt = parser.parse(prepared);
-
-                // Legit assertion - we have H2 query flag above.
-                assert parsedStmt instanceof GridSqlQuery;
-
-                loc = parser.isLocalQuery();
-            }
-
-            if (loc) {
-                if (parser == null) {
-                    parser = new GridSqlQueryParser(false);
-
-                    parser.parse(prepared);
-                }
-
-                GridCacheContext cctx = parser.getFirstPartitionedCache();
-
-                if (cctx != null && cctx.config().getQueryParallelism() > 1) {
-                    loc = false;
-
-                    qry.setDistributedJoins(true);
-                }
-            }
-        }
-
-        SqlFieldsQuery newQry = cloneFieldsQuery(qry).setSql(prepared.getSQL()).setArgs(args);
-
-        if (CommandProcessor.isCommand(prepared)) {
-            GridSqlStatement cmdH2 = new GridSqlQueryParser(false).parse(prepared);
-
-            return new ParsingResult(newQry, null, cmdH2, remainingSql);
-        }
-
-        boolean hasTwoStep = !loc && prepared.isQuery();
-
-        // Let's not cache multiple statements and distributed queries as whole two step query will be cached later on.
-        if (remainingSql != null || hasTwoStep)
-            connMgr.statementCacheForThread().remove(schemaName, qry.getSql());
-
-        if (!hasTwoStep)
-            return new ParsingResult(prepared, newQry, remainingSql);
-
-        // Now we're sure to have a distributed query. Let's try to get a two-step plan from the cache, or perform the
-        // split if needed.
-        H2TwoStepCachedQueryKey cachedQryKey = new H2TwoStepCachedQueryKey(schemaName, qry.getSql(),
-            qry.isCollocated(), qry.isDistributedJoins(), qry.isEnforceJoinOrder(), qry.isLocal());
-
-        H2TwoStepCachedQuery cachedQry = twoStepCache.get(cachedQryKey);
-
-        if (cachedQry == null) {
-            try {
-                GridCacheTwoStepQuery twoStepQry = GridSqlQuerySplitter.split(
-                    connMgr.connectionForThread().connection(newQry.getSchema()),
-                    prepared,
-                    newQry.getArgs(),
-                    newQry.isCollocated(),
-                    newQry.isDistributedJoins(),
-                    newQry.isEnforceJoinOrder(),
-                    newQry.isLocal(),
-                    this
-                );
-
-                List<GridQueryFieldMetadata> meta = H2Utils.meta(stmt.getMetaData());
-
-                cachedQry = new H2TwoStepCachedQuery(meta, twoStepQry);
-
-                if (remainingSql == null && !twoStepQry.explain())
-                    twoStepCache.putIfAbsent(cachedQryKey, cachedQry);
-            }
-            catch (IgniteCheckedException e) {
-                throw new IgniteSQLException("Failed to bind parameters: [qry=" + newQry.getSql() + ", params=" +
-                    Arrays.deepToString(newQry.getArgs()) + "]", IgniteQueryErrorCode.PARSING, e);
-            }
-            catch (SQLException e) {
-                throw new IgniteSQLException(e);
-            }
-            finally {
-                U.close(stmt, log);
-            }
-        }
-
-        return new ParsingResult(prepared, newQry, remainingSql, cachedQry.query(), cachedQryKey, cachedQry.meta());
-    }
-
-    /**
-     * Make a copy of {@link SqlFieldsQuery} with all flags and preserving type.
-     * @param oldQry Query to copy.
-     * @return Query copy.
-     */
-    private SqlFieldsQuery cloneFieldsQuery(SqlFieldsQuery oldQry) {
-        return oldQry.copy().setLocal(oldQry.isLocal()).setPageSize(oldQry.getPageSize());
-    }
-
-    /**
      * @param qry Sql fields query.autoStartTx(qry)
      * @return {@code True} if need to start transaction.
      */
@@ -2400,6 +2133,8 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
         connMgr = new ConnectionManager(ctx);
 
+        parser = new QueryParser(this, connMgr);
+
         schemaMgr = new SchemaManager(ctx, connMgr);
         schemaMgr.start(ctx.config().getSqlSchemas());
 
@@ -2620,27 +2355,18 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         schemaMgr.onCacheDestroyed(cacheName, rmvIdx);
 
         // Unregister connection.
-        connMgr.onCacheUnregistered();
+        connMgr.onCacheDestroyed();
 
         // Clear query cache.
-        int cacheId = CU.cacheId(cacheName);
-
-        for (Iterator<Map.Entry<H2TwoStepCachedQueryKey, H2TwoStepCachedQuery>> it =
-             twoStepCache.entrySet().iterator(); it.hasNext();) {
-            Map.Entry<H2TwoStepCachedQueryKey, H2TwoStepCachedQuery> e = it.next();
-
-            GridCacheTwoStepQuery qry = e.getValue().query();
-
-            if (!F.isEmpty(qry.cacheIds()) && qry.cacheIds().contains(cacheId))
-                it.remove();
-        }
+        clearPlanCache();
     }
 
     /**
      * Remove all cached queries from cached two-steps queries.
      */
     private void clearPlanCache() {
-        twoStepCache = new GridBoundedConcurrentLinkedHashMap<>(TWO_STEP_QRY_CACHE_SIZE);
+        parser.clearCache();
+
         updatePlanCache = new GridBoundedConcurrentLinkedHashMap<>(UPDATE_PLAN_CACHE_SIZE);
     }
 
@@ -2703,22 +2429,6 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         return runningQryMgr.longRunningQueries(duration);
     }
 
-    /**
-     * Gets query history metrics.
-     *
-     * @return Queries history metrics.
-     */
-    public Map<QueryHistoryMetricsKey, QueryHistoryMetrics> queryHistoryMetrics() {
-        return runningQryMgr.queryHistoryMetrics();
-    }
-
-    /**
-     * Reset query history metrics.
-     */
-    public void resetQueryHistoryMetrics() {
-        runningQryMgr.resetQueryHistoryMetrics();
-    }
-
     /** {@inheritDoc} */
     @Override public void cancelQueries(Collection<Long> queries) {
         if (!F.isEmpty(queries)) {
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ParsingResult.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ParsingResult.java
deleted file mode 100644
index 0358f64..0000000
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ParsingResult.java
+++ /dev/null
@@ -1,186 +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.ignite.internal.processors.query.h2;
-
-import java.util.List;
-import org.apache.ignite.cache.query.SqlFieldsQuery;
-import org.apache.ignite.internal.processors.cache.query.GridCacheTwoStepQuery;
-import org.apache.ignite.internal.processors.query.GridQueryFieldMetadata;
-import org.apache.ignite.internal.processors.query.h2.sql.GridSqlStatement;
-import org.apache.ignite.internal.sql.command.SqlCommand;
-import org.h2.command.Prepared;
-
-/**
- * Result of parsing and splitting SQL from {@link SqlFieldsQuery}.
- */
-final class ParsingResult {
-    /** H2 command. */
-    private final Prepared prepared;
-
-    /** New fields query that may be executed right away. */
-    private final SqlFieldsQuery newQry;
-
-    /** Remaining SQL statements. */
-    private final String remainingSql;
-
-    /** Two-step query, or {@code} null if this result is for local query. */
-    private final GridCacheTwoStepQuery twoStepQry;
-
-    /** Two-step query key. */
-    private final H2TwoStepCachedQueryKey twoStepQryKey;
-
-    /** Metadata for two-step query, or {@code} null if this result is for local query. */
-    private final List<GridQueryFieldMetadata> meta;
-
-    /** Command (native). */
-    private final SqlCommand cmdNative;
-
-    /** Command (H2). */
-    private final GridSqlStatement cmdH2;
-
-    /**
-     * Simple constructor.
-     */
-    private ParsingResult(
-        Prepared prepared,
-        SqlFieldsQuery newQry,
-        String remainingSql,
-        GridCacheTwoStepQuery twoStepQry,
-        H2TwoStepCachedQueryKey twoStepQryKey,
-        List<GridQueryFieldMetadata> meta,
-        SqlCommand cmdNative,
-        GridSqlStatement cmdH2
-    ) {
-        this.prepared = prepared;
-        this.newQry = newQry;
-        this.remainingSql = remainingSql;
-        this.twoStepQry = twoStepQry;
-        this.twoStepQryKey = twoStepQryKey;
-        this.meta = meta;
-        this.cmdNative = cmdNative;
-        this.cmdH2 = cmdH2;
-    }
-
-    /**
-     * Construct result in case of h2 parsing and two step query.
-     */
-    public ParsingResult(
-        Prepared prepared,
-        SqlFieldsQuery newQry,
-        String remainingSql,
-        GridCacheTwoStepQuery twoStepQry,
-        H2TwoStepCachedQueryKey twoStepQryKey,
-        List<GridQueryFieldMetadata> meta
-    ) {
-        this(prepared, newQry, remainingSql, twoStepQry, twoStepQryKey, meta, null, null);
-    }
-
-    /**
-     * Construct parsing result in case of native parsing.
-     *
-     * @param newQry leading sql statement of the original multi-statement query.
-     * @param cmdNative Command (native).
-     * @param cmdH2 Command (H2).
-     * @param remainingSql the rest of the original query.
-     */
-    public ParsingResult(SqlFieldsQuery newQry, SqlCommand cmdNative, GridSqlStatement cmdH2, String remainingSql) {
-        this(null, newQry, remainingSql, null, null, null, cmdNative, cmdH2);
-    }
-
-    /**
-     * Result in case we use h2 but don't have two step query.
-     *
-     * @param prepared h2's parsed prepared statement. The leading statement of the original multi-statement query.
-     * @param newQry SqlFields query that prepare represents.
-     * @param remainingSql the rest of the original query.
-     */
-    public ParsingResult(Prepared prepared, SqlFieldsQuery newQry, String remainingSql) {
-        this(prepared, newQry, remainingSql, null, null, null, null, null);
-    }
-
-    /**
-     * @return Metadata for two-step query, or {@code} null if this result is for local query.
-     */
-    List<GridQueryFieldMetadata> meta() {
-        return meta;
-    }
-
-    /**
-     * @return New fields query that may be executed right away.
-     */
-    SqlFieldsQuery newQuery() {
-        return newQry;
-    }
-
-    /**
-     * Command (native).
-     */
-    public SqlCommand commandNative() {
-        return cmdNative;
-    }
-
-    /**
-     * @return Command (H2).
-     */
-    public GridSqlStatement commandH2() {
-        return cmdH2;
-    }
-
-    /**
-     * @return H2 command.
-     */
-    Prepared prepared() {
-        return prepared;
-    }
-
-    /**
-     * @return Remaining SQL statements.
-     */
-    String remainingSql() {
-        return remainingSql;
-    }
-
-    /**
-     * @return Two-step query, or {@code} null if this result is for local query.
-     */
-    GridCacheTwoStepQuery twoStepQuery() {
-        return twoStepQry;
-    }
-
-    /**
-     * @return Two-step query key to cache {@link #twoStepQry}, or {@code null} if there's no need to worry about
-     * two-step caching.
-     */
-    H2TwoStepCachedQueryKey twoStepQueryKey() {
-        return twoStepQryKey;
-    }
-
-    /**
-     * @return Number of parameters.
-     */
-    public int parametersCount() {
-        return prepared != null ? prepared.getParameters().size() : twoStepQry.parametersCount();
-    }
-
-    /**
-     * @return Check whether this is a command.
-     */
-    public boolean isCommand() {
-        return cmdNative != null || cmdH2 != null;
-    }
-}
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
new file mode 100644
index 0000000..0b0baf0
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/QueryParser.java
@@ -0,0 +1,456 @@
+/*
+ * 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;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.IgniteSystemProperties;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.query.GridCacheTwoStepQuery;
+import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
+import org.apache.ignite.internal.processors.cache.query.SqlFieldsQueryEx;
+import org.apache.ignite.internal.processors.query.GridQueryFieldMetadata;
+import org.apache.ignite.internal.processors.query.IgniteSQLException;
+import org.apache.ignite.internal.processors.query.h2.dml.DmlUtils;
+import org.apache.ignite.internal.processors.query.h2.sql.GridSqlQuery;
+import org.apache.ignite.internal.processors.query.h2.sql.GridSqlQueryParser;
+import org.apache.ignite.internal.processors.query.h2.sql.GridSqlQuerySplitter;
+import org.apache.ignite.internal.processors.query.h2.sql.GridSqlStatement;
+import org.apache.ignite.internal.sql.SqlParseException;
+import org.apache.ignite.internal.sql.SqlParser;
+import org.apache.ignite.internal.sql.SqlStrictParseException;
+import org.apache.ignite.internal.sql.command.SqlAlterTableCommand;
+import org.apache.ignite.internal.sql.command.SqlAlterUserCommand;
+import org.apache.ignite.internal.sql.command.SqlBeginTransactionCommand;
+import org.apache.ignite.internal.sql.command.SqlBulkLoadCommand;
+import org.apache.ignite.internal.sql.command.SqlCommand;
+import org.apache.ignite.internal.sql.command.SqlCommitTransactionCommand;
+import org.apache.ignite.internal.sql.command.SqlCreateIndexCommand;
+import org.apache.ignite.internal.sql.command.SqlCreateUserCommand;
+import org.apache.ignite.internal.sql.command.SqlDropIndexCommand;
+import org.apache.ignite.internal.sql.command.SqlDropUserCommand;
+import org.apache.ignite.internal.sql.command.SqlRollbackTransactionCommand;
+import org.apache.ignite.internal.sql.command.SqlSetStreamingCommand;
+import org.apache.ignite.internal.util.GridBoundedConcurrentLinkedHashMap;
+import org.apache.ignite.internal.util.typedef.F;
+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.regex.Pattern;
+
+/**
+ * Parser module. Splits incoming request into a series of parsed results.
+ */
+public class QueryParser {
+    /** */
+    private static final int CACHE_SIZE = 1024;
+
+    /** A pattern for commands having internal implementation in Ignite. */
+    public 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",
+        Pattern.CASE_INSENSITIVE);
+
+    /** Indexing. */
+    private final IgniteH2Indexing idx;
+
+    /** Connection manager. */
+    private final ConnectionManager connMgr;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** */
+    private volatile GridBoundedConcurrentLinkedHashMap<QueryParserCacheKey, QueryParserCacheEntry> cache =
+        new GridBoundedConcurrentLinkedHashMap<>(CACHE_SIZE);
+
+    /**
+     * Constructor.
+     *
+     * @param idx Indexing instance.
+     * @param connMgr Connection manager.
+     */
+    public QueryParser(IgniteH2Indexing idx, ConnectionManager connMgr) {
+        this.idx = idx;
+        this.connMgr = connMgr;
+
+        log = idx.kernalContext().log(QueryParser.class);
+    }
+
+    /**
+     * Parse the query.
+     *
+     * @param schemaName schema name.
+     * @param qry query to parse.
+     * @return Parsing result that contains Parsed leading query and remaining sql script.
+     */
+    public QueryParserResult parse(String schemaName, SqlFieldsQuery qry) {
+        QueryParserResult res = parse0(schemaName, qry);
+
+        checkQueryType(qry, res.isSelect());
+
+        return res;
+    }
+
+    /**
+     * Parse the query.
+     *
+     * @param schemaName schema name.
+     * @param qry query to parse.
+     * @return Parsing result that contains Parsed leading query and remaining sql script.
+     */
+    private QueryParserResult parse0(String schemaName, SqlFieldsQuery qry) {
+        // First, let's check if we already have a two-step query for this statement...
+        QueryParserCacheKey cachedQryKey = new QueryParserCacheKey(
+            schemaName,
+            qry.getSql(),
+            qry.isCollocated(),
+            qry.isDistributedJoins(),
+            qry.isEnforceJoinOrder(),
+            qry.isLocal());
+
+        QueryParserCacheEntry cachedQry = cache.get(cachedQryKey);
+
+        if (cachedQry != null) {
+            QueryParserResultSelect select = new QueryParserResultSelect(
+                cachedQry.query(),
+                cachedQry.meta(),
+                null
+            );
+
+            return new QueryParserResult(qry, null, select, null, null);
+        }
+
+        // Try parting as native command.
+        QueryParserResult parseRes = parseNative(schemaName, qry);
+
+        if (parseRes != null)
+            return parseRes;
+
+        // Parse with H2.
+        return parseH2(schemaName, qry);
+    }
+
+    /**
+     * Tries to parse sql query text using native parser. Only first (leading) sql command of the multi-statement is
+     * actually parsed.
+     *
+     * @param schemaName Schema name.
+     * @param qry which sql text to parse.
+     * @return Command or {@code null} if cannot parse this query.
+     */
+    @SuppressWarnings("IfMayBeConditional")
+    @Nullable
+    private QueryParserResult parseNative(String schemaName, SqlFieldsQuery qry) {
+        String sql = qry.getSql();
+
+        // Heuristic check for fast return.
+        if (!INTERNAL_CMD_RE.matcher(sql.trim()).find())
+            return null;
+
+        try {
+            SqlParser parser = new SqlParser(schemaName, sql);
+
+            SqlCommand nativeCmd = parser.nextCommand();
+
+            assert nativeCmd != null : "Empty query. Parser met end of data";
+
+            if (!(nativeCmd instanceof SqlCreateIndexCommand
+                || nativeCmd instanceof SqlDropIndexCommand
+                || nativeCmd instanceof SqlBeginTransactionCommand
+                || nativeCmd instanceof SqlCommitTransactionCommand
+                || nativeCmd instanceof SqlRollbackTransactionCommand
+                || nativeCmd instanceof SqlBulkLoadCommand
+                || nativeCmd instanceof SqlAlterTableCommand
+                || nativeCmd instanceof SqlSetStreamingCommand
+                || nativeCmd instanceof SqlCreateUserCommand
+                || nativeCmd instanceof SqlAlterUserCommand
+                || nativeCmd instanceof SqlDropUserCommand)
+                )
+                return null;
+
+            SqlFieldsQuery newQry = cloneFieldsQuery(qry).setSql(parser.lastCommandSql());
+
+            SqlFieldsQuery remainingQry;
+
+            if (F.isEmpty(parser.remainingSql()))
+                remainingQry = null;
+            else
+                remainingQry = cloneFieldsQuery(qry).setSql(parser.remainingSql()).setArgs(qry.getArgs());
+
+            QueryParserResultCommand cmd = new QueryParserResultCommand(nativeCmd, null, false);
+
+            return new QueryParserResult(newQry, remainingQry, null, null, cmd);
+        }
+        catch (SqlStrictParseException e) {
+            throw new IgniteSQLException(e.getMessage(), IgniteQueryErrorCode.PARSING, e);
+        }
+        catch (Exception e) {
+            // Cannot parse, return.
+            if (log.isDebugEnabled())
+                log.debug("Failed to parse SQL with native parser [qry=" + sql + ", err=" + e + ']');
+
+            if (!IgniteSystemProperties.getBoolean(IgniteSystemProperties.IGNITE_SQL_PARSER_DISABLE_H2_FALLBACK))
+                return null;
+
+            int code = IgniteQueryErrorCode.PARSING;
+
+            if (e instanceof SqlParseException)                code = ((SqlParseException)e).code();
+
+            throw new IgniteSQLException("Failed to parse DDL statement: " + sql + ": " + e.getMessage(),
+                code, e);
+        }
+    }
+
+    /**
+     * Parse and split query if needed, cache either two-step query or statement.
+     *
+     * @param schemaName Schema name.
+     * @param qry Query.
+     * @return Parsing result.
+     */
+    @SuppressWarnings("IfMayBeConditional")
+    private QueryParserResult parseH2(String schemaName, SqlFieldsQuery qry) {
+        Connection c = connMgr.connectionForThread().connection(schemaName);
+
+        // For queries that are explicitly local, we rely on the flag specified in the query
+        // because this parsing result will be cached and used for queries directly.
+        // For other queries, we enforce join order at this stage to avoid premature optimizations
+        // (and therefore longer parsing) as long as there'll be more parsing at split stage.
+        boolean enforceJoinOrderOnParsing = (!qry.isLocal() || qry.isEnforceJoinOrder());
+
+        H2Utils.setupConnection(c, /*distributedJoins*/false, /*enforceJoinOrder*/enforceJoinOrderOnParsing);
+
+        boolean loc = qry.isLocal();
+
+        PreparedStatement stmt;
+
+        try {
+            stmt = connMgr.prepareStatement(c, qry.getSql());
+        }
+        catch (SQLException e) {
+            throw new IgniteSQLException("Failed to parse query. " + e.getMessage(),
+                IgniteQueryErrorCode.PARSING, e);
+        }
+
+        if (loc && GridSqlQueryParser.checkMultipleStatements(stmt))
+            throw new IgniteSQLException("Multiple statements queries are not supported for local queries.",
+                IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
+
+        GridSqlQueryParser.PreparedWithRemaining prep = GridSqlQueryParser.preparedWithRemaining(stmt);
+
+        Prepared prepared = prep.prepared();
+
+        if (GridSqlQueryParser.isExplainUpdate(prepared))
+            throw new IgniteSQLException("Explains of update queries are not supported.",
+                IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
+
+        int paramsCnt = prepared.getParameters().size();
+
+        Object[] argsOrig = qry.getArgs();
+
+        Object[] args = null;
+        Object[] remainingArgs = null;
+
+        if (!DmlUtils.isBatched(qry) && paramsCnt > 0) {
+            if (argsOrig == null || argsOrig.length < paramsCnt) {
+                throw new IgniteException("Invalid number of query parameters. " +
+                    "Cannot find " + (argsOrig != null ? argsOrig.length + 1 : 1) + " parameter.");
+            }
+
+            args = Arrays.copyOfRange(argsOrig, 0, paramsCnt);
+
+            if (paramsCnt != argsOrig.length)
+                remainingArgs = Arrays.copyOfRange(argsOrig, paramsCnt, argsOrig.length);
+        }
+        else
+            remainingArgs = argsOrig;
+
+        SqlFieldsQuery remainingQry;
+
+        if (F.isEmpty(prep.remainingSql()))
+            remainingQry = null;
+        else
+            remainingQry = cloneFieldsQuery(qry).setSql(prep.remainingSql()).setArgs(remainingArgs);
+
+        SqlFieldsQuery newQry = cloneFieldsQuery(qry).setSql(prepared.getSQL()).setArgs(args);
+
+        // TODO: WTF is that? Modifies global query flag (distr joins), invokes additional parsing.
+        if (prepared.isQuery()) {
+            try {
+                H2Utils.bindParameters(stmt, F.asList(args));
+            }
+            catch (IgniteCheckedException e) {
+                U.closeQuiet(stmt);
+
+                throw new IgniteSQLException("Failed to bind parameters: [qry=" + prepared.getSQL() + ", params=" +
+                    Arrays.deepToString(args) + "]", IgniteQueryErrorCode.PARSING, e);
+            }
+
+            GridSqlQueryParser parser = null;
+
+            if (!loc) {
+                parser = new GridSqlQueryParser(false);
+
+                GridSqlStatement parsedStmt = parser.parse(prepared);
+
+                // Legit assertion - we have H2 query flag above.
+                assert parsedStmt instanceof GridSqlQuery;
+
+                loc = parser.isLocalQuery();
+            }
+
+            if (loc) {
+                if (parser == null) {
+                    parser = new GridSqlQueryParser(false);
+
+                    parser.parse(prepared);
+                }
+
+                GridCacheContext cctx = parser.getFirstPartitionedCache();
+
+                if (cctx != null && cctx.config().getQueryParallelism() > 1) {
+                    loc = false;
+
+                    newQry.setDistributedJoins(true);
+                }
+            }
+        }
+
+        // Do not cache multiple statements and distributed queries as whole two step query will be cached later on.
+        if (remainingQry != null || !loc)
+            connMgr.statementCacheForThread().remove(schemaName, qry.getSql());
+
+        if (CommandProcessor.isCommand(prepared)) {
+            GridSqlStatement cmdH2 = new GridSqlQueryParser(false).parse(prepared);
+
+            QueryParserResultCommand cmd = new QueryParserResultCommand(null, cmdH2, false);
+
+            return new QueryParserResult(newQry, remainingQry, null, null, cmd);
+        }
+        else if (CommandProcessor.isCommandNoOp(prepared)) {
+            QueryParserResultCommand cmd = new QueryParserResultCommand(null, null, true);
+
+            return new QueryParserResult(newQry, remainingQry, null, null, cmd);
+        }
+        else if (GridSqlQueryParser.isDml(prepared))
+            return new QueryParserResult(newQry, remainingQry, null ,new QueryParserResultDml(prepared), null);
+        else if (!prepared.isQuery()) {
+            throw new IgniteSQLException("Unsupported statement: " + newQry.getSql(),
+                IgniteQueryErrorCode.UNSUPPORTED_OPERATION);
+        }
+
+        // At this point only SELECT is possible.
+        if (loc) {
+            // No two-step for local query for now.
+            QueryParserResultSelect select = new QueryParserResultSelect(null, null, prepared);
+
+            return new QueryParserResult(newQry, remainingQry, select, null, null);
+        }
+
+        // Only distirbuted SELECT are possible at this point.
+        QueryParserCacheKey cachedQryKey = new QueryParserCacheKey(
+            schemaName,
+            qry.getSql(),
+            qry.isCollocated(),
+            qry.isDistributedJoins(),
+            qry.isEnforceJoinOrder(),
+            qry.isLocal()
+        );
+
+        QueryParserCacheEntry cachedQry = cache.get(cachedQryKey);
+
+        if (cachedQry == null) {
+            try {
+                GridCacheTwoStepQuery twoStepQry = GridSqlQuerySplitter.split(
+                    connMgr.connectionForThread().connection(newQry.getSchema()),
+                    prepared,
+                    newQry.getArgs(),
+                    newQry.isCollocated(),
+                    newQry.isDistributedJoins(),
+                    newQry.isEnforceJoinOrder(),
+                    newQry.isLocal(),
+                    idx
+                );
+
+                List<GridQueryFieldMetadata> meta = H2Utils.meta(stmt.getMetaData());
+
+                cachedQry = new QueryParserCacheEntry(meta, twoStepQry);
+
+                if (remainingQry == null && !twoStepQry.explain())
+                    cache.putIfAbsent(cachedQryKey, cachedQry);
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteSQLException("Failed to bind parameters: [qry=" + newQry.getSql() + ", params=" +
+                    Arrays.deepToString(newQry.getArgs()) + "]", IgniteQueryErrorCode.PARSING, e);
+            }
+            catch (SQLException e) {
+                throw new IgniteSQLException(e);
+            }
+            finally {
+                U.close(stmt, log);
+            }
+        }
+
+        QueryParserResultSelect select = new QueryParserResultSelect(
+            cachedQry.query(),
+            cachedQry.meta(),
+            prepared
+        );
+
+        return new QueryParserResult(newQry, remainingQry, select, null, null);
+    }
+
+    /**
+     * Clear cached plans.
+     */
+    public void clearCache() {
+        cache = new GridBoundedConcurrentLinkedHashMap<>(CACHE_SIZE);
+    }
+
+    /**
+     * Check expected statement type (when it is set by JDBC) and given statement type.
+     *
+     * @param qry Query.
+     * @param isQry {@code true} for select queries, otherwise (DML/DDL queries) {@code false}.
+     */
+    private static void checkQueryType(SqlFieldsQuery qry, boolean isQry) {
+        Boolean qryFlag = qry instanceof SqlFieldsQueryEx ? ((SqlFieldsQueryEx) qry).isQuery() : null;
+
+        if (qryFlag != null && qryFlag != isQry)
+            throw new IgniteSQLException("Given statement type does not match that declared by JDBC driver",
+                IgniteQueryErrorCode.STMT_TYPE_MISMATCH);
+    }
+
+    /**
+     * Make a copy of {@link SqlFieldsQuery} with all flags and preserving type.
+     *
+     * @param oldQry Query to copy.
+     * @return Query copy.
+     */
+    private static SqlFieldsQuery cloneFieldsQuery(SqlFieldsQuery oldQry) {
+        return oldQry.copy().setLocal(oldQry.isLocal()).setPageSize(oldQry.getPageSize());
+    }
+}
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2TwoStepCachedQuery.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/QueryParserCacheEntry.java
similarity index 89%
copy from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2TwoStepCachedQuery.java
copy to modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/QueryParserCacheEntry.java
index 6139869..2531379 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2TwoStepCachedQuery.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/QueryParserCacheEntry.java
@@ -26,7 +26,7 @@ import java.util.List;
 /**
  * Cached two-step query.
  */
-public class H2TwoStepCachedQuery {
+public class QueryParserCacheEntry {
     /** */
     private final List<GridQueryFieldMetadata> meta;
 
@@ -37,7 +37,7 @@ public class H2TwoStepCachedQuery {
      * @param meta Fields metadata.
      * @param twoStepQry Query.
      */
-    public H2TwoStepCachedQuery(List<GridQueryFieldMetadata> meta, GridCacheTwoStepQuery twoStepQry) {
+    public QueryParserCacheEntry(List<GridQueryFieldMetadata> meta, GridCacheTwoStepQuery twoStepQry) {
         this.meta = meta;
         this.twoStepQry = twoStepQry;
     }
@@ -58,6 +58,6 @@ public class H2TwoStepCachedQuery {
 
     /** {@inheritDoc} */
     @Override public String toString() {
-        return S.toString(H2TwoStepCachedQuery.class, this);
+        return S.toString(QueryParserCacheEntry.class, this);
     }
 }
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2TwoStepCachedQueryKey.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/QueryParserCacheKey.java
similarity index 95%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2TwoStepCachedQueryKey.java
rename to modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/QueryParserCacheKey.java
index f345207..21bf405 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2TwoStepCachedQueryKey.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/QueryParserCacheKey.java
@@ -20,7 +20,7 @@ package org.apache.ignite.internal.processors.query.h2;
 /**
  * Key for cached two-step query.
  */
-public class H2TwoStepCachedQueryKey {
+public class QueryParserCacheKey {
     /** */
     private final String schemaName;
 
@@ -47,7 +47,7 @@ public class H2TwoStepCachedQueryKey {
      * @param enforceJoinOrder Enforce join order of tables.
      * @param isLocal Query is local flag.
      */
-    H2TwoStepCachedQueryKey(String schemaName,
+    QueryParserCacheKey(String schemaName,
         String sql,
         boolean grpByCollocated,
         boolean distributedJoins,
@@ -70,7 +70,7 @@ public class H2TwoStepCachedQueryKey {
         if (o == null || getClass() != o.getClass())
             return false;
 
-        H2TwoStepCachedQueryKey that = (H2TwoStepCachedQueryKey)o;
+        QueryParserCacheKey that = (QueryParserCacheKey)o;
 
         if (grpByCollocated != that.grpByCollocated)
             return false;
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/QueryParserResult.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/QueryParserResult.java
new file mode 100644
index 0000000..57bcf24
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/QueryParserResult.java
@@ -0,0 +1,120 @@
+/*
+ * 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;
+
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Result of parsing and splitting SQL from {@link SqlFieldsQuery}.
+ */
+public class QueryParserResult {
+    /** New fields query that may be executed right away. */
+    private final SqlFieldsQuery qry;
+
+    /** Remaining query. */
+    private final SqlFieldsQuery remainingQry;
+
+    /** Select. */
+    private final QueryParserResultSelect select;
+
+    /** DML. */
+    private final QueryParserResultDml dml;
+
+    /** Command. */
+    private final QueryParserResultCommand cmd;
+
+    /**
+     * Constructor.
+     *
+     * @param qry New query.
+     * @param remainingQry Remaining query.
+     * @param select Select.
+     * @param dml DML.
+     * @param cmd Command.
+     */
+    public QueryParserResult(
+        SqlFieldsQuery qry,
+        SqlFieldsQuery remainingQry,
+        @Nullable QueryParserResultSelect select,
+        @Nullable QueryParserResultDml dml,
+        @Nullable QueryParserResultCommand cmd
+    ) {
+        this.qry = qry;
+        this.remainingQry = remainingQry;
+        this.select = select;
+        this.dml = dml;
+        this.cmd = cmd;
+    }
+
+    /**
+     * @return New fields query that may be executed right away.
+     */
+    public SqlFieldsQuery query() {
+        return qry;
+    }
+
+    /**
+     * @return Remaining query.
+     */
+    @Nullable public SqlFieldsQuery remainingQuery() {
+        return remainingQry;
+    }
+
+    /**
+     * @return SELECT.
+     */
+    @Nullable public QueryParserResultSelect select() {
+        return select;
+    }
+
+    /**
+     * @return DML.
+     */
+    @Nullable public QueryParserResultDml dml() {
+        return dml;
+    }
+
+    /**
+     * @return Command.
+     */
+    @Nullable public QueryParserResultCommand command() {
+        return cmd;
+    }
+
+    /**
+     * @return Check whether this is SELECT.
+     */
+    public boolean isSelect() {
+        return select != null;
+    }
+
+    /**
+     * @return Check whether this is DML.
+     */
+    public boolean isDml() {
+        return dml != null;
+    }
+
+    /**
+     * @return Check whether this is a command.
+     */
+    public boolean isCommand() {
+        return cmd != null;
+    }
+}
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/QueryParserResultCommand.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/QueryParserResultCommand.java
new file mode 100644
index 0000000..5f45de0
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/QueryParserResultCommand.java
@@ -0,0 +1,70 @@
+/*
+ * 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;
+
+import org.apache.ignite.internal.processors.query.h2.sql.GridSqlStatement;
+import org.apache.ignite.internal.sql.command.SqlCommand;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Parsing result: command.
+ */
+public class QueryParserResultCommand {
+    /** Command (native). */
+    private final SqlCommand cmdNative;
+
+    /** Command (H2). */
+    private final GridSqlStatement cmdH2;
+
+    /** Whether this is a no-op command. */
+    private final boolean noOp;
+
+    /**
+     * Constructor.
+     *
+     * @param cmdNative Command (native).
+     * @param cmdH2 Command (H2).
+     * @param noOp Whether this is a no-op command.
+     */
+    public QueryParserResultCommand(@Nullable SqlCommand cmdNative, @Nullable GridSqlStatement cmdH2, boolean noOp) {
+        this.cmdNative = cmdNative;
+        this.cmdH2 = cmdH2;
+        this.noOp = noOp;
+    }
+
+    /**
+     * @return Command (native).
+     */
+    @Nullable public SqlCommand commandNative() {
+        return cmdNative;
+    }
+
+    /**
+     * @return Command (H2).
+     */
+    @Nullable public GridSqlStatement commandH2() {
+        return cmdH2;
+    }
+
+    /**
+     * @return Whether this is a no-op command.
+     */
+    public boolean noOp() {
+        return noOp;
+    }
+}
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/QueryParserResultDml.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/QueryParserResultDml.java
new file mode 100644
index 0000000..5e91288
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/QueryParserResultDml.java
@@ -0,0 +1,44 @@
+/*
+ * 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;
+
+import org.h2.command.Prepared;
+
+/**
+ * Parsing result for DML statement.
+ */
+public class QueryParserResultDml {
+    /** Command. */
+    private final Prepared prepared;
+
+    /**
+     * Constructor.
+     *
+     * @param prepared Command.
+     */
+    public QueryParserResultDml(Prepared prepared) {
+        this.prepared = prepared;
+    }
+
+    /**
+     * @return Command.
+     */
+    public Prepared prepared() {
+        return prepared;
+    }
+}
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2TwoStepCachedQuery.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/QueryParserResultSelect.java
similarity index 54%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2TwoStepCachedQuery.java
rename to modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/QueryParserResultSelect.java
index 6139869..a5787c2 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2TwoStepCachedQuery.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/QueryParserResultSelect.java
@@ -19,45 +19,51 @@ package org.apache.ignite.internal.processors.query.h2;
 
 import org.apache.ignite.internal.processors.cache.query.GridCacheTwoStepQuery;
 import org.apache.ignite.internal.processors.query.GridQueryFieldMetadata;
-import org.apache.ignite.internal.util.typedef.internal.S;
+import org.h2.command.Prepared;
 
 import java.util.List;
 
 /**
- * Cached two-step query.
+ * Parsing result for SELECT.
  */
-public class H2TwoStepCachedQuery {
-    /** */
-    private final List<GridQueryFieldMetadata> meta;
-
-    /** */
+public class QueryParserResultSelect {
+    /** Two-step query, or {@code} null if this result is for local query. */
     private final GridCacheTwoStepQuery twoStepQry;
 
-    /**
-     * @param meta Fields metadata.
-     * @param twoStepQry Query.
-     */
-    public H2TwoStepCachedQuery(List<GridQueryFieldMetadata> meta, GridCacheTwoStepQuery twoStepQry) {
-        this.meta = meta;
+    /** Metadata for two-step query, or {@code} null if this result is for local query. */
+    private final List<GridQueryFieldMetadata> twoStepQryMeta;
+
+    /** Prepared statement for local query. */
+    private final Prepared locPrepared;
+
+    public QueryParserResultSelect(
+        GridCacheTwoStepQuery twoStepQry,
+        List<GridQueryFieldMetadata> twoStepQryMeta,
+        Prepared locPrepared
+    ) {
         this.twoStepQry = twoStepQry;
+        this.twoStepQryMeta = twoStepQryMeta;
+        this.locPrepared = locPrepared;
     }
 
     /**
-     * @return Fields metadata.
+     * @return Two-step query, or {@code} null if this result is for local query.
      */
-    public List<GridQueryFieldMetadata> meta() {
-        return meta;
+    GridCacheTwoStepQuery twoStepQuery() {
+        return twoStepQry;
     }
 
     /**
-     * @return Query.
+     * @return Two-step query metadata.
      */
-    public GridCacheTwoStepQuery query() {
-        return twoStepQry;
+    public List<GridQueryFieldMetadata> twoStepQueryMeta() {
+        return twoStepQryMeta;
     }
 
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(H2TwoStepCachedQuery.class, this);
+    /**
+     * @return Prepared statement for local query.
+     */
+    public Prepared localPrepared() {
+        return locPrepared;
     }
 }
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteCachelessQueriesSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteCachelessQueriesSelfTest.java
index cf35758..1d43835 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteCachelessQueriesSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteCachelessQueriesSelfTest.java
@@ -29,7 +29,7 @@ import org.apache.ignite.cache.query.annotations.QuerySqlField;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.processors.cache.query.GridCacheTwoStepQuery;
-import org.apache.ignite.internal.processors.query.h2.H2TwoStepCachedQuery;
+import org.apache.ignite.internal.processors.query.h2.QueryParserCacheEntry;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.junit.Test;
@@ -319,12 +319,12 @@ public class IgniteCachelessQueriesSelfTest extends GridCommonAbstractTest {
     private GridCacheTwoStepQuery cachedTwoStepQuery() {
         GridQueryIndexing idx = grid(0).context().query().getIndexing();
 
-        Map<?, H2TwoStepCachedQuery> m = U.field(idx, "twoStepCache");
+        Map<?, QueryParserCacheEntry> m = U.field((Object)U.field(idx, "parser"), "cache");
 
         if (m.isEmpty())
             return null;
 
-        H2TwoStepCachedQuery q = m.values().iterator().next();
+        QueryParserCacheEntry q = m.values().iterator().next();
 
         return q.query();
     }
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/SqlQueryHistorySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/SqlQueryHistorySelfTest.java
index cd0fb97..3695710 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/SqlQueryHistorySelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/SqlQueryHistorySelfTest.java
@@ -83,7 +83,8 @@ public class SqlQueryHistorySelfTest extends GridCommonAbstractTest {
 
     /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
-        ((IgniteH2Indexing)queryNode().context().query().getIndexing()).resetQueryHistoryMetrics();
+        ((IgniteH2Indexing)queryNode().context().query().getIndexing()).runningQueryManager()
+            .resetQueryHistoryMetrics();
     }
 
     /**
@@ -199,7 +200,8 @@ public class SqlQueryHistorySelfTest extends GridCommonAbstractTest {
 
         // Check that collected metrics contains correct items: metrics for last N queries.
 
-        Collection<QueryHistoryMetrics> metrics = ((IgniteH2Indexing)queryNode().context().query().getIndexing()).queryHistoryMetrics().values();
+        Collection<QueryHistoryMetrics> metrics = ((IgniteH2Indexing)queryNode().context().query().getIndexing())
+            .runningQueryManager().queryHistoryMetrics().values();
 
         assertEquals(QUERY_HISTORY_SIZE, metrics.size());
 
@@ -289,7 +291,8 @@ public class SqlQueryHistorySelfTest extends GridCommonAbstractTest {
             checkMetrics(QUERY_HISTORY_SIZE, i, 1, 0, false);
 
         // Check that collected metrics contains correct items: metrics for last N queries.
-        Collection<QueryHistoryMetrics> metrics = ((IgniteH2Indexing)queryNode().context().query().getIndexing()).queryHistoryMetrics().values();
+        Collection<QueryHistoryMetrics> metrics = ((IgniteH2Indexing)queryNode().context().query().getIndexing())
+            .runningQueryManager().queryHistoryMetrics().values();
 
         assertEquals(QUERY_HISTORY_SIZE, metrics.size());
 
@@ -420,7 +423,8 @@ public class SqlQueryHistorySelfTest extends GridCommonAbstractTest {
     private void checkMetrics(int sz, int idx, int execs, int failures,
         boolean first) {
 
-        Collection<QueryHistoryMetrics> metrics = ((IgniteH2Indexing)queryNode().context().query().getIndexing()).queryHistoryMetrics().values();
+        Collection<QueryHistoryMetrics> metrics = ((IgniteH2Indexing)queryNode().context().query().getIndexing())
+            .runningQueryManager().queryHistoryMetrics().values();
 
         assertNotNull(metrics);
         assertEquals(sz, metrics.size());
@@ -560,7 +564,8 @@ public class SqlQueryHistorySelfTest extends GridCommonAbstractTest {
      */
     private void waitingFor(final String cond, final int exp) throws IgniteInterruptedCheckedException {
         GridTestUtils.waitForCondition(() -> {
-            Collection<QueryHistoryMetrics> metrics = ((IgniteH2Indexing)queryNode().context().query().getIndexing()).queryHistoryMetrics().values();
+            Collection<QueryHistoryMetrics> metrics = ((IgniteH2Indexing)queryNode().context().query().getIndexing())
+                .runningQueryManager().queryHistoryMetrics().values();
 
             switch (cond) {
                 case "size":