You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by al...@apache.org on 2022/09/01 07:14:37 UTC

[ignite] branch master updated: IGNITE-15424 Move query schema management infrastructure to the core module - Fixes #10200.

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

alexpl 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 7b173278cd1 IGNITE-15424 Move query schema management infrastructure to the core module - Fixes #10200.
7b173278cd1 is described below

commit 7b173278cd19164832f81c533c4ac7818435cb2c
Author: Aleksey Plekhanov <pl...@gmail.com>
AuthorDate: Thu Sep 1 10:07:17 2022 +0300

    IGNITE-15424 Move query schema management infrastructure to the core module - Fixes #10200.
    
    Signed-off-by: Aleksey Plekhanov <pl...@gmail.com>
---
 docs/_docs/SQL/sql-calcite.adoc                    |    2 -
 modules/calcite/pom.xml                            |    4 +-
 .../query/calcite/CalciteQueryProcessor.java       |   37 +-
 .../query/calcite/exec/ExecutionServiceImpl.java   |    8 +-
 .../query/calcite/exec/ddl/DdlCommandHandler.java  |   48 +-
 .../calcite/exec/ddl/NativeCommandHandler.java     |    5 +-
 .../query/calcite/exec/ddl/SchemaManager.java      |   85 --
 .../query/calcite/prepare/QueryPlanCacheImpl.java  |  148 +--
 .../query/calcite/schema/CacheTableImpl.java       |    5 +-
 .../query/calcite/schema/SchemaHolderImpl.java     |   21 +-
 .../processors/query/calcite/QueryChecker.java     |    7 +-
 .../query/calcite/UnstableTopologyTest.java        |   63 +-
 .../CalciteBasicSecondaryIndexIntegrationTest.java |   37 +-
 .../integration/IndexRebuildIntegrationTest.java   |   24 +-
 .../integration/RunningQueriesIntegrationTest.java |   16 +-
 .../ServerStatisticsIntegrationTest.java           |    2 +-
 .../StatisticsCommandDdlIntegrationTest.java       |   23 +-
 .../ignite/jdbc/thin/JdbcThinMetadataSelfTest.java |  960 +++++++-------
 .../SystemViewRowAttributeWalkerGenerator.java     |   25 +-
 .../CompoundIndexCompatibilityTest.java            |  167 +++
 .../IgnitePKIndexesMigrationToUnwrapPkTest.java    |  130 +-
 .../IndexAbstractCompatibilityTest.java            |   20 +-
 .../IgniteCompatibilityBasicTestSuite.java         |    6 +-
 .../util/GridCommandHandlerBrokenIndexTest.java    |    3 +-
 .../apache/ignite/util/SystemViewCommandTest.java  |   50 +-
 .../internal/cache/query/index/IndexProcessor.java |   50 +-
 .../query/index/sorted/QueryIndexDefinition.java   |   27 +-
 .../query/index/sorted/QueryIndexRowHandler.java   |   18 +-
 .../index/sorted/QueryIndexRowHandlerFactory.java  |    2 +-
 .../query/index/sorted/SortedIndexDefinition.java  |   10 +-
 .../query/index/sorted/inline/InlineIndexImpl.java |  183 ++-
 .../sorted}/maintenance/RebuildIndexAction.java    |  208 +--
 .../maintenance/RebuildIndexWorkflowCallback.java  |   25 +-
 .../internal/managers/IgniteMBeansManager.java     |    2 +-
 .../managers/systemview/GridSystemViewManager.java |   17 +-
 .../managers/systemview/SqlViewExporterSpi.java    |   22 +-
 .../systemview/walker/SqlIndexViewWalker.java      |    2 +-
 .../systemview/walker/SqlSchemaViewWalker.java     |    2 +-
 .../walker/SqlTableColumnViewWalker.java           |    2 +-
 .../systemview/walker/SqlTableViewWalker.java      |    2 +-
 .../systemview/walker/SqlViewColumnViewWalker.java |    6 +-
 .../systemview/walker/SqlViewViewWalker.java       |    2 +-
 .../processors/cache/GridCacheProcessor.java       |    2 +-
 .../processors/cache/IgniteCacheProxyImpl.java     |    2 +-
 .../cache/persistence/tree/BPlusTree.java          |    1 +
 .../processors/metric/GridMetricManager.java       |   12 +-
 .../metric}/SqlViewMetricExporterSpi.java          |    2 +-
 .../processors/metric/impl/MetricUtils.java        |   22 +
 .../processors/odbc/jdbc/JdbcMetadataInfo.java     |    6 +-
 .../internal/processors/pool/PoolProcessor.java    |    2 +
 .../processors/query/GridQueryIndexing.java        |  175 +--
 .../processors/query/GridQueryProcessor.java       |  193 ++-
 .../processors/query/GridQuerySchemaManager.java   |   47 -
 .../processors/query/GridQueryTypeDescriptor.java  |    7 +
 .../query/QuerySysIndexDescriptorImpl.java         |   13 +-
 .../processors/query/QueryTypeDescriptorImpl.java  |   10 +-
 .../internal/processors/query/QueryUtils.java      |    3 +
 .../query/schema/AbstractSchemaChangeListener.java |   10 +-
 .../query/schema/SchemaChangeListener.java         |   11 +-
 .../query/schema/SchemaIndexCacheVisitorImpl.java  |   12 +-
 .../management/AbstractIndexDescriptorFactory.java |   63 +
 .../query/schema/management/IndexDescriptor.java   |  144 +++
 .../schema/management/IndexDescriptorFactory.java} |   41 +-
 .../query/schema/management/SchemaDescriptor.java} |   39 +-
 .../query/schema/management/SchemaManager.java     | 1357 ++++++++++++++++++++
 .../management/SortedIndexDescriptorFactory.java   |  181 +++
 .../query/schema/management/TableDescriptor.java   |  113 ++
 .../stat/IgniteStatisticsConfigurationManager.java |   30 +-
 .../query/stat/IgniteStatisticsHelper.java         |   16 +-
 .../query/stat/IgniteStatisticsManagerImpl.java    |   23 +-
 .../ignite/internal/sql/SqlCommandProcessor.java   |   47 +-
 .../spi/systemview/view/sql}/SqlIndexView.java     |   36 +-
 .../spi/systemview/view/sql}/SqlSchemaView.java    |   12 +-
 .../systemview/view/sql}/SqlTableColumnView.java   |   55 +-
 .../spi/systemview/view/sql}/SqlTableView.java     |   36 +-
 .../systemview/view/sql}/SqlViewColumnView.java    |   30 +-
 .../spi/systemview/view/sql}/SqlViewView.java      |   17 +-
 .../IgniteClientCacheInitializationFailTest.java   |   23 -
 .../processors/query/DummyQueryIndexing.java       |  108 +-
 .../processors/query/h2/CommandProcessor.java      |  119 +-
 .../processors/query/h2/ConnectionManager.java     |   43 +-
 .../processors/query/h2/H2IndexFactory.java        |  115 ++
 .../processors/query/h2/H2SchemaManager.java       |  417 ++++++
 .../processors/query/h2/H2TableDescriptor.java     |  302 +----
 .../processors/query/h2/H2TableEngine.java         |    4 +-
 .../internal/processors/query/h2/H2Utils.java      |  227 +---
 .../processors/query/h2/IgniteH2Indexing.java      |  458 +------
 .../internal/processors/query/h2/QueryParser.java  |    2 +-
 .../processors/query/h2/SchemaManager.java         | 1098 ----------------
 .../processors/query/h2/database/H2TreeIndex.java  |   24 +-
 .../processors/query/h2/opt/GridH2IndexBase.java   |   16 +-
 .../processors/query/h2/opt/GridH2ProxyIndex.java  |    7 +-
 .../processors/query/h2/opt/GridH2Table.java       |  330 +----
 .../h2/sys/view}/FiltrableSystemViewLocal.java     |    3 +-
 .../query/h2/sys/view}/SystemViewLocal.java        |    4 +-
 .../ignite/cache/query/IndexQueryFailoverTest.java |    2 +-
 .../cache/index/AbstractRebuildIndexTest.java      |  105 +-
 .../DynamicColumnsAbstractConcurrentSelfTest.java  |   38 +-
 .../index/DynamicEnableIndexingAbstractTest.java   |   25 +-
 .../DynamicIndexAbstractConcurrentSelfTest.java    |  103 +-
 .../cache/index/H2DynamicTableSelfTest.java        |   13 +-
 .../processors/cache/index/IgniteH2IndexingEx.java |  113 --
 .../cache/index/IndexCorruptionRebuildTest.java    |   46 +-
 .../cache/index/ResumeCreateIndexTest.java         |    1 -
 .../cache/metric/SqlViewExporterSpiTest.java       |   45 +-
 .../processors/query/SqlSystemViewsSelfTest.java   |   65 +-
 .../SqlTwoCachesInGroupWithSameEntryTest.java      |    7 +-
 .../query/h2/sql/GridQueryParsingTest.java         |    7 +-
 .../query/stat/ManagerStatisticsTypesTest.java     |    7 +-
 .../query/stat/SqlStatisticsCommandTests.java      |   13 +-
 .../query/stat/StatisticsAbstractTest.java         |   21 +-
 .../query/stat/StatisticsConfigurationTest.java    |    3 +-
 .../internal/systemview/JmxExporterSpiTest.java    |    6 +-
 113 files changed, 4620 insertions(+), 4546 deletions(-)

diff --git a/docs/_docs/SQL/sql-calcite.adoc b/docs/_docs/SQL/sql-calcite.adoc
index 44b8022bb95..d591ae0f2a7 100644
--- a/docs/_docs/SQL/sql-calcite.adoc
+++ b/docs/_docs/SQL/sql-calcite.adoc
@@ -26,8 +26,6 @@ CAUTION: The Calcite-based query engine is currently in beta status.
 
 To use a Calcite-based engine, please make sure that the Calcite module libraries are in a classpath.
 
-CAUTION: Currently, a part of the `ignite-indexing` module functionality is reused, this means the `ignite-indexing` module also has to be present at classpath.
-
 === Standalone Mode
 
 When starting a standalone node, move `optional/ignite-calcite` folder to the `libs` folder before running `ignite.{sh|bat}` script. In this case, the content of the module folder is added to the classpath.
diff --git a/modules/calcite/pom.xml b/modules/calcite/pom.xml
index 5527715b473..9b7e02018c9 100644
--- a/modules/calcite/pom.xml
+++ b/modules/calcite/pom.xml
@@ -55,12 +55,12 @@
         </dependency>
 
         <!--
-            At now the new calcite engine reuses some logic
-            and doesn't work without "old" indexing module.
+            Indexing is required for cross-engines tests.
         -->
         <dependency>
             <groupId>${project.groupId}</groupId>
             <artifactId>ignite-indexing</artifactId>
+            <scope>test</scope>
         </dependency>
 
         <dependency>
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessor.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessor.java
index 538fdb4ff60..6ff731ae2d6 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessor.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessor.java
@@ -194,6 +194,9 @@ public class CalciteQueryProcessor extends GridProcessorAdapter implements Query
     /** */
     private final QueryRegistry qryReg;
 
+    /** */
+    private volatile boolean started;
+
     /**
      * @param ctx Kernal context.
      */
@@ -283,7 +286,7 @@ public class CalciteQueryProcessor extends GridProcessorAdapter implements Query
     }
 
     /** {@inheritDoc} */
-    @Override public void start() {
+    @Override public void onKernalStart(boolean active) {
         onStart(ctx,
             executionSvc,
             mailboxRegistry,
@@ -296,22 +299,28 @@ public class CalciteQueryProcessor extends GridProcessorAdapter implements Query
             exchangeSvc,
             qryReg
         );
+
+        started = true;
     }
 
     /** {@inheritDoc} */
-    @Override public void stop(boolean cancel) {
-        onStop(
-            qryReg,
-            executionSvc,
-            mailboxRegistry,
-            partSvc,
-            schemaHolder,
-            msgSvc,
-            taskExecutor,
-            mappingSvc,
-            qryPlanCache,
-            exchangeSvc
-        );
+    @Override public void onKernalStop(boolean cancel) {
+        if (started) {
+            started = false;
+
+            onStop(
+                qryReg,
+                executionSvc,
+                mailboxRegistry,
+                partSvc,
+                schemaHolder,
+                msgSvc,
+                taskExecutor,
+                mappingSvc,
+                qryPlanCache,
+                exchangeSvc
+            );
+        }
     }
 
     /** {@inheritDoc} */
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionServiceImpl.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionServiceImpl.java
index 60c5ee1648a..490c2644cf8 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionServiceImpl.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionServiceImpl.java
@@ -152,7 +152,7 @@ public class ExecutionServiceImpl<Row> extends AbstractService implements Execut
     private final RowHandler<Row> handler;
 
     /** */
-    private final DdlCommandHandler ddlCmdHnd;
+    private DdlCommandHandler ddlCmdHnd;
 
     /**
      * @param ctx Kernal.
@@ -162,10 +162,6 @@ public class ExecutionServiceImpl<Row> extends AbstractService implements Execut
         this.handler = handler;
 
         discoLsnr = (e, c) -> onNodeLeft(e.eventNode().id());
-
-        ddlCmdHnd = new DdlCommandHandler(
-            ctx::query, ctx.cache(), ctx.security(), () -> schemaHolder().schema(null)
-        );
     }
 
     /**
@@ -384,6 +380,8 @@ public class ExecutionServiceImpl<Row> extends AbstractService implements Execut
         queryRegistry(proc.queryRegistry());
         prepareService(proc.prepareService());
 
+        ddlCmdHnd = new DdlCommandHandler(ctx.query(), ctx.cache(), ctx.security(), () -> schemaHolder().schema(null));
+
         init();
     }
 
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ddl/DdlCommandHandler.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ddl/DdlCommandHandler.java
index 1789eb142b0..3a6b00c8fa1 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ddl/DdlCommandHandler.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ddl/DdlCommandHandler.java
@@ -38,7 +38,6 @@ import org.apache.ignite.internal.processors.cache.GridCacheContextInfo;
 import org.apache.ignite.internal.processors.cache.GridCacheProcessor;
 import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
 import org.apache.ignite.internal.processors.query.GridQueryProcessor;
-import org.apache.ignite.internal.processors.query.GridQuerySchemaManager;
 import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
 import org.apache.ignite.internal.processors.query.IgniteSQLException;
 import org.apache.ignite.internal.processors.query.QueryEntityEx;
@@ -56,6 +55,8 @@ import org.apache.ignite.internal.processors.query.calcite.schema.IgniteCacheTab
 import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
 import org.apache.ignite.internal.processors.query.calcite.util.Commons;
 import org.apache.ignite.internal.processors.query.schema.SchemaOperationException;
+import org.apache.ignite.internal.processors.query.schema.management.SchemaManager;
+import org.apache.ignite.internal.processors.query.schema.management.TableDescriptor;
 import org.apache.ignite.internal.processors.security.IgniteSecurity;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.lang.IgniteUuid;
@@ -67,10 +68,10 @@ import static org.apache.ignite.internal.processors.query.QueryUtils.isDdlOnSche
 /** */
 public class DdlCommandHandler {
     /** */
-    private final Supplier<GridQueryProcessor> qryProcessorSupp;
+    private final GridQueryProcessor qryProc;
 
     /** */
-    private final GridCacheProcessor cacheProcessor;
+    private final GridCacheProcessor cacheProc;
 
     /** */
     private final IgniteSecurity security;
@@ -82,17 +83,17 @@ public class DdlCommandHandler {
     private final NativeCommandHandler nativeCmdHnd;
 
     /** */
-    private final GridQuerySchemaManager schemaMgr;
+    private final SchemaManager schemaMgr;
 
     /** */
-    public DdlCommandHandler(Supplier<GridQueryProcessor> qryProcessorSupp, GridCacheProcessor cacheProcessor,
+    public DdlCommandHandler(GridQueryProcessor qryProc, GridCacheProcessor cacheProc,
         IgniteSecurity security, Supplier<SchemaPlus> schemaSupp) {
-        this.qryProcessorSupp = qryProcessorSupp;
-        this.cacheProcessor = cacheProcessor;
+        this.qryProc = qryProc;
+        this.cacheProc = cacheProc;
         this.security = security;
         this.schemaSupp = schemaSupp;
-        schemaMgr = new SchemaManager(schemaSupp);
-        nativeCmdHnd = new NativeCommandHandler(cacheProcessor.context().kernalContext(), schemaMgr);
+        schemaMgr = qryProc.schemaManager();
+        nativeCmdHnd = new NativeCommandHandler(cacheProc.context().kernalContext());
     }
 
     /** */
@@ -143,13 +144,13 @@ public class DdlCommandHandler {
         ccfg.setSqlSchema(cmd.schemaName());
 
         SchemaOperationException err =
-            QueryUtils.checkQueryEntityConflicts(ccfg, cacheProcessor.cacheDescriptors().values());
+            QueryUtils.checkQueryEntityConflicts(ccfg, cacheProc.cacheDescriptors().values());
 
         if (err != null)
             throw convert(err);
 
-        if (!F.isEmpty(cmd.cacheName()) && cacheProcessor.cacheDescriptor(cmd.cacheName()) != null) {
-            qryProcessorSupp.get().dynamicAddQueryEntity(
+        if (!F.isEmpty(cmd.cacheName()) && cacheProc.cacheDescriptor(cmd.cacheName()) != null) {
+            qryProc.dynamicAddQueryEntity(
                 cmd.cacheName(),
                 cmd.schemaName(),
                 e,
@@ -158,7 +159,7 @@ public class DdlCommandHandler {
             ).get();
         }
         else {
-            qryProcessorSupp.get().dynamicTableCreate(
+            qryProc.dynamicTableCreate(
                 cmd.schemaName(),
                 e,
                 cmd.templateName(),
@@ -195,20 +196,22 @@ public class DdlCommandHandler {
 
         security.authorize(cacheName, SecurityPermission.CACHE_DESTROY);
 
-        qryProcessorSupp.get().dynamicTableDrop(cacheName, cmd.tableName(), cmd.ifExists());
+        qryProc.dynamicTableDrop(cacheName, cmd.tableName(), cmd.ifExists());
     }
 
     /** */
     private void handle0(AlterTableAddCommand cmd) throws IgniteCheckedException {
         isDdlOnSchemaSupported(cmd.schemaName());
 
-        GridQueryTypeDescriptor typeDesc = schemaMgr.typeDescriptorForTable(cmd.schemaName(), cmd.tableName());
+        TableDescriptor tblDesc = schemaMgr.table(cmd.schemaName(), cmd.tableName());
 
-        if (typeDesc == null) {
+        if (tblDesc == null) {
             if (!cmd.ifTableExists())
                 throw new SchemaOperationException(SchemaOperationException.CODE_TABLE_NOT_FOUND, cmd.tableName());
         }
         else {
+            GridQueryTypeDescriptor typeDesc = tblDesc.type();
+
             if (QueryUtils.isSqlType(typeDesc.valueClass())) {
                 throw new SchemaOperationException("Cannot add column(s) because table was created " +
                     "with WRAP_VALUE=false option.");
@@ -243,14 +246,14 @@ public class DdlCommandHandler {
             }
 
             if (!F.isEmpty(cols)) {
-                GridCacheContextInfo<?, ?> ctxInfo = schemaMgr.cacheInfoForTable(cmd.schemaName(), cmd.tableName());
+                GridCacheContextInfo<?, ?> ctxInfo = tblDesc.cacheInfo();
 
                 assert ctxInfo != null;
 
                 if (!allFieldsNullable)
                     QueryUtils.checkNotNullAllowed(ctxInfo.config());
 
-                qryProcessorSupp.get().dynamicColumnAdd(ctxInfo.name(), cmd.schemaName(),
+                qryProc.dynamicColumnAdd(ctxInfo.name(), cmd.schemaName(),
                     typeDesc.tableName(), cols, cmd.ifTableExists(), cmd.ifColumnNotExists()).get();
             }
         }
@@ -260,14 +263,15 @@ public class DdlCommandHandler {
     private void handle0(AlterTableDropCommand cmd) throws IgniteCheckedException {
         isDdlOnSchemaSupported(cmd.schemaName());
 
-        GridQueryTypeDescriptor typeDesc = schemaMgr.typeDescriptorForTable(cmd.schemaName(), cmd.tableName());
+        TableDescriptor tblDesc = schemaMgr.table(cmd.schemaName(), cmd.tableName());
 
-        if (typeDesc == null) {
+        if (tblDesc == null) {
             if (!cmd.ifTableExists())
                 throw new SchemaOperationException(SchemaOperationException.CODE_TABLE_NOT_FOUND, cmd.tableName());
         }
         else {
-            GridCacheContextInfo<?, ?> ctxInfo = schemaMgr.cacheInfoForTable(cmd.schemaName(), cmd.tableName());
+            GridQueryTypeDescriptor typeDesc = tblDesc.type();
+            GridCacheContextInfo<?, ?> ctxInfo = tblDesc.cacheInfo();
 
             GridCacheContext<?, ?> cctx = ctxInfo.cacheContext();
 
@@ -302,7 +306,7 @@ public class DdlCommandHandler {
             }
 
             if (!F.isEmpty(cols)) {
-                qryProcessorSupp.get().dynamicColumnRemove(ctxInfo.name(), cmd.schemaName(),
+                qryProc.dynamicColumnRemove(ctxInfo.name(), cmd.schemaName(),
                     typeDesc.tableName(), cols, cmd.ifTableExists(), cmd.ifColumnExists()).get();
             }
         }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ddl/NativeCommandHandler.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ddl/NativeCommandHandler.java
index 3cca9d61855..edecf493781 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ddl/NativeCommandHandler.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ddl/NativeCommandHandler.java
@@ -21,7 +21,6 @@ import java.util.List;
 import java.util.UUID;
 import org.apache.ignite.cache.query.FieldsQueryCursor;
 import org.apache.ignite.internal.GridKernalContext;
-import org.apache.ignite.internal.processors.query.GridQuerySchemaManager;
 import org.apache.ignite.internal.processors.query.calcite.prepare.ddl.NativeCommandWrapper;
 import org.apache.ignite.internal.sql.SqlCommandProcessor;
 
@@ -35,8 +34,8 @@ public class NativeCommandHandler {
     /**
      * @param ctx Context.
      */
-    public NativeCommandHandler(GridKernalContext ctx, GridQuerySchemaManager schemaMgr) {
-        proc = new SqlCommandProcessor(ctx, schemaMgr);
+    public NativeCommandHandler(GridKernalContext ctx) {
+        proc = new SqlCommandProcessor(ctx);
     }
 
     /**
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ddl/SchemaManager.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ddl/SchemaManager.java
deleted file mode 100644
index 9e2b243f014..00000000000
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ddl/SchemaManager.java
+++ /dev/null
@@ -1,85 +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.calcite.exec.ddl;
-
-import java.util.function.Supplier;
-import org.apache.calcite.schema.SchemaPlus;
-import org.apache.calcite.schema.Table;
-import org.apache.ignite.internal.processors.cache.GridCacheContextInfo;
-import org.apache.ignite.internal.processors.query.GridQuerySchemaManager;
-import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
-import org.apache.ignite.internal.processors.query.calcite.schema.IgniteCacheTable;
-import org.apache.ignite.internal.processors.query.calcite.schema.IgniteTable;
-
-/**
- * Schema manager.
- */
-class SchemaManager implements GridQuerySchemaManager {
-    /** Schema holder. */
-    private final Supplier<SchemaPlus> schemaSupp;
-
-    /**
-     * @param schemaSupp Schema supplier.
-     */
-    SchemaManager(Supplier<SchemaPlus> schemaSupp) {
-        this.schemaSupp = schemaSupp;
-    }
-
-    /** {@inheritDoc} */
-    @Override public GridQueryTypeDescriptor typeDescriptorForTable(String schemaName, String tableName) {
-        SchemaPlus schema = schemaSupp.get().getSubSchema(schemaName);
-
-        if (schema == null)
-            return null;
-
-        IgniteCacheTable tbl = (IgniteCacheTable)schema.getTable(tableName);
-
-        return tbl == null ? null : tbl.descriptor().typeDescription();
-    }
-
-    /** {@inheritDoc} */
-    @Override public GridQueryTypeDescriptor typeDescriptorForIndex(String schemaName, String idxName) {
-        SchemaPlus schema = schemaSupp.get().getSubSchema(schemaName);
-
-        if (schema == null)
-            return null;
-
-        for (String tableName : schema.getTableNames()) {
-            Table tbl = schema.getTable(tableName);
-
-            if (tbl instanceof IgniteCacheTable && ((IgniteTable)tbl).getIndex(idxName) != null)
-                return ((IgniteCacheTable)tbl).descriptor().typeDescription();
-        }
-
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public <K, V> GridCacheContextInfo<K, V> cacheInfoForTable(String schemaName, String tableName) {
-        SchemaPlus schema = schemaSupp.get().getSubSchema(schemaName);
-
-        if (schema == null)
-            return null;
-
-        Table tbl = schema.getTable(tableName);
-
-        IgniteCacheTable cachetbl = tbl instanceof IgniteCacheTable ? (IgniteCacheTable)tbl : null;
-
-        return cachetbl == null ? null : (GridCacheContextInfo<K, V>)cachetbl.descriptor().cacheInfo();
-    }
-}
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/QueryPlanCacheImpl.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/QueryPlanCacheImpl.java
index 74b992cec39..178faf8cd5e 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/QueryPlanCacheImpl.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/QueryPlanCacheImpl.java
@@ -17,26 +17,23 @@
 
 package org.apache.ignite.internal.processors.query.calcite.prepare;
 
-import java.lang.reflect.Method;
 import java.util.List;
 import java.util.Map;
 import java.util.function.Supplier;
 import org.apache.ignite.internal.GridKernalContext;
-import org.apache.ignite.internal.cache.query.index.Index;
 import org.apache.ignite.internal.processors.cache.GridCacheContextInfo;
-import org.apache.ignite.internal.processors.query.GridQueryIndexDescriptor;
 import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
 import org.apache.ignite.internal.processors.query.QueryField;
 import org.apache.ignite.internal.processors.query.calcite.util.AbstractService;
-import org.apache.ignite.internal.processors.query.schema.SchemaChangeListener;
+import org.apache.ignite.internal.processors.query.schema.AbstractSchemaChangeListener;
+import org.apache.ignite.internal.processors.query.schema.management.IndexDescriptor;
 import org.apache.ignite.internal.processors.subscription.GridInternalSubscriptionProcessor;
 import org.apache.ignite.internal.util.GridBoundedConcurrentLinkedHashMap;
-import org.apache.ignite.spi.systemview.view.SystemView;
 
 /**
  *
  */
-public class QueryPlanCacheImpl extends AbstractService implements QueryPlanCache, SchemaChangeListener {
+public class QueryPlanCacheImpl extends AbstractService implements QueryPlanCache {
     /** */
     private static final int CACHE_SIZE = 1024;
 
@@ -67,7 +64,7 @@ public class QueryPlanCacheImpl extends AbstractService implements QueryPlanCach
 
     /** {@inheritDoc} */
     @Override public void init() {
-        subscriptionProcessor.registerSchemaChangeListener(this);
+        subscriptionProcessor.registerSchemaChangeListener(new SchemaListener());
     }
 
     /** {@inheritDoc} */
@@ -96,82 +93,65 @@ public class QueryPlanCacheImpl extends AbstractService implements QueryPlanCach
         cache = new GridBoundedConcurrentLinkedHashMap<>(CACHE_SIZE);
     }
 
-    /** {@inheritDoc} */
-    @Override public void onSchemaDropped(String schemaName) {
-        clear();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onSqlTypeDropped(
-        String schemaName,
-        GridQueryTypeDescriptor typeDescriptor,
-        boolean destroy
-    ) {
-        clear();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onIndexCreated(String schemaName, String tblName, String idxName,
-        GridQueryIndexDescriptor idxDesc, Index idx) {
-        clear();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onIndexDropped(String schemaName, String tblName, String idxName) {
-        clear();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onIndexRebuildStarted(String schemaName, String tblName) {
-        clear();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onIndexRebuildFinished(String schemaName, String tblName) {
-        clear();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onSchemaCreated(String schemaName) {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onSqlTypeCreated(
-        String schemaName,
-        GridQueryTypeDescriptor typeDesc,
-        GridCacheContextInfo<?, ?> cacheInfo
-    ) {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onColumnsAdded(
-        String schemaName,
-        GridQueryTypeDescriptor typeDesc,
-        GridCacheContextInfo<?, ?> cacheInfo,
-        List<QueryField> cols
-    ) {
-        clear();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onColumnsDropped(
-        String schemaName,
-        GridQueryTypeDescriptor typeDesc,
-        GridCacheContextInfo<?, ?> cacheInfo,
-        List<String> cols
-    ) {
-        clear();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onFunctionCreated(String schemaName, String name, Method method) {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onSystemViewCreated(String schemaName, SystemView<?> sysView) {
-        // No-op.
+    /** Schema change listener. */
+    private class SchemaListener extends AbstractSchemaChangeListener {
+        /** {@inheritDoc} */
+        @Override public void onSchemaDropped(String schemaName) {
+            clear();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onSqlTypeDropped(
+            String schemaName,
+            GridQueryTypeDescriptor typeDescriptor,
+            boolean destroy
+        ) {
+            clear();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onIndexCreated(
+            String schemaName,
+            String tblName,
+            String idxName,
+            IndexDescriptor idxDesc
+        ) {
+            clear();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onIndexDropped(String schemaName, String tblName, String idxName) {
+            clear();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onIndexRebuildStarted(String schemaName, String tblName) {
+            clear();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onIndexRebuildFinished(String schemaName, String tblName) {
+            clear();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onColumnsAdded(
+            String schemaName,
+            GridQueryTypeDescriptor typeDesc,
+            GridCacheContextInfo<?, ?> cacheInfo,
+            List<QueryField> cols
+        ) {
+            clear();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onColumnsDropped(
+            String schemaName,
+            GridQueryTypeDescriptor typeDesc,
+            GridCacheContextInfo<?, ?> cacheInfo,
+            List<String> cols
+        ) {
+            clear();
+        }
     }
 }
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/CacheTableImpl.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/CacheTableImpl.java
index 1d036527d78..0487725cfa1 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/CacheTableImpl.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/CacheTableImpl.java
@@ -41,7 +41,6 @@ import org.apache.ignite.internal.processors.query.calcite.prepare.MappingQueryC
 import org.apache.ignite.internal.processors.query.calcite.rel.logical.IgniteLogicalTableScan;
 import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistribution;
 import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
-import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
 import org.apache.ignite.internal.processors.query.stat.ObjectStatisticsImpl;
 import org.apache.ignite.internal.processors.query.stat.StatisticsKey;
 import org.apache.ignite.internal.util.typedef.internal.U;
@@ -79,12 +78,10 @@ public class CacheTableImpl extends AbstractTable implements IgniteCacheTable {
 
     /** {@inheritDoc} */
     @Override public Statistic getStatistic() {
-        IgniteH2Indexing idx = (IgniteH2Indexing)ctx.query().getIndexing();
-
         final String tblName = desc.typeDescription().tableName();
         final String schemaName = desc.typeDescription().schemaName();
 
-        ObjectStatisticsImpl statistics = (ObjectStatisticsImpl)idx.statsManager().getLocalStatistics(
+        ObjectStatisticsImpl statistics = (ObjectStatisticsImpl)ctx.query().statsManager().getLocalStatistics(
             new StatisticsKey(schemaName, tblName));
 
         if (statistics != null)
diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/SchemaHolderImpl.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/SchemaHolderImpl.java
index 9ea703a577b..a810d8e56bf 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/SchemaHolderImpl.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/SchemaHolderImpl.java
@@ -33,9 +33,9 @@ import org.apache.ignite.cache.affinity.AffinityFunction;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.internal.GridKernalContext;
-import org.apache.ignite.internal.cache.query.index.Index;
+import org.apache.ignite.internal.cache.query.index.SortOrder;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyDefinition;
 import org.apache.ignite.internal.processors.cache.GridCacheContextInfo;
-import org.apache.ignite.internal.processors.query.GridQueryIndexDescriptor;
 import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
 import org.apache.ignite.internal.processors.query.QueryField;
 import org.apache.ignite.internal.processors.query.calcite.exec.exp.IgniteScalarFunction;
@@ -43,6 +43,7 @@ import org.apache.ignite.internal.processors.query.calcite.trait.TraitUtils;
 import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
 import org.apache.ignite.internal.processors.query.calcite.util.AbstractService;
 import org.apache.ignite.internal.processors.query.schema.SchemaChangeListener;
+import org.apache.ignite.internal.processors.query.schema.management.IndexDescriptor;
 import org.apache.ignite.internal.processors.subscription.GridInternalSubscriptionProcessor;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.lang.IgnitePredicate;
@@ -230,7 +231,7 @@ public class SchemaHolderImpl extends AbstractService implements SchemaHolder, S
 
     /** {@inheritDoc} */
     @Override public synchronized void onIndexCreated(String schemaName, String tblName, String idxName,
-        GridQueryIndexDescriptor idxDesc, @Nullable Index gridIdx) {
+        IndexDescriptor idxDesc) {
         IgniteSchema schema = igniteSchemas.get(schemaName);
         assert schema != null;
 
@@ -239,7 +240,7 @@ public class SchemaHolderImpl extends AbstractService implements SchemaHolder, S
 
         RelCollation idxCollation = deriveSecondaryIndexCollation(idxDesc, tbl);
 
-        IgniteIndex idx = new CacheIndexImpl(idxCollation, idxName, gridIdx, tbl);
+        IgniteIndex idx = new CacheIndexImpl(idxCollation, idxName, idxDesc.index(), tbl);
         tbl.addIndex(idx);
     }
 
@@ -247,18 +248,18 @@ public class SchemaHolderImpl extends AbstractService implements SchemaHolder, S
      * @return Index collation.
      */
     @NotNull private static RelCollation deriveSecondaryIndexCollation(
-        GridQueryIndexDescriptor idxDesc,
+        IndexDescriptor idxDesc,
         IgniteCacheTable tbl
     ) {
         CacheTableDescriptor tblDesc = tbl.descriptor();
-        List<RelFieldCollation> collations = new ArrayList<>(idxDesc.fields().size());
+        List<RelFieldCollation> collations = new ArrayList<>(idxDesc.keyDefinitions().size());
 
-        for (String idxField : idxDesc.fields()) {
-            ColumnDescriptor fieldDesc = tblDesc.columnDescriptor(idxField);
+        for (Map.Entry<String, IndexKeyDefinition> keyDef : idxDesc.keyDefinitions().entrySet()) {
+            ColumnDescriptor fieldDesc = tblDesc.columnDescriptor(keyDef.getKey());
 
             assert fieldDesc != null;
 
-            boolean descending = idxDesc.descending(idxField);
+            boolean descending = keyDef.getValue().order().sortOrder() == SortOrder.DESC;
             int fieldIdx = fieldDesc.fieldIndex();
 
             collations.add(TraitUtils.createFieldCollation(fieldIdx, !descending));
@@ -303,7 +304,7 @@ public class SchemaHolderImpl extends AbstractService implements SchemaHolder, S
     }
 
     /** {@inheritDoc} */
-    @Override public void onFunctionCreated(String schemaName, String name, Method method) {
+    @Override public void onFunctionCreated(String schemaName, String name, boolean deterministic, Method method) {
         IgniteSchema schema = igniteSchemas.computeIfAbsent(schemaName, IgniteSchema::new);
 
         schema.addFunction(name.toUpperCase(), IgniteScalarFunction.create(method));
diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/QueryChecker.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/QueryChecker.java
index 4649703a7bf..4c2f92a9a25 100644
--- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/QueryChecker.java
+++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/QueryChecker.java
@@ -35,6 +35,7 @@ import org.apache.ignite.internal.IgniteInterruptedCheckedException;
 import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache;
 import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition;
 import org.apache.ignite.internal.processors.query.QueryEngine;
+import org.apache.ignite.internal.processors.query.schema.management.SchemaManager;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.G;
 import org.apache.ignite.internal.util.typedef.X;
@@ -87,7 +88,11 @@ public abstract class QueryChecker {
      * @return Matcher.
      */
     public static Matcher<String> containsIndexScan(String schema, String tblName, String idxName) {
-        return containsSubPlan("IgniteIndexScan(table=[[" + schema + ", " + tblName + "]], index=[" + idxName + ']');
+        return CoreMatchers.anyOf(
+            containsSubPlan("IgniteIndexScan(table=[[" + schema + ", " + tblName + "]], index=[" + idxName + ']'),
+            containsSubPlan("IgniteIndexScan(table=[[" + schema + ", " + tblName + "]], index=[" +
+                SchemaManager.generateProxyIdxName(idxName) + ']')
+        );
     }
 
     /**
diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/UnstableTopologyTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/UnstableTopologyTest.java
index fa448fa5361..b1e5897690e 100644
--- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/UnstableTopologyTest.java
+++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/UnstableTopologyTest.java
@@ -17,14 +17,16 @@
 
 package org.apache.ignite.internal.processors.query.calcite;
 
+import java.util.Collection;
 import java.util.Collections;
+import java.util.Iterator;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Random;
 import java.util.concurrent.ThreadLocalRandom;
+import javax.cache.Cache;
 import com.google.common.collect.ImmutableList;
 import org.apache.ignite.Ignite;
-import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteDataStreamer;
 import org.apache.ignite.binary.BinaryObject;
 import org.apache.ignite.cache.CacheAtomicityMode;
@@ -37,16 +39,15 @@ import org.apache.ignite.cluster.ClusterState;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.IgniteEx;
-import org.apache.ignite.internal.processors.cache.GridCacheContext;
-import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
-import org.apache.ignite.internal.processors.query.GridQueryProcessor;
-import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
 import org.apache.ignite.internal.processors.query.QueryEngine;
 import org.apache.ignite.internal.processors.query.calcite.util.Commons;
-import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
 import org.apache.ignite.internal.util.typedef.G;
-import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.spi.IgniteSpiAdapter;
+import org.apache.ignite.spi.IgniteSpiException;
+import org.apache.ignite.spi.indexing.IndexingQueryFilter;
+import org.apache.ignite.spi.indexing.IndexingSpi;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.jetbrains.annotations.Nullable;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
@@ -183,24 +184,52 @@ public class UnstableTopologyTest extends GridCommonAbstractTest {
         }
     }
 
-    /** Start with custon QueryIndexing. */
+    /** Start with custon indexing SPI. */
     private IgniteEx ignitionStart(int idx, boolean slow) throws Exception {
+        IgniteConfiguration cfg = getConfiguration(getTestIgniteInstanceName(idx));
+
         if (slow)
-            GridQueryProcessor.idxCls = BlockingIndexing.class;
+            cfg.setIndexingSpi(new BlockingIndexingSpi());
 
-        return startGrid(getConfiguration(getTestIgniteInstanceName(idx)));
+        return startGrid(cfg);
     }
 
     /**
-     * Simple blocking indexing processor.
+     * Simple blocking indexing SPI.
      */
-    private static class BlockingIndexing extends IgniteH2Indexing {
-        /** */
-        @Override public void remove(GridCacheContext cctx, GridQueryTypeDescriptor type, CacheDataRow row)
-            throws IgniteCheckedException {
-            U.sleep(50);
+    private static class BlockingIndexingSpi extends IgniteSpiAdapter implements IndexingSpi {
+        /** {@inheritDoc} */
+        @Override public void spiStart(@Nullable String igniteInstanceName) throws IgniteSpiException {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
+        @Override public void spiStop() throws IgniteSpiException {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
+        @Override public Iterator<Cache.Entry<?, ?>> query(
+            @Nullable String cacheName,
+            Collection<Object> params,
+            @Nullable IndexingQueryFilter filters
+        ) throws IgniteSpiException {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void store(
+            @Nullable String cacheName,
+            Object key,
+            Object val,
+            long expirationTime
+        ) throws IgniteSpiException {
+            // No-op.
+        }
 
-            super.remove(cctx, type, row);
+        /** {@inheritDoc} */
+        @Override public void remove(@Nullable String cacheName, Object key) throws IgniteSpiException {
+            doSleep(50L);
         }
     }
 }
diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/CalciteBasicSecondaryIndexIntegrationTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/CalciteBasicSecondaryIndexIntegrationTest.java
index 5e0e948baad..9987405a5c7 100644
--- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/CalciteBasicSecondaryIndexIntegrationTest.java
+++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/CalciteBasicSecondaryIndexIntegrationTest.java
@@ -23,10 +23,9 @@ import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.cache.QueryEntity;
 import org.apache.ignite.cache.QueryIndex;
 import org.apache.ignite.cache.QueryIndexType;
-import org.apache.ignite.cache.query.SqlFieldsQuery;
 import org.apache.ignite.cache.query.annotations.QuerySqlField;
 import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.processors.query.QueryUtils;
 import org.apache.ignite.internal.processors.query.calcite.CalciteQueryProcessorTest;
 import org.apache.ignite.internal.util.typedef.F;
 import org.junit.Ignore;
@@ -40,15 +39,19 @@ import static org.apache.ignite.internal.processors.query.calcite.QueryChecker.c
 import static org.apache.ignite.internal.processors.query.calcite.QueryChecker.containsSubPlan;
 import static org.apache.ignite.internal.processors.query.calcite.QueryChecker.containsTableScan;
 import static org.apache.ignite.internal.processors.query.calcite.QueryChecker.containsUnion;
-import static org.apache.ignite.internal.processors.query.h2.H2TableDescriptor.AFFINITY_KEY_IDX_NAME;
-import static org.apache.ignite.internal.processors.query.h2.H2TableDescriptor.PK_IDX_NAME;
-import static org.apache.ignite.internal.processors.query.h2.opt.GridH2Table.generateProxyIdxName;
+import static org.apache.ignite.internal.processors.query.schema.management.SchemaManager.generateProxyIdxName;
 import static org.hamcrest.CoreMatchers.not;
 
 /**
  * Basic index tests.
  */
 public class CalciteBasicSecondaryIndexIntegrationTest extends AbstractBasicIntegrationTest {
+    /** */
+    private static final String PK_IDX_NAME = QueryUtils.PRIMARY_KEY_INDEX;
+
+    /** */
+    private static final String AFFINITY_KEY_IDX_NAME = QueryUtils.AFFINITY_KEY_INDEX;
+
     /** */
     private static final String DEPID_IDX = "DEPID_IDX";
 
@@ -188,19 +191,17 @@ public class CalciteBasicSecondaryIndexIntegrationTest extends AbstractBasicInte
         tblConstrPk.put(1, new CalciteQueryProcessorTest.Developer("Petr", 10));
         tblConstrPk.put(2, new CalciteQueryProcessorTest.Developer("Ivan", 11));
 
-        GridQueryProcessor qryProc = client.context().query();
-
-        qryProc.querySqlFields(new SqlFieldsQuery("CREATE TABLE PUBLIC.UNWRAP_PK" + " (F1 VARCHAR, F2 LONG, F3 LONG, F4 LONG, " +
-            "CONSTRAINT PK PRIMARY KEY (F2, F1)) WITH \"backups=0, affinity_key=F1\""), true).getAll();
+        executeSql("CREATE TABLE PUBLIC.UNWRAP_PK" + " (F1 VARCHAR, F2 BIGINT, F3 BIGINT, F4 BIGINT, " +
+            "CONSTRAINT PK PRIMARY KEY (F2, F1)) WITH \"backups=0, affinity_key=F1\"");
 
-        qryProc.querySqlFields(new SqlFieldsQuery("INSERT INTO PUBLIC.UNWRAP_PK(F1, F2, F3, F4) values ('Petr', 1, 2, 3)"), true);
-        qryProc.querySqlFields(new SqlFieldsQuery("INSERT INTO PUBLIC.UNWRAP_PK(F1, F2, F3, F4) values ('Ivan', 2, 2, 4)"), true);
+        executeSql("INSERT INTO PUBLIC.UNWRAP_PK(F1, F2, F3, F4) values ('Petr', 1, 2, 3)");
+        executeSql("INSERT INTO PUBLIC.UNWRAP_PK(F1, F2, F3, F4) values ('Ivan', 2, 2, 4)");
 
-        qryProc.querySqlFields(new SqlFieldsQuery("INSERT INTO PUBLIC.UNWRAP_PK(F1, F2, F3, F4) values ('Ivan1', 21, 2, 4)"), true);
-        qryProc.querySqlFields(new SqlFieldsQuery("INSERT INTO PUBLIC.UNWRAP_PK(F1, F2, F3, F4) values ('Ivan2', 22, 2, 4)"), true);
-        qryProc.querySqlFields(new SqlFieldsQuery("INSERT INTO PUBLIC.UNWRAP_PK(F1, F2, F3, F4) values ('Ivan3', 23, 2, 4)"), true);
-        qryProc.querySqlFields(new SqlFieldsQuery("INSERT INTO PUBLIC.UNWRAP_PK(F1, F2, F3, F4) values ('Ivan4', 24, 2, 4)"), true);
-        qryProc.querySqlFields(new SqlFieldsQuery("INSERT INTO PUBLIC.UNWRAP_PK(F1, F2, F3, F4) values ('Ivan5', 25, 2, 4)"), true);
+        executeSql("INSERT INTO PUBLIC.UNWRAP_PK(F1, F2, F3, F4) values ('Ivan1', 21, 2, 4)");
+        executeSql("INSERT INTO PUBLIC.UNWRAP_PK(F1, F2, F3, F4) values ('Ivan2', 22, 2, 4)");
+        executeSql("INSERT INTO PUBLIC.UNWRAP_PK(F1, F2, F3, F4) values ('Ivan3', 23, 2, 4)");
+        executeSql("INSERT INTO PUBLIC.UNWRAP_PK(F1, F2, F3, F4) values ('Ivan4', 24, 2, 4)");
+        executeSql("INSERT INTO PUBLIC.UNWRAP_PK(F1, F2, F3, F4) values ('Ivan5', 25, 2, 4)");
 
         awaitPartitionMapExchange();
     }
@@ -228,7 +229,7 @@ public class CalciteBasicSecondaryIndexIntegrationTest extends AbstractBasicInte
     @Test
     public void testEqualsFilterWithUnwrpKey() {
         assertQuery("SELECT F1 FROM UNWRAP_PK WHERE F2=2")
-            .matches(containsIndexScan("PUBLIC", "UNWRAP_PK", PK_IDX_NAME))
+            .matches(containsIndexScan("PUBLIC", "UNWRAP_PK", QueryUtils.PRIMARY_KEY_INDEX))
             .returns("Ivan")
             .check();
     }
@@ -237,7 +238,7 @@ public class CalciteBasicSecondaryIndexIntegrationTest extends AbstractBasicInte
     @Test
     public void testEqualsFilterWithUnwrpKeyAndAff() {
         assertQuery("SELECT F2 FROM UNWRAP_PK WHERE F1='Ivan'")
-            .matches(containsIndexScan("PUBLIC", "UNWRAP_PK", AFFINITY_KEY_IDX_NAME))
+            .matches(containsIndexScan("PUBLIC", "UNWRAP_PK", QueryUtils.AFFINITY_KEY_INDEX))
             .check();
     }
 
diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/IndexRebuildIntegrationTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/IndexRebuildIntegrationTest.java
index 39f32840ec7..152f91e1156 100644
--- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/IndexRebuildIntegrationTest.java
+++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/IndexRebuildIntegrationTest.java
@@ -90,7 +90,6 @@ public class IndexRebuildIntegrationTest extends AbstractBasicIntegrationTest {
         executeSql("CREATE TABLE tbl (id INT PRIMARY KEY, val VARCHAR, val2 VARCHAR) WITH CACHE_NAME=\"test\"");
         executeSql("CREATE INDEX idx_id_val ON tbl (id DESC, val)");
         executeSql("CREATE INDEX idx_id_val2 ON tbl (id, val2 DESC)");
-        executeSql("CREATE INDEX idx_val ON tbl (val DESC)");
 
         for (int i = 0; i < 100; i++)
             executeSql("INSERT INTO tbl VALUES (?, ?, ?)", i, "val" + i, "val" + i);
@@ -190,16 +189,23 @@ public class IndexRebuildIntegrationTest extends AbstractBasicIntegrationTest {
         checkRebuildIndexQuery(grid(1), checker, checker);
 
         // Order by another collation.
-        sql = "SELECT * FROM tbl WHERE val BETWEEN 'val10' AND 'val15' ORDER BY val";
+        executeSql("CREATE INDEX idx_val ON tbl (val DESC)");
 
-        checker = assertQuery(initNode, sql)
-            .matches(QueryChecker.containsSubPlan("IgniteSort"))
-            .matches(QueryChecker.containsIndexScan("PUBLIC", "TBL", "IDX_VAL"))
-            .returns(10, "val10", "val10").returns(11, "val11", "val11").returns(12, "val12", "val12")
-            .returns(13, "val13", "val13").returns(14, "val14", "val14").returns(15, "val15", "val15")
-            .ordered();
+        try {
+            sql = "SELECT * FROM tbl WHERE val BETWEEN 'val10' AND 'val15' ORDER BY val";
 
-        checkRebuildIndexQuery(grid(1), checker, checker);
+            checker = assertQuery(initNode, sql)
+                .matches(QueryChecker.containsSubPlan("IgniteSort"))
+                .matches(QueryChecker.containsIndexScan("PUBLIC", "TBL", "IDX_VAL"))
+                .returns(10, "val10", "val10").returns(11, "val11", "val11").returns(12, "val12", "val12")
+                .returns(13, "val13", "val13").returns(14, "val14", "val14").returns(15, "val15", "val15")
+                .ordered();
+
+            checkRebuildIndexQuery(grid(1), checker, checker);
+        }
+        finally {
+            executeSql("DROP INDEX idx_val");
+        }
     }
 
     /**
diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/RunningQueriesIntegrationTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/RunningQueriesIntegrationTest.java
index 7df14b3f31d..b927ff6b3d7 100644
--- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/RunningQueriesIntegrationTest.java
+++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/RunningQueriesIntegrationTest.java
@@ -36,6 +36,8 @@ import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.calcite.CalciteQueryEngineConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.processors.query.IgniteSQLException;
@@ -85,6 +87,14 @@ public class RunningQueriesIntegrationTest extends AbstractBasicIntegrationTest
         srv = grid(0);
     }
 
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+        cfg.getSqlConfiguration().setQueryEnginesConfiguration(new CalciteQueryEngineConfiguration());
+
+        return cfg;
+    }
+
     /**
      * Execute query with a lot of JOINs to produce very long planning phase.
      * Cancel query on planning phase and check query registry is empty on the all nodes of the cluster.
@@ -317,17 +327,21 @@ public class RunningQueriesIntegrationTest extends AbstractBasicIntegrationTest
         String initiatorId = "initiator";
 
         GridTestUtils.runAsync(() -> client.cache("cache").query(new SqlFieldsQuery("SELECT * FROM t")
-            .setQueryInitiatorId(initiatorId)));
+            .setQueryInitiatorId(initiatorId)).getAll());
 
         try {
             SystemView<SqlQueryView> view = client.context().systemView().view(SQL_QRY_VIEW);
 
             assertTrue(GridTestUtils.waitForCondition(() -> !F.isEmpty(view), 1_000));
 
+            assertFalse(F.isEmpty(engine.runningQueries()));
+
             assertEquals(1, F.size(view.iterator(), v -> initiatorId.equals(v.initiatorId())));
         }
         finally {
             latch.countDown();
         }
+
+        assertTrue(GridTestUtils.waitForCondition(() -> F.isEmpty(engine.runningQueries()), PLANNER_TIMEOUT * 2));
     }
 }
diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/ServerStatisticsIntegrationTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/ServerStatisticsIntegrationTest.java
index 28d6d97a315..bc05ce087b8 100644
--- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/ServerStatisticsIntegrationTest.java
+++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/ServerStatisticsIntegrationTest.java
@@ -446,7 +446,7 @@ public class ServerStatisticsIntegrationTest extends AbstractBasicIntegrationTes
 
     /**
      * Drop statistics for specified key.
-     * 
+     *
      * @param key Statistics key to collect statistics for.
      */
     protected void dropStatistics(StatisticsKey key) {
diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/StatisticsCommandDdlIntegrationTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/StatisticsCommandDdlIntegrationTest.java
index d42ae8a7df4..5b6448f7c94 100644
--- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/StatisticsCommandDdlIntegrationTest.java
+++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/StatisticsCommandDdlIntegrationTest.java
@@ -23,10 +23,9 @@ import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.IgniteInterruptedCheckedException;
-import org.apache.ignite.internal.processors.query.GridQueryIndexing;
 import org.apache.ignite.internal.processors.query.IgniteSQLException;
 import org.apache.ignite.internal.processors.query.stat.ColumnStatistics;
-import org.apache.ignite.internal.processors.query.stat.IgniteStatisticsManagerImpl;
+import org.apache.ignite.internal.processors.query.stat.IgniteStatisticsManager;
 import org.apache.ignite.internal.processors.query.stat.ObjectStatistics;
 import org.apache.ignite.internal.processors.query.stat.ObjectStatisticsImpl;
 import org.apache.ignite.internal.processors.query.stat.StatisticsKey;
@@ -308,9 +307,8 @@ public class StatisticsCommandDdlIntegrationTest extends AbstractDdlIntegrationT
                 if (node.cluster().localNode().isClient())
                     continue;
 
-                GridQueryIndexing indexing = ((IgniteEx)node).context().query().getIndexing();
-
-                ObjectStatistics localStat = indexing.statsManager().getLocalStatistics(new StatisticsKey(schema, obj));
+                ObjectStatistics localStat = statisticsMgr((IgniteEx)node).getLocalStatistics(
+                    new StatisticsKey(schema, obj));
 
                 if (!(isNull == (localStat == null)))
                     return false;
@@ -335,11 +333,8 @@ public class StatisticsCommandDdlIntegrationTest extends AbstractDdlIntegrationT
                 if (node.cluster().localNode().isClient())
                     continue;
 
-                GridQueryIndexing indexing = ((IgniteEx)node).context().query().getIndexing();
-
-                ObjectStatisticsImpl localStat = (ObjectStatisticsImpl)indexing.statsManager().getLocalStatistics(
-                    new StatisticsKey(schema, obj)
-                );
+                ObjectStatisticsImpl localStat = (ObjectStatisticsImpl)statisticsMgr((IgniteEx)node)
+                    .getLocalStatistics(new StatisticsKey(schema, obj));
 
                 long sumVer = localStat.columnsStatistics().values().stream()
                     .mapToLong(ColumnStatistics::version)
@@ -369,14 +364,12 @@ public class StatisticsCommandDdlIntegrationTest extends AbstractDdlIntegrationT
     }
 
     /** */
-    private IgniteStatisticsManagerImpl statisticsMgr(int idx) {
+    private IgniteStatisticsManager statisticsMgr(int idx) {
         return statisticsMgr(grid(idx));
     }
 
     /** */
-    private IgniteStatisticsManagerImpl statisticsMgr(IgniteEx ign) {
-        GridQueryIndexing indexing = ign.context().query().getIndexing();
-
-        return (IgniteStatisticsManagerImpl)indexing.statsManager();
+    private IgniteStatisticsManager statisticsMgr(IgniteEx ign) {
+        return ign.context().query().statsManager();
     }
 }
diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinMetadataSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinMetadataSelfTest.java
index f6e2ef87df1..3851004fa50 100644
--- a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinMetadataSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinMetadataSelfTest.java
@@ -633,9 +633,9 @@ public class JdbcThinMetadataSelfTest extends JdbcThinAbstractSelfTest {
                 "pers.PERSON.ORGID.null"
             ));
 
-            Set<String> actualUserCols = new HashSet<>(expectedCols.size());
+            Set<String> actualUserCols = new TreeSet<>();
 
-            Set<String> actualSystemCols = new HashSet<>();
+            Set<String> actualSystemCols = new TreeSet<>();
 
             while (rs.next()) {
                 int precision = rs.getInt("COLUMN_SIZE");
@@ -660,484 +660,484 @@ public class JdbcThinMetadataSelfTest extends JdbcThinAbstractSelfTest {
             Assert.assertEquals(expectedCols, actualUserCols);
 
             expectedCols = new TreeSet<>(Arrays.asList(
-                "SYS.BASELINE_NODES.CONSISTENT_ID.null.2147483647",
-                "SYS.BASELINE_NODES.ONLINE.null.1",
-                "SYS.BASELINE_NODE_ATTRIBUTES.NODE_CONSISTENT_ID.null.2147483647",
-                "SYS.BASELINE_NODE_ATTRIBUTES.NAME.null.2147483647",
-                "SYS.BASELINE_NODE_ATTRIBUTES.VALUE.null.2147483647",
-                "SYS.CACHES.CACHE_GROUP_ID.null.10",
-                "SYS.CACHES.CACHE_GROUP_NAME.null.2147483647",
-                "SYS.CACHES.CACHE_ID.null.10",
-                "SYS.CACHES.CACHE_NAME.null.2147483647",
-                "SYS.CACHES.CACHE_TYPE.null.2147483647",
-                "SYS.CACHES.CACHE_MODE.null.2147483647",
-                "SYS.CACHES.ATOMICITY_MODE.null.2147483647",
-                "SYS.CACHES.IS_ONHEAP_CACHE_ENABLED.null.1",
-                "SYS.CACHES.IS_COPY_ON_READ.null.1",
-                "SYS.CACHES.IS_LOAD_PREVIOUS_VALUE.null.1",
-                "SYS.CACHES.IS_READ_FROM_BACKUP.null.1",
-                "SYS.CACHES.PARTITION_LOSS_POLICY.null.2147483647",
-                "SYS.CACHES.NODE_FILTER.null.2147483647",
-                "SYS.CACHES.TOPOLOGY_VALIDATOR.null.2147483647",
-                "SYS.CACHES.IS_EAGER_TTL.null.1",
-                "SYS.CACHES.WRITE_SYNCHRONIZATION_MODE.null.2147483647",
-                "SYS.CACHES.IS_INVALIDATE.null.1",
-                "SYS.CACHES.IS_EVENTS_DISABLED.null.1",
-                "SYS.CACHES.IS_STATISTICS_ENABLED.null.1",
-                "SYS.CACHES.IS_MANAGEMENT_ENABLED.null.1",
-                "SYS.CACHES.BACKUPS.null.10",
-                "SYS.CACHES.AFFINITY.null.2147483647",
-                "SYS.CACHES.AFFINITY_MAPPER.null.2147483647",
-                "SYS.CACHES.REBALANCE_MODE.null.2147483647",
-                "SYS.CACHES.REBALANCE_BATCH_SIZE.null.10",
-                "SYS.CACHES.REBALANCE_TIMEOUT.null.19",
-                "SYS.CACHES.REBALANCE_DELAY.null.19",
-                "SYS.CACHES.REBALANCE_THROTTLE.null.19",
-                "SYS.CACHES.REBALANCE_BATCHES_PREFETCH_COUNT.null.19",
-                "SYS.CACHES.REBALANCE_ORDER.null.10",
-                "SYS.CACHES.EVICTION_FILTER.null.2147483647",
-                "SYS.CACHES.EVICTION_POLICY_FACTORY.null.2147483647",
-                "SYS.CACHES.CONFLICT_RESOLVER.null.2147483647",
-                "SYS.CACHES.IS_NEAR_CACHE_ENABLED.null.1",
-                "SYS.CACHES.NEAR_CACHE_EVICTION_POLICY_FACTORY.null.2147483647",
-                "SYS.CACHES.NEAR_CACHE_START_SIZE.null.10",
-                "SYS.CACHES.DEFAULT_LOCK_TIMEOUT.null.19",
-                "SYS.CACHES.INTERCEPTOR.null.2147483647",
-                "SYS.CACHES.CACHE_STORE_FACTORY.null.2147483647",
-                "SYS.CACHES.IS_STORE_KEEP_BINARY.null.1",
-                "SYS.CACHES.IS_READ_THROUGH.null.1",
-                "SYS.CACHES.IS_WRITE_THROUGH.null.1",
-                "SYS.CACHES.IS_WRITE_BEHIND_ENABLED.null.1",
-                "SYS.CACHES.WRITE_BEHIND_COALESCING.null.1",
-                "SYS.CACHES.WRITE_BEHIND_FLUSH_SIZE.null.10",
-                "SYS.CACHES.WRITE_BEHIND_FLUSH_FREQUENCY.null.19",
-                "SYS.CACHES.WRITE_BEHIND_FLUSH_THREAD_COUNT.null.10",
-                "SYS.CACHES.WRITE_BEHIND_BATCH_SIZE.null.10",
-                "SYS.CACHES.MAX_CONCURRENT_ASYNC_OPERATIONS.null.10",
-                "SYS.CACHES.CACHE_LOADER_FACTORY.null.2147483647",
-                "SYS.CACHES.CACHE_WRITER_FACTORY.null.2147483647",
-                "SYS.CACHES.EXPIRY_POLICY_FACTORY.null.2147483647",
-                "SYS.CACHES.IS_SQL_ESCAPE_ALL.null.1",
-                "SYS.CACHES.IS_ENCRYPTION_ENABLED.null.1",
-                "SYS.CACHES.SQL_SCHEMA.null.2147483647",
-                "SYS.CACHES.SQL_INDEX_MAX_INLINE_SIZE.null.10",
-                "SYS.CACHES.IS_SQL_ONHEAP_CACHE_ENABLED.null.1",
-                "SYS.CACHES.SQL_ONHEAP_CACHE_MAX_SIZE.null.10",
-                "SYS.CACHES.QUERY_DETAIL_METRICS_SIZE.null.10",
-                "SYS.CACHES.QUERY_PARALLELISM.null.10",
-                "SYS.CACHES.MAX_QUERY_ITERATORS_COUNT.null.10",
-                "SYS.CACHES.DATA_REGION_NAME.null.2147483647",
-                "SYS.CACHE_GROUPS.CACHE_GROUP_ID.null.10",
-                "SYS.CACHE_GROUPS.CACHE_GROUP_NAME.null.2147483647",
-                "SYS.CACHE_GROUPS.IS_SHARED.null.1",
-                "SYS.CACHE_GROUPS.CACHE_COUNT.null.10",
-                "SYS.CACHE_GROUPS.CACHE_MODE.null.2147483647",
-                "SYS.CACHE_GROUPS.ATOMICITY_MODE.null.2147483647",
-                "SYS.CACHE_GROUPS.AFFINITY.null.2147483647",
-                "SYS.CACHE_GROUPS.PARTITIONS_COUNT.null.10",
-                "SYS.CACHE_GROUPS.NODE_FILTER.null.2147483647",
-                "SYS.CACHE_GROUPS.DATA_REGION_NAME.null.2147483647",
-                "SYS.CACHE_GROUPS.TOPOLOGY_VALIDATOR.null.2147483647",
-                "SYS.CACHE_GROUPS.PARTITION_LOSS_POLICY.null.2147483647",
-                "SYS.CACHE_GROUPS.REBALANCE_MODE.null.2147483647",
-                "SYS.CACHE_GROUPS.REBALANCE_DELAY.null.19",
-                "SYS.CACHE_GROUPS.REBALANCE_ORDER.null.10",
-                "SYS.CACHE_GROUPS.BACKUPS.null.10",
-                "SYS.INDEXES.CACHE_GROUP_ID.null.10",
-                "SYS.INDEXES.CACHE_GROUP_NAME.null.2147483647",
-                "SYS.INDEXES.CACHE_ID.null.10",
-                "SYS.INDEXES.CACHE_NAME.null.2147483647",
-                "SYS.INDEXES.SCHEMA_NAME.null.2147483647",
-                "SYS.INDEXES.TABLE_NAME.null.2147483647",
-                "SYS.INDEXES.INDEX_NAME.null.2147483647",
-                "SYS.INDEXES.INDEX_TYPE.null.2147483647",
-                "SYS.INDEXES.COLUMNS.null.2147483647",
-                "SYS.INDEXES.IS_PK.null.1",
-                "SYS.INDEXES.IS_UNIQUE.null.1",
-                "SYS.INDEXES.INLINE_SIZE.null.10",
-                "SYS.LOCAL_CACHE_GROUPS_IO.CACHE_GROUP_ID.null.10",
-                "SYS.LOCAL_CACHE_GROUPS_IO.CACHE_GROUP_NAME.null.2147483647",
-                "SYS.LOCAL_CACHE_GROUPS_IO.PHYSICAL_READS.null.19",
-                "SYS.LOCAL_CACHE_GROUPS_IO.LOGICAL_READS.null.19",
-                "SYS.SQL_QUERIES_HISTORY.SCHEMA_NAME.null.2147483647",
-                "SYS.SQL_QUERIES_HISTORY.SQL.null.2147483647",
-                "SYS.SQL_QUERIES_HISTORY.LOCAL.null.1",
-                "SYS.SQL_QUERIES_HISTORY.EXECUTIONS.null.19",
-                "SYS.SQL_QUERIES_HISTORY.FAILURES.null.19",
-                "SYS.SQL_QUERIES_HISTORY.DURATION_MIN.null.19",
-                "SYS.SQL_QUERIES_HISTORY.DURATION_MAX.null.19",
-                "SYS.SQL_QUERIES_HISTORY.LAST_START_TIME.null.26.6",
-                "SYS.SQL_QUERIES.QUERY_ID.null.2147483647",
-                "SYS.SQL_QUERIES.SQL.null.2147483647",
-                "SYS.SQL_QUERIES.SCHEMA_NAME.null.2147483647",
-                "SYS.SQL_QUERIES.LOCAL.null.1",
-                "SYS.SQL_QUERIES.START_TIME.null.26.6",
-                "SYS.SQL_QUERIES.DURATION.null.19",
-                "SYS.SQL_QUERIES.ORIGIN_NODE_ID.null.2147483647",
-                "SYS.SQL_QUERIES.INITIATOR_ID.null.2147483647",
-                "SYS.SQL_QUERIES.SUBJECT_ID.null.2147483647",
-                "SYS.SCAN_QUERIES.START_TIME.null.19",
-                "SYS.SCAN_QUERIES.TRANSFORMER.null.2147483647",
-                "SYS.SCAN_QUERIES.LOCAL.null.1",
-                "SYS.SCAN_QUERIES.QUERY_ID.null.19",
-                "SYS.SCAN_QUERIES.PARTITION.null.10",
-                "SYS.SCAN_QUERIES.CACHE_GROUP_ID.null.10",
-                "SYS.SCAN_QUERIES.CACHE_NAME.null.2147483647",
-                "SYS.SCAN_QUERIES.TOPOLOGY.null.2147483647",
-                "SYS.SCAN_QUERIES.CACHE_GROUP_NAME.null.2147483647",
-                "SYS.SCAN_QUERIES.TASK_NAME.null.2147483647",
-                "SYS.SCAN_QUERIES.DURATION.null.19",
-                "SYS.SCAN_QUERIES.KEEP_BINARY.null.1",
-                "SYS.SCAN_QUERIES.FILTER.null.2147483647",
-                "SYS.SCAN_QUERIES.SUBJECT_ID.null.2147483647",
-                "SYS.SCAN_QUERIES.CANCELED.null.1",
-                "SYS.SCAN_QUERIES.CACHE_ID.null.10",
-                "SYS.SCAN_QUERIES.PAGE_SIZE.null.10",
-                "SYS.SCAN_QUERIES.ORIGIN_NODE_ID.null.2147483647",
-                "SYS.NODES.NODE_ID.null.2147483647",
-                "SYS.NODES.CONSISTENT_ID.null.2147483647",
-                "SYS.NODES.VERSION.null.2147483647",
-                "SYS.NODES.IS_CLIENT.null.1",
-                "SYS.NODES.IS_DAEMON.null.1",
-                "SYS.NODES.IS_LOCAL.null.1",
-                "SYS.NODES.NODE_ORDER.null.19",
-                "SYS.NODES.ADDRESSES.null.2147483647",
-                "SYS.NODES.HOSTNAMES.null.2147483647",
-                "SYS.NODE_ATTRIBUTES.NODE_ID.null.2147483647",
-                "SYS.NODE_ATTRIBUTES.NAME.null.2147483647",
-                "SYS.NODE_ATTRIBUTES.VALUE.null.2147483647",
-                "SYS.NODE_METRICS.NODE_ID.null.2147483647",
-                "SYS.NODE_METRICS.LAST_UPDATE_TIME.null.26.6",
-                "SYS.NODE_METRICS.MAX_ACTIVE_JOBS.null.10",
-                "SYS.NODE_METRICS.CUR_ACTIVE_JOBS.null.10",
-                "SYS.NODE_METRICS.AVG_ACTIVE_JOBS.null.7",
-                "SYS.NODE_METRICS.MAX_WAITING_JOBS.null.10",
-                "SYS.NODE_METRICS.CUR_WAITING_JOBS.null.10",
-                "SYS.NODE_METRICS.AVG_WAITING_JOBS.null.7",
-                "SYS.NODE_METRICS.MAX_REJECTED_JOBS.null.10",
-                "SYS.NODE_METRICS.CUR_REJECTED_JOBS.null.10",
-                "SYS.NODE_METRICS.AVG_REJECTED_JOBS.null.7",
-                "SYS.NODE_METRICS.TOTAL_REJECTED_JOBS.null.10",
-                "SYS.NODE_METRICS.MAX_CANCELED_JOBS.null.10",
-                "SYS.NODE_METRICS.CUR_CANCELED_JOBS.null.10",
-                "SYS.NODE_METRICS.AVG_CANCELED_JOBS.null.7",
-                "SYS.NODE_METRICS.TOTAL_CANCELED_JOBS.null.10",
-                "SYS.NODE_METRICS.MAX_JOBS_WAIT_TIME.null.19",
-                "SYS.NODE_METRICS.CUR_JOBS_WAIT_TIME.null.19",
-                "SYS.NODE_METRICS.AVG_JOBS_WAIT_TIME.null.19",
-                "SYS.NODE_METRICS.MAX_JOBS_EXECUTE_TIME.null.19",
-                "SYS.NODE_METRICS.CUR_JOBS_EXECUTE_TIME.null.19",
-                "SYS.NODE_METRICS.AVG_JOBS_EXECUTE_TIME.null.19",
-                "SYS.NODE_METRICS.TOTAL_JOBS_EXECUTE_TIME.null.19",
-                "SYS.NODE_METRICS.TOTAL_EXECUTED_JOBS.null.10",
-                "SYS.NODE_METRICS.TOTAL_EXECUTED_TASKS.null.10",
-                "SYS.NODE_METRICS.TOTAL_BUSY_TIME.null.19",
-                "SYS.NODE_METRICS.TOTAL_IDLE_TIME.null.19",
-                "SYS.NODE_METRICS.CUR_IDLE_TIME.null.19",
-                "SYS.NODE_METRICS.BUSY_TIME_PERCENTAGE.null.7",
-                "SYS.NODE_METRICS.IDLE_TIME_PERCENTAGE.null.7",
-                "SYS.NODE_METRICS.TOTAL_CPU.null.10",
-                "SYS.NODE_METRICS.CUR_CPU_LOAD.null.17",
-                "SYS.NODE_METRICS.AVG_CPU_LOAD.null.17",
-                "SYS.NODE_METRICS.CUR_GC_CPU_LOAD.null.17",
-                "SYS.NODE_METRICS.HEAP_MEMORY_INIT.null.19",
-                "SYS.NODE_METRICS.HEAP_MEMORY_USED.null.19",
-                "SYS.NODE_METRICS.HEAP_MEMORY_COMMITED.null.19",
-                "SYS.NODE_METRICS.HEAP_MEMORY_MAX.null.19",
-                "SYS.NODE_METRICS.HEAP_MEMORY_TOTAL.null.19",
-                "SYS.NODE_METRICS.NONHEAP_MEMORY_INIT.null.19",
-                "SYS.NODE_METRICS.NONHEAP_MEMORY_USED.null.19",
-                "SYS.NODE_METRICS.NONHEAP_MEMORY_COMMITED.null.19",
-                "SYS.NODE_METRICS.NONHEAP_MEMORY_MAX.null.19",
-                "SYS.NODE_METRICS.NONHEAP_MEMORY_TOTAL.null.19",
-                "SYS.NODE_METRICS.UPTIME.null.19",
-                "SYS.NODE_METRICS.JVM_START_TIME.null.26.6",
-                "SYS.NODE_METRICS.NODE_START_TIME.null.26.6",
-                "SYS.NODE_METRICS.LAST_DATA_VERSION.null.19",
-                "SYS.NODE_METRICS.CUR_THREAD_COUNT.null.10",
-                "SYS.NODE_METRICS.MAX_THREAD_COUNT.null.10",
-                "SYS.NODE_METRICS.TOTAL_THREAD_COUNT.null.19",
-                "SYS.NODE_METRICS.CUR_DAEMON_THREAD_COUNT.null.10",
-                "SYS.NODE_METRICS.SENT_MESSAGES_COUNT.null.10",
-                "SYS.NODE_METRICS.SENT_BYTES_COUNT.null.19",
-                "SYS.NODE_METRICS.RECEIVED_MESSAGES_COUNT.null.10",
-                "SYS.NODE_METRICS.RECEIVED_BYTES_COUNT.null.19",
-                "SYS.NODE_METRICS.OUTBOUND_MESSAGES_QUEUE.null.10",
-                "SYS.TABLES.CACHE_GROUP_ID.null.10",
-                "SYS.TABLES.CACHE_GROUP_NAME.null.2147483647",
-                "SYS.TABLES.CACHE_ID.null.10",
-                "SYS.TABLES.CACHE_NAME.null.2147483647",
-                "SYS.TABLES.SCHEMA_NAME.null.2147483647",
-                "SYS.TABLES.TABLE_NAME.null.2147483647",
-                "SYS.TABLES.AFFINITY_KEY_COLUMN.null.2147483647",
-                "SYS.TABLES.KEY_ALIAS.null.2147483647",
-                "SYS.TABLES.VALUE_ALIAS.null.2147483647",
-                "SYS.TABLES.KEY_TYPE_NAME.null.2147483647",
-                "SYS.TABLES.VALUE_TYPE_NAME.null.2147483647",
-                "SYS.TABLES.IS_INDEX_REBUILD_IN_PROGRESS.null.1",
-                "SYS.METRICS.NAME.null.2147483647",
-                "SYS.METRICS.VALUE.null.2147483647",
-                "SYS.METRICS.DESCRIPTION.null.2147483647",
-                "SYS.SERVICES.SERVICE_ID.null.2147483647",
-                "SYS.SERVICES.NAME.null.2147483647",
-                "SYS.SERVICES.SERVICE_CLASS.null.2147483647",
-                "SYS.SERVICES.CACHE_NAME.null.2147483647",
-                "SYS.SERVICES.ORIGIN_NODE_ID.null.2147483647",
-                "SYS.SERVICES.TOTAL_COUNT.null.10",
-                "SYS.SERVICES.MAX_PER_NODE_COUNT.null.10",
-                "SYS.SERVICES.AFFINITY_KEY.null.2147483647",
-                "SYS.SERVICES.NODE_FILTER.null.2147483647",
-                "SYS.SERVICES.STATICALLY_CONFIGURED.null.1",
-                "SYS.SERVICES.SERVICE_ID.null.2147483647",
-                "SYS.TASKS.AFFINITY_CACHE_NAME.null.2147483647",
-                "SYS.TASKS.INTERNAL.null.1",
-                "SYS.TASKS.END_TIME.null.19",
-                "SYS.TASKS.START_TIME.null.19",
-                "SYS.TASKS.USER_VERSION.null.2147483647",
-                "SYS.TASKS.TASK_NAME.null.2147483647",
-                "SYS.TASKS.TASK_NODE_ID.null.2147483647",
-                "SYS.TASKS.JOB_ID.null.2147483647",
-                "SYS.TASKS.ID.null.2147483647",
-                "SYS.TASKS.SESSION_ID.null.2147483647",
-                "SYS.TASKS.AFFINITY_PARTITION_ID.null.10",
-                "SYS.TASKS.TASK_CLASS_NAME.null.2147483647",
-                "SYS.JOBS.IS_STARTED.null.1",
-                "SYS.JOBS.EXECUTOR_NAME.null.2147483647",
-                "SYS.JOBS.IS_TIMED_OUT.null.1",
-                "SYS.JOBS.ID.null.2147483647",
-                "SYS.JOBS.FINISH_TIME.null.19",
-                "SYS.JOBS.IS_INTERNAL.null.1",
-                "SYS.JOBS.CREATE_TIME.null.19",
-                "SYS.JOBS.AFFINITY_PARTITION_ID.null.10",
-                "SYS.JOBS.ORIGIN_NODE_ID.null.2147483647",
-                "SYS.JOBS.TASK_NAME.null.2147483647",
-                "SYS.JOBS.TASK_CLASS_NAME.null.2147483647",
-                "SYS.JOBS.SESSION_ID.null.2147483647",
-                "SYS.JOBS.IS_FINISHING.null.1",
-                "SYS.JOBS.START_TIME.null.19",
-                "SYS.JOBS.AFFINITY_CACHE_IDS.null.2147483647",
-                "SYS.JOBS.STATE.null.2147483647",
-                "SYS.CLIENT_CONNECTIONS.CONNECTION_ID.null.19",
-                "SYS.CLIENT_CONNECTIONS.LOCAL_ADDRESS.null.2147483647",
-                "SYS.CLIENT_CONNECTIONS.REMOTE_ADDRESS.null.2147483647",
-                "SYS.CLIENT_CONNECTIONS.TYPE.null.2147483647",
-                "SYS.CLIENT_CONNECTIONS.USER.null.2147483647",
-                "SYS.CLIENT_CONNECTIONS.VERSION.null.2147483647",
-                "SYS.TASKS.EXEC_NAME.null.2147483647",
-                "SYS.TRANSACTIONS.LOCAL_NODE_ID.null.2147483647",
-                "SYS.TRANSACTIONS.STATE.null.2147483647",
-                "SYS.TRANSACTIONS.XID.null.2147483647",
-                "SYS.TRANSACTIONS.LABEL.null.2147483647",
-                "SYS.TRANSACTIONS.START_TIME.null.19",
-                "SYS.TRANSACTIONS.ISOLATION.null.2147483647",
-                "SYS.TRANSACTIONS.CONCURRENCY.null.2147483647",
-                "SYS.TRANSACTIONS.COLOCATED.null.1",
-                "SYS.TRANSACTIONS.DHT.null.1",
-                "SYS.TRANSACTIONS.IMPLICIT.null.1",
-                "SYS.TRANSACTIONS.IMPLICIT_SINGLE.null.1",
-                "SYS.TRANSACTIONS.INTERNAL.null.1",
-                "SYS.TRANSACTIONS.LOCAL.null.1",
-                "SYS.TRANSACTIONS.NEAR.null.1",
-                "SYS.TRANSACTIONS.ONE_PHASE_COMMIT.null.1",
-                "SYS.TRANSACTIONS.SUBJECT_ID.null.2147483647",
-                "SYS.TRANSACTIONS.SYSTEM.null.1",
-                "SYS.TRANSACTIONS.THREAD_ID.null.19",
-                "SYS.TRANSACTIONS.TIMEOUT.null.19",
-                "SYS.TRANSACTIONS.DURATION.null.19",
-                "SYS.TRANSACTIONS.ORIGINATING_NODE_ID.null.2147483647",
-                "SYS.TRANSACTIONS.OTHER_NODE_ID.null.2147483647",
-                "SYS.TRANSACTIONS.TOP_VER.null.2147483647",
-                "SYS.TRANSACTIONS.KEYS_COUNT.null.10",
-                "SYS.TRANSACTIONS.CACHE_IDS.null.2147483647",
-                "SYS.SCHEMAS.SCHEMA_NAME.null.2147483647",
-                "SYS.SCHEMAS.PREDEFINED.null.1",
-                "SYS.VIEWS.NAME.null.2147483647",
-                "SYS.VIEWS.DESCRIPTION.null.2147483647",
-                "SYS.VIEWS.SCHEMA.null.2147483647",
-                "SYS.TABLE_COLUMNS.AFFINITY_COLUMN.null.1",
-                "SYS.TABLE_COLUMNS.COLUMN_NAME.null.2147483647",
-                "SYS.TABLE_COLUMNS.SCALE.null.10",
-                "SYS.TABLE_COLUMNS.PK.null.1",
-                "SYS.TABLE_COLUMNS.TYPE.null.2147483647",
-                "SYS.TABLE_COLUMNS.DEFAULT_VALUE.null.2147483647",
-                "SYS.TABLE_COLUMNS.SCHEMA_NAME.null.2147483647",
-                "SYS.TABLE_COLUMNS.TABLE_NAME.null.2147483647",
-                "SYS.TABLE_COLUMNS.NULLABLE.null.1",
-                "SYS.TABLE_COLUMNS.PRECISION.null.10",
-                "SYS.TABLE_COLUMNS.AUTO_INCREMENT.null.1",
-                "SYS.VIEW_COLUMNS.NULLABLE.null.1",
-                "SYS.VIEW_COLUMNS.SCHEMA_NAME.null.2147483647",
-                "SYS.VIEW_COLUMNS.COLUMN_NAME.null.2147483647",
-                "SYS.VIEW_COLUMNS.TYPE.null.2147483647",
-                "SYS.VIEW_COLUMNS.PRECISION.null.19",
-                "SYS.VIEW_COLUMNS.DEFAULT_VALUE.null.2147483647",
-                "SYS.VIEW_COLUMNS.SCALE.null.10",
-                "SYS.VIEW_COLUMNS.VIEW_NAME.null.2147483647",
-                "SYS.CONTINUOUS_QUERIES.NOTIFY_EXISTING.null.1",
-                "SYS.CONTINUOUS_QUERIES.OLD_VALUE_REQUIRED.null.1",
-                "SYS.CONTINUOUS_QUERIES.KEEP_BINARY.null.1",
-                "SYS.CONTINUOUS_QUERIES.IS_MESSAGING.null.1",
-                "SYS.CONTINUOUS_QUERIES.AUTO_UNSUBSCRIBE.null.1",
-                "SYS.CONTINUOUS_QUERIES.LAST_SEND_TIME.null.19",
-                "SYS.CONTINUOUS_QUERIES.LOCAL_TRANSFORMED_LISTENER.null.2147483647",
-                "SYS.CONTINUOUS_QUERIES.TOPIC.null.2147483647",
-                "SYS.CONTINUOUS_QUERIES.BUFFER_SIZE.null.10",
-                "SYS.CONTINUOUS_QUERIES.REMOTE_TRANSFORMER.null.2147483647",
-                "SYS.CONTINUOUS_QUERIES.DELAYED_REGISTER.null.1",
-                "SYS.CONTINUOUS_QUERIES.IS_QUERY.null.1",
-                "SYS.CONTINUOUS_QUERIES.NODE_ID.null.2147483647",
-                "SYS.CONTINUOUS_QUERIES.INTERVAL.null.19",
-                "SYS.CONTINUOUS_QUERIES.IS_EVENTS.null.1",
-                "SYS.CONTINUOUS_QUERIES.ROUTINE_ID.null.2147483647",
-                "SYS.CONTINUOUS_QUERIES.REMOTE_FILTER.null.2147483647",
-                "SYS.CONTINUOUS_QUERIES.CACHE_NAME.null.2147483647",
-                "SYS.CONTINUOUS_QUERIES.LOCAL_LISTENER.null.2147483647",
-                "SYS.STRIPED_THREADPOOL_QUEUE.STRIPE_INDEX.null.10",
-                "SYS.STRIPED_THREADPOOL_QUEUE.DESCRIPTION.null.2147483647",
-                "SYS.STRIPED_THREADPOOL_QUEUE.THREAD_NAME.null.2147483647",
-                "SYS.STRIPED_THREADPOOL_QUEUE.TASK_NAME.null.2147483647",
-                "SYS.DATASTREAM_THREADPOOL_QUEUE.STRIPE_INDEX.null.10",
-                "SYS.DATASTREAM_THREADPOOL_QUEUE.DESCRIPTION.null.2147483647",
-                "SYS.DATASTREAM_THREADPOOL_QUEUE.THREAD_NAME.null.2147483647",
-                "SYS.DATASTREAM_THREADPOOL_QUEUE.TASK_NAME.null.2147483647",
-                "SYS.CACHE_GROUP_PAGE_LISTS.CACHE_GROUP_ID.null.10",
-                "SYS.CACHE_GROUP_PAGE_LISTS.PARTITION_ID.null.10",
-                "SYS.CACHE_GROUP_PAGE_LISTS.NAME.null.2147483647",
-                "SYS.CACHE_GROUP_PAGE_LISTS.BUCKET_NUMBER.null.10",
-                "SYS.CACHE_GROUP_PAGE_LISTS.BUCKET_SIZE.null.19",
-                "SYS.CACHE_GROUP_PAGE_LISTS.STRIPES_COUNT.null.10",
-                "SYS.CACHE_GROUP_PAGE_LISTS.CACHED_PAGES_COUNT.null.10",
-                "SYS.DATA_REGION_PAGE_LISTS.NAME.null.2147483647",
-                "SYS.DATA_REGION_PAGE_LISTS.BUCKET_NUMBER.null.10",
-                "SYS.DATA_REGION_PAGE_LISTS.BUCKET_SIZE.null.19",
-                "SYS.DATA_REGION_PAGE_LISTS.STRIPES_COUNT.null.10",
-                "SYS.DATA_REGION_PAGE_LISTS.CACHED_PAGES_COUNT.null.10",
-                "SYS.PARTITION_STATES.CACHE_GROUP_ID.null.10",
-                "SYS.PARTITION_STATES.PARTITION_ID.null.10",
-                "SYS.PARTITION_STATES.NODE_ID.null.2147483647",
-                "SYS.PARTITION_STATES.STATE.null.2147483647",
-                "SYS.PARTITION_STATES.IS_PRIMARY.null.1",
-                "SYS.BINARY_METADATA.FIELDS.null.2147483647",
-                "SYS.BINARY_METADATA.AFF_KEY_FIELD_NAME.null.2147483647",
-                "SYS.BINARY_METADATA.SCHEMAS_IDS.null.2147483647",
-                "SYS.BINARY_METADATA.TYPE_ID.null.10",
-                "SYS.BINARY_METADATA.IS_ENUM.null.1",
-                "SYS.BINARY_METADATA.FIELDS_COUNT.null.10",
-                "SYS.BINARY_METADATA.TYPE_NAME.null.2147483647",
-                "SYS.DISTRIBUTED_METASTORAGE.NAME.null.2147483647",
-                "SYS.DISTRIBUTED_METASTORAGE.VALUE.null.2147483647",
-                "SYS.DS_ATOMICLONGS.GROUP_ID.null.10",
-                "SYS.DS_ATOMICLONGS.GROUP_NAME.null.2147483647",
-                "SYS.DS_ATOMICLONGS.NAME.null.2147483647",
-                "SYS.DS_ATOMICLONGS.REMOVED.null.1",
-                "SYS.DS_ATOMICLONGS.VALUE.null.19",
-                "SYS.DS_ATOMICREFERENCES.GROUP_ID.null.10",
-                "SYS.DS_ATOMICREFERENCES.GROUP_NAME.null.2147483647",
-                "SYS.DS_ATOMICREFERENCES.NAME.null.2147483647",
-                "SYS.DS_ATOMICREFERENCES.REMOVED.null.1",
-                "SYS.DS_ATOMICREFERENCES.VALUE.null.2147483647",
-                "SYS.DS_ATOMICSEQUENCES.BATCH_SIZE.null.19",
-                "SYS.DS_ATOMICSEQUENCES.GROUP_ID.null.10",
-                "SYS.DS_ATOMICSEQUENCES.GROUP_NAME.null.2147483647",
-                "SYS.DS_ATOMICSEQUENCES.NAME.null.2147483647",
-                "SYS.DS_ATOMICSEQUENCES.REMOVED.null.1",
-                "SYS.DS_ATOMICSEQUENCES.VALUE.null.19",
-                "SYS.DS_ATOMICSTAMPED.GROUP_ID.null.10",
-                "SYS.DS_ATOMICSTAMPED.GROUP_NAME.null.2147483647",
-                "SYS.DS_ATOMICSTAMPED.NAME.null.2147483647",
-                "SYS.DS_ATOMICSTAMPED.REMOVED.null.1",
-                "SYS.DS_ATOMICSTAMPED.STAMP.null.2147483647",
-                "SYS.DS_ATOMICSTAMPED.VALUE.null.2147483647",
-                "SYS.DS_COUNTDOWNLATCHES.AUTO_DELETE.null.1",
-                "SYS.DS_COUNTDOWNLATCHES.COUNT.null.10",
-                "SYS.DS_COUNTDOWNLATCHES.GROUP_ID.null.10",
-                "SYS.DS_COUNTDOWNLATCHES.GROUP_NAME.null.2147483647",
-                "SYS.DS_COUNTDOWNLATCHES.INITIAL_COUNT.null.10",
-                "SYS.DS_COUNTDOWNLATCHES.NAME.null.2147483647",
-                "SYS.DS_COUNTDOWNLATCHES.REMOVED.null.1",
-                "SYS.DS_QUEUES.BOUNDED.null.1",
-                "SYS.DS_QUEUES.CAPACITY.null.10",
-                "SYS.DS_QUEUES.SIZE.null.10",
-                "SYS.DS_QUEUES.COLLOCATED.null.1",
-                "SYS.DS_QUEUES.GROUP_ID.null.10",
-                "SYS.DS_QUEUES.GROUP_NAME.null.2147483647",
-                "SYS.DS_QUEUES.ID.null.2147483647",
-                "SYS.DS_QUEUES.NAME.null.2147483647",
-                "SYS.DS_QUEUES.REMOVED.null.1",
-                "SYS.DS_REENTRANTLOCKS.BROKEN.null.1",
-                "SYS.DS_REENTRANTLOCKS.FAILOVER_SAFE.null.1",
-                "SYS.DS_REENTRANTLOCKS.FAIR.null.1",
-                "SYS.DS_REENTRANTLOCKS.GROUP_ID.null.10",
-                "SYS.DS_REENTRANTLOCKS.GROUP_NAME.null.2147483647",
-                "SYS.DS_REENTRANTLOCKS.HAS_QUEUED_THREADS.null.1",
-                "SYS.DS_REENTRANTLOCKS.LOCKED.null.1",
-                "SYS.DS_REENTRANTLOCKS.NAME.null.2147483647",
-                "SYS.DS_REENTRANTLOCKS.REMOVED.null.1",
-                "SYS.DS_SEMAPHORES.AVAILABLE_PERMITS.null.19",
-                "SYS.DS_SEMAPHORES.BROKEN.null.1",
-                "SYS.DS_SEMAPHORES.FAILOVER_SAFE.null.1",
-                "SYS.DS_SEMAPHORES.GROUP_ID.null.10",
-                "SYS.DS_SEMAPHORES.GROUP_NAME.null.2147483647",
-                "SYS.DS_SEMAPHORES.HAS_QUEUED_THREADS.null.1",
-                "SYS.DS_SEMAPHORES.NAME.null.2147483647",
-                "SYS.DS_SEMAPHORES.QUEUE_LENGTH.null.10",
-                "SYS.DS_SEMAPHORES.REMOVED.null.1",
-                "SYS.DS_SETS.COLLOCATED.null.1",
-                "SYS.DS_SETS.GROUP_ID.null.10",
-                "SYS.DS_SETS.GROUP_NAME.null.2147483647",
-                "SYS.DS_SETS.ID.null.2147483647",
-                "SYS.DS_SETS.NAME.null.2147483647",
-                "SYS.DS_SETS.REMOVED.null.1",
-                "SYS.DS_SETS.SIZE.null.10",
-                "SYS.STATISTICS_LOCAL_DATA.LAST_UPDATE_TIME.null.2147483647",
-                "SYS.STATISTICS_LOCAL_DATA.NAME.null.2147483647",
-                "SYS.STATISTICS_LOCAL_DATA.TOTAL.null.19",
-                "SYS.STATISTICS_PARTITION_DATA.VERSION.null.19",
-                "SYS.STATISTICS_CONFIGURATION.TYPE.null.2147483647",
-                "SYS.STATISTICS_PARTITION_DATA.NAME.null.2147483647",
-                "SYS.STATISTICS_CONFIGURATION.COLUMN.null.2147483647",
-                "SYS.STATISTICS_LOCAL_DATA.ROWS_COUNT.null.19",
-                "SYS.STATISTICS_PARTITION_DATA.TYPE.null.2147483647",
-                "SYS.STATISTICS_LOCAL_DATA.DISTINCT.null.19",
-                "SYS.STATISTICS_LOCAL_DATA.SIZE.null.10",
-                "SYS.STATISTICS_PARTITION_DATA.LAST_UPDATE_TIME.null.19",
-                "SYS.STATISTICS_CONFIGURATION.MAX_PARTITION_OBSOLESCENCE_PERCENT.null.3",
-                "SYS.STATISTICS_LOCAL_DATA.VERSION.null.19",
-                "SYS.STATISTICS_LOCAL_DATA.COLUMN.null.2147483647",
-                "SYS.STATISTICS_CONFIGURATION.SCHEMA.null.2147483647",
-                "SYS.STATISTICS_PARTITION_DATA.TOTAL.null.19",
-                "SYS.STATISTICS_PARTITION_DATA.PARTITION.null.10",
-                "SYS.STATISTICS_PARTITION_DATA.SCHEMA.null.2147483647",
-                "SYS.STATISTICS_PARTITION_DATA.ROWS_COUNT.null.19",
-                "SYS.STATISTICS_PARTITION_DATA.SIZE.null.10",
-                "SYS.STATISTICS_PARTITION_DATA.UPDATE_COUNTER.null.19",
-                "SYS.STATISTICS_CONFIGURATION.NAME.null.2147483647",
-                "SYS.STATISTICS_PARTITION_DATA.DISTINCT.null.19",
-                "SYS.STATISTICS_LOCAL_DATA.NULLS.null.19",
-                "SYS.STATISTICS_CONFIGURATION.VERSION.null.19",
-                "SYS.STATISTICS_CONFIGURATION.MANUAL_SIZE.null.10",
-                "SYS.STATISTICS_CONFIGURATION.MANUAL_DISTINCT.null.19",
-                "SYS.STATISTICS_CONFIGURATION.MANUAL_NULLS.null.19",
-                "SYS.STATISTICS_CONFIGURATION.MANUAL_TOTAL.null.19",
-                "SYS.STATISTICS_LOCAL_DATA.TYPE.null.2147483647",
-                "SYS.STATISTICS_PARTITION_DATA.NULLS.null.19",
-                "SYS.STATISTICS_PARTITION_DATA.COLUMN.null.2147483647",
-                "SYS.STATISTICS_LOCAL_DATA.SCHEMA.null.2147483647",
-                "SYS.STATISTICS_GLOBAL_DATA.SCHEMA.null.2147483647",
-                "SYS.STATISTICS_GLOBAL_DATA.TYPE.null.2147483647",
-                "SYS.STATISTICS_GLOBAL_DATA.NAME.null.2147483647",
-                "SYS.STATISTICS_GLOBAL_DATA.COLUMN.null.2147483647",
-                "SYS.STATISTICS_GLOBAL_DATA.ROWS_COUNT.null.19",
-                "SYS.STATISTICS_GLOBAL_DATA.DISTINCT.null.19",
-                "SYS.STATISTICS_GLOBAL_DATA.NULLS.null.19",
-                "SYS.STATISTICS_GLOBAL_DATA.TOTAL.null.19",
-                "SYS.STATISTICS_GLOBAL_DATA.SIZE.null.10",
-                "SYS.STATISTICS_GLOBAL_DATA.VERSION.null.19",
-                "SYS.STATISTICS_GLOBAL_DATA.LAST_UPDATE_TIME.null.2147483647",
-                "SYS.PAGES_TIMESTAMP_HISTOGRAM.DATA_REGION_NAME.null.2147483647",
-                "SYS.PAGES_TIMESTAMP_HISTOGRAM.INTERVAL_START.null.26.6",
-                "SYS.PAGES_TIMESTAMP_HISTOGRAM.INTERVAL_END.null.26.6",
-                "SYS.PAGES_TIMESTAMP_HISTOGRAM.PAGES_COUNT.null.19"
+                "SYS.BASELINE_NODES.CONSISTENT_ID.null",
+                "SYS.BASELINE_NODES.ONLINE.null",
+                "SYS.BASELINE_NODE_ATTRIBUTES.NODE_CONSISTENT_ID.null",
+                "SYS.BASELINE_NODE_ATTRIBUTES.NAME.null",
+                "SYS.BASELINE_NODE_ATTRIBUTES.VALUE.null",
+                "SYS.CACHES.CACHE_GROUP_ID.null",
+                "SYS.CACHES.CACHE_GROUP_NAME.null",
+                "SYS.CACHES.CACHE_ID.null",
+                "SYS.CACHES.CACHE_NAME.null",
+                "SYS.CACHES.CACHE_TYPE.null",
+                "SYS.CACHES.CACHE_MODE.null",
+                "SYS.CACHES.ATOMICITY_MODE.null",
+                "SYS.CACHES.IS_ONHEAP_CACHE_ENABLED.null",
+                "SYS.CACHES.IS_COPY_ON_READ.null",
+                "SYS.CACHES.IS_LOAD_PREVIOUS_VALUE.null",
+                "SYS.CACHES.IS_READ_FROM_BACKUP.null",
+                "SYS.CACHES.PARTITION_LOSS_POLICY.null",
+                "SYS.CACHES.NODE_FILTER.null",
+                "SYS.CACHES.TOPOLOGY_VALIDATOR.null",
+                "SYS.CACHES.IS_EAGER_TTL.null",
+                "SYS.CACHES.WRITE_SYNCHRONIZATION_MODE.null",
+                "SYS.CACHES.IS_INVALIDATE.null",
+                "SYS.CACHES.IS_EVENTS_DISABLED.null",
+                "SYS.CACHES.IS_STATISTICS_ENABLED.null",
+                "SYS.CACHES.IS_MANAGEMENT_ENABLED.null",
+                "SYS.CACHES.BACKUPS.null",
+                "SYS.CACHES.AFFINITY.null",
+                "SYS.CACHES.AFFINITY_MAPPER.null",
+                "SYS.CACHES.REBALANCE_MODE.null",
+                "SYS.CACHES.REBALANCE_BATCH_SIZE.null",
+                "SYS.CACHES.REBALANCE_TIMEOUT.null",
+                "SYS.CACHES.REBALANCE_DELAY.null",
+                "SYS.CACHES.REBALANCE_THROTTLE.null",
+                "SYS.CACHES.REBALANCE_BATCHES_PREFETCH_COUNT.null",
+                "SYS.CACHES.REBALANCE_ORDER.null",
+                "SYS.CACHES.EVICTION_FILTER.null",
+                "SYS.CACHES.EVICTION_POLICY_FACTORY.null",
+                "SYS.CACHES.CONFLICT_RESOLVER.null",
+                "SYS.CACHES.IS_NEAR_CACHE_ENABLED.null",
+                "SYS.CACHES.NEAR_CACHE_EVICTION_POLICY_FACTORY.null",
+                "SYS.CACHES.NEAR_CACHE_START_SIZE.null",
+                "SYS.CACHES.DEFAULT_LOCK_TIMEOUT.null",
+                "SYS.CACHES.INTERCEPTOR.null",
+                "SYS.CACHES.CACHE_STORE_FACTORY.null",
+                "SYS.CACHES.IS_STORE_KEEP_BINARY.null",
+                "SYS.CACHES.IS_READ_THROUGH.null",
+                "SYS.CACHES.IS_WRITE_THROUGH.null",
+                "SYS.CACHES.IS_WRITE_BEHIND_ENABLED.null",
+                "SYS.CACHES.WRITE_BEHIND_COALESCING.null",
+                "SYS.CACHES.WRITE_BEHIND_FLUSH_SIZE.null",
+                "SYS.CACHES.WRITE_BEHIND_FLUSH_FREQUENCY.null",
+                "SYS.CACHES.WRITE_BEHIND_FLUSH_THREAD_COUNT.null",
+                "SYS.CACHES.WRITE_BEHIND_BATCH_SIZE.null",
+                "SYS.CACHES.MAX_CONCURRENT_ASYNC_OPERATIONS.null",
+                "SYS.CACHES.CACHE_LOADER_FACTORY.null",
+                "SYS.CACHES.CACHE_WRITER_FACTORY.null",
+                "SYS.CACHES.EXPIRY_POLICY_FACTORY.null",
+                "SYS.CACHES.IS_SQL_ESCAPE_ALL.null",
+                "SYS.CACHES.IS_ENCRYPTION_ENABLED.null",
+                "SYS.CACHES.SQL_SCHEMA.null",
+                "SYS.CACHES.SQL_INDEX_MAX_INLINE_SIZE.null",
+                "SYS.CACHES.IS_SQL_ONHEAP_CACHE_ENABLED.null",
+                "SYS.CACHES.SQL_ONHEAP_CACHE_MAX_SIZE.null",
+                "SYS.CACHES.QUERY_DETAIL_METRICS_SIZE.null",
+                "SYS.CACHES.QUERY_PARALLELISM.null",
+                "SYS.CACHES.MAX_QUERY_ITERATORS_COUNT.null",
+                "SYS.CACHES.DATA_REGION_NAME.null",
+                "SYS.CACHE_GROUPS.CACHE_GROUP_ID.null",
+                "SYS.CACHE_GROUPS.CACHE_GROUP_NAME.null",
+                "SYS.CACHE_GROUPS.IS_SHARED.null",
+                "SYS.CACHE_GROUPS.CACHE_COUNT.null",
+                "SYS.CACHE_GROUPS.CACHE_MODE.null",
+                "SYS.CACHE_GROUPS.ATOMICITY_MODE.null",
+                "SYS.CACHE_GROUPS.AFFINITY.null",
+                "SYS.CACHE_GROUPS.PARTITIONS_COUNT.null",
+                "SYS.CACHE_GROUPS.NODE_FILTER.null",
+                "SYS.CACHE_GROUPS.DATA_REGION_NAME.null",
+                "SYS.CACHE_GROUPS.TOPOLOGY_VALIDATOR.null",
+                "SYS.CACHE_GROUPS.PARTITION_LOSS_POLICY.null",
+                "SYS.CACHE_GROUPS.REBALANCE_MODE.null",
+                "SYS.CACHE_GROUPS.REBALANCE_DELAY.null",
+                "SYS.CACHE_GROUPS.REBALANCE_ORDER.null",
+                "SYS.CACHE_GROUPS.BACKUPS.null",
+                "SYS.INDEXES.CACHE_GROUP_ID.null",
+                "SYS.INDEXES.CACHE_GROUP_NAME.null",
+                "SYS.INDEXES.CACHE_ID.null",
+                "SYS.INDEXES.CACHE_NAME.null",
+                "SYS.INDEXES.SCHEMA_NAME.null",
+                "SYS.INDEXES.TABLE_NAME.null",
+                "SYS.INDEXES.INDEX_NAME.null",
+                "SYS.INDEXES.INDEX_TYPE.null",
+                "SYS.INDEXES.COLUMNS.null",
+                "SYS.INDEXES.IS_PK.null",
+                "SYS.INDEXES.IS_UNIQUE.null",
+                "SYS.INDEXES.INLINE_SIZE.null",
+                "SYS.LOCAL_CACHE_GROUPS_IO.CACHE_GROUP_ID.null",
+                "SYS.LOCAL_CACHE_GROUPS_IO.CACHE_GROUP_NAME.null",
+                "SYS.LOCAL_CACHE_GROUPS_IO.PHYSICAL_READS.null",
+                "SYS.LOCAL_CACHE_GROUPS_IO.LOGICAL_READS.null",
+                "SYS.SQL_QUERIES_HISTORY.SCHEMA_NAME.null",
+                "SYS.SQL_QUERIES_HISTORY.SQL.null",
+                "SYS.SQL_QUERIES_HISTORY.LOCAL.null",
+                "SYS.SQL_QUERIES_HISTORY.EXECUTIONS.null",
+                "SYS.SQL_QUERIES_HISTORY.FAILURES.null",
+                "SYS.SQL_QUERIES_HISTORY.DURATION_MIN.null",
+                "SYS.SQL_QUERIES_HISTORY.DURATION_MAX.null",
+                "SYS.SQL_QUERIES_HISTORY.LAST_START_TIME.null",
+                "SYS.SQL_QUERIES.QUERY_ID.null",
+                "SYS.SQL_QUERIES.SQL.null",
+                "SYS.SQL_QUERIES.SCHEMA_NAME.null",
+                "SYS.SQL_QUERIES.LOCAL.null",
+                "SYS.SQL_QUERIES.START_TIME.null",
+                "SYS.SQL_QUERIES.DURATION.null",
+                "SYS.SQL_QUERIES.ORIGIN_NODE_ID.null",
+                "SYS.SQL_QUERIES.INITIATOR_ID.null",
+                "SYS.SQL_QUERIES.SUBJECT_ID.null",
+                "SYS.SCAN_QUERIES.START_TIME.null",
+                "SYS.SCAN_QUERIES.TRANSFORMER.null",
+                "SYS.SCAN_QUERIES.LOCAL.null",
+                "SYS.SCAN_QUERIES.QUERY_ID.null",
+                "SYS.SCAN_QUERIES.PARTITION.null",
+                "SYS.SCAN_QUERIES.CACHE_GROUP_ID.null",
+                "SYS.SCAN_QUERIES.CACHE_NAME.null",
+                "SYS.SCAN_QUERIES.TOPOLOGY.null",
+                "SYS.SCAN_QUERIES.CACHE_GROUP_NAME.null",
+                "SYS.SCAN_QUERIES.TASK_NAME.null",
+                "SYS.SCAN_QUERIES.DURATION.null",
+                "SYS.SCAN_QUERIES.KEEP_BINARY.null",
+                "SYS.SCAN_QUERIES.FILTER.null",
+                "SYS.SCAN_QUERIES.SUBJECT_ID.null",
+                "SYS.SCAN_QUERIES.CANCELED.null",
+                "SYS.SCAN_QUERIES.CACHE_ID.null",
+                "SYS.SCAN_QUERIES.PAGE_SIZE.null",
+                "SYS.SCAN_QUERIES.ORIGIN_NODE_ID.null",
+                "SYS.NODES.NODE_ID.null",
+                "SYS.NODES.CONSISTENT_ID.null",
+                "SYS.NODES.VERSION.null",
+                "SYS.NODES.IS_CLIENT.null",
+                "SYS.NODES.IS_DAEMON.null",
+                "SYS.NODES.IS_LOCAL.null",
+                "SYS.NODES.NODE_ORDER.null",
+                "SYS.NODES.ADDRESSES.null",
+                "SYS.NODES.HOSTNAMES.null",
+                "SYS.NODE_ATTRIBUTES.NODE_ID.null",
+                "SYS.NODE_ATTRIBUTES.NAME.null",
+                "SYS.NODE_ATTRIBUTES.VALUE.null",
+                "SYS.NODE_METRICS.NODE_ID.null",
+                "SYS.NODE_METRICS.LAST_UPDATE_TIME.null",
+                "SYS.NODE_METRICS.MAX_ACTIVE_JOBS.null",
+                "SYS.NODE_METRICS.CUR_ACTIVE_JOBS.null",
+                "SYS.NODE_METRICS.AVG_ACTIVE_JOBS.null",
+                "SYS.NODE_METRICS.MAX_WAITING_JOBS.null",
+                "SYS.NODE_METRICS.CUR_WAITING_JOBS.null",
+                "SYS.NODE_METRICS.AVG_WAITING_JOBS.null",
+                "SYS.NODE_METRICS.MAX_REJECTED_JOBS.null",
+                "SYS.NODE_METRICS.CUR_REJECTED_JOBS.null",
+                "SYS.NODE_METRICS.AVG_REJECTED_JOBS.null",
+                "SYS.NODE_METRICS.TOTAL_REJECTED_JOBS.null",
+                "SYS.NODE_METRICS.MAX_CANCELED_JOBS.null",
+                "SYS.NODE_METRICS.CUR_CANCELED_JOBS.null",
+                "SYS.NODE_METRICS.AVG_CANCELED_JOBS.null",
+                "SYS.NODE_METRICS.TOTAL_CANCELED_JOBS.null",
+                "SYS.NODE_METRICS.MAX_JOBS_WAIT_TIME.null",
+                "SYS.NODE_METRICS.CUR_JOBS_WAIT_TIME.null",
+                "SYS.NODE_METRICS.AVG_JOBS_WAIT_TIME.null",
+                "SYS.NODE_METRICS.MAX_JOBS_EXECUTE_TIME.null",
+                "SYS.NODE_METRICS.CUR_JOBS_EXECUTE_TIME.null",
+                "SYS.NODE_METRICS.AVG_JOBS_EXECUTE_TIME.null",
+                "SYS.NODE_METRICS.TOTAL_JOBS_EXECUTE_TIME.null",
+                "SYS.NODE_METRICS.TOTAL_EXECUTED_JOBS.null",
+                "SYS.NODE_METRICS.TOTAL_EXECUTED_TASKS.null",
+                "SYS.NODE_METRICS.TOTAL_BUSY_TIME.null",
+                "SYS.NODE_METRICS.TOTAL_IDLE_TIME.null",
+                "SYS.NODE_METRICS.CUR_IDLE_TIME.null",
+                "SYS.NODE_METRICS.BUSY_TIME_PERCENTAGE.null",
+                "SYS.NODE_METRICS.IDLE_TIME_PERCENTAGE.null",
+                "SYS.NODE_METRICS.TOTAL_CPU.null",
+                "SYS.NODE_METRICS.CUR_CPU_LOAD.null",
+                "SYS.NODE_METRICS.AVG_CPU_LOAD.null",
+                "SYS.NODE_METRICS.CUR_GC_CPU_LOAD.null",
+                "SYS.NODE_METRICS.HEAP_MEMORY_INIT.null",
+                "SYS.NODE_METRICS.HEAP_MEMORY_USED.null",
+                "SYS.NODE_METRICS.HEAP_MEMORY_COMMITED.null",
+                "SYS.NODE_METRICS.HEAP_MEMORY_MAX.null",
+                "SYS.NODE_METRICS.HEAP_MEMORY_TOTAL.null",
+                "SYS.NODE_METRICS.NONHEAP_MEMORY_INIT.null",
+                "SYS.NODE_METRICS.NONHEAP_MEMORY_USED.null",
+                "SYS.NODE_METRICS.NONHEAP_MEMORY_COMMITED.null",
+                "SYS.NODE_METRICS.NONHEAP_MEMORY_MAX.null",
+                "SYS.NODE_METRICS.NONHEAP_MEMORY_TOTAL.null",
+                "SYS.NODE_METRICS.UPTIME.null",
+                "SYS.NODE_METRICS.JVM_START_TIME.null",
+                "SYS.NODE_METRICS.NODE_START_TIME.null",
+                "SYS.NODE_METRICS.LAST_DATA_VERSION.null",
+                "SYS.NODE_METRICS.CUR_THREAD_COUNT.null",
+                "SYS.NODE_METRICS.MAX_THREAD_COUNT.null",
+                "SYS.NODE_METRICS.TOTAL_THREAD_COUNT.null",
+                "SYS.NODE_METRICS.CUR_DAEMON_THREAD_COUNT.null",
+                "SYS.NODE_METRICS.SENT_MESSAGES_COUNT.null",
+                "SYS.NODE_METRICS.SENT_BYTES_COUNT.null",
+                "SYS.NODE_METRICS.RECEIVED_MESSAGES_COUNT.null",
+                "SYS.NODE_METRICS.RECEIVED_BYTES_COUNT.null",
+                "SYS.NODE_METRICS.OUTBOUND_MESSAGES_QUEUE.null",
+                "SYS.TABLES.CACHE_GROUP_ID.null",
+                "SYS.TABLES.CACHE_GROUP_NAME.null",
+                "SYS.TABLES.CACHE_ID.null",
+                "SYS.TABLES.CACHE_NAME.null",
+                "SYS.TABLES.SCHEMA_NAME.null",
+                "SYS.TABLES.TABLE_NAME.null",
+                "SYS.TABLES.AFFINITY_KEY_COLUMN.null",
+                "SYS.TABLES.KEY_ALIAS.null",
+                "SYS.TABLES.VALUE_ALIAS.null",
+                "SYS.TABLES.KEY_TYPE_NAME.null",
+                "SYS.TABLES.VALUE_TYPE_NAME.null",
+                "SYS.TABLES.IS_INDEX_REBUILD_IN_PROGRESS.null",
+                "SYS.METRICS.NAME.null",
+                "SYS.METRICS.VALUE.null",
+                "SYS.METRICS.DESCRIPTION.null",
+                "SYS.SERVICES.SERVICE_ID.null",
+                "SYS.SERVICES.NAME.null",
+                "SYS.SERVICES.SERVICE_CLASS.null",
+                "SYS.SERVICES.CACHE_NAME.null",
+                "SYS.SERVICES.ORIGIN_NODE_ID.null",
+                "SYS.SERVICES.TOTAL_COUNT.null",
+                "SYS.SERVICES.MAX_PER_NODE_COUNT.null",
+                "SYS.SERVICES.AFFINITY_KEY.null",
+                "SYS.SERVICES.NODE_FILTER.null",
+                "SYS.SERVICES.STATICALLY_CONFIGURED.null",
+                "SYS.SERVICES.SERVICE_ID.null",
+                "SYS.TASKS.AFFINITY_CACHE_NAME.null",
+                "SYS.TASKS.INTERNAL.null",
+                "SYS.TASKS.END_TIME.null",
+                "SYS.TASKS.START_TIME.null",
+                "SYS.TASKS.USER_VERSION.null",
+                "SYS.TASKS.TASK_NAME.null",
+                "SYS.TASKS.TASK_NODE_ID.null",
+                "SYS.TASKS.JOB_ID.null",
+                "SYS.TASKS.ID.null",
+                "SYS.TASKS.SESSION_ID.null",
+                "SYS.TASKS.AFFINITY_PARTITION_ID.null",
+                "SYS.TASKS.TASK_CLASS_NAME.null",
+                "SYS.JOBS.IS_STARTED.null",
+                "SYS.JOBS.EXECUTOR_NAME.null",
+                "SYS.JOBS.IS_TIMED_OUT.null",
+                "SYS.JOBS.ID.null",
+                "SYS.JOBS.FINISH_TIME.null",
+                "SYS.JOBS.IS_INTERNAL.null",
+                "SYS.JOBS.CREATE_TIME.null",
+                "SYS.JOBS.AFFINITY_PARTITION_ID.null",
+                "SYS.JOBS.ORIGIN_NODE_ID.null",
+                "SYS.JOBS.TASK_NAME.null",
+                "SYS.JOBS.TASK_CLASS_NAME.null",
+                "SYS.JOBS.SESSION_ID.null",
+                "SYS.JOBS.IS_FINISHING.null",
+                "SYS.JOBS.START_TIME.null",
+                "SYS.JOBS.AFFINITY_CACHE_IDS.null",
+                "SYS.JOBS.STATE.null",
+                "SYS.CLIENT_CONNECTIONS.CONNECTION_ID.null",
+                "SYS.CLIENT_CONNECTIONS.LOCAL_ADDRESS.null",
+                "SYS.CLIENT_CONNECTIONS.REMOTE_ADDRESS.null",
+                "SYS.CLIENT_CONNECTIONS.TYPE.null",
+                "SYS.CLIENT_CONNECTIONS.USER.null",
+                "SYS.CLIENT_CONNECTIONS.VERSION.null",
+                "SYS.TASKS.EXEC_NAME.null",
+                "SYS.TRANSACTIONS.LOCAL_NODE_ID.null",
+                "SYS.TRANSACTIONS.STATE.null",
+                "SYS.TRANSACTIONS.XID.null",
+                "SYS.TRANSACTIONS.LABEL.null",
+                "SYS.TRANSACTIONS.START_TIME.null",
+                "SYS.TRANSACTIONS.ISOLATION.null",
+                "SYS.TRANSACTIONS.CONCURRENCY.null",
+                "SYS.TRANSACTIONS.COLOCATED.null",
+                "SYS.TRANSACTIONS.DHT.null",
+                "SYS.TRANSACTIONS.IMPLICIT.null",
+                "SYS.TRANSACTIONS.IMPLICIT_SINGLE.null",
+                "SYS.TRANSACTIONS.INTERNAL.null",
+                "SYS.TRANSACTIONS.LOCAL.null",
+                "SYS.TRANSACTIONS.NEAR.null",
+                "SYS.TRANSACTIONS.ONE_PHASE_COMMIT.null",
+                "SYS.TRANSACTIONS.SUBJECT_ID.null",
+                "SYS.TRANSACTIONS.SYSTEM.null",
+                "SYS.TRANSACTIONS.THREAD_ID.null",
+                "SYS.TRANSACTIONS.TIMEOUT.null",
+                "SYS.TRANSACTIONS.DURATION.null",
+                "SYS.TRANSACTIONS.ORIGINATING_NODE_ID.null",
+                "SYS.TRANSACTIONS.OTHER_NODE_ID.null",
+                "SYS.TRANSACTIONS.TOP_VER.null",
+                "SYS.TRANSACTIONS.KEYS_COUNT.null",
+                "SYS.TRANSACTIONS.CACHE_IDS.null",
+                "SYS.SCHEMAS.SCHEMA_NAME.null",
+                "SYS.SCHEMAS.PREDEFINED.null",
+                "SYS.VIEWS.NAME.null",
+                "SYS.VIEWS.DESCRIPTION.null",
+                "SYS.VIEWS.SCHEMA.null",
+                "SYS.TABLE_COLUMNS.AFFINITY_COLUMN.null",
+                "SYS.TABLE_COLUMNS.COLUMN_NAME.null",
+                "SYS.TABLE_COLUMNS.SCALE.null",
+                "SYS.TABLE_COLUMNS.PK.null",
+                "SYS.TABLE_COLUMNS.TYPE.null",
+                "SYS.TABLE_COLUMNS.DEFAULT_VALUE.null",
+                "SYS.TABLE_COLUMNS.SCHEMA_NAME.null",
+                "SYS.TABLE_COLUMNS.TABLE_NAME.null",
+                "SYS.TABLE_COLUMNS.NULLABLE.null",
+                "SYS.TABLE_COLUMNS.PRECISION.null",
+                "SYS.TABLE_COLUMNS.AUTO_INCREMENT.null",
+                "SYS.VIEW_COLUMNS.NULLABLE.null",
+                "SYS.VIEW_COLUMNS.SCHEMA_NAME.null",
+                "SYS.VIEW_COLUMNS.COLUMN_NAME.null",
+                "SYS.VIEW_COLUMNS.TYPE.null",
+                "SYS.VIEW_COLUMNS.PRECISION.null",
+                "SYS.VIEW_COLUMNS.DEFAULT_VALUE.null",
+                "SYS.VIEW_COLUMNS.SCALE.null",
+                "SYS.VIEW_COLUMNS.VIEW_NAME.null",
+                "SYS.CONTINUOUS_QUERIES.NOTIFY_EXISTING.null",
+                "SYS.CONTINUOUS_QUERIES.OLD_VALUE_REQUIRED.null",
+                "SYS.CONTINUOUS_QUERIES.KEEP_BINARY.null",
+                "SYS.CONTINUOUS_QUERIES.IS_MESSAGING.null",
+                "SYS.CONTINUOUS_QUERIES.AUTO_UNSUBSCRIBE.null",
+                "SYS.CONTINUOUS_QUERIES.LAST_SEND_TIME.null",
+                "SYS.CONTINUOUS_QUERIES.LOCAL_TRANSFORMED_LISTENER.null",
+                "SYS.CONTINUOUS_QUERIES.TOPIC.null",
+                "SYS.CONTINUOUS_QUERIES.BUFFER_SIZE.null",
+                "SYS.CONTINUOUS_QUERIES.REMOTE_TRANSFORMER.null",
+                "SYS.CONTINUOUS_QUERIES.DELAYED_REGISTER.null",
+                "SYS.CONTINUOUS_QUERIES.IS_QUERY.null",
+                "SYS.CONTINUOUS_QUERIES.NODE_ID.null",
+                "SYS.CONTINUOUS_QUERIES.INTERVAL.null",
+                "SYS.CONTINUOUS_QUERIES.IS_EVENTS.null",
+                "SYS.CONTINUOUS_QUERIES.ROUTINE_ID.null",
+                "SYS.CONTINUOUS_QUERIES.REMOTE_FILTER.null",
+                "SYS.CONTINUOUS_QUERIES.CACHE_NAME.null",
+                "SYS.CONTINUOUS_QUERIES.LOCAL_LISTENER.null",
+                "SYS.STRIPED_THREADPOOL_QUEUE.STRIPE_INDEX.null",
+                "SYS.STRIPED_THREADPOOL_QUEUE.DESCRIPTION.null",
+                "SYS.STRIPED_THREADPOOL_QUEUE.THREAD_NAME.null",
+                "SYS.STRIPED_THREADPOOL_QUEUE.TASK_NAME.null",
+                "SYS.DATASTREAM_THREADPOOL_QUEUE.STRIPE_INDEX.null",
+                "SYS.DATASTREAM_THREADPOOL_QUEUE.DESCRIPTION.null",
+                "SYS.DATASTREAM_THREADPOOL_QUEUE.THREAD_NAME.null",
+                "SYS.DATASTREAM_THREADPOOL_QUEUE.TASK_NAME.null",
+                "SYS.CACHE_GROUP_PAGE_LISTS.CACHE_GROUP_ID.null",
+                "SYS.CACHE_GROUP_PAGE_LISTS.PARTITION_ID.null",
+                "SYS.CACHE_GROUP_PAGE_LISTS.NAME.null",
+                "SYS.CACHE_GROUP_PAGE_LISTS.BUCKET_NUMBER.null",
+                "SYS.CACHE_GROUP_PAGE_LISTS.BUCKET_SIZE.null",
+                "SYS.CACHE_GROUP_PAGE_LISTS.STRIPES_COUNT.null",
+                "SYS.CACHE_GROUP_PAGE_LISTS.CACHED_PAGES_COUNT.null",
+                "SYS.DATA_REGION_PAGE_LISTS.NAME.null",
+                "SYS.DATA_REGION_PAGE_LISTS.BUCKET_NUMBER.null",
+                "SYS.DATA_REGION_PAGE_LISTS.BUCKET_SIZE.null",
+                "SYS.DATA_REGION_PAGE_LISTS.STRIPES_COUNT.null",
+                "SYS.DATA_REGION_PAGE_LISTS.CACHED_PAGES_COUNT.null",
+                "SYS.PARTITION_STATES.CACHE_GROUP_ID.null",
+                "SYS.PARTITION_STATES.PARTITION_ID.null",
+                "SYS.PARTITION_STATES.NODE_ID.null",
+                "SYS.PARTITION_STATES.STATE.null",
+                "SYS.PARTITION_STATES.IS_PRIMARY.null",
+                "SYS.BINARY_METADATA.FIELDS.null",
+                "SYS.BINARY_METADATA.AFF_KEY_FIELD_NAME.null",
+                "SYS.BINARY_METADATA.SCHEMAS_IDS.null",
+                "SYS.BINARY_METADATA.TYPE_ID.null",
+                "SYS.BINARY_METADATA.IS_ENUM.null",
+                "SYS.BINARY_METADATA.FIELDS_COUNT.null",
+                "SYS.BINARY_METADATA.TYPE_NAME.null",
+                "SYS.DISTRIBUTED_METASTORAGE.NAME.null",
+                "SYS.DISTRIBUTED_METASTORAGE.VALUE.null",
+                "SYS.DS_ATOMICLONGS.GROUP_ID.null",
+                "SYS.DS_ATOMICLONGS.GROUP_NAME.null",
+                "SYS.DS_ATOMICLONGS.NAME.null",
+                "SYS.DS_ATOMICLONGS.REMOVED.null",
+                "SYS.DS_ATOMICLONGS.VALUE.null",
+                "SYS.DS_ATOMICREFERENCES.GROUP_ID.null",
+                "SYS.DS_ATOMICREFERENCES.GROUP_NAME.null",
+                "SYS.DS_ATOMICREFERENCES.NAME.null",
+                "SYS.DS_ATOMICREFERENCES.REMOVED.null",
+                "SYS.DS_ATOMICREFERENCES.VALUE.null",
+                "SYS.DS_ATOMICSEQUENCES.BATCH_SIZE.null",
+                "SYS.DS_ATOMICSEQUENCES.GROUP_ID.null",
+                "SYS.DS_ATOMICSEQUENCES.GROUP_NAME.null",
+                "SYS.DS_ATOMICSEQUENCES.NAME.null",
+                "SYS.DS_ATOMICSEQUENCES.REMOVED.null",
+                "SYS.DS_ATOMICSEQUENCES.VALUE.null",
+                "SYS.DS_ATOMICSTAMPED.GROUP_ID.null",
+                "SYS.DS_ATOMICSTAMPED.GROUP_NAME.null",
+                "SYS.DS_ATOMICSTAMPED.NAME.null",
+                "SYS.DS_ATOMICSTAMPED.REMOVED.null",
+                "SYS.DS_ATOMICSTAMPED.STAMP.null",
+                "SYS.DS_ATOMICSTAMPED.VALUE.null",
+                "SYS.DS_COUNTDOWNLATCHES.AUTO_DELETE.null",
+                "SYS.DS_COUNTDOWNLATCHES.COUNT.null",
+                "SYS.DS_COUNTDOWNLATCHES.GROUP_ID.null",
+                "SYS.DS_COUNTDOWNLATCHES.GROUP_NAME.null",
+                "SYS.DS_COUNTDOWNLATCHES.INITIAL_COUNT.null",
+                "SYS.DS_COUNTDOWNLATCHES.NAME.null",
+                "SYS.DS_COUNTDOWNLATCHES.REMOVED.null",
+                "SYS.DS_QUEUES.BOUNDED.null",
+                "SYS.DS_QUEUES.CAPACITY.null",
+                "SYS.DS_QUEUES.SIZE.null",
+                "SYS.DS_QUEUES.COLLOCATED.null",
+                "SYS.DS_QUEUES.GROUP_ID.null",
+                "SYS.DS_QUEUES.GROUP_NAME.null",
+                "SYS.DS_QUEUES.ID.null",
+                "SYS.DS_QUEUES.NAME.null",
+                "SYS.DS_QUEUES.REMOVED.null",
+                "SYS.DS_REENTRANTLOCKS.BROKEN.null",
+                "SYS.DS_REENTRANTLOCKS.FAILOVER_SAFE.null",
+                "SYS.DS_REENTRANTLOCKS.FAIR.null",
+                "SYS.DS_REENTRANTLOCKS.GROUP_ID.null",
+                "SYS.DS_REENTRANTLOCKS.GROUP_NAME.null",
+                "SYS.DS_REENTRANTLOCKS.HAS_QUEUED_THREADS.null",
+                "SYS.DS_REENTRANTLOCKS.LOCKED.null",
+                "SYS.DS_REENTRANTLOCKS.NAME.null",
+                "SYS.DS_REENTRANTLOCKS.REMOVED.null",
+                "SYS.DS_SEMAPHORES.AVAILABLE_PERMITS.null",
+                "SYS.DS_SEMAPHORES.BROKEN.null",
+                "SYS.DS_SEMAPHORES.FAILOVER_SAFE.null",
+                "SYS.DS_SEMAPHORES.GROUP_ID.null",
+                "SYS.DS_SEMAPHORES.GROUP_NAME.null",
+                "SYS.DS_SEMAPHORES.HAS_QUEUED_THREADS.null",
+                "SYS.DS_SEMAPHORES.NAME.null",
+                "SYS.DS_SEMAPHORES.QUEUE_LENGTH.null",
+                "SYS.DS_SEMAPHORES.REMOVED.null",
+                "SYS.DS_SETS.COLLOCATED.null",
+                "SYS.DS_SETS.GROUP_ID.null",
+                "SYS.DS_SETS.GROUP_NAME.null",
+                "SYS.DS_SETS.ID.null",
+                "SYS.DS_SETS.NAME.null",
+                "SYS.DS_SETS.REMOVED.null",
+                "SYS.DS_SETS.SIZE.null",
+                "SYS.STATISTICS_LOCAL_DATA.LAST_UPDATE_TIME.null",
+                "SYS.STATISTICS_LOCAL_DATA.NAME.null",
+                "SYS.STATISTICS_LOCAL_DATA.TOTAL.null",
+                "SYS.STATISTICS_PARTITION_DATA.VERSION.null",
+                "SYS.STATISTICS_CONFIGURATION.TYPE.null",
+                "SYS.STATISTICS_PARTITION_DATA.NAME.null",
+                "SYS.STATISTICS_CONFIGURATION.COLUMN.null",
+                "SYS.STATISTICS_LOCAL_DATA.ROWS_COUNT.null",
+                "SYS.STATISTICS_PARTITION_DATA.TYPE.null",
+                "SYS.STATISTICS_LOCAL_DATA.DISTINCT.null",
+                "SYS.STATISTICS_LOCAL_DATA.SIZE.null",
+                "SYS.STATISTICS_PARTITION_DATA.LAST_UPDATE_TIME.null",
+                "SYS.STATISTICS_CONFIGURATION.MAX_PARTITION_OBSOLESCENCE_PERCENT.null",
+                "SYS.STATISTICS_LOCAL_DATA.VERSION.null",
+                "SYS.STATISTICS_LOCAL_DATA.COLUMN.null",
+                "SYS.STATISTICS_CONFIGURATION.SCHEMA.null",
+                "SYS.STATISTICS_PARTITION_DATA.TOTAL.null",
+                "SYS.STATISTICS_PARTITION_DATA.PARTITION.null",
+                "SYS.STATISTICS_PARTITION_DATA.SCHEMA.null",
+                "SYS.STATISTICS_PARTITION_DATA.ROWS_COUNT.null",
+                "SYS.STATISTICS_PARTITION_DATA.SIZE.null",
+                "SYS.STATISTICS_PARTITION_DATA.UPDATE_COUNTER.null",
+                "SYS.STATISTICS_CONFIGURATION.NAME.null",
+                "SYS.STATISTICS_PARTITION_DATA.DISTINCT.null",
+                "SYS.STATISTICS_LOCAL_DATA.NULLS.null",
+                "SYS.STATISTICS_CONFIGURATION.VERSION.null",
+                "SYS.STATISTICS_CONFIGURATION.MANUAL_SIZE.null",
+                "SYS.STATISTICS_CONFIGURATION.MANUAL_DISTINCT.null",
+                "SYS.STATISTICS_CONFIGURATION.MANUAL_NULLS.null",
+                "SYS.STATISTICS_CONFIGURATION.MANUAL_TOTAL.null",
+                "SYS.STATISTICS_LOCAL_DATA.TYPE.null",
+                "SYS.STATISTICS_PARTITION_DATA.NULLS.null",
+                "SYS.STATISTICS_PARTITION_DATA.COLUMN.null",
+                "SYS.STATISTICS_LOCAL_DATA.SCHEMA.null",
+                "SYS.STATISTICS_GLOBAL_DATA.SCHEMA.null",
+                "SYS.STATISTICS_GLOBAL_DATA.TYPE.null",
+                "SYS.STATISTICS_GLOBAL_DATA.NAME.null",
+                "SYS.STATISTICS_GLOBAL_DATA.COLUMN.null",
+                "SYS.STATISTICS_GLOBAL_DATA.ROWS_COUNT.null",
+                "SYS.STATISTICS_GLOBAL_DATA.DISTINCT.null",
+                "SYS.STATISTICS_GLOBAL_DATA.NULLS.null",
+                "SYS.STATISTICS_GLOBAL_DATA.TOTAL.null",
+                "SYS.STATISTICS_GLOBAL_DATA.SIZE.null",
+                "SYS.STATISTICS_GLOBAL_DATA.VERSION.null",
+                "SYS.STATISTICS_GLOBAL_DATA.LAST_UPDATE_TIME.null",
+                "SYS.PAGES_TIMESTAMP_HISTOGRAM.DATA_REGION_NAME.null",
+                "SYS.PAGES_TIMESTAMP_HISTOGRAM.INTERVAL_START.null",
+                "SYS.PAGES_TIMESTAMP_HISTOGRAM.INTERVAL_END.null",
+                "SYS.PAGES_TIMESTAMP_HISTOGRAM.PAGES_COUNT.null"
                 ));
 
             Assert.assertEquals(expectedCols, actualSystemCols);
diff --git a/modules/codegen/src/main/java/org/apache/ignite/codegen/SystemViewRowAttributeWalkerGenerator.java b/modules/codegen/src/main/java/org/apache/ignite/codegen/SystemViewRowAttributeWalkerGenerator.java
index 850ce8ffd9f..c04e97773fe 100644
--- a/modules/codegen/src/main/java/org/apache/ignite/codegen/SystemViewRowAttributeWalkerGenerator.java
+++ b/modules/codegen/src/main/java/org/apache/ignite/codegen/SystemViewRowAttributeWalkerGenerator.java
@@ -62,14 +62,8 @@ import org.apache.ignite.spi.systemview.view.PartitionStateView;
 import org.apache.ignite.spi.systemview.view.ScanQueryView;
 import org.apache.ignite.spi.systemview.view.ServiceView;
 import org.apache.ignite.spi.systemview.view.SnapshotView;
-import org.apache.ignite.spi.systemview.view.SqlIndexView;
 import org.apache.ignite.spi.systemview.view.SqlQueryHistoryView;
 import org.apache.ignite.spi.systemview.view.SqlQueryView;
-import org.apache.ignite.spi.systemview.view.SqlSchemaView;
-import org.apache.ignite.spi.systemview.view.SqlTableColumnView;
-import org.apache.ignite.spi.systemview.view.SqlTableView;
-import org.apache.ignite.spi.systemview.view.SqlViewColumnView;
-import org.apache.ignite.spi.systemview.view.SqlViewView;
 import org.apache.ignite.spi.systemview.view.StripedExecutorTaskView;
 import org.apache.ignite.spi.systemview.view.SystemView;
 import org.apache.ignite.spi.systemview.view.SystemViewRowAttributeWalker;
@@ -83,9 +77,14 @@ import org.apache.ignite.spi.systemview.view.datastructures.QueueView;
 import org.apache.ignite.spi.systemview.view.datastructures.ReentrantLockView;
 import org.apache.ignite.spi.systemview.view.datastructures.SemaphoreView;
 import org.apache.ignite.spi.systemview.view.datastructures.SetView;
+import org.apache.ignite.spi.systemview.view.sql.SqlIndexView;
+import org.apache.ignite.spi.systemview.view.sql.SqlSchemaView;
+import org.apache.ignite.spi.systemview.view.sql.SqlTableColumnView;
+import org.apache.ignite.spi.systemview.view.sql.SqlTableView;
+import org.apache.ignite.spi.systemview.view.sql.SqlViewColumnView;
+import org.apache.ignite.spi.systemview.view.sql.SqlViewView;
 
 import static org.apache.ignite.codegen.MessageCodeGenerator.DFLT_SRC_DIR;
-import static org.apache.ignite.codegen.MessageCodeGenerator.INDEXING_SRC_DIR;
 
 /**
  * Application for code generation of {@link SystemViewRowAttributeWalker}.
@@ -148,12 +147,12 @@ public class SystemViewRowAttributeWalkerGenerator {
         gen.generateAndWrite(MetricsView.class, DFLT_SRC_DIR);
         gen.generateAndWrite(PagesTimestampHistogramView.class, DFLT_SRC_DIR);
 
-        gen.generateAndWrite(SqlSchemaView.class, INDEXING_SRC_DIR);
-        gen.generateAndWrite(SqlTableView.class, INDEXING_SRC_DIR);
-        gen.generateAndWrite(SqlViewView.class, INDEXING_SRC_DIR);
-        gen.generateAndWrite(SqlIndexView.class, INDEXING_SRC_DIR);
-        gen.generateAndWrite(SqlTableColumnView.class, INDEXING_SRC_DIR);
-        gen.generateAndWrite(SqlViewColumnView.class, INDEXING_SRC_DIR);
+        gen.generateAndWrite(SqlSchemaView.class, DFLT_SRC_DIR);
+        gen.generateAndWrite(SqlTableView.class, DFLT_SRC_DIR);
+        gen.generateAndWrite(SqlViewView.class, DFLT_SRC_DIR);
+        gen.generateAndWrite(SqlIndexView.class, DFLT_SRC_DIR);
+        gen.generateAndWrite(SqlTableColumnView.class, DFLT_SRC_DIR);
+        gen.generateAndWrite(SqlViewColumnView.class, DFLT_SRC_DIR);
 
         gen.generateAndWrite(StatisticsColumnConfigurationView.class, DFLT_SRC_DIR);
         gen.generateAndWrite(StatisticsColumnLocalDataView.class, DFLT_SRC_DIR);
diff --git a/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/CompoundIndexCompatibilityTest.java b/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/CompoundIndexCompatibilityTest.java
new file mode 100644
index 00000000000..71d44135270
--- /dev/null
+++ b/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/CompoundIndexCompatibilityTest.java
@@ -0,0 +1,167 @@
+/*
+ * 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.compatibility.persistence;
+
+import java.math.BigDecimal;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.cluster.ClusterState;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.cache.query.index.IndexName;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.InlineIndexImpl;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.junit.Test;
+
+/**
+ * Tests compatibility for compound indexes.
+ */
+public class CompoundIndexCompatibilityTest extends IndexAbstractCompatibilityTest {
+    /** */
+    @Test
+    public void testSecondaryIndexesMigration_2_7_6() throws Exception {
+        // 2.7.6 - version before _KEY unwrapping.
+        doTestStartupWithOldVersion("2.7.6", () -> {
+            checkIndex("_key_PK", null, "_KEY", "NAME");
+            checkIndex("AFFINITY_KEY", "name='name1'", "NAME", "_KEY");
+            checkIndex("IDX_CITY_AGE", "city='city1'", "CITY", "AGE", "_KEY", "NAME");
+            checkIndex("IDX_AGE_NAME", "age=1", "AGE", "NAME", "_KEY");
+            checkIndex("IDX_SALARY", "salary=0.1", "SALARY", "_KEY", "NAME");
+            checkIndex("IDX_COMPANY", "company='company1'", "COMPANY", "_KEY", "NAME");
+        });
+    }
+
+    /** */
+    @Test
+    public void testSecondaryIndexesMigration_2_13_0() throws Exception {
+        // 2.13.0 - version with _KEY unwrapping, but before index processing moved to the core module.
+        doTestStartupWithOldVersion("2.13.0", () -> {
+            checkIndex("_key_PK", "id=1", "ID", "NAME", "CITY");
+            checkIndex("AFFINITY_KEY", "name='name1'", "NAME", "ID", "CITY");
+            checkIndex("IDX_CITY_AGE", "city='city1'", "CITY", "AGE", "ID", "NAME");
+            checkIndex("IDX_AGE_NAME", "age=1", "AGE", "NAME", "ID", "CITY");
+            checkIndex("IDX_SALARY", "salary=0.1", "SALARY", "ID", "NAME", "CITY");
+            checkIndex("IDX_COMPANY", "company='company1'", "COMPANY", "ID", "NAME", "CITY");
+        });
+    }
+
+    /** */
+    private void doTestStartupWithOldVersion(String ver, Runnable idxChecker) throws Exception {
+        try {
+            startGrid(1, ver, new PersistenceBasicCompatibilityTest.ConfigurationClosure(true),
+                new PostStartupClosure());
+
+            stopAllGrids();
+
+            IgniteEx igniteEx = startGrid(0);
+
+            igniteEx.cluster().state(ClusterState.ACTIVE);
+
+            fillData(igniteEx, 100, 200);
+
+            idxChecker.run();
+
+            igniteEx.cluster().state(ClusterState.INACTIVE);
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
+
+    /** */
+    private static class PostStartupClosure implements IgniteInClosure<Ignite> {
+        /** {@inheritDoc} */
+        @Override public void apply(Ignite ignite) {
+            ignite.active(true);
+
+            IgniteEx igniteEx = (IgniteEx)ignite;
+
+            initializeTables(igniteEx);
+
+            ignite.active(false);
+        }
+    }
+
+    /** */
+    private static void initializeTables(IgniteEx igniteEx) {
+        executeSql(igniteEx, "CREATE TABLE TEST (" +
+            "id int, " +
+            "name varchar, " +
+            "age int, " +
+            "city varchar(8), " +
+            "salary decimal(10,2), " +
+            "primary key (id, name, city)) WITH \"affinity_key=name,cache_name=cache\"");
+
+        // Check different inline sizes (variable length columns, fixed length columns, not inlined types).
+        executeSql(igniteEx, "CREATE INDEX IDX_CITY_AGE ON TEST (city, age)");
+        executeSql(igniteEx, "CREATE INDEX IDX_AGE_NAME ON TEST (age, name)");
+        executeSql(igniteEx, "CREATE INDEX IDX_SALARY ON TEST (salary)");
+
+        // In versions before 2.14 fixed length columns added by "alter table" column inlined as variable length columns.
+        executeSql(igniteEx, "ALTER TABLE TEST ADD company varchar(10)");
+        executeSql(igniteEx, "CREATE INDEX IDX_COMPANY ON TEST (COMPANY)");
+
+        fillData(igniteEx, 0, 100);
+    }
+
+    /** */
+    private static void fillData(IgniteEx igniteEx, int from, int to) {
+        for (int i = from; i < to; i++) {
+            executeSql(igniteEx, "INSERT INTO TEST (id, name, age, company, city, salary) VALUES (?, ?, ?, ?, ?, ?)",
+                i, "name" + i, i, "company" + i, "city" + i, BigDecimal.valueOf(i, 1));
+        }
+    }
+
+    /**
+     * @param name Index name.
+     * @param cond Condition to select first row of the table by this index (if null - don't check query on index).
+     * @param cols Index columns.
+     */
+    private void checkIndex(String name, String cond, String... cols) {
+        // Check index columns.
+        InlineIndexImpl idx = grid(0).context().indexProcessor().index(new IndexName("cache",
+            "PUBLIC", "TEST", name)).unwrap(InlineIndexImpl.class);
+
+        assertEquals(F.asList(cols), new ArrayList<>(idx.indexDefinition().indexKeyDefinitions().keySet()));
+
+        // Check index is used by condition.
+        if (cond == null)
+            return;
+
+        String sql = "SELECT id FROM TEST WHERE " + cond;
+
+        List<List<?>> plan = executeSql(grid(0), "explain " + sql);
+
+        assertFalse(plan.isEmpty());
+        assertTrue("Actual plan: " + plan.get(0).get(0).toString() + " expected index: " + name,
+            plan.get(0).get(0).toString().toLowerCase().contains(name.toLowerCase()));
+
+        // Check query on index.
+        assertScalarResult(grid(0), sql, 1);
+    }
+
+    /** */
+    private static void assertScalarResult(IgniteEx igniteEx, String sql, Object expRes, Object... args) {
+        List<List<?>> results = executeSql(igniteEx, sql, args);
+
+        assertEquals(1, results.size());
+        assertEquals(expRes, results.get(0).get(0));
+    }
+}
diff --git a/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/IgnitePKIndexesMigrationToUnwrapPkTest.java b/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/IgnitePKIndexesMigrationToUnwrapPkTest.java
index 0e157202c4f..5393eb96f4c 100644
--- a/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/IgnitePKIndexesMigrationToUnwrapPkTest.java
+++ b/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/IgnitePKIndexesMigrationToUnwrapPkTest.java
@@ -18,58 +18,19 @@
 
 package org.apache.ignite.compatibility.persistence;
 
-import java.util.Collection;
 import java.util.List;
-import java.util.Set;
 import org.apache.ignite.Ignite;
-import org.apache.ignite.cache.query.SqlFieldsQuery;
-import org.apache.ignite.compatibility.testframework.junits.Dependency;
-import org.apache.ignite.configuration.DataRegionConfiguration;
-import org.apache.ignite.configuration.DataStorageConfiguration;
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.configuration.SystemDataRegionConfiguration;
+import org.apache.ignite.cluster.ClusterState;
 import org.apache.ignite.internal.IgniteEx;
-import org.apache.ignite.internal.processors.cache.GridCacheAbstractFullApiSelfTest;
 import org.apache.ignite.lang.IgniteInClosure;
-import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
-import org.jetbrains.annotations.NotNull;
-import org.junit.Ignore;
 import org.junit.Test;
 
 /**
  * Test to check that starting node with PK index of the old format present doesn't break anything.
  */
-@Ignore("https://issues.apache.org/jira/browse/IGNITE-13723")
-public class IgnitePKIndexesMigrationToUnwrapPkTest extends IgnitePersistenceCompatibilityAbstractTest {
+public class IgnitePKIndexesMigrationToUnwrapPkTest extends IndexAbstractCompatibilityTest {
     /** */
-    private static String TABLE_NAME = "TEST_IDX_TABLE";
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
-
-        new ConfigurationClosure().apply(cfg);
-
-        return cfg;
-    }
-
-    /** {@inheritDoc} */
-    @Override @NotNull protected Collection<Dependency> getDependencies(String igniteVer) {
-        Collection<Dependency> dependencies = super.getDependencies(igniteVer);
-
-        dependencies.add(new Dependency("h2", "com.h2database", "h2", "1.4.195", false));
-
-        return dependencies;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected Set<String> getExcluded(String ver, Collection<Dependency> dependencies) {
-        Set<String> excluded = super.getExcluded(ver, dependencies);
-
-        excluded.add("h2");
-
-        return excluded;
-    }
+    private static final String TABLE_NAME = "TEST_IDX_TABLE";
 
     /**
      * Tests opportunity to read data from previous Ignite DB version.
@@ -97,28 +58,30 @@ public class IgnitePKIndexesMigrationToUnwrapPkTest extends IgnitePersistenceCom
      * @param ver 3-digits version of ignite
      * @throws Exception If failed.
      */
-    @SuppressWarnings("unchecked")
     private void doTestStartupWithOldVersion(String ver) throws Exception {
         try {
-            startGrid(1, ver, new ConfigurationClosure(), new PostStartupClosure(true));
+            startGrid(1, ver, new PersistenceBasicCompatibilityTest.ConfigurationClosure(true),
+                new PostStartupClosure());
 
             stopAllGrids();
 
             IgniteEx igniteEx = startGrid(0);
 
-            new PostStartupClosure(false).apply(igniteEx);
-
-            igniteEx.active(true);
+            igniteEx.cluster().state(ClusterState.ACTIVE);
 
             assertDontUsingPkIndex(igniteEx, TABLE_NAME);
 
+            assertQueryWorks(igniteEx, TABLE_NAME);
+
             String newTblName = TABLE_NAME + "_NEW";
 
             initializeTable(igniteEx, newTblName);
 
             checkUsingIndexes(igniteEx, newTblName);
 
-            igniteEx.active(false);
+            assertQueryWorks(igniteEx, newTblName);
+
+            igniteEx.cluster().state(ClusterState.INACTIVE);
         }
         finally {
             stopAllGrids();
@@ -127,25 +90,13 @@ public class IgnitePKIndexesMigrationToUnwrapPkTest extends IgnitePersistenceCom
 
     /** */
     private static class PostStartupClosure implements IgniteInClosure<Ignite> {
-
-        /** */
-        boolean createTable;
-
-        /**
-         * @param createTable {@code true} In case table should be created
-         */
-        public PostStartupClosure(boolean createTable) {
-            this.createTable = createTable;
-        }
-
         /** {@inheritDoc} */
         @Override public void apply(Ignite ignite) {
             ignite.active(true);
 
             IgniteEx igniteEx = (IgniteEx)ignite;
 
-            if (createTable)
-                initializeTable(igniteEx, TABLE_NAME);
+            initializeTable(igniteEx, TABLE_NAME);
 
             assertDontUsingPkIndex(igniteEx, TABLE_NAME);
 
@@ -157,7 +108,7 @@ public class IgnitePKIndexesMigrationToUnwrapPkTest extends IgnitePersistenceCom
      * @param igniteEx Ignite instance.
      * @param tblName Table name.
      */
-    @NotNull private static void initializeTable(IgniteEx igniteEx, String tblName) {
+    private static void initializeTable(IgniteEx igniteEx, String tblName) {
         executeSql(igniteEx, "CREATE TABLE " + tblName + " (id int, name varchar, age int, company varchar, city varchar, " +
             "primary key (id, name, city)) WITH \"affinity_key=name\"");
 
@@ -167,18 +118,6 @@ public class IgnitePKIndexesMigrationToUnwrapPkTest extends IgnitePersistenceCom
             executeSql(igniteEx, "INSERT INTO " + tblName + " (id, name, age, company, city) VALUES(?,'name',2,'company', 'city')", i);
     }
 
-    /**
-     * Run SQL statement on specified node.
-     *
-     * @param node node to execute query.
-     * @param stmt Statement to run.
-     * @param args arguments of statements
-     * @return Run result.
-     */
-    private static List<List<?>> executeSql(IgniteEx node, String stmt, Object... args) {
-        return node.context().query().querySqlFields(new SqlFieldsQuery(stmt).setArgs(args), true).getAll();
-    }
-
     /**
      * Check using PK indexes for few cases.
      *
@@ -207,7 +146,7 @@ public class IgnitePKIndexesMigrationToUnwrapPkTest extends IgnitePersistenceCom
      * @param results Result list of explain of query.
      */
     private static void assertUsingPkIndex(List<List<?>> results) {
-        assertEquals(2, results.size());
+        assertFalse(results.isEmpty());
 
         String explainPlan = (String)results.get(0).get(0);
 
@@ -225,7 +164,7 @@ public class IgnitePKIndexesMigrationToUnwrapPkTest extends IgnitePersistenceCom
     private static void assertDontUsingPkIndex(IgniteEx igniteEx, String tblName) {
         List<List<?>> results = executeSql(igniteEx, "explain SELECT * FROM " + tblName + " WHERE id=1");
 
-        assertEquals(2, results.size());
+        assertFalse(results.isEmpty());
 
         String explainPlan = (String)results.get(0).get(0);
 
@@ -236,33 +175,16 @@ public class IgnitePKIndexesMigrationToUnwrapPkTest extends IgnitePersistenceCom
         assertTrue(explainPlan, explainPlan.contains("_SCAN_"));
     }
 
-    /** */
-    private static class ConfigurationClosure implements IgniteInClosure<IgniteConfiguration> {
-        /** {@inheritDoc} */
-        @Override public void apply(IgniteConfiguration cfg) {
-            cfg.setLocalHost("127.0.0.1");
-
-            TcpDiscoverySpi disco = new TcpDiscoverySpi();
-            disco.setIpFinder(GridCacheAbstractFullApiSelfTest.LOCAL_IP_FINDER);
-
-            cfg.setDiscoverySpi(disco);
-
-            cfg.setPeerClassLoadingEnabled(false);
-
-            DataStorageConfiguration storageCfg = new DataStorageConfiguration()
-                .setDefaultDataRegionConfiguration(
-                    new DataRegionConfiguration()
-                            .setPersistenceEnabled(true)
-                            .setInitialSize(10 * 1024 * 1024)
-                            .setMaxSize(15 * 1024 * 1024)
-                )
-                .setSystemDataRegionConfiguration(
-                    new SystemDataRegionConfiguration()
-                            .setInitialSize(10 * 1024 * 1024)
-                            .setMaxSize(15 * 1024 * 1024)
-                );
-
-            cfg.setDataStorageConfiguration(storageCfg);
-        }
+    /**
+     * Check that query on table works correctly.
+     *
+     * @param igniteEx Ignite instance.
+     * @param tblName Name of table.
+     */
+    private static void assertQueryWorks(IgniteEx igniteEx, String tblName) {
+        List<List<?>> results = executeSql(igniteEx, "SELECT * FROM " + tblName + " WHERE id=1");
+
+        assertEquals(1, results.size());
+        assertEquals(1, results.get(0).get(0));
     }
 }
diff --git a/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/IndexAbstractCompatibilityTest.java b/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/IndexAbstractCompatibilityTest.java
index bed0637729a..17b6b8387f1 100644
--- a/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/IndexAbstractCompatibilityTest.java
+++ b/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/IndexAbstractCompatibilityTest.java
@@ -19,15 +19,19 @@
 package org.apache.ignite.compatibility.persistence;
 
 import java.util.Collection;
+import java.util.List;
 import java.util.Set;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.compatibility.IgniteReleasedVersion;
 import org.apache.ignite.compatibility.testframework.junits.Dependency;
 import org.apache.ignite.configuration.BinaryConfiguration;
 import org.apache.ignite.configuration.DataRegionConfiguration;
 import org.apache.ignite.configuration.DataStorageConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.configuration.WALMode;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.lang.IgniteProductVersion;
 import org.jetbrains.annotations.NotNull;
 
 /**
@@ -62,7 +66,7 @@ public abstract class IndexAbstractCompatibilityTest extends IgnitePersistenceCo
     @Override @NotNull protected Collection<Dependency> getDependencies(String igniteVer) {
         Collection<Dependency> dependencies = super.getDependencies(igniteVer);
 
-        if ("2.6.0".equals(igniteVer)) {
+        if (IgniteProductVersion.fromString(igniteVer).compareTo(IgniteReleasedVersion.VER_2_7_0.version()) < 0) {
             dependencies.add(new Dependency("h2", "com.h2database", "h2", "1.4.195", false));
 
             dependencies.add(new Dependency("h2", "org.apache.lucene", "lucene-core", "5.5.2", false));
@@ -79,7 +83,7 @@ public abstract class IndexAbstractCompatibilityTest extends IgnitePersistenceCo
     @Override protected Set<String> getExcluded(String ver, Collection<Dependency> dependencies) {
         Set<String> excluded = super.getExcluded(ver, dependencies);
 
-        if ("2.6.0".equals(ver))
+        if (IgniteProductVersion.fromString(ver).compareTo(IgniteReleasedVersion.VER_2_7_0.version()) < 0)
             excluded.add("h2");
 
         return excluded;
@@ -89,4 +93,16 @@ public abstract class IndexAbstractCompatibilityTest extends IgnitePersistenceCo
     protected void checkIndexUsed(IgniteCache<?, ?> cache, SqlFieldsQuery qry, String idxName) {
         assertTrue("Query does not use index.", queryPlan(cache, qry).toLowerCase().contains(idxName.toLowerCase()));
     }
+
+    /**
+     * Run SQL statement on specified node.
+     *
+     * @param node node to execute query.
+     * @param stmt Statement to run.
+     * @param args arguments of statements
+     * @return Run result.
+     */
+    protected static List<List<?>> executeSql(IgniteEx node, String stmt, Object... args) {
+        return node.context().query().querySqlFields(new SqlFieldsQuery(stmt).setArgs(args), true).getAll();
+    }
 }
diff --git a/modules/compatibility/src/test/java/org/apache/ignite/compatibility/testsuites/IgniteCompatibilityBasicTestSuite.java b/modules/compatibility/src/test/java/org/apache/ignite/compatibility/testsuites/IgniteCompatibilityBasicTestSuite.java
index c155d1979cf..ee0f1d09ba9 100644
--- a/modules/compatibility/src/test/java/org/apache/ignite/compatibility/testsuites/IgniteCompatibilityBasicTestSuite.java
+++ b/modules/compatibility/src/test/java/org/apache/ignite/compatibility/testsuites/IgniteCompatibilityBasicTestSuite.java
@@ -19,7 +19,9 @@ package org.apache.ignite.compatibility.testsuites;
 
 import org.apache.ignite.compatibility.clients.JavaThinCompatibilityTest;
 import org.apache.ignite.compatibility.clients.JdbcThinCompatibilityTest;
+import org.apache.ignite.compatibility.persistence.CompoundIndexCompatibilityTest;
 import org.apache.ignite.compatibility.persistence.FoldersReuseCompatibilityTest;
+import org.apache.ignite.compatibility.persistence.IgnitePKIndexesMigrationToUnwrapPkTest;
 import org.apache.ignite.compatibility.persistence.IndexTypesCompatibilityTest;
 import org.apache.ignite.compatibility.persistence.InlineJavaObjectCompatibilityTest;
 import org.apache.ignite.compatibility.persistence.MetaStorageCompatibilityTest;
@@ -42,7 +44,9 @@ import org.junit.runners.Suite;
     MetaStorageCompatibilityTest.class,
     MoveBinaryMetadataCompatibility.class,
     JdbcThinCompatibilityTest.class,
-    JavaThinCompatibilityTest.class
+    JavaThinCompatibilityTest.class,
+    IgnitePKIndexesMigrationToUnwrapPkTest.class,
+    CompoundIndexCompatibilityTest.class,
 })
 public class IgniteCompatibilityBasicTestSuite {
 }
diff --git a/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerBrokenIndexTest.java b/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerBrokenIndexTest.java
index 1f8f996910e..e4a7c441bcc 100644
--- a/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerBrokenIndexTest.java
+++ b/modules/control-utility/src/test/java/org/apache/ignite/util/GridCommandHandlerBrokenIndexTest.java
@@ -152,7 +152,8 @@ public class GridCommandHandlerBrokenIndexTest extends GridCommandHandlerCluster
                     IndexName idxName = idxDef.idxName();
 
                     SortedIndexDefinition badIdxDef = new QueryIndexDefinition(
-                        idxDef.rowDescriptor(),
+                        idxDef.typeDescriptor(),
+                        idxDef.cacheInfo(),
                         new IndexName(idxName.cacheName(), idxName.schemaName(), idxName.tableName(), IDX_NAME),
                         idxDef.treeName(),
                         idxDef.idxRowCache(),
diff --git a/modules/control-utility/src/test/java/org/apache/ignite/util/SystemViewCommandTest.java b/modules/control-utility/src/test/java/org/apache/ignite/util/SystemViewCommandTest.java
index 230565d864c..5f621e27fe0 100644
--- a/modules/control-utility/src/test/java/org/apache/ignite/util/SystemViewCommandTest.java
+++ b/modules/control-utility/src/test/java/org/apache/ignite/util/SystemViewCommandTest.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.util;
 
 import java.lang.reflect.Field;
+import java.net.InetSocketAddress;
 import java.sql.Connection;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -74,7 +75,6 @@ import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.transactions.Transaction;
 import org.junit.Test;
 
-import static java.lang.Integer.MAX_VALUE;
 import static java.util.Arrays.asList;
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
 import static java.util.regex.Pattern.quote;
@@ -108,11 +108,11 @@ import static org.apache.ignite.internal.processors.pool.PoolProcessor.STREAM_PO
 import static org.apache.ignite.internal.processors.pool.PoolProcessor.SYS_POOL_QUEUE_VIEW;
 import static org.apache.ignite.internal.processors.query.QueryUtils.DFLT_SCHEMA;
 import static org.apache.ignite.internal.processors.query.QueryUtils.SCHEMA_SYS;
-import static org.apache.ignite.internal.processors.query.h2.SchemaManager.SQL_SCHEMA_VIEW;
-import static org.apache.ignite.internal.processors.query.h2.SchemaManager.SQL_TBLS_VIEW;
-import static org.apache.ignite.internal.processors.query.h2.SchemaManager.SQL_TBL_COLS_VIEW;
-import static org.apache.ignite.internal.processors.query.h2.SchemaManager.SQL_VIEWS_VIEW;
-import static org.apache.ignite.internal.processors.query.h2.SchemaManager.SQL_VIEW_COLS_VIEW;
+import static org.apache.ignite.internal.processors.query.schema.management.SchemaManager.SQL_SCHEMA_VIEW;
+import static org.apache.ignite.internal.processors.query.schema.management.SchemaManager.SQL_TBLS_VIEW;
+import static org.apache.ignite.internal.processors.query.schema.management.SchemaManager.SQL_TBL_COLS_VIEW;
+import static org.apache.ignite.internal.processors.query.schema.management.SchemaManager.SQL_VIEWS_VIEW;
+import static org.apache.ignite.internal.processors.query.schema.management.SchemaManager.SQL_VIEW_COLS_VIEW;
 import static org.apache.ignite.internal.processors.service.IgniteServiceProcessor.SVCS_VIEW;
 import static org.apache.ignite.internal.processors.task.GridTaskProcessor.TASKS_VIEW;
 import static org.apache.ignite.internal.util.IgniteUtils.toStringSafe;
@@ -514,7 +514,7 @@ public class SystemViewCommandTest extends GridCommandHandlerClusterByClassAbstr
     public void testTableColumns() {
         assertTrue(systemView(ignite0, SQL_TBL_COLS_VIEW).isEmpty());
 
-        executeSql(ignite0, "CREATE TABLE T1(ID LONG PRIMARY KEY, NAME VARCHAR(40))");
+        executeSql(ignite0, "CREATE TABLE T1(ID LONG PRIMARY KEY, NAME VARCHAR(40) DEFAULT 'name')");
 
         Set<?> actCols = systemView(ignite0, SQL_TBL_COLS_VIEW).stream()
             .map(row -> row.get(0)) // columnName
@@ -524,15 +524,17 @@ public class SystemViewCommandTest extends GridCommandHandlerClusterByClassAbstr
 
         executeSql(ignite0, "CREATE TABLE T2(ID LONG PRIMARY KEY, NAME VARCHAR(50))");
 
+        // Columns order: COLUMN_NAME, TABLE_NAME, SCHEMA_NAME, AFFINITY_COLUMN, AUTO_INCREMENT, DEFAULT_VALUE,
+        // NULLABLE, PK, PRECESION, SCALE, TYPE.
         Set<List<String>> expSqlTableColumnsView = new HashSet<>(asList(
-            asList("ID", "T1", "PUBLIC", "false", "false", "null", "true", "true", "-1", "-1", Long.class.getName()),
-            asList("NAME", "T1", "PUBLIC", "false", "false", "null", "true", "false", "40", "-1", String.class.getName()),
-            asList("_KEY", "T1", "PUBLIC", "true", "false", "null", "false", "true", "-1", "-1", "null"),
-            asList("_VAL", "T1", "PUBLIC", "false", "false", "null", "true", "false", "-1", "-1", "null"),
-            asList("ID", "T2", "PUBLIC", "false", "false", "null", "true", "true", "-1", "-1", Long.class.getName()),
+            asList("ID", "T1", "PUBLIC", "true", "false", "null", "true", "true", "-1", "-1", Long.class.getName()),
+            asList("NAME", "T1", "PUBLIC", "false", "false", "name", "true", "false", "40", "-1", String.class.getName()),
+            asList("_KEY", "T1", "PUBLIC", "true", "false", "null", "false", "true", "-1", "-1", Long.class.getName()),
+            asList("_VAL", "T1", "PUBLIC", "false", "false", "null", "false", "false", "-1", "-1", Object.class.getName()),
+            asList("ID", "T2", "PUBLIC", "true", "false", "null", "true", "true", "-1", "-1", Long.class.getName()),
             asList("NAME", "T2", "PUBLIC", "false", "false", "null", "true", "false", "50", "-1", String.class.getName()),
-            asList("_KEY", "T2", "PUBLIC", "true", "false", "null", "false", "true", "-1", "-1", "null"),
-            asList("_VAL", "T2", "PUBLIC", "false", "false", "null", "true", "false", "-1", "-1", "null")
+            asList("_KEY", "T2", "PUBLIC", "true", "false", "null", "false", "true", "-1", "-1", Long.class.getName()),
+            asList("_VAL", "T2", "PUBLIC", "false", "false", "null", "false", "false", "-1", "-1", Object.class.getName())
         ));
 
         Set<List<String>> sqlTableColumnsView = new HashSet<>(systemView(ignite0, SQL_TBL_COLS_VIEW));
@@ -548,18 +550,16 @@ public class SystemViewCommandTest extends GridCommandHandlerClusterByClassAbstr
     /** */
     @Test
     public void testViewColumns() {
+        // Columns order: COLUMN_NAME, VIEW_NAME, SCHEMA_NAME, DEFAULT_VALUE, NULLABLE, PRECESION, SCALE, TYPE.
         Set<List<String>> expsqlViewColumnsView = new HashSet<>(asList(
-            asList("CONNECTION_ID", "CLIENT_CONNECTIONS", SCHEMA_SYS, "null", "true", "19", "0", Long.class.getName()),
-            asList("LOCAL_ADDRESS", "CLIENT_CONNECTIONS", SCHEMA_SYS, "null", "true", Integer.toString(MAX_VALUE), "0",
-                String.class.getName()),
-            asList("REMOTE_ADDRESS", "CLIENT_CONNECTIONS", SCHEMA_SYS, "null", "true", Integer.toString(MAX_VALUE), "0",
-                String.class.getName()),
-            asList("TYPE", "CLIENT_CONNECTIONS", SCHEMA_SYS, "null", "true", Integer.toString(MAX_VALUE), "0",
-                String.class.getName()),
-            asList("USER", "CLIENT_CONNECTIONS", SCHEMA_SYS, "null", "true", Integer.toString(MAX_VALUE), "0",
-                String.class.getName()),
-            asList("VERSION", "CLIENT_CONNECTIONS", SCHEMA_SYS, "null", "true", Integer.toString(MAX_VALUE), "0",
-                String.class.getName())
+            asList("CONNECTION_ID", "CLIENT_CONNECTIONS", SCHEMA_SYS, "null", "false", "-1", "-1", long.class.getName()),
+            asList("LOCAL_ADDRESS", "CLIENT_CONNECTIONS", SCHEMA_SYS, "null", "true", "-1", "-1",
+                InetSocketAddress.class.getName()),
+            asList("REMOTE_ADDRESS", "CLIENT_CONNECTIONS", SCHEMA_SYS, "null", "true", "-1", "-1",
+                InetSocketAddress.class.getName()),
+            asList("TYPE", "CLIENT_CONNECTIONS", SCHEMA_SYS, "null", "true", "-1", "-1", String.class.getName()),
+            asList("USER", "CLIENT_CONNECTIONS", SCHEMA_SYS, "null", "true", "-1", "-1", String.class.getName()),
+            asList("VERSION", "CLIENT_CONNECTIONS", SCHEMA_SYS, "null", "true", "-1", "-1", String.class.getName())
         ));
 
         Set<List<String>> sqlViewColumnsView = systemView(ignite0, SQL_VIEW_COLS_VIEW).stream()
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexProcessor.java
index ef5b2a70b14..9c1086e0cce 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/IndexProcessor.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.cache.query.index;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.UUID;
@@ -29,6 +30,7 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyTypeSettings;
 import org.apache.ignite.internal.cache.query.index.sorted.IndexRow;
 import org.apache.ignite.internal.cache.query.index.sorted.IndexRowCache;
 import org.apache.ignite.internal.cache.query.index.sorted.IndexRowCacheRegistry;
@@ -61,6 +63,7 @@ import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
 import org.apache.ignite.internal.processors.cache.persistence.tree.reuse.ReuseList;
 import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
 import org.apache.ignite.internal.processors.query.IgniteSQLException;
+import org.apache.ignite.internal.processors.query.QueryUtils;
 import org.apache.ignite.internal.processors.query.schema.IndexRebuildCancelToken;
 import org.apache.ignite.internal.processors.query.schema.SchemaIndexCacheVisitor;
 import org.apache.ignite.internal.util.GridAtomicLong;
@@ -105,6 +108,9 @@ public class IndexProcessor extends GridProcessorAdapter {
     /** Row cache. */
     private final IndexRowCacheRegistry idxRowCacheRegistry = new IndexRowCacheRegistry();
 
+    /** Default key type settings. */
+    private final IndexKeyTypeSettings keyTypeSettings = new IndexKeyTypeSettings();
+
     /**
      * Registry of all indexes. High key is a cache name, lower key is an unique index name.
      */
@@ -259,11 +265,10 @@ public class IndexProcessor extends GridProcessorAdapter {
     /**
      * Removes an index.
      *
-     * @param cctx Cache context.
      * @param idxName Index name.
      * @param softDelete whether it's required to delete underlying structures.
      */
-    public void removeIndex(GridCacheContext<?, ?> cctx, IndexName idxName, boolean softDelete) {
+    public void removeIndex(IndexName idxName, boolean softDelete) {
         ddlLock.writeLock().lock();
 
         try {
@@ -535,6 +540,13 @@ public class IndexProcessor extends GridProcessorAdapter {
         tree.destroy();
     }
 
+    /**
+     * @return Default key type settings.
+     */
+    public IndexKeyTypeSettings keyTypeSettings() {
+        return keyTypeSettings;
+    }
+
     /**
      * @return {@code true} In case of use an unwrapped PK for the index.
      */
@@ -613,4 +625,38 @@ public class IndexProcessor extends GridProcessorAdapter {
     public IgniteLogger logger() {
         return log;
     }
+
+    /**
+     * Information about secondary indexes efficient (actual) inline size.
+     *
+     * @return Map with inline sizes. The key of entry is a full index name (with schema and table name), the value of
+     * entry is a inline size.
+     */
+    public Map<String, Integer> secondaryIndexesInlineSize() {
+        Map<String, Integer> map = new HashMap<>();
+
+        ddlLock.readLock().lock();
+
+        try {
+            for (Map<String, Index> idxs : cacheToIdx.values()) {
+                for (Index idx : idxs.values()) {
+                    if (idx instanceof InlineIndex && !QueryUtils.PRIMARY_KEY_INDEX.equals(idx.name())) {
+                        InlineIndex idx0 = (InlineIndex)idx;
+                        IndexDefinition idxDef = indexDefinition(idx.id());
+                        IndexName idxName = idxDef.idxName();
+
+                        map.put(
+                            idxName.schemaName() + "#" + idxName.tableName() + "#" + idxName.idxName(),
+                            idx0.inlineSize()
+                        );
+                    }
+                }
+            }
+        }
+        finally {
+            ddlLock.readLock().unlock();
+        }
+
+        return map;
+    }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/QueryIndexDefinition.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/QueryIndexDefinition.java
index 6be3d5f8e69..626727acb84 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/QueryIndexDefinition.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/QueryIndexDefinition.java
@@ -19,7 +19,8 @@ package org.apache.ignite.internal.cache.query.index.sorted;
 
 import java.util.LinkedHashMap;
 import org.apache.ignite.internal.cache.query.index.IndexName;
-import org.apache.ignite.internal.processors.query.GridQueryRowDescriptor;
+import org.apache.ignite.internal.processors.cache.GridCacheContextInfo;
+import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
 
 /**
  * Define query index.
@@ -28,8 +29,11 @@ public class QueryIndexDefinition implements SortedIndexDefinition {
     /** Wrapped key definitions. */
     private final LinkedHashMap<String, IndexKeyDefinition> keyDefs;
 
-    /** Table row descriptor. */
-    private final GridQueryRowDescriptor rowDescriptor;
+    /** Type descriptor. */
+    private final GridQueryTypeDescriptor typeDesc;
+
+    /** Cache info. */
+    private final GridCacheContextInfo<?, ?> cacheInfo;
 
     /** Index name. */
     private final IndexName idxName;
@@ -63,7 +67,8 @@ public class QueryIndexDefinition implements SortedIndexDefinition {
 
     /** */
     public QueryIndexDefinition(
-        GridQueryRowDescriptor rowDescriptor,
+        GridQueryTypeDescriptor typeDesc,
+        GridCacheContextInfo<?, ?> cacheInfo,
         IndexName idxName,
         String treeName,
         IndexRowCache idxRowCache,
@@ -73,11 +78,12 @@ public class QueryIndexDefinition implements SortedIndexDefinition {
         int cfgInlineSize,
         IndexKeyTypeSettings keyTypeSettings
     ) {
-        this.rowDescriptor = rowDescriptor;
+        this.typeDesc = typeDesc;
+        this.cacheInfo = cacheInfo;
         this.idxName = idxName;
         this.treeName = treeName;
         this.idxRowCache = idxRowCache;
-        this.segments = rowDescriptor.context().config().getQueryParallelism();
+        this.segments = cacheInfo.cacheContext().config().getQueryParallelism();
         this.inlineSize = cfgInlineSize;
         this.isPrimary = isPrimary;
         this.isAffinity = isAffinity;
@@ -142,8 +148,13 @@ public class QueryIndexDefinition implements SortedIndexDefinition {
     }
 
     /** {@inheritDoc} */
-    @Override public GridQueryRowDescriptor rowDescriptor() {
-        return rowDescriptor;
+    @Override public GridQueryTypeDescriptor typeDescriptor() {
+        return typeDesc;
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridCacheContextInfo<?, ?> cacheInfo() {
+        return cacheInfo;
     }
 
     /** {@inheritDoc} */
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/QueryIndexRowHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/QueryIndexRowHandler.java
index 7db7f0b1166..7a7d4e69dbe 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/QueryIndexRowHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/QueryIndexRowHandler.java
@@ -29,16 +29,16 @@ import org.apache.ignite.internal.cache.query.index.sorted.keys.IndexKey;
 import org.apache.ignite.internal.cache.query.index.sorted.keys.IndexKeyFactory;
 import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.CacheObjectContext;
+import org.apache.ignite.internal.processors.cache.GridCacheContextInfo;
 import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
 import org.apache.ignite.internal.processors.query.GridQueryProperty;
-import org.apache.ignite.internal.processors.query.GridQueryRowDescriptor;
 import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
 import org.apache.ignite.internal.processors.query.QueryUtils;
 
 /** Maps CacheDataRow to IndexRow using columns references. */
 public class QueryIndexRowHandler implements InlineIndexRowHandler {
-    /** Cache descriptor. */
-    private final GridQueryRowDescriptor rowDescriptor;
+    /** Cache info. */
+    private final GridCacheContextInfo<?, ?> cacheInfo;
 
     /** List of key types for inlined index keys. */
     private final List<InlineIndexKeyType> keyTypes;
@@ -54,7 +54,8 @@ public class QueryIndexRowHandler implements InlineIndexRowHandler {
 
     /** */
     public QueryIndexRowHandler(
-        GridQueryRowDescriptor rowDescriptor,
+        GridQueryTypeDescriptor type,
+        GridCacheContextInfo<?, ?> cacheInfo,
         LinkedHashMap<String, IndexKeyDefinition> keyDefs,
         List<InlineIndexKeyType> keyTypes,
         IndexKeyTypeSettings keyTypeSettings
@@ -64,7 +65,6 @@ public class QueryIndexRowHandler implements InlineIndexRowHandler {
 
         props = new GridQueryProperty[keyDefs.size()];
         int propIdx = 0;
-        GridQueryTypeDescriptor type = rowDescriptor.type();
 
         for (String propName : keyDefs.keySet()) {
             GridQueryProperty prop;
@@ -83,7 +83,7 @@ public class QueryIndexRowHandler implements InlineIndexRowHandler {
             props[propIdx++] = prop;
         }
 
-        this.rowDescriptor = rowDescriptor;
+        this.cacheInfo = cacheInfo;
         this.keyTypeSettings = keyTypeSettings;
     }
 
@@ -93,7 +93,7 @@ public class QueryIndexRowHandler implements InlineIndexRowHandler {
             return IndexKeyFactory.wrap(
                 props[idx].value(row.key(), row.value()),
                 keyDefs.get(idx).idxType(),
-                rowDescriptor.context().cacheObjectContext(),
+                cacheInfo.cacheContext().cacheObjectContext(),
                 keyTypeSettings
             );
         }
@@ -121,7 +121,7 @@ public class QueryIndexRowHandler implements InlineIndexRowHandler {
     @Override public int partition(CacheDataRow row) {
         Object key = unwrap(row.key());
 
-        return rowDescriptor.context().affinity().partition(key);
+        return cacheInfo.cacheContext().affinity().partition(key);
     }
 
     /** {@inheritDoc} */
@@ -141,7 +141,7 @@ public class QueryIndexRowHandler implements InlineIndexRowHandler {
         if (o != null)
             return o;
 
-        CacheObjectContext coctx = rowDescriptor.context().cacheObjectContext();
+        CacheObjectContext coctx = cacheInfo.cacheContext().cacheObjectContext();
 
         return val.value(coctx, false);
     }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/QueryIndexRowHandlerFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/QueryIndexRowHandlerFactory.java
index dbdacc06253..1a0224c174e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/QueryIndexRowHandlerFactory.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/QueryIndexRowHandlerFactory.java
@@ -32,6 +32,6 @@ public class QueryIndexRowHandlerFactory implements InlineIndexRowHandlerFactory
 
         List<InlineIndexKeyType> keyTypes = InlineIndexKeyTypeRegistry.types(keyDefs.values(), keyTypeSettings);
 
-        return new QueryIndexRowHandler(sdef.rowDescriptor(), keyDefs, keyTypes, keyTypeSettings);
+        return new QueryIndexRowHandler(sdef.typeDescriptor(), sdef.cacheInfo(), keyDefs, keyTypes, keyTypeSettings);
     }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/SortedIndexDefinition.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/SortedIndexDefinition.java
index e38bdcc3399..5324df13e6d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/SortedIndexDefinition.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/SortedIndexDefinition.java
@@ -18,7 +18,8 @@
 package org.apache.ignite.internal.cache.query.index.sorted;
 
 import org.apache.ignite.internal.cache.query.index.IndexDefinition;
-import org.apache.ignite.internal.processors.query.GridQueryRowDescriptor;
+import org.apache.ignite.internal.processors.cache.GridCacheContextInfo;
+import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
 
 /**
  * Represents a definition of a sorted index.
@@ -39,8 +40,11 @@ public interface SortedIndexDefinition extends IndexDefinition {
     /** Cache of index rows. */
     public IndexRowCache idxRowCache();
 
-    /** Table row descriptor. */
-    public GridQueryRowDescriptor rowDescriptor();
+    /** Type descriptor. */
+    public GridQueryTypeDescriptor typeDescriptor();
+
+    /** Cache info. */
+    public GridCacheContextInfo<?, ?> cacheInfo();
 
     /** Amount of index tree segments.*/
     public int segments();
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineIndexImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineIndexImpl.java
index a437e40e32d..bbabaf91d6e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineIndexImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineIndexImpl.java
@@ -21,6 +21,8 @@ import java.util.Comparator;
 import java.util.PriorityQueue;
 import java.util.UUID;
 import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.failure.FailureContext;
@@ -74,6 +76,9 @@ public class InlineIndexImpl extends AbstractIndex implements InlineIndex {
     /** Row handler. */
     private final InlineIndexRowHandler rowHnd;
 
+    /** */
+    private final ReadWriteLock lock = new ReentrantReadWriteLock();
+
     /** Constructor. */
     public InlineIndexImpl(GridCacheContext<?, ?> cctx, SortedIndexDefinition def, InlineIndexTree[] segments,
         IoStatisticsHolderIndex stats) {
@@ -96,17 +101,24 @@ public class InlineIndexImpl extends AbstractIndex implements InlineIndex {
     ) throws IgniteCheckedException {
         InlineTreeFilterClosure closure = filterClosure(qryCtx);
 
-        // If it is known that only one row will be returned an optimization is employed
-        if (isSingleRowLookup(lower, upper)) {
-            IndexRowImpl row = segments[segment].findOne(lower, closure, null);
+        lock.readLock().lock();
 
-            if (row == null || isExpired(row))
-                return IndexValueCursor.EMPTY;
+        try {
+            // If it is known that only one row will be returned an optimization is employed
+            if (isSingleRowLookup(lower, upper)) {
+                IndexRowImpl row = segments[segment].findOne(lower, closure, null);
 
-            return new SingleCursor<>(row);
-        }
+                if (row == null || isExpired(row))
+                    return IndexValueCursor.EMPTY;
+
+                return new SingleCursor<>(row);
+            }
 
-        return segments[segment].find(lower, upper, lowIncl, upIncl, closure, null);
+            return segments[segment].find(lower, upper, lowIncl, upIncl, closure, null);
+        }
+        finally {
+            lock.readLock().unlock();
+        }
     }
 
     /** {@inheritDoc} */
@@ -117,27 +129,48 @@ public class InlineIndexImpl extends AbstractIndex implements InlineIndex {
         boolean upIncl,
         IndexQueryContext qryCtx
     ) throws IgniteCheckedException {
-        int segmentsCnt = segmentsCount();
+        lock.readLock().lock();
 
-        if (segmentsCnt == 1)
-            return find(lower, upper, lowIncl, upIncl, 0, qryCtx);
+        try {
+            int segmentsCnt = segmentsCount();
 
-        final GridCursor<IndexRow>[] segmentCursors = new GridCursor[segmentsCnt];
+            if (segmentsCnt == 1)
+                return find(lower, upper, lowIncl, upIncl, 0, qryCtx);
 
-        for (int i = 0; i < segmentsCnt; i++)
-            segmentCursors[i] = find(lower, upper, lowIncl, upIncl, i, qryCtx);
+            final GridCursor<IndexRow>[] segmentCursors = new GridCursor[segmentsCnt];
 
-        return new SegmentedIndexCursor(segmentCursors, def);
+            for (int i = 0; i < segmentsCnt; i++)
+                segmentCursors[i] = find(lower, upper, lowIncl, upIncl, i, qryCtx);
+
+            return new SegmentedIndexCursor(segmentCursors, def);
+        }
+        finally {
+            lock.readLock().unlock();
+        }
     }
 
     /** {@inheritDoc} */
     @Override public long count(int segment) throws IgniteCheckedException {
-        return segments[segment].size();
+        lock.readLock().lock();
+
+        try {
+            return segments[segment].size();
+        }
+        finally {
+            lock.readLock().unlock();
+        }
     }
 
     /** {@inheritDoc} */
     @Override public long count(int segment, IndexQueryContext qryCtx) throws IgniteCheckedException {
-        return segments[segment].size(filterClosure(qryCtx));
+        lock.readLock().lock();
+
+        try {
+            return segments[segment].size(filterClosure(qryCtx));
+        }
+        finally {
+            lock.readLock().unlock();
+        }
     }
 
     /**
@@ -147,12 +180,19 @@ public class InlineIndexImpl extends AbstractIndex implements InlineIndex {
      * @throws IgniteCheckedException If failed.
      */
     @Override public long totalCount() throws IgniteCheckedException {
-        long ret = 0;
+        lock.readLock().lock();
+
+        try {
+            long ret = 0;
 
-        for (int i = 0; i < segmentsCount(); i++)
-            ret += segments[i].size();
+            for (int i = 0; i < segmentsCount(); i++)
+                ret += segments[i].size();
 
-        return ret;
+            return ret;
+        }
+        finally {
+            lock.readLock().unlock();
+        }
     }
 
     /** */
@@ -215,24 +255,38 @@ public class InlineIndexImpl extends AbstractIndex implements InlineIndex {
     @Override public GridCursor<IndexRow> findFirst(int segment, IndexQueryContext qryCtx) throws IgniteCheckedException {
         InlineTreeFilterClosure closure = filterClosure(qryCtx);
 
-        IndexRow found = segments[segment].findFirst(closure);
+        lock.readLock().lock();
 
-        if (found == null || isExpired(found))
-            return IndexValueCursor.EMPTY;
+        try {
+            IndexRow found = segments[segment].findFirst(closure);
 
-        return new SingleCursor<>(found);
+            if (found == null || isExpired(found))
+                return IndexValueCursor.EMPTY;
+
+            return new SingleCursor<>(found);
+        }
+        finally {
+            lock.readLock().unlock();
+        }
     }
 
     /** {@inheritDoc} */
     @Override public GridCursor<IndexRow> findLast(int segment, IndexQueryContext qryCtx) throws IgniteCheckedException {
         InlineTreeFilterClosure closure = filterClosure(qryCtx);
 
-        IndexRow found = segments[segment].findLast(closure);
+        lock.readLock().lock();
+
+        try {
+            IndexRow found = segments[segment].findLast(closure);
 
-        if (found == null || isExpired(found))
-            return IndexValueCursor.EMPTY;
+            if (found == null || isExpired(found))
+                return IndexValueCursor.EMPTY;
 
-        return new SingleCursor<>(found);
+            return new SingleCursor<>(found);
+        }
+        finally {
+            lock.readLock().unlock();
+        }
     }
 
     /** {@inheritDoc} */
@@ -283,6 +337,8 @@ public class InlineIndexImpl extends AbstractIndex implements InlineIndex {
 
     /** */
     private boolean putx(IndexRowImpl idxRow, int segment, boolean flag) throws IgniteCheckedException {
+        lock.readLock().lock();
+
         try {
             boolean replaced;
 
@@ -302,10 +358,15 @@ public class InlineIndexImpl extends AbstractIndex implements InlineIndex {
 
             throw t;
         }
+        finally {
+            lock.readLock().unlock();
+        }
     }
 
     /** */
     private void remove(CacheDataRow row) throws IgniteCheckedException {
+        lock.readLock().lock();
+
         try {
             int segment = segmentForRow(row);
 
@@ -321,6 +382,9 @@ public class InlineIndexImpl extends AbstractIndex implements InlineIndex {
 
             throw t;
         }
+        finally {
+            lock.readLock().unlock();
+        }
     }
 
     /**
@@ -331,12 +395,15 @@ public class InlineIndexImpl extends AbstractIndex implements InlineIndex {
     public void putIndexRow(IndexRowImpl row) throws IgniteCheckedException {
         int segment = segmentForRow(row.cacheDataRow());
 
+        lock.readLock().lock();
+
         try {
             ThreadLocalRowHandlerHolder.rowHandler(rowHnd);
 
             segments[segment].putx(row);
         }
         finally {
+            lock.readLock().unlock();
             ThreadLocalRowHandlerHolder.clearRowHandler();
         }
     }
@@ -434,10 +501,9 @@ public class InlineIndexImpl extends AbstractIndex implements InlineIndex {
     /** {@inheritDoc} */
     @Override public void destroy(boolean softDel) {
         try {
-            destroy0(softDel, false);
+            destroy0(softDel);
         }
         catch (IgniteCheckedException e) {
-            // Should NEVER happen because renameImmediately is false here, but just in case:
             throw new IgniteException(e);
         }
     }
@@ -446,41 +512,48 @@ public class InlineIndexImpl extends AbstractIndex implements InlineIndex {
      * Destroys the index and if {@code renameImmediately} is {@code true} renames index trees.
      *
      * @param softDel If {@code true} then perform logical deletion.
-     * @param renameImmediately If {@code true} then rename index trees immediately.
      * @throws IgniteCheckedException If failed to rename index trees.
      */
-    public void destroy0(boolean softDel, boolean renameImmediately) throws IgniteCheckedException {
+    private void destroy0(boolean softDel) throws IgniteCheckedException {
         // Already destroyed.
         if (!destroyed.compareAndSet(false, true))
             return;
 
         if (cctx.affinityNode() && !softDel) {
-            for (InlineIndexTree segment : segments) {
-                segment.markDestroyed();
+            lock.writeLock().lock();
 
-                segment.close();
-            }
+            try {
+                for (InlineIndexTree segment : segments) {
+                    segment.markDestroyed();
 
-            cctx.kernalContext().metric().remove(stats.metricRegistryName());
-
-            if (cctx.group().persistenceEnabled() ||
-                cctx.shared().kernalContext().state().clusterState().state() != INACTIVE) {
-                // Actual destroy index task.
-                DurableBackgroundCleanupIndexTreeTaskV2 task = new DurableBackgroundCleanupIndexTreeTaskV2(
-                    cctx.group().name(),
-                    cctx.name(),
-                    def.idxName().idxName(),
-                    treeName,
-                    UUID.randomUUID().toString(),
-                    segments.length,
-                    segments
-                );
-
-                if (renameImmediately) {
-                    task.renameIndexTrees(cctx.group());
+                    segment.close();
                 }
 
-                cctx.kernalContext().durableBackgroundTask().executeAsync(task, cctx.config());
+                cctx.kernalContext().metric().remove(stats.metricRegistryName());
+
+                if (cctx.group().persistenceEnabled() ||
+                    cctx.shared().kernalContext().state().clusterState().state() != INACTIVE) {
+                    // Actual destroy index task.
+                    DurableBackgroundCleanupIndexTreeTaskV2 task = new DurableBackgroundCleanupIndexTreeTaskV2(
+                        cctx.group().name(),
+                        cctx.name(),
+                        def.idxName().idxName(),
+                        treeName,
+                        UUID.randomUUID().toString(),
+                        segments.length,
+                        segments
+                    );
+
+                    // In maintenance mode, durable task is not started immediately (only after restart and activation),
+                    // but to rebuild the index we need to create a new tree, and old tree should be renamed prior to this.
+                    if (cctx.kernalContext().maintenanceRegistry().isMaintenanceMode())
+                        task.renameIndexTrees(cctx.group());
+
+                    cctx.kernalContext().durableBackgroundTask().executeAsync(task, cctx.config());
+                }
+            }
+            finally {
+                lock.writeLock().unlock();
             }
         }
     }
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/maintenance/RebuildIndexAction.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/maintenance/RebuildIndexAction.java
similarity index 50%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/maintenance/RebuildIndexAction.java
rename to modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/maintenance/RebuildIndexAction.java
index 73afacf0aac..5bd696805d3 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/maintenance/RebuildIndexAction.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/maintenance/RebuildIndexAction.java
@@ -15,35 +15,32 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.h2.maintenance;
+package org.apache.ignite.internal.cache.query.index.sorted.maintenance;
 
 import java.util.List;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cache.QueryIndexType;
 import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.cache.query.index.Index;
+import org.apache.ignite.internal.cache.query.index.IndexDefinition;
 import org.apache.ignite.internal.cache.query.index.IndexProcessor;
-import org.apache.ignite.internal.cache.query.index.sorted.SortedIndexDefinition;
 import org.apache.ignite.internal.cache.query.index.sorted.inline.InlineIndexFactory;
 import org.apache.ignite.internal.cache.query.index.sorted.inline.InlineIndexImpl;
-import org.apache.ignite.internal.cache.query.index.sorted.maintenance.MaintenanceRebuildIndexTarget;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
 import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointManager;
-import org.apache.ignite.internal.processors.query.GridQueryProcessor;
 import org.apache.ignite.internal.processors.query.aware.IndexBuildStatusStorage;
-import org.apache.ignite.internal.processors.query.h2.H2TableDescriptor;
-import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
-import org.apache.ignite.internal.processors.query.h2.SchemaManager;
-import org.apache.ignite.internal.processors.query.h2.database.H2TreeIndex;
-import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table;
 import org.apache.ignite.internal.processors.query.schema.IndexRebuildCancelToken;
 import org.apache.ignite.internal.processors.query.schema.SchemaIndexCacheVisitorClosure;
 import org.apache.ignite.internal.processors.query.schema.SchemaIndexCacheVisitorImpl;
+import org.apache.ignite.internal.processors.query.schema.management.IndexDescriptor;
+import org.apache.ignite.internal.processors.query.schema.management.SchemaManager;
+import org.apache.ignite.internal.processors.query.schema.management.TableDescriptor;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.maintenance.MaintenanceAction;
-import org.h2.engine.Session;
-import org.h2.index.Index;
 import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.internal.cache.query.index.sorted.maintenance.MaintenanceRebuildIndexUtils.INDEX_REBUILD_MNTC_TASK_NAME;
@@ -56,8 +53,8 @@ public class RebuildIndexAction implements MaintenanceAction<Boolean> {
     /** Indexes to rebuild. */
     private final List<MaintenanceRebuildIndexTarget> indexesToRebuild;
 
-    /** Ignite indexing. */
-    private final IgniteH2Indexing indexing;
+    /** Context. */
+    private final GridKernalContext ctx;
 
     /** Logger. */
     private final IgniteLogger log;
@@ -66,35 +63,32 @@ public class RebuildIndexAction implements MaintenanceAction<Boolean> {
      * Constructor.
      *
      * @param indexesToRebuild Indexes to rebuild.
-     * @param indexing Indexing.
-     * @param log Logger.
+     * @param ctx Context.
      */
-    public RebuildIndexAction(List<MaintenanceRebuildIndexTarget> indexesToRebuild, IgniteH2Indexing indexing, IgniteLogger log) {
+    public RebuildIndexAction(List<MaintenanceRebuildIndexTarget> indexesToRebuild, GridKernalContext ctx) {
         this.indexesToRebuild = indexesToRebuild;
-        this.indexing = indexing;
-        this.log = log;
+        this.ctx = ctx;
+
+        log = ctx.log(RebuildIndexAction.class);
     }
 
     /** {@inheritDoc} */
     @Override public Boolean execute() {
-        GridKernalContext kernalContext = indexing.kernalContext();
-
-        GridCacheDatabaseSharedManager database = (GridCacheDatabaseSharedManager)kernalContext.cache()
-            .context().database();
+        GridCacheDatabaseSharedManager db = (GridCacheDatabaseSharedManager)ctx.cache().context().database();
 
-        CheckpointManager manager = database.getCheckpointManager();
+        CheckpointManager cpMgr = db.getCheckpointManager();
 
-        IndexBuildStatusStorage storage = getIndexBuildStatusStorage(kernalContext);
+        IndexBuildStatusStorage storage = ctx.query().getIdxBuildStatusStorage();
 
         try {
-            prepareForRebuild(database, manager, storage);
+            prepareForRebuild(db, cpMgr, storage);
 
             for (MaintenanceRebuildIndexTarget params : indexesToRebuild) {
                 int cacheId = params.cacheId();
                 String idxName = params.idxName();
 
                 try {
-                    execute0(cacheId, idxName, kernalContext, storage, manager);
+                    execute0(cacheId, idxName, storage, cpMgr);
                 }
                 catch (Exception e) {
                     log.error("Rebuilding index " + idxName + " for cache " + cacheId + " failed", e);
@@ -108,10 +102,10 @@ public class RebuildIndexAction implements MaintenanceAction<Boolean> {
             return false;
         }
         finally {
-            cleanUpAfterRebuild(manager, storage);
+            cleanUpAfterRebuild(cpMgr, storage);
         }
 
-        unregisterMaintenanceTask(kernalContext);
+        unregisterMaintenanceTask();
 
         return true;
     }
@@ -121,68 +115,64 @@ public class RebuildIndexAction implements MaintenanceAction<Boolean> {
      *
      * @param cacheId Cache id.
      * @param idxName Name of the index.
-     * @param kernalContext Context.
      * @param storage Index build status storage.
-     * @param manager Checkpoint manager.
+     * @param cpMgr Checkpoint manager.
      * @throws Exception If failed to execute rebuild.
      */
     private void execute0(
         int cacheId,
         String idxName,
-        GridKernalContext kernalContext,
         IndexBuildStatusStorage storage,
-        CheckpointManager manager
+        CheckpointManager cpMgr
     ) throws Exception {
-        GridCacheContext<?, ?> context = kernalContext.cache().context().cacheContext(cacheId);
+        GridCacheContext<?, ?> cctx = ctx.cache().context().cacheContext(cacheId);
 
-        String cacheName = context.name();
+        String cacheName = cctx.name();
 
-        SchemaManager schemaManager = indexing.schemaManager();
+        SchemaManager schemaMgr = ctx.query().schemaManager();
 
-        H2TreeIndex targetIndex = findIndex(cacheName, idxName, schemaManager);
+        String schemaName = schemaMgr.schemaName(cacheName);
 
-        if (targetIndex == null) {
-            // Our job here is already done.
+        if (F.isEmpty(schemaName))
             return;
-        }
 
-        GridH2Table targetTable = targetIndex.getTable();
+        IndexDescriptor idxDesc = schemaMgr.index(schemaName, idxName);
+
+        if (idxDesc == null)
+            return;
 
-        destroyOldIndex(targetIndex, targetTable);
+        schemaMgr.dropIndex(schemaName, idxName, true);
 
-        recreateIndex(targetIndex, context, cacheName, storage, schemaManager, targetTable);
+        recreateIndex(idxDesc, cctx, cacheName, storage, schemaMgr);
 
-        manager.forceCheckpoint("afterIndexRebuild", null).futureFor(FINISHED).get();
+        cpMgr.forceCheckpoint("afterIndexRebuild", null).futureFor(FINISHED).get();
     }
 
     /**
      * Creates new index from the old one and builds it.
      *
-     * @param oldIndex Old index.
-     * @param context Cache context.
+     * @param oldIdxDesc Old index.
+     * @param cctx Cache context.
      * @param cacheName Cache name.
      * @param storage Index build status storage.
-     * @param schemaManager Schema manager.
-     * @param targetTable Table for the index.
-     * @throws IgniteCheckedException If failed to recreate an index.
+     * @param schemaMgr Schema manager.
      */
     private void recreateIndex(
-        H2TreeIndex oldIndex,
-        GridCacheContext<?, ?> context,
+        IndexDescriptor oldIdxDesc,
+        GridCacheContext<?, ?> cctx,
         String cacheName,
         IndexBuildStatusStorage storage,
-        SchemaManager schemaManager,
-        GridH2Table targetTable
-    ) throws IgniteCheckedException {
+        SchemaManager schemaMgr
+    ) {
         GridFutureAdapter<Void> createIdxFut = new GridFutureAdapter<>();
 
         IndexRebuildCancelToken token = new IndexRebuildCancelToken();
 
-        SchemaIndexCacheVisitorImpl visitor = new SchemaIndexCacheVisitorImpl(context, token, createIdxFut) {
+        SchemaIndexCacheVisitorImpl visitor = new SchemaIndexCacheVisitorImpl(cctx, token, createIdxFut) {
             /** {@inheritDoc} */
             @Override public void visit(SchemaIndexCacheVisitorClosure clo) {
                 // Rebuild index after it is created.
-                storage.onStartRebuildIndexes(context);
+                storage.onStartRebuildIndexes(cctx);
 
                 try {
                     super.visit(clo);
@@ -198,126 +188,70 @@ public class RebuildIndexAction implements MaintenanceAction<Boolean> {
             }
         };
 
-        IndexProcessor indexProcessor = context.kernalContext().indexProcessor();
+        IndexProcessor idxProc = ctx.indexProcessor();
 
-        SortedIndexDefinition definition = oldIndex.index().indexDefinition();
+        assert oldIdxDesc.type() == QueryIndexType.SORTED : oldIdxDesc.type();
 
-        org.apache.ignite.internal.cache.query.index.Index newIndex = indexProcessor.createIndexDynamically(
-            targetTable.cacheContext(), InlineIndexFactory.INSTANCE, definition, visitor);
+        TableDescriptor tblDesc = oldIdxDesc.table();
+        IndexDefinition definition = oldIdxDesc.index().unwrap(InlineIndexImpl.class).indexDefinition();
 
-        InlineIndexImpl queryIndex = newIndex.unwrap(InlineIndexImpl.class);
+        Index newIdx = idxProc.createIndexDynamically(tblDesc.cacheInfo().cacheContext(),
+            InlineIndexFactory.INSTANCE, definition, visitor);
 
-        H2TreeIndex newIdx = oldIndex.createCopy(queryIndex, definition);
+        IndexDescriptor newIdxDesc = new IndexDescriptor(tblDesc, oldIdxDesc.name(), oldIdxDesc.type(),
+            oldIdxDesc.keyDefinitions(), oldIdxDesc.isPk(), oldIdxDesc.isAffinity(), oldIdxDesc.inlineSize(), newIdx);
 
-        schemaManager.createIndex(
-            targetTable.getSchema().getName(),
-            targetTable.getName(),
-            newIdx,
-            true
-        );
+        try {
+            schemaMgr.addIndex(tblDesc, newIdxDesc);
+        }
+        catch (IgniteCheckedException e) {
+            log.error("Failed to register index in schema manager", e);
+        }
 
         // This future must be already finished by the schema index cache visitor above
         assert createIdxFut.isDone();
     }
 
-    /**
-     * Destroys old index.
-     *
-     * @param index Index.
-     * @param table Table.
-     * @throws IgniteCheckedException If failed to destroy index.
-     */
-    private void destroyOldIndex(H2TreeIndex index, GridH2Table table) throws IgniteCheckedException {
-        index.destroyImmediately();
-
-        Session session = table.getDatabase().getSystemSession();
-
-        table.removeIndex(session, index);
-    }
-
-    /**
-     * Finds index for the cache by the name.
-     *
-     * @param cacheName Cache name.
-     * @param idxName Index name.
-     * @param schemaManager Schema manager.
-     * @return Index or {@code null} if index was not found.
-     */
-    @Nullable private H2TreeIndex findIndex(String cacheName, String idxName, SchemaManager schemaManager) {
-        H2TreeIndex targetIndex = null;
-
-        for (H2TableDescriptor tblDesc : schemaManager.tablesForCache(cacheName)) {
-            GridH2Table tbl = tblDesc.table();
-
-            assert tbl != null;
-
-            Index index = tbl.getIndex(idxName);
-
-            if (index != null) {
-                assert index instanceof H2TreeIndex;
-
-                targetIndex = (H2TreeIndex)index;
-                break;
-            }
-        }
-
-        return targetIndex;
-    }
-
     /**
      * Prepares system for the rebuild.
      *
-     * @param database Database manager.
-     * @param manager Checkpoint manager.
+     * @param db Database manager.
+     * @param cpMgr Checkpoint manager.
      * @param storage Index build status storage.
      * @throws IgniteCheckedException If failed.
      */
     private void prepareForRebuild(
-        GridCacheDatabaseSharedManager database,
-        CheckpointManager manager,
+        GridCacheDatabaseSharedManager db,
+        CheckpointManager cpMgr,
         IndexBuildStatusStorage storage
     ) throws IgniteCheckedException {
         // Enable WAL
-        database.resumeWalLogging();
+        db.resumeWalLogging();
 
         // Enable checkpointer
-        database.onStateRestored(null);
+        db.onStateRestored(null);
 
         // IndexBuildStatusStorage listens for checkpoint to update the status of the rebuild in the metastorage.
         // We need to set up the listener manually here, because it's maintenance mode.
-        manager.addCheckpointListener(storage, null);
+        cpMgr.addCheckpointListener(storage, null);
     }
 
     /**
      * Cleans up after the rebuild.
      *
-     * @param manager Checkpoint manager.
+     * @param cpMgr Checkpoint manager.
      * @param storage Index build status storage.
      */
-    private void cleanUpAfterRebuild(CheckpointManager manager, IndexBuildStatusStorage storage) {
+    private void cleanUpAfterRebuild(CheckpointManager cpMgr, IndexBuildStatusStorage storage) {
         // Remove the checkpoint listener
-        manager.removeCheckpointListener(storage);
+        cpMgr.removeCheckpointListener(storage);
     }
 
     /**
      * Removes maintenance task.
-     *
-     * @param kernalContext Kernal context.
      */
-    private void unregisterMaintenanceTask(GridKernalContext kernalContext) {
-        kernalContext.maintenanceRegistry().unregisterMaintenanceTask(INDEX_REBUILD_MNTC_TASK_NAME);
-    }
-
-    /**
-     * Gets index build status storage.
-     *
-     * @param kernalContext Kernal context.
-     * @return Index build status storage.
-     */
-    private IndexBuildStatusStorage getIndexBuildStatusStorage(GridKernalContext kernalContext) {
-        GridQueryProcessor query = kernalContext.query();
-
-        return query.getIdxBuildStatusStorage();
+    private void unregisterMaintenanceTask() {
+        ctx.maintenanceRegistry().unregisterMaintenanceTask(INDEX_REBUILD_MNTC_TASK_NAME);
     }
 
     /** {@inheritDoc} */
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/maintenance/RebuildIndexWorkflowCallback.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/maintenance/RebuildIndexWorkflowCallback.java
similarity index 72%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/maintenance/RebuildIndexWorkflowCallback.java
rename to modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/maintenance/RebuildIndexWorkflowCallback.java
index 0f1202da3e8..d38cd29cf02 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/maintenance/RebuildIndexWorkflowCallback.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/maintenance/RebuildIndexWorkflowCallback.java
@@ -15,13 +15,11 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.h2.maintenance;
+package org.apache.ignite.internal.cache.query.index.sorted.maintenance;
 
 import java.util.Collections;
 import java.util.List;
-import org.apache.ignite.IgniteLogger;
-import org.apache.ignite.internal.cache.query.index.sorted.maintenance.MaintenanceRebuildIndexTarget;
-import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
+import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.maintenance.MaintenanceAction;
 import org.apache.ignite.maintenance.MaintenanceWorkflowCallback;
 import org.jetbrains.annotations.Nullable;
@@ -33,23 +31,18 @@ public class RebuildIndexWorkflowCallback implements MaintenanceWorkflowCallback
     /** Indexes to rebuild. */
     private final List<MaintenanceRebuildIndexTarget> indexesToRebuild;
 
-    /** Indexing. */
-    private final IgniteH2Indexing indexing;
-
-    /** Logger. */
-    private final IgniteLogger log;
+    /** Context. */
+    private final GridKernalContext ctx;
 
     /**
      * Constructor.
      *
      * @param indexesToRebuild Indexes to rebuild.
-     * @param indexing Indexing.
-     * @param log Logger.
+     * @param ctx Context.
      */
-    public RebuildIndexWorkflowCallback(List<MaintenanceRebuildIndexTarget> indexesToRebuild, IgniteH2Indexing indexing, IgniteLogger log) {
+    public RebuildIndexWorkflowCallback(List<MaintenanceRebuildIndexTarget> indexesToRebuild, GridKernalContext ctx) {
         this.indexesToRebuild = indexesToRebuild;
-        this.indexing = indexing;
-        this.log = log;
+        this.ctx = ctx;
     }
 
     /** {@inheritDoc} */
@@ -59,11 +52,11 @@ public class RebuildIndexWorkflowCallback implements MaintenanceWorkflowCallback
 
     /** {@inheritDoc} */
     @Override public List<MaintenanceAction<?>> allActions() {
-        return Collections.singletonList(new RebuildIndexAction(indexesToRebuild, indexing, log));
+        return Collections.singletonList(new RebuildIndexAction(indexesToRebuild, ctx));
     }
 
     /** {@inheritDoc} */
     @Override public @Nullable MaintenanceAction<?> automaticAction() {
-        return new RebuildIndexAction(indexesToRebuild, indexing, log);
+        return new RebuildIndexAction(indexesToRebuild, ctx);
     }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/IgniteMBeansManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/IgniteMBeansManager.java
index 3e1eafdde7c..4e256253c58 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/IgniteMBeansManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/IgniteMBeansManager.java
@@ -167,7 +167,7 @@ public class IgniteMBeansManager {
         registerMBean("Kernal", blockOpCtrlMXBean.getClass().getSimpleName(), blockOpCtrlMXBean,
             FailureHandlingMxBean.class);
 
-        if (ctx.query().moduleEnabled())
+        if (ctx.query().indexingEnabled())
             ctx.query().getIndexing().registerMxBeans(this);
 
         PerformanceStatisticsMBeanImpl performanceStatMbean = new PerformanceStatisticsMBeanImpl(ctx);
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/systemview/GridSystemViewManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/systemview/GridSystemViewManager.java
index f63bc53ab8e..0d6d4e82d85 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/systemview/GridSystemViewManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/systemview/GridSystemViewManager.java
@@ -29,12 +29,10 @@ import java.util.function.Consumer;
 import java.util.function.Function;
 import java.util.function.Supplier;
 import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.IgniteException;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.IgniteComponentType;
 import org.apache.ignite.internal.managers.GridManagerAdapter;
 import org.apache.ignite.internal.util.typedef.F;
-import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.spi.systemview.ReadOnlySystemViewRegistry;
 import org.apache.ignite.spi.systemview.SystemViewExporterSpi;
 import org.apache.ignite.spi.systemview.view.SystemView;
@@ -52,9 +50,6 @@ import static org.apache.ignite.internal.util.IgniteUtils.notifyListeners;
  */
 public class GridSystemViewManager extends GridManagerAdapter<SystemViewExporterSpi>
     implements ReadOnlySystemViewRegistry {
-    /** Class name for a SQL view exporter of system views. */
-    public static final String SYSTEM_VIEW_SQL_SPI = "org.apache.ignite.internal.managers.systemview.SqlViewExporterSpi";
-
     /** Registered system views. */
     private final ConcurrentHashMap<String, SystemView<?>> systemViews = new ConcurrentHashMap<>();
 
@@ -236,7 +231,7 @@ public class GridSystemViewManager extends GridManagerAdapter<SystemViewExporter
     private static SystemViewExporterSpi[] addStandardExporters(SystemViewExporterSpi[] spis) {
         int newSz = F.isEmpty(spis) ? 1 : spis.length + 1;
 
-        boolean addSql = IgniteComponentType.INDEXING.inClassPath();
+        boolean addSql = IgniteComponentType.INDEXING.inClassPath() || IgniteComponentType.QUERY_ENGINE.inClassPath();
 
         if (addSql)
             newSz += 1;
@@ -246,14 +241,8 @@ public class GridSystemViewManager extends GridManagerAdapter<SystemViewExporter
         if (!F.isEmpty(spis))
             System.arraycopy(spis, 0, newSpis, 0, spis.length);
 
-        if (addSql) {
-            try {
-                newSpis[newSpis.length - 2] = U.newInstance(SYSTEM_VIEW_SQL_SPI);
-            }
-            catch (IgniteCheckedException e) {
-                throw new IgniteException(e);
-            }
-        }
+        if (addSql)
+            newSpis[newSpis.length - 2] = new SqlViewExporterSpi();
 
         newSpis[newSpis.length - 1] = new JmxSystemViewExporterSpi();
 
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/managers/systemview/SqlViewExporterSpi.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/systemview/SqlViewExporterSpi.java
similarity index 71%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/managers/systemview/SqlViewExporterSpi.java
rename to modules/core/src/main/java/org/apache/ignite/internal/managers/systemview/SqlViewExporterSpi.java
index 09b61e45564..6a0fc52b923 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/managers/systemview/SqlViewExporterSpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/systemview/SqlViewExporterSpi.java
@@ -19,8 +19,6 @@ package org.apache.ignite.internal.managers.systemview;
 
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.IgniteEx;
-import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
-import org.apache.ignite.internal.processors.query.h2.SchemaManager;
 import org.apache.ignite.spi.IgniteSpiContext;
 import org.apache.ignite.spi.IgniteSpiException;
 import org.apache.ignite.spi.systemview.view.SystemView;
@@ -29,24 +27,16 @@ import static org.apache.ignite.internal.processors.query.QueryUtils.SCHEMA_SYS;
 
 /**
  * This SPI implementation exports metrics as SQL views.
- *
- * Note, instance of this class created with reflection.
- * @see GridSystemViewManager#SYSTEM_VIEW_SQL_SPI
  */
 class SqlViewExporterSpi extends AbstractSystemViewExporterSpi {
-    /** Schema manager. */
-    private SchemaManager mgr;
+    /** */
+    private GridKernalContext ctx;
 
     /** {@inheritDoc} */
     @Override protected void onContextInitialized0(IgniteSpiContext spiCtx) throws IgniteSpiException {
-        GridKernalContext ctx = ((IgniteEx)ignite()).context();
-
-        if (ctx.query().getIndexing() instanceof IgniteH2Indexing) {
-            mgr = ((IgniteH2Indexing)ctx.query().getIndexing()).schemaManager();
-
-            sysViewReg.forEach(this::register);
-            sysViewReg.addSystemViewCreationListener(this::register);
-        }
+        ctx = ((IgniteEx)ignite()).context();
+        sysViewReg.forEach(this::register);
+        sysViewReg.addSystemViewCreationListener(this::register);
     }
 
     /**
@@ -58,6 +48,6 @@ class SqlViewExporterSpi extends AbstractSystemViewExporterSpi {
         if (log.isDebugEnabled())
             log.debug("Found new system view [name=" + sysView.name() + ']');
 
-        mgr.createSystemView(SCHEMA_SYS, sysView);
+        ctx.query().schemaManager().createSystemView(SCHEMA_SYS, sysView);
     }
 }
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/managers/systemview/walker/SqlIndexViewWalker.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/systemview/walker/SqlIndexViewWalker.java
similarity index 97%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/managers/systemview/walker/SqlIndexViewWalker.java
rename to modules/core/src/main/java/org/apache/ignite/internal/managers/systemview/walker/SqlIndexViewWalker.java
index 6384c716fd2..db48ccc4e36 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/managers/systemview/walker/SqlIndexViewWalker.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/systemview/walker/SqlIndexViewWalker.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.managers.systemview.walker;
 
-import org.apache.ignite.spi.systemview.view.SqlIndexView;
 import org.apache.ignite.spi.systemview.view.SystemViewRowAttributeWalker;
+import org.apache.ignite.spi.systemview.view.sql.SqlIndexView;
 
 /**
  * Generated by {@code org.apache.ignite.codegen.SystemViewRowAttributeWalkerGenerator}.
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/managers/systemview/walker/SqlSchemaViewWalker.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/systemview/walker/SqlSchemaViewWalker.java
similarity index 96%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/managers/systemview/walker/SqlSchemaViewWalker.java
rename to modules/core/src/main/java/org/apache/ignite/internal/managers/systemview/walker/SqlSchemaViewWalker.java
index ca440e89b6b..09dad8a4298 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/managers/systemview/walker/SqlSchemaViewWalker.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/systemview/walker/SqlSchemaViewWalker.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.managers.systemview.walker;
 
-import org.apache.ignite.spi.systemview.view.SqlSchemaView;
 import org.apache.ignite.spi.systemview.view.SystemViewRowAttributeWalker;
+import org.apache.ignite.spi.systemview.view.sql.SqlSchemaView;
 
 /**
  * Generated by {@code org.apache.ignite.codegen.SystemViewRowAttributeWalkerGenerator}.
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/managers/systemview/walker/SqlTableColumnViewWalker.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/systemview/walker/SqlTableColumnViewWalker.java
similarity index 97%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/managers/systemview/walker/SqlTableColumnViewWalker.java
rename to modules/core/src/main/java/org/apache/ignite/internal/managers/systemview/walker/SqlTableColumnViewWalker.java
index 41f791ad8e2..99e8127b28d 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/managers/systemview/walker/SqlTableColumnViewWalker.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/systemview/walker/SqlTableColumnViewWalker.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.managers.systemview.walker;
 
-import org.apache.ignite.spi.systemview.view.SqlTableColumnView;
 import org.apache.ignite.spi.systemview.view.SystemViewRowAttributeWalker;
+import org.apache.ignite.spi.systemview.view.sql.SqlTableColumnView;
 
 /**
  * Generated by {@code org.apache.ignite.codegen.SystemViewRowAttributeWalkerGenerator}.
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/managers/systemview/walker/SqlTableViewWalker.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/systemview/walker/SqlTableViewWalker.java
similarity index 97%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/managers/systemview/walker/SqlTableViewWalker.java
rename to modules/core/src/main/java/org/apache/ignite/internal/managers/systemview/walker/SqlTableViewWalker.java
index 0826618376a..54cca487567 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/managers/systemview/walker/SqlTableViewWalker.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/systemview/walker/SqlTableViewWalker.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.managers.systemview.walker;
 
-import org.apache.ignite.spi.systemview.view.SqlTableView;
 import org.apache.ignite.spi.systemview.view.SystemViewRowAttributeWalker;
+import org.apache.ignite.spi.systemview.view.sql.SqlTableView;
 
 /**
  * Generated by {@code org.apache.ignite.codegen.SystemViewRowAttributeWalkerGenerator}.
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/managers/systemview/walker/SqlViewColumnViewWalker.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/systemview/walker/SqlViewColumnViewWalker.java
similarity index 93%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/managers/systemview/walker/SqlViewColumnViewWalker.java
rename to modules/core/src/main/java/org/apache/ignite/internal/managers/systemview/walker/SqlViewColumnViewWalker.java
index 558982e7534..43c72927f8b 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/managers/systemview/walker/SqlViewColumnViewWalker.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/systemview/walker/SqlViewColumnViewWalker.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.managers.systemview.walker;
 
-import org.apache.ignite.spi.systemview.view.SqlViewColumnView;
 import org.apache.ignite.spi.systemview.view.SystemViewRowAttributeWalker;
+import org.apache.ignite.spi.systemview.view.sql.SqlViewColumnView;
 
 /**
  * Generated by {@code org.apache.ignite.codegen.SystemViewRowAttributeWalkerGenerator}.
@@ -34,7 +34,7 @@ public class SqlViewColumnViewWalker implements SystemViewRowAttributeWalker<Sql
         v.accept(2, "schemaName", String.class);
         v.accept(3, "defaultValue", String.class);
         v.accept(4, "nullable", boolean.class);
-        v.accept(5, "precision", long.class);
+        v.accept(5, "precision", int.class);
         v.accept(6, "scale", int.class);
         v.accept(7, "type", String.class);
     }
@@ -46,7 +46,7 @@ public class SqlViewColumnViewWalker implements SystemViewRowAttributeWalker<Sql
         v.accept(2, "schemaName", String.class, row.schemaName());
         v.accept(3, "defaultValue", String.class, row.defaultValue());
         v.acceptBoolean(4, "nullable", row.nullable());
-        v.acceptLong(5, "precision", row.precision());
+        v.acceptInt(5, "precision", row.precision());
         v.acceptInt(6, "scale", row.scale());
         v.accept(7, "type", String.class, row.type());
     }
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/managers/systemview/walker/SqlViewViewWalker.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/systemview/walker/SqlViewViewWalker.java
similarity index 96%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/managers/systemview/walker/SqlViewViewWalker.java
rename to modules/core/src/main/java/org/apache/ignite/internal/managers/systemview/walker/SqlViewViewWalker.java
index 6926133e97b..bc3f49f67e3 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/managers/systemview/walker/SqlViewViewWalker.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/systemview/walker/SqlViewViewWalker.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.managers.systemview.walker;
 
-import org.apache.ignite.spi.systemview.view.SqlViewView;
 import org.apache.ignite.spi.systemview.view.SystemViewRowAttributeWalker;
+import org.apache.ignite.spi.systemview.view.sql.SqlViewView;
 
 /**
  * Generated by {@code org.apache.ignite.codegen.SystemViewRowAttributeWalkerGenerator}.
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index d800dfb55e7..22b66b4787d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -1066,7 +1066,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                 ctx.kernalContext().query().onCacheStop(cacheInfo, rmvIdx, clearIdx);
             }
             else
-                ctx.kernalContext().query().getIndexing().closeCacheOnClient(ctx.name());
+                ctx.kernalContext().query().onClientCacheStop(cacheInfo);
 
             if (isNearEnabled(ctx)) {
                 GridDhtCacheAdapter dht = ctx.near().dht();
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxyImpl.java
index 007552b7f52..06fd7bab341 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxyImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxyImpl.java
@@ -936,7 +936,7 @@ public class IgniteCacheProxyImpl<K, V> extends AsyncSupportAdapter<IgniteCache<
         if (!ctx.kernalContext().query().moduleEnabled() &&
             (qry instanceof SqlQuery || qry instanceof SqlFieldsQuery || qry instanceof TextQuery))
             throw new CacheException("Failed to execute query. Add module 'ignite-indexing' to the classpath " +
-                    "of all Ignite nodes.");
+                    "of all Ignite nodes or configure any query engine.");
     }
 
     /** {@inheritDoc} */
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/BPlusTree.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/BPlusTree.java
index 69ae6a17171..b37cb87222d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/BPlusTree.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/BPlusTree.java
@@ -1586,6 +1586,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             switch (findDown(g, g.rootId, 0L, g.rootLvl)) {
                 case RETRY:
                 case RETRY_ROOT:
+                    checkDestroyed();
                     checkInterrupted();
 
                     continue;
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/metric/GridMetricManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/metric/GridMetricManager.java
index 78bb0a9e383..ebd62e3ce01 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/metric/GridMetricManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/metric/GridMetricManager.java
@@ -71,9 +71,6 @@ import static org.apache.ignite.internal.util.IgniteUtils.notifyListeners;
  * @see MetricRegistry
  */
 public class GridMetricManager extends GridManagerAdapter<MetricExporterSpi> implements ReadOnlyMetricManager {
-    /** Class name for a SQL view metrics exporter. */
-    public static final String SQL_SPI = "org.apache.ignite.internal.processors.metric.sql.SqlViewMetricExporterSpi";
-
     /** Metrics update frequency. */
     private static final long METRICS_UPDATE_FREQ = 3000;
 
@@ -198,7 +195,7 @@ public class GridMetricManager extends GridManagerAdapter<MetricExporterSpi> imp
         super(ctx, ((Supplier<MetricExporterSpi[]>)() -> {
             MetricExporterSpi[] spi = ctx.config().getMetricExporterSpi();
 
-            if (!IgniteComponentType.INDEXING.inClassPath())
+            if (!IgniteComponentType.INDEXING.inClassPath() && !IgniteComponentType.QUERY_ENGINE.inClassPath())
                 return spi;
 
             MetricExporterSpi[] spiWithSql = new MetricExporterSpi[spi != null ? spi.length + 1 : 1];
@@ -206,12 +203,7 @@ public class GridMetricManager extends GridManagerAdapter<MetricExporterSpi> imp
             if (!F.isEmpty(spi))
                 System.arraycopy(spi, 0, spiWithSql, 0, spi.length);
 
-            try {
-                spiWithSql[spiWithSql.length - 1] = U.newInstance(SQL_SPI);
-            }
-            catch (IgniteCheckedException e) {
-                throw new IgniteException(e);
-            }
+            spiWithSql[spiWithSql.length - 1] = new SqlViewMetricExporterSpi();
 
             return spiWithSql;
         }).get());
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/metric/sql/SqlViewMetricExporterSpi.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/metric/SqlViewMetricExporterSpi.java
similarity index 97%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/metric/sql/SqlViewMetricExporterSpi.java
rename to modules/core/src/main/java/org/apache/ignite/internal/processors/metric/SqlViewMetricExporterSpi.java
index 50145c572ac..226023ab93f 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/metric/sql/SqlViewMetricExporterSpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/metric/SqlViewMetricExporterSpi.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.metric.sql;
+package org.apache.ignite.internal.processors.metric;
 
 import java.util.function.Predicate;
 import org.apache.ignite.internal.GridKernalContext;
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/metric/impl/MetricUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/metric/impl/MetricUtils.java
index cf62dc55b31..8e9c1d25f2f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/metric/impl/MetricUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/metric/impl/MetricUtils.java
@@ -17,10 +17,14 @@
 
 package org.apache.ignite.internal.processors.metric.impl;
 
+import java.util.LinkedHashMap;
+import java.util.Map;
 import org.apache.ignite.internal.processors.metric.GridMetricManager;
 import org.apache.ignite.internal.processors.metric.MetricRegistry;
 import org.apache.ignite.internal.util.typedef.T2;
 import org.apache.ignite.spi.metric.HistogramMetric;
+import org.apache.ignite.spi.systemview.view.SystemView;
+import org.apache.ignite.spi.systemview.view.SystemViewRowAttributeWalker;
 
 import static org.apache.ignite.internal.processors.cache.CacheGroupMetricsImpl.CACHE_GROUP_METRICS_PREFIX;
 import static org.apache.ignite.internal.processors.cache.CacheMetricsImpl.CACHE_METRICS;
@@ -188,4 +192,22 @@ public class MetricUtils {
             .replaceAll("([A-Z])", "_$1")
             .replaceAll('\\' + SEPARATOR, "_").toUpperCase();
     }
+
+    /**
+     * Extract attributes for system view.
+     *
+     * @param sysView System view.
+     * @return Attributes map.
+     */
+    public static Map<String, Class<?>> systemViewAttributes(SystemView<?> sysView) {
+        Map<String, Class<?>> attrs = new LinkedHashMap<>(sysView.walker().count());
+
+        sysView.walker().visitAll(new SystemViewRowAttributeWalker.AttributeVisitor() {
+            @Override public <T> void accept(int idx, String name, Class<T> clazz) {
+                attrs.put(name, clazz);
+            }
+        });
+
+        return attrs;
+    }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetadataInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetadataInfo.java
index 9450fa10207..7f8e4ba37e8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetadataInfo.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetadataInfo.java
@@ -151,7 +151,7 @@ public class JdbcMetadataInfo {
      * @return List of metadatas of tables that matches.
      */
     public List<JdbcTableMeta> getTablesMeta(String schemaNamePtrn, String tblNamePtrn, String[] tblTypes) {
-        Collection<TableInformation> tblsMeta = ctx.query().getIndexing()
+        Collection<TableInformation> tblsMeta = ctx.query().schemaManager()
             .tablesInformation(schemaNamePtrn, tblNamePtrn, tblTypes);
 
         return tblsMeta.stream()
@@ -176,7 +176,7 @@ public class JdbcMetadataInfo {
 
         Collection<JdbcColumnMeta> metas = new LinkedHashSet<>();
 
-        Collection<ColumnInformation> colsInfo = ctx.query().getIndexing()
+        Collection<ColumnInformation> colsInfo = ctx.query().schemaManager()
             .columnsInformation(schemaNamePtrn, tblNamePtrn, colNamePtrn);
 
         colsInfo.stream().sorted(bySchemaThenTabNameThenColOrder)
@@ -217,7 +217,7 @@ public class JdbcMetadataInfo {
     public SortedSet<String> getSchemasMeta(String schemaNamePtrn) {
         SortedSet<String> schemas = new TreeSet<>(); // to have values sorted.
 
-        for (String schema : ctx.query().getIndexing().schemasNames()) {
+        for (String schema : ctx.query().schemaManager().schemaNames()) {
             if (matches(schema, schemaNamePtrn))
                 schemas.add(schema);
         }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/pool/PoolProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/pool/PoolProcessor.java
index e47dfb88dab..7b72f035148 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/pool/PoolProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/pool/PoolProcessor.java
@@ -460,7 +460,9 @@ public class PoolProcessor extends GridProcessorAdapter {
                 GridIoPolicy.IDX_POOL,
                 oomeHnd
             );
+        }
 
+        if (IgniteComponentType.INDEXING.inClassPath() || IgniteComponentType.QUERY_ENGINE.inClassPath()) {
             int buildIdxThreadPoolSize = cfg.getBuildIndexThreadPoolSize();
 
             validateThreadPoolSize(buildIdxThreadPoolSize, "build-idx");
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
index c1114691f3f..c8c91394e96 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
@@ -18,11 +18,7 @@
 package org.apache.ignite.internal.processors.query;
 
 import java.sql.SQLException;
-import java.util.Collection;
-import java.util.Collections;
 import java.util.List;
-import java.util.Map;
-import java.util.Set;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteDataStreamer;
 import org.apache.ignite.cache.query.FieldsQueryCursor;
@@ -36,8 +32,6 @@ import org.apache.ignite.internal.processors.cache.GridCacheContextInfo;
 import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot;
 import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
 import org.apache.ignite.internal.processors.odbc.jdbc.JdbcParameterMeta;
-import org.apache.ignite.internal.processors.query.schema.SchemaIndexCacheVisitor;
-import org.apache.ignite.internal.processors.query.stat.IgniteStatisticsManager;
 import org.apache.ignite.internal.util.GridSpinBusyLock;
 import org.apache.ignite.internal.util.lang.GridCloseableIterator;
 import org.apache.ignite.lang.IgniteBiTuple;
@@ -141,57 +135,6 @@ public interface GridQueryIndexing {
     public <K, V> GridCloseableIterator<IgniteBiTuple<K, V>> queryLocalText(String schemaName, String cacheName,
         String qry, String typeName, IndexingQueryFilter filter, int limit) throws IgniteCheckedException;
 
-    /**
-     * Create new index locally.
-     *
-     * @param schemaName Schema name.
-     * @param tblName Table name.
-     * @param idxDesc Index descriptor.
-     * @param ifNotExists Ignore operation if index exists (instead of throwing an error).
-     * @param cacheVisitor Cache visitor
-     * @throws IgniteCheckedException if failed.
-     */
-    public void dynamicIndexCreate(String schemaName, String tblName, QueryIndexDescriptorImpl idxDesc,
-        boolean ifNotExists, SchemaIndexCacheVisitor cacheVisitor) throws IgniteCheckedException;
-
-    /**
-     * Remove index from the cache.
-     *
-     * @param schemaName Schema name.
-     * @param idxName Index name.
-     * @param ifExists Ignore operation if index does not exist (instead of throwing an error).
-     * @throws IgniteCheckedException If failed.
-     */
-    public void dynamicIndexDrop(String schemaName, String idxName, boolean ifExists) throws IgniteCheckedException;
-
-    /**
-     * Add columns to dynamic table.
-     *
-     * @param schemaName Schema name.
-     * @param tblName Table name.
-     * @param cols Columns to add.
-     * @param ifTblExists Ignore operation if target table does not exist (instead of throwing an error).
-     * @param ifColNotExists Ignore operation if column already exists (instead of throwing an error) - is honored only
-     *     for single column case.
-     * @throws IgniteCheckedException If failed.
-     */
-    public void dynamicAddColumn(String schemaName, String tblName, List<QueryField> cols, boolean ifTblExists,
-        boolean ifColNotExists) throws IgniteCheckedException;
-
-    /**
-     * Drop columns from dynamic table.
-     *
-     * @param schemaName Schema name.
-     * @param tblName Table name.
-     * @param cols Columns to drop.
-     * @param ifTblExists Ignore operation if target table does not exist (instead of throwing an error).
-     * @param ifColExists Ignore operation if column does not exist (instead of throwing an error) - is honored only
-     *     for single column case.
-     * @throws IgniteCheckedException If failed.
-     */
-    public void dynamicDropColumn(String schemaName, String tblName, List<String> cols, boolean ifTblExists,
-        boolean ifColExists) throws IgniteCheckedException;
-
     /**
      * Registers cache.
      *
@@ -207,11 +150,8 @@ public interface GridQueryIndexing {
      * Unregisters cache.
      *
      * @param cacheInfo Cache context info.
-     * @param rmvIdx If {@code true}, will remove index.
-     * @param clearIdx If {@code true}, will clear the index.
-     * @throws IgniteCheckedException If failed to drop cache schema.
      */
-    public void unregisterCache(GridCacheContextInfo cacheInfo, boolean rmvIdx, boolean clearIdx) throws IgniteCheckedException;
+    public void unregisterCache(GridCacheContextInfo<?, ?> cacheInfo);
 
     /**
      *
@@ -245,18 +185,6 @@ public interface GridQueryIndexing {
         GridQueryCancel cancel
     ) throws IgniteCheckedException;
 
-    /**
-     * Registers type if it was not known before or updates it otherwise.
-     *
-     * @param cacheInfo Cache context info.
-     * @param desc Type descriptor.
-     * @param isSql {@code true} in case table has been created from SQL.
-     * @throws IgniteCheckedException If failed.
-     * @return {@code True} if type was registered, {@code false} if for some reason it was rejected.
-     */
-    public boolean registerType(GridCacheContextInfo cacheInfo, GridQueryTypeDescriptor desc,
-        boolean isSql) throws IgniteCheckedException;
-
     /**
      * Jdbc parameters metadata of the specified query.
      *
@@ -306,13 +234,6 @@ public interface GridQueryIndexing {
     public void remove(GridCacheContext cctx, GridQueryTypeDescriptor type, CacheDataRow row)
         throws IgniteCheckedException;
 
-    /**
-     * Mark as rebuild needed for the given cache.
-     *
-     * @param cctx Cache context.
-     */
-    public void markAsRebuildNeeded(GridCacheContext cctx, boolean val);
-
     /**
      * Returns backup filter.
      *
@@ -339,21 +260,6 @@ public interface GridQueryIndexing {
      */
     public void onKernalStop();
 
-    /**
-     * Gets database schema from cache name.
-     *
-     * @param cacheName Cache name. {@code null} would be converted to an empty string.
-     * @return Schema name. Should not be null since we should not fail for an invalid cache name.
-     */
-    public String schema(String cacheName);
-
-    /**
-     * Gets database schemas names.
-     *
-     * @return Schema names.
-     */
-    public Set<String> schemasNames();
-
     /**
      * Whether passed sql statement is single insert statement eligible for streaming.
      *
@@ -362,83 +268,13 @@ public interface GridQueryIndexing {
      */
     public boolean isStreamableInsertStatement(String schemaName, SqlFieldsQuery sql) throws SQLException;
 
-    /**
-     * Return context for registered cache info.
-     *
-     * @param cacheName Cache name.
-     * @return Cache context for registered cache or {@code null} in case the cache has not been registered.
-     */
-    @Nullable public GridCacheContextInfo registeredCacheInfo(String cacheName);
-
-    /**
-     * Clear cache info and clear parser cache on call cache.close() on client node.
-     *
-     * @param cacheName Cache name to clear.
-     */
-    public void closeCacheOnClient(String cacheName);
-
-    /**
-     * Initialize table's cache context created for not started cache.
-     *
-     * @param ctx Cache context.
-     * @throws IgniteCheckedException If failed.
-     *
-     * @return {@code true} If context has been initialized.
-     */
-    public boolean initCacheContext(GridCacheContext ctx) throws IgniteCheckedException;
-
     /**
      * Register SQL JMX beans.
      *
      * @param mbMgr Ignite MXBean manager.
      * @throws IgniteCheckedException On bean registration error.
      */
-    void registerMxBeans(IgniteMBeansManager mbMgr) throws IgniteCheckedException;
-
-    /**
-     * Return table information filtered by given patterns.
-     *
-     * @param schemaNamePtrn Filter by schema name. Can be {@code null} to don't use the filter.
-     * @param tblNamePtrn Filter by table name. Can be {@code null} to don't use the filter.
-     * @param tblTypes Filter by table type. As Of now supported only 'TABLES' and 'VIEWS'.
-     * Can be {@code null} or empty to don't use the filter.
-     *
-     * @return Column information filtered by given patterns.
-     */
-    Collection<TableInformation> tablesInformation(String schemaNamePtrn, String tblNamePtrn, String... tblTypes);
-
-    /**
-     * Return column information filtered by given patterns.
-     *
-     * @param schemaNamePtrn Filter by schema name. Can be {@code null} to don't use the filter.
-     * @param tblNamePtrn Filter by table name. Can be {@code null} to don't use the filter.
-     * @param colNamePtrn Filter by column name. Can be {@code null} to don't use the filter.
-     *
-     * @return Column information filtered by given patterns.
-     */
-    Collection<ColumnInformation> columnsInformation(String schemaNamePtrn, String tblNamePtrn, String colNamePtrn);
-
-    /**
-     * Return index size by schema, table and index name.
-     *
-     * @param schemaName Schema name.
-     * @param tblName Table name.
-     * @param idxName Index name.
-     * @return Index size (Number of elements) or {@code 0} if index not found.
-     */
-    default long indexSize(String schemaName, String tblName, String idxName) throws IgniteCheckedException {
-        return 0;
-    }
-
-    /**
-     * Information about secondary indexes efficient (actual) inline size.
-     *
-     * @return Map with inline sizes. The key of entry is a full index name (with schema and table name), the value of
-     * entry is a inline size.
-     */
-    default Map<String, Integer> secondaryIndexesInlineSize() {
-        return Collections.emptyMap();
-    }
+    public void registerMxBeans(IgniteMBeansManager mbMgr) throws IgniteCheckedException;
 
     /**
      * Checks if object of the specified class can be stored in the specified table column by the query engine.
@@ -451,10 +287,5 @@ public interface GridQueryIndexing {
      *         query engine.
      * @throws IgniteSQLException if table or column with specified name was not found.
      */
-    boolean isConvertibleToColumnType(String schemaName, String tblName, String colName, Class<?> cls);
-
-    /**
-     * @return Ignite query statistics manager.
-     */
-    public IgniteStatisticsManager statsManager();
+    public boolean isConvertibleToColumnType(String schemaName, String tblName, String colName, Class<?> cls);
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
index 2c0b6aa9b4f..35aa38929b5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
@@ -68,6 +68,7 @@ import org.apache.ignite.internal.binary.BinaryMetadata;
 import org.apache.ignite.internal.cache.query.index.IndexProcessor;
 import org.apache.ignite.internal.cache.query.index.IndexQueryProcessor;
 import org.apache.ignite.internal.cache.query.index.IndexQueryResult;
+import org.apache.ignite.internal.cache.query.index.sorted.maintenance.RebuildIndexWorkflowCallback;
 import org.apache.ignite.internal.managers.communication.GridMessageListener;
 import org.apache.ignite.internal.processors.GridProcessorAdapter;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
@@ -106,6 +107,7 @@ import org.apache.ignite.internal.processors.query.schema.SchemaOperationClientF
 import org.apache.ignite.internal.processors.query.schema.SchemaOperationException;
 import org.apache.ignite.internal.processors.query.schema.SchemaOperationManager;
 import org.apache.ignite.internal.processors.query.schema.SchemaOperationWorker;
+import org.apache.ignite.internal.processors.query.schema.management.SchemaManager;
 import org.apache.ignite.internal.processors.query.schema.message.SchemaAbstractDiscoveryMessage;
 import org.apache.ignite.internal.processors.query.schema.message.SchemaFinishDiscoveryMessage;
 import org.apache.ignite.internal.processors.query.schema.message.SchemaOperationStatusMessage;
@@ -116,6 +118,8 @@ import org.apache.ignite.internal.processors.query.schema.operation.SchemaAlterT
 import org.apache.ignite.internal.processors.query.schema.operation.SchemaAlterTableDropColumnOperation;
 import org.apache.ignite.internal.processors.query.schema.operation.SchemaIndexCreateOperation;
 import org.apache.ignite.internal.processors.query.schema.operation.SchemaIndexDropOperation;
+import org.apache.ignite.internal.processors.query.stat.IgniteStatisticsManager;
+import org.apache.ignite.internal.processors.query.stat.IgniteStatisticsManagerImpl;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor;
 import org.apache.ignite.internal.util.GridBoundedConcurrentLinkedHashSet;
 import org.apache.ignite.internal.util.GridSpinBusyLock;
@@ -157,6 +161,8 @@ import static org.apache.ignite.internal.GridTopic.TOPIC_SCHEMA;
 import static org.apache.ignite.internal.IgniteComponentType.INDEXING;
 import static org.apache.ignite.internal.binary.BinaryUtils.fieldTypeName;
 import static org.apache.ignite.internal.binary.BinaryUtils.typeByClass;
+import static org.apache.ignite.internal.cache.query.index.sorted.maintenance.MaintenanceRebuildIndexUtils.INDEX_REBUILD_MNTC_TASK_NAME;
+import static org.apache.ignite.internal.cache.query.index.sorted.maintenance.MaintenanceRebuildIndexUtils.parseMaintenanceTaskParameters;
 import static org.apache.ignite.internal.managers.communication.GridIoPolicy.SCHEMA_POOL;
 import static org.apache.ignite.internal.processors.query.schema.SchemaOperationException.CODE_COLUMN_EXISTS;
 
@@ -272,6 +278,9 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     /** Index build statuses. */
     private final IndexBuildStatusStorage idxBuildStatusStorage;
 
+    /** Statistic manager. */
+    private IgniteStatisticsManager statsMgr;
+
     /** Default query engine. */
     private QueryEngine dfltQryEngine;
 
@@ -284,6 +293,9 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     /** Running query manager. */
     private RunningQueryManager runningQryMgr;
 
+    /** Schema manager. */
+    private final SchemaManager schemaMgr;
+
     /**
      * Constructor.
      *
@@ -300,6 +312,8 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         else
             idx = INDEXING.inClassPath() ? U.newInstance(INDEXING.className()) : null;
 
+        schemaMgr = new SchemaManager(ctx);
+
         idxProc = ctx.indexProcessor();
 
         idxQryPrc = new IndexQueryProcessor(idxProc);
@@ -327,14 +341,18 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     @Override public void start() throws IgniteCheckedException {
         super.start();
 
+        runningQryMgr = new RunningQueryManager(ctx);
+        runningQryMgr.start(busyLock);
+
         if (idx != null) {
             ctx.resource().injectGeneric(idx);
 
             idx.start(ctx, busyLock);
         }
 
-        runningQryMgr = new RunningQueryManager(ctx);
-        runningQryMgr.start(busyLock);
+        statsMgr = new IgniteStatisticsManagerImpl(ctx);
+
+        schemaMgr.start(ctx.config().getSqlConfiguration().getSqlSchemas());
 
         ctx.io().addMessageListener(TOPIC_SCHEMA, ioLsnr);
 
@@ -344,6 +362,11 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                 ctxs.queries().evictDetailMetrics();
         }, QRY_DETAIL_METRICS_EVICTION_FREQ, QRY_DETAIL_METRICS_EVICTION_FREQ);
 
+        ctx.maintenanceRegistry().registerWorkflowCallbackIfTaskExists(
+            INDEX_REBUILD_MNTC_TASK_NAME,
+            task -> new RebuildIndexWorkflowCallback(parseMaintenanceTaskParameters(task.parameters()), ctx)
+        );
+
         idxBuildStatusStorage.start();
 
         registerMetadataForRegisteredCaches(false);
@@ -382,6 +405,8 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             idx.stop();
 
         runningQryMgr.stop();
+        schemaMgr.stop();
+        statsMgr.stop();
 
         U.closeQuiet(qryDetailMetricsEvictTask);
     }
@@ -531,9 +556,20 @@ public class GridQueryProcessor extends GridProcessorAdapter {
 
         QueryEngineConfiguration[] qryEnginesCfg = ctx.config().getSqlConfiguration().getQueryEnginesConfiguration();
 
-        // No query engines explicitly configured - indexing will be used.
-        if (F.isEmpty(qryEnginesCfg))
+        if (F.isEmpty(qryEnginesCfg)) {
+            // No query engines explicitly configured - indexing will be used.
+            // If indexing is disabled, try to find any query engine in components.
+            if (!indexingEnabled()) {
+                for (GridComponent cmp : ctx.components()) {
+                    if (cmp instanceof QueryEngine) {
+                        qryEngines = new QueryEngine[] {(QueryEngine)cmp};
+                        dfltQryEngine = (QueryEngine)cmp;
+                    }
+                }
+            }
+
             return;
+        }
 
         this.qryEnginesCfg = new QueryEngineConfigurationEx[qryEnginesCfg.length];
 
@@ -596,12 +632,11 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     }
 
     /**
-     *
      * @return Information about secondary indexes inline size. Key is a full index name, value is a effective inline size.
-     * @see GridQueryIndexing#secondaryIndexesInlineSize()
+     * @see IndexProcessor#secondaryIndexesInlineSize()
      */
     public Map<String, Integer> secondaryIndexesInlineSize() {
-        return idx != null ? idx.secondaryIndexesInlineSize() : Collections.emptyMap();
+        return moduleEnabled() ? ctx.indexProcessor().secondaryIndexesInlineSize() : Collections.emptyMap();
     }
 
     /**
@@ -976,16 +1011,23 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     /**
      * @return {@code true} If indexing module is in classpath and successfully initialized.
      */
-    public boolean moduleEnabled() {
+    public boolean indexingEnabled() {
         return idx != null;
     }
 
+    /**
+     * @return {@code true} If indexing module is enabled or any query engine is enabled.
+     */
+    public boolean moduleEnabled() {
+        return indexingEnabled() || dfltQryEngine != null;
+    }
+
     /**
      * @return Indexing.
      * @throws IgniteException If module is not enabled.
      */
     public GridQueryIndexing getIndexing() throws IgniteException {
-        checkxEnabled();
+        checkxIndexingEnabled();
 
         return idx;
     }
@@ -1032,16 +1074,20 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         ctx.cache().context().database().checkpointReadLock();
 
         try {
-            if (cacheInfo.isClientCache() && cacheInfo.isCacheContextInited() && idx.initCacheContext(cacheInfo.cacheContext()))
+            String cacheName = cacheInfo.name();
+            String schemaName = QueryUtils.normalizeSchemaName(cacheName, cacheInfo.config().getSqlSchema());
+
+            if (cacheInfo.isClientCache() && cacheInfo.isCacheContextInited()
+                && schemaMgr.initCacheContext(cacheInfo.cacheContext())) {
+                if (idx != null)
+                    idx.registerCache(cacheName, schemaName, cacheInfo);
+
                 return;
+            }
 
             synchronized (stateMux) {
                 boolean escape = cacheInfo.config().isSqlEscapeAll();
 
-                String cacheName = cacheInfo.name();
-
-                String schemaName = QueryUtils.normalizeSchemaName(cacheName, cacheInfo.config().getSqlSchema());
-
                 T3<Collection<QueryTypeCandidate>, Map<String, QueryTypeDescriptorImpl>, Map<String, QueryTypeDescriptorImpl>>
                     candRes = createQueryCandidates(cacheName, schemaName, cacheInfo, schema.entities(), escape);
 
@@ -1194,9 +1240,12 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @param isSql {@code true} in case create cache initialized from SQL.
      * @throws IgniteCheckedException If failed.
      */
-    public void onCacheStart(GridCacheContextInfo cacheInfo, QuerySchema schema,
-        boolean isSql) throws IgniteCheckedException {
-        if (idx == null)
+    public void onCacheStart(
+        GridCacheContextInfo cacheInfo,
+        QuerySchema schema,
+        boolean isSql
+    ) throws IgniteCheckedException {
+        if (!moduleEnabled())
             return;
 
         if (!busyLock.enterBusy())
@@ -1216,10 +1265,10 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @param cacheName Cache name.
      */
     public void onCacheStop(String cacheName) {
-        if (idx == null)
+        if (!moduleEnabled())
             return;
 
-        GridCacheContextInfo cacheInfo = idx.registeredCacheInfo(cacheName);
+        GridCacheContextInfo cacheInfo = schemaMgr.cacheInfo(cacheName);
 
         if (cacheInfo != null)
             onCacheStop(cacheInfo, true, true);
@@ -1231,7 +1280,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @param clearIdx If {@code true}, will clear the index.
      */
     public void onCacheStop(GridCacheContextInfo cacheInfo, boolean removeIdx, boolean clearIdx) {
-        if (idx == null)
+        if (!moduleEnabled())
             return;
 
         if (!busyLock.enterBusy())
@@ -1245,6 +1294,27 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         }
     }
 
+    /**
+     * @param cacheInfo Cache context info.
+     */
+    public void onClientCacheStop(GridCacheContextInfo cacheInfo) {
+        if (!moduleEnabled())
+            return;
+
+        if (!busyLock.enterBusy())
+            return;
+
+        try {
+            if (schemaMgr.clearCacheContext(cacheInfo.cacheContext())) {
+                if (idx != null)
+                    idx.unregisterCache(cacheInfo);
+            }
+        }
+        finally {
+            busyLock.leaveBusy();
+        }
+    }
+
     /**
      * @return Skip field lookup flag.
      */
@@ -1995,7 +2065,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
 
         }
         else
-            cacheInfo = idx.registeredCacheInfo(cacheName);
+            cacheInfo = schemaMgr.cacheInfo(cacheName);
 
         if (cacheInfo == null || !F.eq(depId, cacheInfo.dynamicDeploymentId()))
             throw new SchemaOperationException(SchemaOperationException.CODE_CACHE_NOT_FOUND, cacheName);
@@ -2054,28 +2124,26 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                     //For not started caches we shouldn't add any data to index.
                     visitor = clo -> {};
 
-                idx.dynamicIndexCreate(op0.schemaName(), op0.tableName(), idxDesc, op0.ifNotExists(), visitor);
+                schemaMgr.createIndex(op0.schemaName(), op0.tableName(), idxDesc, op0.ifNotExists(), visitor);
             }
             else if (op instanceof SchemaIndexDropOperation) {
                 SchemaIndexDropOperation op0 = (SchemaIndexDropOperation)op;
 
-                idx.dynamicIndexDrop(op0.schemaName(), op0.indexName(), op0.ifExists());
+                schemaMgr.dropIndex(op0.schemaName(), op0.indexName(), op0.ifExists());
             }
             else if (op instanceof SchemaAlterTableAddColumnOperation) {
                 SchemaAlterTableAddColumnOperation op0 = (SchemaAlterTableAddColumnOperation)op;
 
                 processDynamicAddColumn(type, op0.columns());
 
-                idx.dynamicAddColumn(op0.schemaName(), op0.tableName(), op0.columns(), op0.ifTableExists(),
-                    op0.ifNotExists());
+                schemaMgr.addColumn(op0.schemaName(), op0.tableName(), op0.columns(), op0.ifTableExists(), op0.ifNotExists());
             }
             else if (op instanceof SchemaAlterTableDropColumnOperation) {
                 SchemaAlterTableDropColumnOperation op0 = (SchemaAlterTableDropColumnOperation)op;
 
                 processDynamicDropColumn(type, op0.columns());
 
-                idx.dynamicDropColumn(op0.schemaName(), op0.tableName(), op0.columns(), op0.ifTableExists(),
-                    op0.ifExists());
+                schemaMgr.dropColumn(op0.schemaName(), op0.tableName(), op0.columns(), op0.ifTableExists(), op0.ifExists());
             }
             else if (op instanceof SchemaAddQueryEntityOperation) {
                 SchemaAddQueryEntityOperation op0 = (SchemaAddQueryEntityOperation)op;
@@ -2257,8 +2325,14 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         boolean isSql
     ) throws IgniteCheckedException {
         synchronized (stateMux) {
-            if (idx != null)
-                idx.registerCache(cacheName, schemaName, cacheInfo);
+            if (moduleEnabled()) {
+                ctx.indexProcessor().idxRowCacheRegistry().onCacheRegistered(cacheInfo);
+
+                schemaMgr.onCacheCreated(cacheName, schemaName, cacheInfo.config().getSqlFunctionClasses());
+
+                if (idx != null)
+                    idx.registerCache(cacheName, schemaName, cacheInfo);
+            }
 
             try {
                 for (QueryTypeCandidate cand : cands) {
@@ -2288,8 +2362,8 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                         }
                     }
 
-                    if (idx != null)
-                        idx.registerType(cacheInfo, desc, isSql);
+                    if (moduleEnabled())
+                        schemaMgr.onCacheTypeCreated(cacheInfo, desc, isSql);
                 }
 
                 cacheNames.add(CU.mask(cacheName));
@@ -2311,7 +2385,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @param clearIdx Clear flag.
      */
     public void onCacheStop0(GridCacheContextInfo cacheInfo, boolean destroy, boolean clearIdx) {
-        if (idx == null || !cacheNames.contains(cacheInfo.name()))
+        if (!moduleEnabled() || !cacheNames.contains(cacheInfo.name()))
             return;
 
         String cacheName = cacheInfo.name();
@@ -2348,12 +2422,17 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                     op.manager().worker().cancel();
             }
 
-            // Notify indexing.
             try {
-                idx.unregisterCache(cacheInfo, destroy, clearIdx);
+                ctx.indexProcessor().unregisterCache(cacheInfo);
+
+                schemaMgr.onCacheDestroyed(cacheName, destroy, clearIdx);
+
+                // Notify indexing.
+                if (idx != null)
+                    idx.unregisterCache(cacheInfo);
             }
             catch (Exception e) {
-                U.error(log, "Failed to clear indexing on cache unregister (will ignore): " + cacheName, e);
+                U.error(log, "Failed to clear schema manager on cache unregister (will ignore): " + cacheName, e);
             }
 
             cacheNames.remove(cacheName);
@@ -2449,7 +2528,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
 
         idxProc.markRebuildIndexesForCache(cctx, val);
 
-        idx.markAsRebuildNeeded(cctx, val);
+        schemaMgr.markIndexRebuild(cctx.name(), val);
     }
 
     /**
@@ -2459,8 +2538,8 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      */
     @SuppressWarnings("RedundantIfStatement")
     private boolean rebuildIsMeaningless(GridCacheContext cctx) {
-        // Indexing module is disabled, nothing to rebuild.
-        if (idx == null)
+        // Query module is disabled, nothing to rebuild.
+        if (!moduleEnabled())
             return true;
 
         // No data on non-affinity nodes.
@@ -2794,7 +2873,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     /**
      * @throws IgniteCheckedException If failed.
      */
-    private void checkEnabled() throws IgniteCheckedException {
+    private void checkIndexingEnabled() throws IgniteCheckedException {
         if (idx == null)
             throw new IgniteCheckedException("Indexing is disabled.");
     }
@@ -2802,7 +2881,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     /**
      * @throws IgniteException If indexing is disabled.
      */
-    private void checkxEnabled() throws IgniteException {
+    private void checkxIndexingEnabled() throws IgniteException {
         if (idx == null)
             throw new IgniteException("Failed to execute query because indexing is disabled (consider adding module " +
                 INDEXING.module() + " to classpath or moving it from 'optional' to 'libs' folder).");
@@ -2840,7 +2919,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         MvccSnapshot mvccSnapshot,
         GridQueryCancel cancel
     ) throws IgniteCheckedException {
-        checkxEnabled();
+        checkxIndexingEnabled();
 
         return idx.executeUpdateOnDataNodeTransactional(
             cctx,
@@ -2975,8 +3054,12 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         GridCacheQueryType qryType,
         @Nullable final GridQueryCancel cancel
     ) {
-        // Validate.
-        checkxEnabled();
+        if (!moduleEnabled()) {
+            throw new IgniteException("Failed to execute query because indexing is disabled and no query engine is " +
+                "configured (consider adding module " + INDEXING.module() + " to classpath or moving it " +
+                "from 'optional' to 'libs' folder or configuring any query engine with " +
+                "IgniteConfiguration.SqlConfiguration.QueryEnginesConfiguration property).");
+        }
 
         if (qry.isDistributedJoins() && qry.getPartitions() != null)
             throw new CacheException("Using both partitions and distributed JOINs is not supported for the same query");
@@ -2985,8 +3068,6 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             throw new CacheException("Execution of local SqlFieldsQuery on client node disallowed.");
 
         return executeQuerySafe(cctx, () -> {
-            assert idx != null;
-
             final String schemaName = qry.getSchema() == null ? schemaName(cctx) : qry.getSchema();
 
             IgniteOutClosureX<List<FieldsQueryCursor<List<?>>>> clo =
@@ -3044,7 +3125,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      */
     public String schemaName(GridCacheContext<?, ?> cctx) {
         if (cctx != null) {
-            String cacheSchemaName = idx.schema(cctx.name());
+            String cacheSchemaName = schemaMgr.schemaName(cctx.name());
 
             if (!F.isEmpty(cacheSchemaName))
                 return cacheSchemaName;
@@ -3159,7 +3240,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      */
     public List<Long> streamBatchedUpdateQuery(final String schemaName, final SqlClientContext cliCtx,
         final String qry, final List<Object[]> args, String qryInitiatorId) {
-        checkxEnabled();
+        checkxIndexingEnabled();
 
         if (!busyLock.enterBusy())
             throw new IllegalStateException("Failed to execute query (grid is stopping).");
@@ -3507,7 +3588,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
 
         idxProc.remove(cctx.name(), row);
 
-        if (idx != null)
+        if (indexingEnabled())
             idx.remove(cctx, desc, row);
     }
 
@@ -3524,7 +3605,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      */
     public <K, V> GridCloseableIterator<IgniteBiTuple<K, V>> queryText(final String cacheName, final String clause,
         final String resType, final IndexingQueryFilter filters, int limit) throws IgniteCheckedException {
-        checkEnabled();
+        checkIndexingEnabled();
 
         if (!busyLock.enterBusy())
             throw new IllegalStateException("Failed to execute query (grid is stopping).");
@@ -3536,7 +3617,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                 new IgniteOutClosureX<GridCloseableIterator<IgniteBiTuple<K, V>>>() {
                     @Override public GridCloseableIterator<IgniteBiTuple<K, V>> applyx() throws IgniteCheckedException {
                         String typeName = typeName(cacheName, resType);
-                        String schemaName = idx.schema(cacheName);
+                        String schemaName = schemaMgr.schemaName(cacheName);
 
                         return idx.queryLocalText(schemaName, cacheName, clause, typeName, filters, limit);
                     }
@@ -4203,4 +4284,16 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     public IndexBuildStatusStorage getIdxBuildStatusStorage() {
         return idxBuildStatusStorage;
     }
+
+    /**
+     * @return Schema manager.
+     */
+    public SchemaManager schemaManager() {
+        return schemaMgr;
+    }
+
+    /** @return Statistics manager. */
+    public IgniteStatisticsManager statsManager() {
+        return statsMgr;
+    }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQuerySchemaManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQuerySchemaManager.java
deleted file mode 100644
index b89d2b8f0b4..00000000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQuerySchemaManager.java
+++ /dev/null
@@ -1,47 +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;
-
-import org.apache.ignite.internal.processors.cache.GridCacheContextInfo;
-import org.jetbrains.annotations.Nullable;
-
-/**
- * Provides information about query engine schemas.
- */
-public interface GridQuerySchemaManager {
-    /**
-     * Find type descriptor by schema and table name.
-     *
-     * @return Query type descriptor or {@code null} if descriptor was not found.
-     */
-    public @Nullable GridQueryTypeDescriptor typeDescriptorForTable(String schemaName, String tableName);
-
-    /**
-     * Find type descriptor by schema and index name.
-     *
-     * @return Query type descriptor or {@code null} if descriptor was not found.
-     */
-    public @Nullable GridQueryTypeDescriptor typeDescriptorForIndex(String schemaName, String idxName);
-
-    /**
-     * Find cache info by schema and table name.
-     *
-     * @return Cache info or {@code null} if cache info was not found.
-     */
-    public @Nullable <K, V> GridCacheContextInfo<K, V> cacheInfoForTable(String schemaName, String tableName);
-}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryTypeDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryTypeDescriptor.java
index 1c18fe02861..02a62128489 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryTypeDescriptor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryTypeDescriptor.java
@@ -27,6 +27,13 @@ import org.jetbrains.annotations.Nullable;
  * Value descriptor which allows to extract fields from value object of given type.
  */
 public interface GridQueryTypeDescriptor {
+    /**
+     * Gets cache name of this type.
+     *
+     * @return Cache name.
+     */
+    public String cacheName();
+
     /**
      * Gets type name which uniquely identifies this type.
      *
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QuerySysIndexDescriptorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QuerySysIndexDescriptorImpl.java
index db2f4d64d8e..dc19373715a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QuerySysIndexDescriptorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QuerySysIndexDescriptorImpl.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.processors.query;
 
 import java.util.Collection;
+import java.util.Collections;
 import org.apache.ignite.cache.QueryIndexType;
 
 /**
@@ -30,15 +31,19 @@ public class QuerySysIndexDescriptorImpl implements GridQueryIndexDescriptor {
     /** Index fields. */
     private final Collection<String> fields;
 
+    /** Inline size. */
+    private final int inlineSize;
+
     /**
      * Constructor.
      *
      * @param name Index name.
      * @param fields Index fields.
      */
-    public QuerySysIndexDescriptorImpl(String name, Collection<String> fields) {
+    public QuerySysIndexDescriptorImpl(String name, Collection<String> fields, int inlineSize) {
         this.name = name;
-        this.fields = fields;
+        this.fields = Collections.unmodifiableCollection(fields);
+        this.inlineSize = inlineSize;
     }
 
     /** {@inheritDoc} */
@@ -58,11 +63,11 @@ public class QuerySysIndexDescriptorImpl implements GridQueryIndexDescriptor {
 
     /** {@inheritDoc} */
     @Override public QueryIndexType type() {
-        throw new UnsupportedOperationException("Not implemented");
+        return QueryIndexType.SORTED;
     }
 
     /** {@inheritDoc} */
     @Override public int inlineSize() {
-        throw new UnsupportedOperationException("Not implemented");
+        return inlineSize;
     }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeDescriptorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeDescriptorImpl.java
index bca1593bac2..4da4d68b249 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeDescriptorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeDescriptorImpl.java
@@ -166,10 +166,8 @@ public class QueryTypeDescriptorImpl implements GridQueryTypeDescriptor {
         this.log = coCtx.kernalContext().log(getClass());
     }
 
-    /**
-     * @return Cache name.
-     */
-    public String cacheName() {
+    /** {@inheritDoc} */
+    @Override public String cacheName() {
         return cacheName;
     }
 
@@ -726,9 +724,7 @@ public class QueryTypeDescriptorImpl implements GridQueryTypeDescriptor {
 
             GridQueryIndexing indexing = coCtx.kernalContext().query().getIndexing();
 
-            assert indexing != null;
-
-            if (indexing.isConvertibleToColumnType(schemaName, tableName(), colName, val.getClass()))
+            if (indexing != null && indexing.isConvertibleToColumnType(schemaName, tableName(), colName, val.getClass()))
                 return true;
 
             return expColType.isArray()
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java
index 6ac233ac295..d9270156dd3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java
@@ -106,6 +106,9 @@ public class QueryUtils {
     /** Name of Primary Key index for every table. */
     public static final String PRIMARY_KEY_INDEX = "_key_PK";
 
+    /** Affinity key index name. */
+    public static final String AFFINITY_KEY_INDEX = "AFFINITY_KEY";
+
     /** Schema for system view. */
     public static final String SCHEMA_SYS = getBoolean(IGNITE_SQL_SYSTEM_SCHEMA_NAME_IGNITE) ? "IGNITE" : "SYS";
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/AbstractSchemaChangeListener.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/AbstractSchemaChangeListener.java
index c3d52136a7a..9e3f8674fa1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/AbstractSchemaChangeListener.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/AbstractSchemaChangeListener.java
@@ -19,11 +19,10 @@ package org.apache.ignite.internal.processors.query.schema;
 
 import java.lang.reflect.Method;
 import java.util.List;
-import org.apache.ignite.internal.cache.query.index.Index;
 import org.apache.ignite.internal.processors.cache.GridCacheContextInfo;
-import org.apache.ignite.internal.processors.query.GridQueryIndexDescriptor;
 import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
 import org.apache.ignite.internal.processors.query.QueryField;
+import org.apache.ignite.internal.processors.query.schema.management.IndexDescriptor;
 import org.apache.ignite.spi.systemview.view.SystemView;
 
 /**
@@ -45,8 +44,7 @@ public abstract class AbstractSchemaChangeListener implements SchemaChangeListen
         String schemaName,
         String tblName,
         String idxName,
-        GridQueryIndexDescriptor idxDesc,
-        Index idx
+        IndexDescriptor idxDesc
     ) {
         // No-op.
     }
@@ -91,7 +89,7 @@ public abstract class AbstractSchemaChangeListener implements SchemaChangeListen
         GridQueryTypeDescriptor typeDesc,
         GridCacheContextInfo<?, ?> cacheInfo,
         List<String> cols
-    ){
+    ) {
         // No-op.
     }
 
@@ -105,7 +103,7 @@ public abstract class AbstractSchemaChangeListener implements SchemaChangeListen
     }
 
     /** {@inheritDoc} */
-    @Override public void onFunctionCreated(String schemaName, String name, Method method) {
+    @Override public void onFunctionCreated(String schemaName, String name, boolean deterministic, Method method) {
         // No-op.
     }
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/SchemaChangeListener.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/SchemaChangeListener.java
index 47ef2c575d2..d95fbf7104f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/SchemaChangeListener.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/SchemaChangeListener.java
@@ -19,13 +19,11 @@ package org.apache.ignite.internal.processors.query.schema;
 
 import java.lang.reflect.Method;
 import java.util.List;
-import org.apache.ignite.internal.cache.query.index.Index;
 import org.apache.ignite.internal.processors.cache.GridCacheContextInfo;
-import org.apache.ignite.internal.processors.query.GridQueryIndexDescriptor;
 import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
 import org.apache.ignite.internal.processors.query.QueryField;
+import org.apache.ignite.internal.processors.query.schema.management.IndexDescriptor;
 import org.apache.ignite.spi.systemview.view.SystemView;
-import org.jetbrains.annotations.Nullable;
 
 /**
  *
@@ -104,10 +102,8 @@ public interface SchemaChangeListener {
      * @param tblName Table name.
      * @param idxName Index name.
      * @param idxDesc Index descriptor.
-     * @param idx Index.
      */
-    public void onIndexCreated(String schemaName, String tblName, String idxName, GridQueryIndexDescriptor idxDesc,
-        @Nullable Index idx);
+    public void onIndexCreated(String schemaName, String tblName, String idxName, IndexDescriptor idxDesc);
 
     /**
      * Callback on index drop.
@@ -139,9 +135,10 @@ public interface SchemaChangeListener {
      *
      * @param schemaName Schema name.
      * @param name Function name.
+     * @param deterministic Specifies if the function is deterministic (result depends only on input parameters)
      * @param method Public static method, implementing this function.
      */
-    public void onFunctionCreated(String schemaName, String name, Method method);
+    public void onFunctionCreated(String schemaName, String name, boolean deterministic, Method method);
 
     /**
      * Callback method.
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/SchemaIndexCacheVisitorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/SchemaIndexCacheVisitorImpl.java
index 6be6755ed43..7de4716668d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/SchemaIndexCacheVisitorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/SchemaIndexCacheVisitorImpl.java
@@ -24,11 +24,13 @@ import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.cache.query.index.IndexName;
+import org.apache.ignite.internal.cache.query.index.IndexProcessor;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.InlineIndex;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearCacheAdapter;
 import org.apache.ignite.internal.processors.query.GridQueryIndexDescriptor;
-import org.apache.ignite.internal.processors.query.GridQueryIndexing;
 import org.apache.ignite.internal.processors.query.QueryTypeDescriptorImpl;
 import org.apache.ignite.internal.processors.query.QueryUtils;
 import org.apache.ignite.internal.util.future.GridCompoundFuture;
@@ -169,7 +171,7 @@ public class SchemaIndexCacheVisitorImpl implements SchemaIndexCacheVisitor {
         res.a("   Scanned rows " + stat.scannedKeys() + ", visited types " + stat.typeNames());
         res.a(U.nl());
 
-        final GridQueryIndexing idx = cctx.kernalContext().query().getIndexing();
+        IndexProcessor idxProc = cctx.kernalContext().indexProcessor();
 
         for (QueryTypeDescriptorImpl type : stat.types()) {
             res.a("        Type name=" + type.name());
@@ -178,11 +180,13 @@ public class SchemaIndexCacheVisitorImpl implements SchemaIndexCacheVisitor {
             String pk = QueryUtils.PRIMARY_KEY_INDEX;
             String tblName = type.tableName();
 
-            res.a("            Index: name=" + pk + ", size=" + idx.indexSize(type.schemaName(), tblName, pk));
+            res.a("            Index: name=" + pk + ", size=" + idxProc.index(new IndexName(
+                cctx.cache().name(), type.schemaName(), tblName, pk)).unwrap(InlineIndex.class).totalCount());
             res.a(U.nl());
 
             for (GridQueryIndexDescriptor descriptor : type.indexes().values()) {
-                long size = idx.indexSize(type.schemaName(), tblName, descriptor.name());
+                long size = idxProc.index(new IndexName(
+                    cctx.cache().name(), type.schemaName(), tblName, pk)).unwrap(InlineIndex.class).totalCount();
 
                 res.a("            Index: name=" + descriptor.name() + ", size=" + size);
                 res.a(U.nl());
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/management/AbstractIndexDescriptorFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/management/AbstractIndexDescriptorFactory.java
new file mode 100644
index 00000000000..18f281adadc
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/management/AbstractIndexDescriptorFactory.java
@@ -0,0 +1,63 @@
+/*
+ * 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.schema.management;
+
+import java.util.LinkedHashMap;
+import org.apache.ignite.internal.cache.query.index.Order;
+import org.apache.ignite.internal.cache.query.index.SortOrder;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyDefinition;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyType;
+import org.apache.ignite.internal.processors.query.GridQueryIndexDescriptor;
+import org.apache.ignite.internal.processors.query.GridQueryProperty;
+import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
+import org.apache.ignite.internal.processors.query.QueryUtils;
+import org.apache.ignite.internal.util.typedef.F;
+
+/** Abstract index descriptor factory (with helper methods). */
+public abstract class AbstractIndexDescriptorFactory implements IndexDescriptorFactory {
+    /** */
+    protected static LinkedHashMap<String, IndexKeyDefinition> indexDescriptorToKeysDefinition(
+        GridQueryIndexDescriptor idxDesc,
+        GridQueryTypeDescriptor typeDesc
+    ) {
+        LinkedHashMap<String, IndexKeyDefinition> keyDefs = new LinkedHashMap<>(idxDesc.fields().size());
+
+        for (String field : idxDesc.fields())
+            keyDefs.put(field, keyDefinition(typeDesc, field, !idxDesc.descending(field)));
+
+        return keyDefs;
+    }
+
+    /** */
+    protected static IndexKeyDefinition keyDefinition(GridQueryTypeDescriptor typeDesc, String field, boolean ascOrder) {
+        Order order = new Order(ascOrder ? SortOrder.ASC : SortOrder.DESC, null);
+
+        GridQueryProperty prop = typeDesc.property(field);
+
+        // Try to find property by alternative key field name.
+        if (prop == null && F.eq(field, QueryUtils.KEY_FIELD_NAME) && !F.isEmpty(typeDesc.keyFieldName()))
+            prop = typeDesc.property(typeDesc.keyFieldName());
+
+        Class<?> fieldType = F.eq(field, QueryUtils.KEY_FIELD_NAME) ? typeDesc.keyClass() :
+            F.eq(field, QueryUtils.VAL_FIELD_NAME) ? typeDesc.valueClass() : prop.type();
+
+        int fieldPrecision = prop != null ? prop.precision() : -1;
+
+        return new IndexKeyDefinition(IndexKeyType.forClass(fieldType).code(), order, fieldPrecision);
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/management/IndexDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/management/IndexDescriptor.java
new file mode 100644
index 00000000000..8174b5c4ddc
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/management/IndexDescriptor.java
@@ -0,0 +1,144 @@
+/*
+ * 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.schema.management;
+
+import java.util.LinkedHashMap;
+import org.apache.ignite.cache.QueryIndexType;
+import org.apache.ignite.internal.cache.query.index.Index;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyDefinition;
+
+/**
+ * Local database index object.
+ */
+public class IndexDescriptor {
+    /** */
+    private final String name;
+
+    /** */
+    private final QueryIndexType type;
+
+    /** */
+    private final LinkedHashMap<String, IndexKeyDefinition> keyDefs;
+
+    /** */
+    private final boolean isPk;
+
+    /** */
+    private final boolean isAff;
+
+    /** */
+    private final int inlineSize;
+
+    /** Index handler. */
+    private final Index idx;
+
+    /** Table descriptor. */
+    private final TableDescriptor tbl;
+
+    /** Target index descriptor for proxy index. */
+    private final IndexDescriptor targetIdx;
+
+    /** */
+    public IndexDescriptor(
+        TableDescriptor tbl,
+        String name,
+        QueryIndexType type,
+        LinkedHashMap<String, IndexKeyDefinition> keyDefs,
+        boolean isPk,
+        boolean isAff,
+        int inlineSize,
+        Index idx
+    ) {
+        this.tbl = tbl;
+        this.name = name;
+        this.type = type;
+        this.keyDefs = keyDefs;
+        this.isPk = isPk;
+        this.isAff = isAff;
+        this.targetIdx = null;
+        this.inlineSize = inlineSize;
+        this.idx = idx;
+    }
+
+    /** Constructor for proxy index descriptor. */
+    public IndexDescriptor(
+        String name,
+        LinkedHashMap<String, IndexKeyDefinition> keyDefs,
+        IndexDescriptor targetIdx
+    ) {
+        this.name = name;
+        this.keyDefs = keyDefs;
+        isPk = false;
+        isAff = false;
+        this.targetIdx = targetIdx;
+        tbl = targetIdx.table();
+        type = targetIdx.type();
+        inlineSize = targetIdx.inlineSize();
+        idx = targetIdx.index();
+    }
+
+    /** */
+    public String name() {
+        return name;
+    }
+
+    /** */
+    public QueryIndexType type() {
+        return type;
+    }
+
+    /** */
+    public LinkedHashMap<String, IndexKeyDefinition> keyDefinitions() {
+        return keyDefs;
+    }
+
+    /** */
+    public boolean isPk() {
+        return isPk;
+    }
+
+    /** */
+    public boolean isAffinity() {
+        return isAff;
+    }
+
+    /** */
+    public boolean isProxy() {
+        return targetIdx != null;
+    }
+
+    /** */
+    public int inlineSize() {
+        return inlineSize;
+    }
+
+    /** Index handler. */
+    public Index index() {
+        return idx;
+    }
+
+    /** Target index descriptor for proxy index. */
+    public IndexDescriptor targetIdx() {
+        return targetIdx;
+    }
+
+    /** Table descriptor. */
+    public TableDescriptor table() {
+        return tbl;
+    }
+}
diff --git a/modules/indexing/src/main/java/org/apache/ignite/spi/systemview/view/SqlSchemaView.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/management/IndexDescriptorFactory.java
similarity index 56%
copy from modules/indexing/src/main/java/org/apache/ignite/spi/systemview/view/SqlSchemaView.java
copy to modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/management/IndexDescriptorFactory.java
index 7a1ecbd415f..fb4497c3278 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/spi/systemview/view/SqlSchemaView.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/management/IndexDescriptorFactory.java
@@ -15,33 +15,20 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.spi.systemview.view;
+package org.apache.ignite.internal.processors.query.schema.management;
 
-import org.apache.ignite.internal.managers.systemview.walker.Order;
-import org.apache.ignite.internal.processors.query.h2.H2Schema;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.processors.query.GridQueryIndexDescriptor;
+import org.apache.ignite.internal.processors.query.schema.SchemaIndexCacheVisitor;
+import org.jetbrains.annotations.Nullable;
 
-/**
- * Sql schema system view representation.
- */
-public class SqlSchemaView {
-    /** H2 schema. */
-    private final H2Schema schema;
-
-    /**
-     * @param schema H2 schema.
-     */
-    public SqlSchemaView(H2Schema schema) {
-        this.schema = schema;
-    }
-
-    /** @return Schema name. */
-    @Order
-    public String schemaName() {
-        return schema.schemaName();
-    }
-
-    /** @return {@code True} if schema is predefined, {@code false} otherwise. */
-    public boolean predefined() {
-        return schema.predefined();
-    }
+/** Factory interface, to create index descriptors. */
+public interface IndexDescriptorFactory {
+    /** */
+    public IndexDescriptor create(
+        GridKernalContext ctx,
+        GridQueryIndexDescriptor idxDesc,
+        TableDescriptor tbl,
+        @Nullable SchemaIndexCacheVisitor cacheVisitor
+    );
 }
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2Schema.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/management/SchemaDescriptor.java
similarity index 68%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2Schema.java
rename to modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/management/SchemaDescriptor.java
index fba85c3a14a..792e78aba5a 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2Schema.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/management/SchemaDescriptor.java
@@ -15,24 +15,25 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.internal.processors.query.h2;
+package org.apache.ignite.internal.processors.query.schema.management;
 
 import java.util.Collection;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
+import org.apache.ignite.internal.processors.query.QueryTypeNameKey;
 
 /**
- * Database schema object.
+ * Local database schema object.
  */
-public class H2Schema {
+public class SchemaDescriptor {
     /** */
     private final String schemaName;
 
     /** */
-    private final ConcurrentMap<String, H2TableDescriptor> tbls = new ConcurrentHashMap<>();
+    private final ConcurrentMap<String, TableDescriptor> tbls = new ConcurrentHashMap<>();
 
     /** */
-    private final ConcurrentMap<H2TypeKey, H2TableDescriptor> typeToTbl = new ConcurrentHashMap<>();
+    private final ConcurrentMap<QueryTypeNameKey, TableDescriptor> typeToTbl = new ConcurrentHashMap<>();
 
     /** Whether schema is predefined and cannot be dorpped. */
     private final boolean predefined;
@@ -46,7 +47,7 @@ public class H2Schema {
      * @param schemaName Schema name.
      * @param predefined Predefined flag.
      */
-    public H2Schema(String schemaName, boolean predefined) {
+    public SchemaDescriptor(String schemaName, boolean predefined) {
         this.schemaName = schemaName;
         this.predefined = predefined;
     }
@@ -78,7 +79,7 @@ public class H2Schema {
     /**
      * @return Tables.
      */
-    public Collection<H2TableDescriptor> tables() {
+    public Collection<TableDescriptor> tables() {
         return tbls.values();
     }
 
@@ -86,7 +87,7 @@ public class H2Schema {
      * @param tblName Table name.
      * @return Table.
      */
-    public H2TableDescriptor tableByName(String tblName) {
+    public TableDescriptor tableByName(String tblName) {
         return tbls.get(tblName);
     }
 
@@ -94,19 +95,19 @@ public class H2Schema {
      * @param typeName Type name.
      * @return Table.
      */
-    public H2TableDescriptor tableByTypeName(String cacheName, String typeName) {
-        return typeToTbl.get(new H2TypeKey(cacheName, typeName));
+    public TableDescriptor tableByTypeName(String cacheName, String typeName) {
+        return typeToTbl.get(new QueryTypeNameKey(cacheName, typeName));
     }
 
     /**
      * @param tbl Table descriptor.
      */
-    public void add(H2TableDescriptor tbl) {
-        if (tbls.putIfAbsent(tbl.tableName(), tbl) != null)
-            throw new IllegalStateException("Table already registered: " + tbl.fullTableName());
+    public void add(TableDescriptor tbl) {
+        if (tbls.putIfAbsent(tbl.type().tableName(), tbl) != null)
+            throw new IllegalStateException("Table already registered: " + tbl.type().tableName());
 
-        if (typeToTbl.putIfAbsent(new H2TypeKey(tbl.cacheName(), tbl.typeName()), tbl) != null)
-            throw new IllegalStateException("Table already registered: " + tbl.fullTableName());
+        if (typeToTbl.putIfAbsent(new QueryTypeNameKey(tbl.cacheInfo().name(), tbl.type().name()), tbl) != null)
+            throw new IllegalStateException("Table already registered: " + tbl.type().tableName());
     }
 
     /**
@@ -114,12 +115,10 @@ public class H2Schema {
      *
      * @param tbl Table to be removed.
      */
-    public void drop(H2TableDescriptor tbl) {
-        tbl.onDrop();
+    public void drop(TableDescriptor tbl) {
+        tbls.remove(tbl.type().tableName());
 
-        tbls.remove(tbl.tableName());
-
-        typeToTbl.remove(new H2TypeKey(tbl.cacheName(), tbl.typeName()));
+        typeToTbl.remove(new QueryTypeNameKey(tbl.cacheInfo().name(), tbl.type().name()));
     }
 
     /**
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/management/SchemaManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/management/SchemaManager.java
new file mode 100644
index 00000000000..9a94bc9368c
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/management/SchemaManager.java
@@ -0,0 +1,1357 @@
+/*
+ * 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.schema.management;
+
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.text.MessageFormat;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.IgniteSystemProperties;
+import org.apache.ignite.cache.QueryIndexType;
+import org.apache.ignite.cache.query.annotations.QuerySqlFunction;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.cache.query.index.IndexName;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyDefinition;
+import org.apache.ignite.internal.jdbc2.JdbcUtils;
+import org.apache.ignite.internal.managers.systemview.walker.SqlIndexViewWalker;
+import org.apache.ignite.internal.managers.systemview.walker.SqlSchemaViewWalker;
+import org.apache.ignite.internal.managers.systemview.walker.SqlTableColumnViewWalker;
+import org.apache.ignite.internal.managers.systemview.walker.SqlTableViewWalker;
+import org.apache.ignite.internal.managers.systemview.walker.SqlViewColumnViewWalker;
+import org.apache.ignite.internal.managers.systemview.walker.SqlViewViewWalker;
+import org.apache.ignite.internal.processors.cache.CacheGroupDescriptor;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.GridCacheContextInfo;
+import org.apache.ignite.internal.processors.metric.impl.MetricUtils;
+import org.apache.ignite.internal.processors.query.ColumnInformation;
+import org.apache.ignite.internal.processors.query.GridQueryIndexDescriptor;
+import org.apache.ignite.internal.processors.query.GridQueryProperty;
+import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
+import org.apache.ignite.internal.processors.query.QueryField;
+import org.apache.ignite.internal.processors.query.QueryIndexDescriptorImpl;
+import org.apache.ignite.internal.processors.query.QuerySysIndexDescriptorImpl;
+import org.apache.ignite.internal.processors.query.QueryUtils;
+import org.apache.ignite.internal.processors.query.QueryUtils.KeyOrValProperty;
+import org.apache.ignite.internal.processors.query.TableInformation;
+import org.apache.ignite.internal.processors.query.schema.AbstractSchemaChangeListener;
+import org.apache.ignite.internal.processors.query.schema.SchemaChangeListener;
+import org.apache.ignite.internal.processors.query.schema.SchemaIndexCacheVisitor;
+import org.apache.ignite.internal.processors.query.schema.SchemaOperationException;
+import org.apache.ignite.internal.util.GridConcurrentHashSet;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.T2;
+import org.apache.ignite.spi.systemview.view.SystemView;
+import org.apache.ignite.spi.systemview.view.sql.SqlIndexView;
+import org.apache.ignite.spi.systemview.view.sql.SqlSchemaView;
+import org.apache.ignite.spi.systemview.view.sql.SqlTableColumnView;
+import org.apache.ignite.spi.systemview.view.sql.SqlTableView;
+import org.apache.ignite.spi.systemview.view.sql.SqlViewColumnView;
+import org.apache.ignite.spi.systemview.view.sql.SqlViewView;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.internal.processors.metric.impl.MetricUtils.metricName;
+import static org.apache.ignite.internal.processors.query.QueryUtils.KEY_FIELD_NAME;
+import static org.apache.ignite.internal.processors.query.QueryUtils.VAL_FIELD_NAME;
+import static org.apache.ignite.internal.processors.query.QueryUtils.matches;
+
+/**
+ * Schema manager. Responsible for all manipulations on schema objects.
+ */
+public class SchemaManager {
+    /** */
+    public static final String SQL_SCHEMA_VIEW = "schemas";
+
+    /** */
+    public static final String SQL_SCHEMA_VIEW_DESC = "SQL schemas";
+
+    /** */
+    public static final String SQL_TBLS_VIEW = "tables";
+
+    /** */
+    public static final String SQL_TBLS_VIEW_DESC = "SQL tables";
+
+    /** */
+    public static final String SQL_VIEWS_VIEW = "views";
+
+    /** */
+    public static final String SQL_VIEWS_VIEW_DESC = "SQL views";
+
+    /** */
+    public static final String SQL_IDXS_VIEW = "indexes";
+
+    /** */
+    public static final String SQL_IDXS_VIEW_DESC = "SQL indexes";
+
+    /** */
+    public static final String SQL_TBL_COLS_VIEW = metricName("table", "columns");
+
+    /** */
+    public static final String SQL_TBL_COLS_VIEW_DESC = "SQL table columns";
+
+    /** */
+    public static final String SQL_VIEW_COLS_VIEW = metricName("view", "columns");
+
+    /** */
+    public static final String SQL_VIEW_COLS_VIEW_DESC = "SQL view columns";
+
+    /** */
+    private static final Map<QueryIndexType, IndexDescriptorFactory>
+        IDX_DESC_FACTORY = new EnumMap<>(QueryIndexType.class);
+
+    /** Index descriptor factory for current instance. */
+    private final Map<QueryIndexType, IndexDescriptorFactory> idxDescFactory = new EnumMap<>(QueryIndexType.class);
+
+    /** */
+    private volatile SchemaChangeListener lsnr;
+
+    /** Collection of schemaNames and registered tables. */
+    private final ConcurrentMap<String, SchemaDescriptor> schemas = new ConcurrentHashMap<>();
+
+    /** Cache name -> schema name. */
+    private final Map<String, String> cacheName2schema = new ConcurrentHashMap<>();
+
+    /** Map from table identifier to table. */
+    private final ConcurrentMap<T2<String, String>, TableDescriptor> id2tbl = new ConcurrentHashMap<>();
+
+    /** System VIEW collection. */
+    private final Set<SystemView<?>> sysViews = new GridConcurrentHashSet<>();
+
+    /** Lock to synchronize schema operations. */
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /**
+     * Constructor.
+     *
+     * @param ctx Kernal context.
+     */
+    public SchemaManager(GridKernalContext ctx) {
+        this.ctx = ctx;
+
+        log = ctx.log(SchemaManager.class);
+    }
+
+    /** Register index descriptor factory for custom index type. */
+    public static void registerIndexDescriptorFactory(QueryIndexType type, IndexDescriptorFactory factory) {
+        IDX_DESC_FACTORY.put(type, factory);
+    }
+
+    /** Unregister index descriptor factory for custom index type. */
+    public static void unregisterIndexDescriptorFactory(QueryIndexType type) {
+        IDX_DESC_FACTORY.remove(type);
+    }
+
+    /**
+     * Handle node start.
+     *
+     * @param schemaNames Schema names.
+     */
+    public void start(String[] schemaNames) throws IgniteCheckedException {
+        lsnr = schemaChangeListener(ctx);
+
+        idxDescFactory.putAll(IDX_DESC_FACTORY);
+
+        // Register default index descriptor factory for tree indexes if it wasn't overrided.
+        if (!idxDescFactory.containsKey(QueryIndexType.SORTED))
+            idxDescFactory.put(QueryIndexType.SORTED, new SortedIndexDescriptorFactory(log));
+
+        ctx.systemView().registerView(SQL_SCHEMA_VIEW, SQL_SCHEMA_VIEW_DESC,
+            new SqlSchemaViewWalker(),
+            schemas.values(),
+            SqlSchemaView::new);
+
+        ctx.systemView().registerView(SQL_TBLS_VIEW, SQL_TBLS_VIEW_DESC,
+            new SqlTableViewWalker(),
+            id2tbl.values(),
+            SqlTableView::new);
+
+        ctx.systemView().registerView(SQL_VIEWS_VIEW, SQL_VIEWS_VIEW_DESC,
+            new SqlViewViewWalker(),
+            sysViews,
+            SqlViewView::new);
+
+        ctx.systemView().registerInnerCollectionView(SQL_IDXS_VIEW, SQL_IDXS_VIEW_DESC,
+            new SqlIndexViewWalker(),
+            id2tbl.values(),
+            t -> t.indexes().values(),
+            SqlIndexView::new);
+
+        ctx.systemView().registerInnerCollectionView(SQL_TBL_COLS_VIEW, SQL_TBL_COLS_VIEW_DESC,
+            new SqlTableColumnViewWalker(),
+            id2tbl.values(),
+            this::tableColumns,
+            SqlTableColumnView::new);
+
+        ctx.systemView().registerInnerCollectionView(SQL_VIEW_COLS_VIEW, SQL_VIEW_COLS_VIEW_DESC,
+            new SqlViewColumnViewWalker(),
+            sysViews,
+            v -> MetricUtils.systemViewAttributes(v).entrySet(),
+            SqlViewColumnView::new);
+
+        lock.writeLock().lock();
+
+        try {
+            // Register PUBLIC schema which is always present.
+            createSchema(QueryUtils.DFLT_SCHEMA, true);
+
+            // Create schemas listed in node's configuration.
+            createPredefinedSchemas(schemaNames);
+        }
+        finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    /** */
+    private Collection<GridQueryProperty> tableColumns(TableDescriptor tblDesc) {
+        GridQueryTypeDescriptor typeDesc = tblDesc.type();
+        Collection<GridQueryProperty> props = typeDesc.properties().values();
+
+        if (!tblDesc.type().properties().containsKey(KEY_FIELD_NAME))
+            props = F.concat(false, new KeyOrValProperty(true, KEY_FIELD_NAME, typeDesc.keyClass()), props);
+
+        if (!tblDesc.type().properties().containsKey(VAL_FIELD_NAME))
+            props = F.concat(false, new KeyOrValProperty(false, VAL_FIELD_NAME, typeDesc.valueClass()), props);
+
+        return props;
+    }
+
+    /**
+     * Handle node stop.
+     */
+    public void stop() {
+        schemas.clear();
+        cacheName2schema.clear();
+    }
+
+    /**
+     * Registers new system view.
+     *
+     * @param schema Schema to create view in.
+     * @param view System view.
+     */
+    public void createSystemView(String schema, SystemView<?> view) {
+        boolean disabled = IgniteSystemProperties.getBoolean(IgniteSystemProperties.IGNITE_SQL_DISABLE_SYSTEM_VIEWS);
+
+        if (disabled) {
+            if (log.isInfoEnabled()) {
+                log.info("SQL system views will not be created because they are disabled (see " +
+                    IgniteSystemProperties.IGNITE_SQL_DISABLE_SYSTEM_VIEWS + " system property)");
+            }
+
+            return;
+        }
+
+        lock.writeLock().lock();
+
+        try {
+            createSchema(schema, true);
+
+            sysViews.add(view);
+
+            lsnr.onSystemViewCreated(schema, view);
+        }
+        finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    /**
+     * Create predefined schemas.
+     *
+     * @param schemaNames Schema names.
+     */
+    private void createPredefinedSchemas(String[] schemaNames) {
+        assert lock.isWriteLockedByCurrentThread();
+
+        if (F.isEmpty(schemaNames))
+            return;
+
+        Collection<String> schemaNames0 = new LinkedHashSet<>();
+
+        for (String schemaName : schemaNames) {
+            if (F.isEmpty(schemaName))
+                continue;
+
+            schemaName = QueryUtils.normalizeSchemaName(null, schemaName);
+
+            schemaNames0.add(schemaName);
+        }
+
+        for (String schemaName : schemaNames0)
+            createSchema(schemaName, true);
+    }
+
+    /**
+     * Invoked when cache is created.
+     *
+     * @param cacheName Cache name.
+     * @param schemaName Schema name.
+     * @param sqlFuncs Custom SQL functions.
+     * @throws IgniteCheckedException If failed.
+     */
+    public void onCacheCreated(String cacheName, String schemaName, Class<?>[] sqlFuncs) throws IgniteCheckedException {
+        lock.writeLock().lock();
+
+        try {
+            createSchema(schemaName, false);
+
+            cacheName2schema.put(cacheName, schemaName);
+
+            createSqlFunctions(schemaName, sqlFuncs);
+        }
+        finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    /**
+     * Registers new class description.
+     *
+     * @param cacheInfo Cache info.
+     * @param type Type descriptor.
+     * @param isSql Whether SQL enabled.
+     * @throws IgniteCheckedException If failed.
+     */
+    public void onCacheTypeCreated(
+        GridCacheContextInfo<?, ?> cacheInfo,
+        GridQueryTypeDescriptor type,
+        boolean isSql
+    ) throws IgniteCheckedException {
+        validateTypeDescriptor(type);
+
+        lock.writeLock().lock();
+
+        try {
+            String schemaName = schemaName(cacheInfo.name());
+
+            SchemaDescriptor schema = schema(schemaName);
+
+            TableDescriptor tbl = new TableDescriptor(cacheInfo, type, isSql);
+
+            schema.add(tbl);
+
+            T2<String, String> tableId = new T2<>(schemaName, type.tableName());
+
+            if (id2tbl.putIfAbsent(tableId, tbl) != null)
+                throw new IllegalStateException("Table already exists: " + schemaName + '.' + type.tableName());
+
+            lsnr.onSqlTypeCreated(schemaName, type, cacheInfo);
+
+            // Create system indexes.
+            createPkIndex(tbl);
+            createAffinityIndex(tbl);
+
+            // Create initial user indexes.
+            for (GridQueryIndexDescriptor idxDesc : tbl.type().indexes().values())
+                createIndex0(idxDesc, tbl, null);
+        }
+        finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    /**
+     * Validates properties described by query types.
+     *
+     * @param type Type descriptor.
+     * @throws IgniteCheckedException If validation failed.
+     */
+    private static void validateTypeDescriptor(GridQueryTypeDescriptor type) throws IgniteCheckedException {
+        assert type != null;
+
+        Collection<String> names = new HashSet<>(type.fields().keySet());
+
+        if (names.size() < type.fields().size())
+            throw new IgniteCheckedException("Found duplicated properties with the same name [keyType=" +
+                type.keyClass().getName() + ", valueType=" + type.valueClass().getName() + "]");
+
+        String ptrn = "Name ''{0}'' is reserved and cannot be used as a field name [type=" + type.name() + "]";
+
+        for (String name : names) {
+            if (name.equalsIgnoreCase(KEY_FIELD_NAME) || name.equalsIgnoreCase(VAL_FIELD_NAME))
+                throw new IgniteCheckedException(MessageFormat.format(ptrn, name));
+        }
+    }
+
+    /**
+     * Handle cache destroy.
+     *
+     * @param cacheName Cache name.
+     * @param rmvIdx Whether to remove indexes.
+     * @param clearIdx Whether to clear the index.
+     */
+    public void onCacheDestroyed(String cacheName, boolean rmvIdx, boolean clearIdx) {
+        lock.writeLock().lock();
+
+        try {
+            String schemaName = schemaName(cacheName);
+
+            SchemaDescriptor schema = schemas.get(schemaName);
+
+            // Remove this mapping only after callback to DML proc - it needs that mapping internally.
+            cacheName2schema.remove(cacheName);
+
+            for (TableDescriptor tbl : schema.tables()) {
+                if (F.eq(tbl.cacheInfo().name(), cacheName)) {
+                    Collection<IndexDescriptor> idxs = new ArrayList<>(tbl.indexes().values());
+
+                    for (IndexDescriptor idx : idxs) {
+                        if (!idx.isProxy()) // Proxies will be deleted implicitly after deleting target index.
+                            dropIndex(tbl, idx.name(), !clearIdx);
+                    }
+
+                    lsnr.onSqlTypeDropped(schemaName, tbl.type(), rmvIdx);
+
+                    schema.drop(tbl);
+
+                    T2<String, String> tableId = new T2<>(tbl.type().schemaName(), tbl.type().tableName());
+                    id2tbl.remove(tableId, tbl);
+                }
+            }
+
+            if (schema.decrementUsageCount()) {
+                schemas.remove(schemaName);
+
+                lsnr.onSchemaDropped(schemaName);
+            }
+        }
+        finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    /**
+     * Create and register schema if needed.
+     *
+     * @param schemaName Schema name.
+     * @param predefined Whether this is predefined schema.
+     */
+    private void createSchema(String schemaName, boolean predefined) {
+        assert lock.isWriteLockedByCurrentThread();
+
+        if (!predefined)
+            predefined = F.eq(QueryUtils.DFLT_SCHEMA, schemaName);
+
+        SchemaDescriptor schema = new SchemaDescriptor(schemaName, predefined);
+
+        SchemaDescriptor oldSchema = schemas.putIfAbsent(schemaName, schema);
+
+        if (oldSchema == null)
+            lsnr.onSchemaCreated(schemaName);
+        else
+            schema = oldSchema;
+
+        schema.incrementUsageCount();
+    }
+
+    /**
+     * Registers SQL functions.
+     *
+     * @param schema Schema.
+     * @param clss Classes.
+     * @throws IgniteCheckedException If failed.
+     */
+    private void createSqlFunctions(String schema, Class<?>[] clss) throws IgniteCheckedException {
+        assert lock.isWriteLockedByCurrentThread();
+
+        if (F.isEmpty(clss))
+            return;
+
+        for (Class<?> cls : clss) {
+            for (Method m : cls.getDeclaredMethods()) {
+                QuerySqlFunction ann = m.getAnnotation(QuerySqlFunction.class);
+
+                if (ann != null) {
+                    int modifiers = m.getModifiers();
+
+                    if (!Modifier.isStatic(modifiers) || !Modifier.isPublic(modifiers))
+                        throw new IgniteCheckedException("Method " + m.getName() + " must be public static.");
+
+                    String alias = ann.alias().isEmpty() ? m.getName() : ann.alias();
+
+                    lsnr.onFunctionCreated(schema, alias, ann.deterministic(), m);
+                }
+            }
+        }
+    }
+
+    /**
+     * Get schema name for cache.
+     *
+     * @param cacheName Cache name.
+     * @return Schema name.
+     */
+    public String schemaName(String cacheName) {
+        String res = cacheName2schema.get(cacheName);
+
+        if (res == null)
+            res = "";
+
+        return res;
+    }
+
+    /**
+     * Get schemas names.
+     *
+     * @return Schemas names.
+     */
+    public Set<String> schemaNames() {
+        return new HashSet<>(schemas.keySet());
+    }
+
+    /**
+     * Get schema by name.
+     *
+     * @param schemaName Schema name.
+     * @return Schema.
+     */
+    private SchemaDescriptor schema(String schemaName) {
+        return schemas.get(schemaName);
+    }
+
+    /** */
+    private void createPkIndex(TableDescriptor tbl) throws IgniteCheckedException {
+        assert lock.isWriteLockedByCurrentThread();
+
+        GridQueryIndexDescriptor idxDesc = new QuerySysIndexDescriptorImpl(QueryUtils.PRIMARY_KEY_INDEX,
+            Collections.emptyList(), tbl.type().primaryKeyInlineSize()); // _KEY field will be added implicitly.
+
+        // Add primary key index.
+        createIndex0(
+            idxDesc,
+            tbl,
+            null
+        );
+    }
+
+    /** */
+    private void createAffinityIndex(TableDescriptor tbl) throws IgniteCheckedException {
+        assert lock.isWriteLockedByCurrentThread();
+
+        // Locate index where affinity column is first (if any).
+        if (tbl.affinityKey() != null && !tbl.affinityKey().equals(KEY_FIELD_NAME)) {
+            boolean affIdxFound = false;
+
+            for (GridQueryIndexDescriptor idxDesc : tbl.type().indexes().values()) {
+                if (idxDesc.type() != QueryIndexType.SORTED)
+                    continue;
+
+                affIdxFound |= F.eq(tbl.affinityKey(), F.first(idxDesc.fields()));
+            }
+
+            // Add explicit affinity key index if nothing alike was found.
+            if (!affIdxFound) {
+                GridQueryIndexDescriptor idxDesc = new QuerySysIndexDescriptorImpl(QueryUtils.AFFINITY_KEY_INDEX,
+                    Collections.singleton(tbl.affinityKey()), tbl.type().affinityFieldInlineSize());
+
+                createIndex0(
+                    idxDesc,
+                    tbl,
+                    null
+                );
+            }
+        }
+    }
+
+    /** */
+    private void createIndex0(
+        GridQueryIndexDescriptor idxDesc,
+        TableDescriptor tbl,
+        @Nullable SchemaIndexCacheVisitor cacheVisitor
+    ) throws IgniteCheckedException {
+        // If cacheVisitor is not null, index creation can be durable, schema lock should not be acquired in this case
+        // to avoid blocking of other schema operations.
+        assert cacheVisitor == null || !lock.isWriteLockedByCurrentThread();
+
+        IndexDescriptorFactory factory = idxDescFactory.get(idxDesc.type());
+
+        if (factory == null)
+            throw new IllegalStateException("Not found factory for index type: " + idxDesc.type());
+
+        IndexDescriptor desc = factory.create(ctx, idxDesc, tbl, cacheVisitor);
+
+        addIndex(tbl, desc);
+    }
+
+    /** Create proxy index for real index if needed. */
+    private void createProxyIndex(
+        IndexDescriptor idxDesc,
+        TableDescriptor tbl
+    ) {
+        assert lock.isWriteLockedByCurrentThread();
+
+        GridQueryTypeDescriptor typeDesc = tbl.type();
+
+        if (F.isEmpty(typeDesc.keyFieldName()) && F.isEmpty(typeDesc.valueFieldName()))
+            return;
+
+        String keyAlias = typeDesc.keyFieldAlias();
+        String valAlias = typeDesc.valueFieldAlias();
+
+        LinkedHashMap<String, IndexKeyDefinition> proxyKeyDefs = new LinkedHashMap<>(idxDesc.keyDefinitions().size());
+
+        boolean modified = false;
+
+        for (Map.Entry<String, IndexKeyDefinition> keyDef : idxDesc.keyDefinitions().entrySet()) {
+            String oldFieldName = keyDef.getKey();
+            String newFieldName = oldFieldName;
+
+            // Replace _KEY/_VAL field with aliases and vice versa.
+            if (F.eq(oldFieldName, QueryUtils.KEY_FIELD_NAME) && !F.isEmpty(keyAlias))
+                newFieldName = keyAlias;
+            else if (F.eq(oldFieldName, QueryUtils.VAL_FIELD_NAME) && !F.isEmpty(valAlias))
+                newFieldName = valAlias;
+            else if (F.eq(oldFieldName, keyAlias))
+                newFieldName = QueryUtils.KEY_FIELD_NAME;
+            else if (F.eq(oldFieldName, valAlias))
+                newFieldName = QueryUtils.VAL_FIELD_NAME;
+
+            modified |= !F.eq(oldFieldName, newFieldName);
+
+            proxyKeyDefs.put(newFieldName, keyDef.getValue());
+        }
+
+        if (!modified)
+            return;
+
+        String proxyName = generateProxyIdxName(idxDesc.name());
+
+        IndexDescriptor proxyDesc = new IndexDescriptor(proxyName, proxyKeyDefs, idxDesc);
+
+        tbl.addIndex(proxyName, proxyDesc);
+
+        lsnr.onIndexCreated(tbl.type().schemaName(), tbl.type().tableName(), proxyName, proxyDesc);
+    }
+
+    /** */
+    public static String generateProxyIdxName(String idxName) {
+        return idxName + "_proxy";
+    }
+
+    /**
+     * Create index dynamically.
+     *
+     * @param schemaName Schema name.
+     * @param tblName Table name.
+     * @param idxDesc Index descriptor.
+     * @param ifNotExists If-not-exists.
+     * @param cacheVisitor Cache visitor.
+     * @throws IgniteCheckedException If failed.
+     */
+    public void createIndex(
+        String schemaName,
+        String tblName,
+        QueryIndexDescriptorImpl idxDesc,
+        boolean ifNotExists,
+        SchemaIndexCacheVisitor cacheVisitor
+    ) throws IgniteCheckedException {
+        TableDescriptor tbl;
+
+        lock.readLock().lock();
+
+        try {
+            // Locate table.
+            tbl = table(schemaName, tblName);
+
+            if (tbl == null)
+                throw new SchemaOperationException(SchemaOperationException.CODE_TABLE_NOT_FOUND, tblName);
+
+            if (tbl.indexes().containsKey(idxDesc.name())) {
+                if (ifNotExists)
+                    return;
+                else
+                    throw new SchemaOperationException(SchemaOperationException.CODE_INDEX_EXISTS, idxDesc.name());
+            }
+        }
+        finally {
+            lock.readLock().unlock();
+        }
+
+        createIndex0(idxDesc, tbl, cacheVisitor);
+    }
+
+    /**
+     * Add index to the schema.
+     *
+     * @param tbl Table descriptor.
+     * @param idxDesc Index descriptor.
+     */
+    public void addIndex(TableDescriptor tbl, IndexDescriptor idxDesc) throws IgniteCheckedException {
+        lock.writeLock().lock();
+
+        try {
+            // Check under the lock if table is still exists.
+            if (table(tbl.type().schemaName(), tbl.type().tableName()) == null) {
+                ctx.indexProcessor().removeIndex(new IndexName(tbl.cacheInfo().name(), tbl.type().schemaName(),
+                        tbl.type().tableName(), idxDesc.name()), false);
+
+                throw new SchemaOperationException(SchemaOperationException.CODE_TABLE_NOT_FOUND, tbl.type().tableName());
+            }
+
+            tbl.addIndex(idxDesc.name(), idxDesc);
+
+            lsnr.onIndexCreated(tbl.type().schemaName(), tbl.type().tableName(), idxDesc.name(), idxDesc);
+
+            createProxyIndex(idxDesc, tbl);
+        }
+        finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    /**
+     * Drop index.
+     *
+     * @param schemaName Schema name.
+     * @param idxName Index name.
+     * @param ifExists If exists.
+     */
+    public void dropIndex(String schemaName, String idxName, boolean ifExists) throws IgniteCheckedException {
+        lock.writeLock().lock();
+
+        try {
+            IndexDescriptor idxDesc = index(schemaName, idxName);
+
+            if (idxDesc == null) {
+                if (ifExists)
+                    return;
+                else
+                    throw new SchemaOperationException(SchemaOperationException.CODE_INDEX_NOT_FOUND, idxName);
+            }
+
+            dropIndex(idxDesc.table(), idxName, false);
+        }
+        finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    /**
+     * Drop index.
+     *
+     * @param tbl Table descriptor.
+     * @param idxName Index name.
+     */
+    private void dropIndex(TableDescriptor tbl, String idxName, boolean softDelete) {
+        assert lock.isWriteLockedByCurrentThread();
+
+        String schemaName = tbl.type().schemaName();
+        String cacheName = tbl.type().cacheName();
+        String tableName = tbl.type().tableName();
+
+        IndexDescriptor idxDesc = tbl.dropIndex(idxName);
+
+        assert idxDesc != null;
+
+        if (!idxDesc.isProxy()) {
+            ctx.indexProcessor().removeIndex(
+                new IndexName(cacheName, schemaName, tableName, idxName),
+                softDelete
+            );
+        }
+
+        lsnr.onIndexDropped(schemaName, tableName, idxName);
+
+        // Drop proxy for target index.
+        for (IndexDescriptor proxyDesc : tbl.indexes().values()) {
+            if (proxyDesc.targetIdx() == idxDesc) {
+                tbl.dropIndex(proxyDesc.name());
+
+                lsnr.onIndexDropped(schemaName, tableName, proxyDesc.name());
+            }
+        }
+    }
+
+    /**
+     * Add column.
+     *
+     * @param schemaName Schema name.
+     * @param tblName Table name.
+     * @param cols Columns.
+     * @param ifTblExists If table exists.
+     * @param ifColNotExists If column not exists.
+     * @throws IgniteCheckedException If failed.
+     */
+    public void addColumn(
+        String schemaName,
+        String tblName,
+        List<QueryField> cols,
+        boolean ifTblExists,
+        boolean ifColNotExists
+    ) throws IgniteCheckedException {
+        assert !ifColNotExists || cols.size() == 1;
+
+        lock.writeLock().lock();
+
+        try {
+            // Locate table.
+            TableDescriptor tbl = table(schemaName, tblName);
+
+            if (tbl == null) {
+                if (!ifTblExists) {
+                    throw new IgniteCheckedException("Table not found in schema manager [schemaName=" + schemaName +
+                        ", tblName=" + tblName + ']');
+                }
+                else
+                    return;
+            }
+
+            lsnr.onColumnsAdded(schemaName, tbl.type(), tbl.cacheInfo(), cols);
+        }
+        finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    /**
+     * Drop column.
+     *
+     * @param schemaName Schema name.
+     * @param tblName Table name.
+     * @param cols Columns.
+     * @param ifTblExists If table exists.
+     * @param ifColExists If column exists.
+     * @throws IgniteCheckedException If failed.
+     */
+    public void dropColumn(
+        String schemaName,
+        String tblName,
+        List<String> cols,
+        boolean ifTblExists,
+        boolean ifColExists
+    ) throws IgniteCheckedException {
+        assert !ifColExists || cols.size() == 1;
+
+        lock.writeLock().lock();
+
+        try {
+            // Locate table.
+            TableDescriptor tbl = table(schemaName, tblName);
+
+            if (tbl == null) {
+                if (!ifTblExists) {
+                    throw new IgniteCheckedException("Table not found in schema manager [schemaName=" + schemaName +
+                        ",tblName=" + tblName + ']');
+                }
+                else
+                    return;
+            }
+
+            lsnr.onColumnsDropped(schemaName, tbl.type(), tbl.cacheInfo(), cols);
+        }
+        finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    /**
+     * Initialize table's cache context created for not started cache.
+     *
+     * @param cctx Cache context.
+     * @return {@code true} If context has been initialized.
+     */
+    public boolean initCacheContext(GridCacheContext<?, ?> cctx) {
+        lock.writeLock().lock();
+
+        try {
+            GridCacheContextInfo<?, ?> cacheInfo = cacheInfo(cctx.name());
+
+            if (cacheInfo != null) {
+                assert !cacheInfo.isCacheContextInited() : cacheInfo.name();
+                assert cacheInfo.name().equals(cctx.name()) : cacheInfo.name() + " != " + cctx.name();
+
+                cacheInfo.initCacheContext((GridCacheContext)cctx);
+
+                return true;
+            }
+
+            return false;
+        }
+        finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    /**
+     * Clear cache context on call cache.close() on non-affinity node.
+     *
+     * @param cctx Cache context.
+     * @return {@code true} If context has been cleared.
+     */
+    public boolean clearCacheContext(GridCacheContext<?, ?> cctx) {
+        lock.writeLock().lock();
+
+        try {
+            GridCacheContextInfo<?, ?> cacheInfo = cacheInfo(cctx.name());
+
+            if (cacheInfo != null && cacheInfo.isCacheContextInited()) {
+                cacheInfo.clearCacheContext();
+
+                return true;
+            }
+
+            return false;
+        }
+        finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    /**
+     * Mark tables for index rebuild, so that their indexes are not used.
+     *
+     * @param cacheName Cache name.
+     * @param mark Mark/unmark flag, {@code true} if index rebuild started, {@code false} if finished.
+     */
+    public void markIndexRebuild(String cacheName, boolean mark) {
+        lock.writeLock().lock();
+
+        try {
+            for (TableDescriptor tbl : tablesForCache(cacheName)) {
+                tbl.markIndexRebuildInProgress(mark);
+
+                if (mark)
+                    lsnr.onIndexRebuildStarted(tbl.type().schemaName(), tbl.type().tableName());
+                else
+                    lsnr.onIndexRebuildFinished(tbl.type().schemaName(), tbl.type().tableName());
+            }
+        }
+        finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    /**
+     * Get table descriptor.
+     *
+     * @param schemaName Schema name.
+     * @param cacheName Cache name.
+     * @param type Type name.
+     * @return Descriptor.
+     */
+    @Nullable public GridQueryTypeDescriptor typeDescriptorForType(String schemaName, String cacheName, String type) {
+        lock.readLock().lock();
+
+        try {
+            SchemaDescriptor schema = schema(schemaName);
+
+            if (schema == null)
+                return null;
+
+            TableDescriptor tbl = schema.tableByTypeName(cacheName, type);
+
+            return tbl == null ? null : tbl.type();
+        }
+        finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Gets collection of table for given schema name.
+     *
+     * @param cacheName Cache name.
+     * @return Collection of table descriptors.
+     */
+    public Collection<TableDescriptor> tablesForCache(String cacheName) {
+        lock.readLock().lock();
+
+        try {
+            SchemaDescriptor schema = schema(schemaName(cacheName));
+
+            if (schema == null)
+                return Collections.emptySet();
+
+            List<TableDescriptor> tbls = new ArrayList<>();
+
+            for (TableDescriptor tbl : schema.tables()) {
+                if (F.eq(tbl.cacheInfo().name(), cacheName))
+                    tbls.add(tbl);
+            }
+
+            return tbls;
+        }
+        finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Find registered cache info by it's name.
+     */
+    @Nullable public GridCacheContextInfo<?, ?> cacheInfo(String cacheName) {
+        lock.readLock().lock();
+
+        try {
+            SchemaDescriptor schema = schema(schemaName(cacheName));
+
+            if (schema == null)
+                return null;
+
+            for (TableDescriptor tbl : schema.tables()) {
+                if (F.eq(tbl.cacheInfo().name(), cacheName))
+                    return tbl.cacheInfo();
+            }
+
+            return null;
+        }
+        finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Find table by it's identifier.
+     *
+     * @param schemaName Schema name.
+     * @param tblName Table name.
+     * @return Table descriptor or {@code null} if none found.
+     */
+    @Nullable public TableDescriptor table(String schemaName, String tblName) {
+        lock.readLock().lock();
+
+        try {
+            return id2tbl.get(new T2<>(schemaName, tblName));
+        }
+        finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Find index by it's identifier.
+     *
+     * @param schemaName Schema name.
+     * @param idxName Index name.
+     * @return Index or {@code null} if none found.
+     */
+    @Nullable public IndexDescriptor index(String schemaName, String idxName) {
+        lock.readLock().lock();
+
+        try {
+            SchemaDescriptor schema = schema(schemaName);
+
+            if (schema == null)
+                return null;
+
+            for (TableDescriptor tbl : schema.tables()) {
+                IndexDescriptor idx = tbl.indexes().get(idxName);
+
+                if (idx != null)
+                    return idx;
+            }
+
+            return null;
+        }
+        finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Return table information filtered by given patterns.
+     *
+     * @param schemaNamePtrn Filter by schema name. Can be {@code null} to don't use the filter.
+     * @param tblNamePtrn Filter by table name. Can be {@code null} to don't use the filter.
+     * @param tblTypes Filter by table type. As Of now supported only 'TABLES' and 'VIEWS'.
+     * Can be {@code null} or empty to don't use the filter.
+     *
+     * @return Table information filtered by given patterns.
+     */
+    public Collection<TableInformation> tablesInformation(
+        String schemaNamePtrn,
+        String tblNamePtrn,
+        String... tblTypes
+    ) {
+        Set<String> types = F.isEmpty(tblTypes) ? Collections.emptySet() : new HashSet<>(Arrays.asList(tblTypes));
+
+        Collection<TableInformation> infos = new ArrayList<>();
+
+        boolean allTypes = F.isEmpty(tblTypes);
+
+        if (allTypes || types.contains(JdbcUtils.TYPE_TABLE)) {
+            id2tbl.values().stream()
+                .filter(t -> matches(t.type().schemaName(), schemaNamePtrn))
+                .filter(t -> matches(t.type().tableName(), tblNamePtrn))
+                .map(t -> {
+                    int cacheGrpId = t.cacheInfo().groupId();
+
+                    CacheGroupDescriptor cacheGrpDesc = ctx.cache().cacheGroupDescriptors().get(cacheGrpId);
+
+                    // We should skip table in case regarding cache group has been removed.
+                    if (cacheGrpDesc == null)
+                        return null;
+
+                    GridQueryTypeDescriptor type = t.type();
+
+                    return new TableInformation(t.type().schemaName(), t.type().tableName(),
+                        JdbcUtils.TYPE_TABLE, cacheGrpId, cacheGrpDesc.cacheOrGroupName(), t.cacheInfo().cacheId(),
+                        t.cacheInfo().name(), type.affinityKey(), type.keyFieldAlias(), type.valueFieldAlias(),
+                        type.keyTypeName(), type.valueTypeName());
+                })
+                .filter(Objects::nonNull)
+                .forEach(infos::add);
+        }
+
+        if ((allTypes || types.contains(JdbcUtils.TYPE_VIEW)) && matches(QueryUtils.SCHEMA_SYS, schemaNamePtrn)) {
+            sysViews.stream()
+                .filter(t -> matches(MetricUtils.toSqlName(t.name()), tblNamePtrn))
+                .map(v -> new TableInformation(QueryUtils.SCHEMA_SYS, MetricUtils.toSqlName(v.name()), JdbcUtils.TYPE_VIEW))
+                .forEach(infos::add);
+        }
+
+        return infos;
+    }
+
+    /**
+     * Return column information filtered by given patterns.
+     *
+     * @param schemaNamePtrn Filter by schema name. Can be {@code null} to don't use the filter.
+     * @param tblNamePtrn Filter by table name. Can be {@code null} to don't use the filter.
+     * @param colNamePtrn Filter by column name. Can be {@code null} to don't use the filter.
+     *
+     * @return Column information filtered by given patterns.
+     */
+    public Collection<ColumnInformation> columnsInformation(
+        String schemaNamePtrn,
+        String tblNamePtrn,
+        String colNamePtrn
+    ) {
+        Collection<ColumnInformation> infos = new ArrayList<>();
+
+        // Gather information about tables.
+        id2tbl.values().stream()
+            .filter(t -> matches(t.type().schemaName(), schemaNamePtrn))
+            .filter(t -> matches(t.type().tableName(), tblNamePtrn))
+            .forEach(tbl -> {
+                AtomicInteger cnt = new AtomicInteger(1); // Column ordinal position, start from 1.
+
+                GridQueryTypeDescriptor d = tbl.type();
+
+                // Add default columns if fields not specified explicitely.
+                if (F.isEmpty(d.fields())) {
+                    if (matches(KEY_FIELD_NAME, colNamePtrn)) {
+                        infos.add(new ColumnInformation(cnt.getAndIncrement(), d.schemaName(), d.tableName(),
+                            KEY_FIELD_NAME, d.keyClass(), false, null, -1, -1, false));
+                    }
+
+                    if (matches(VAL_FIELD_NAME, colNamePtrn)) {
+                        infos.add(new ColumnInformation(cnt.getAndIncrement(), d.schemaName(), d.tableName(),
+                            VAL_FIELD_NAME, d.valueClass(), false, null, -1, -1, false));
+                    }
+                }
+                else {
+                    d.fields().keySet().stream()
+                        .filter(field -> matches(field, colNamePtrn))
+                        .forEach(field -> {
+                            GridQueryProperty prop = d.property(field);
+
+                            infos.add(new ColumnInformation(
+                                cnt.getAndIncrement(),
+                                d.schemaName(),
+                                d.tableName(),
+                                field,
+                                prop.type(),
+                                !prop.notNull(),
+                                prop.defaultValue(),
+                                prop.precision(),
+                                prop.scale(),
+                                field.equals(d.affinityKey()))
+                            );
+                        });
+                }
+            });
+
+        // Gather information about system views.
+        if (matches(QueryUtils.SCHEMA_SYS, schemaNamePtrn)) {
+            sysViews.stream()
+                .filter(v -> matches(MetricUtils.toSqlName(v.name()), tblNamePtrn))
+                .flatMap(
+                    view -> {
+                        AtomicInteger cnt = new AtomicInteger(1); // Column ordinal position, start from 1.
+
+                        return MetricUtils.systemViewAttributes(view).entrySet().stream()
+                            .filter(c -> matches(MetricUtils.toSqlName(c.getKey()), colNamePtrn))
+                            .map(c -> new ColumnInformation(
+                                cnt.getAndIncrement(),
+                                QueryUtils.SCHEMA_SYS,
+                                MetricUtils.toSqlName(view.name()),
+                                MetricUtils.toSqlName(c.getKey()),
+                                c.getValue(),
+                                true,
+                                null,
+                                -1,
+                                -1,
+                                false)
+                            );
+                    }
+                ).forEach(infos::add);
+        }
+
+        return infos;
+    }
+
+    /** */
+    private SchemaChangeListener schemaChangeListener(GridKernalContext ctx) {
+        List<SchemaChangeListener> subscribers = new ArrayList<>(ctx.internalSubscriptionProcessor().getSchemaChangeSubscribers());
+
+        if (F.isEmpty(subscribers))
+            return new NoOpSchemaChangeListener();
+
+        return new CompoundSchemaChangeListener(ctx, subscribers);
+    }
+
+    /** */
+    private static final class NoOpSchemaChangeListener extends AbstractSchemaChangeListener {
+        // No-op.
+    }
+
+    /** */
+    private static final class CompoundSchemaChangeListener implements SchemaChangeListener {
+        /** */
+        private final List<SchemaChangeListener> lsnrs;
+
+        /** */
+        private final IgniteLogger log;
+
+        /**
+         * @param ctx Kernal context.
+         * @param lsnrs Lsnrs.
+         */
+        private CompoundSchemaChangeListener(GridKernalContext ctx, List<SchemaChangeListener> lsnrs) {
+            this.lsnrs = lsnrs;
+            log = ctx.log(CompoundSchemaChangeListener.class);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onSchemaCreated(String schemaName) {
+            lsnrs.forEach(lsnr -> executeSafe(() -> lsnr.onSchemaCreated(schemaName)));
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onSchemaDropped(String schemaName) {
+            lsnrs.forEach(lsnr -> executeSafe(() -> lsnr.onSchemaDropped(schemaName)));
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onSqlTypeCreated(
+            String schemaName,
+            GridQueryTypeDescriptor typeDesc,
+            GridCacheContextInfo<?, ?> cacheInfo
+        ) {
+            lsnrs.forEach(lsnr -> executeSafe(() -> lsnr.onSqlTypeCreated(schemaName, typeDesc, cacheInfo)));
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onColumnsAdded(
+            String schemaName,
+            GridQueryTypeDescriptor typeDesc,
+            GridCacheContextInfo<?, ?> cacheInfo,
+            List<QueryField> cols
+        ) {
+            lsnrs.forEach(lsnr -> executeSafe(() -> lsnr.onColumnsAdded(schemaName, typeDesc, cacheInfo, cols)));
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onColumnsDropped(
+            String schemaName,
+            GridQueryTypeDescriptor typeDesc,
+            GridCacheContextInfo<?, ?> cacheInfo,
+            List<String> cols
+        ) {
+            lsnrs.forEach(lsnr -> executeSafe(() -> lsnr.onColumnsDropped(schemaName, typeDesc, cacheInfo, cols)));
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onSqlTypeDropped(
+            String schemaName,
+            GridQueryTypeDescriptor typeDescriptor,
+            boolean destroy
+        ) {
+            lsnrs.forEach(lsnr -> executeSafe(() -> lsnr.onSqlTypeDropped(schemaName, typeDescriptor, destroy)));
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onIndexCreated(
+            String schemaName,
+            String tblName,
+            String idxName,
+            IndexDescriptor idxDesc
+        ) {
+            lsnrs.forEach(lsnr -> executeSafe(() -> lsnr.onIndexCreated(schemaName, tblName, idxName, idxDesc)));
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onIndexDropped(String schemaName, String tblName, String idxName) {
+            lsnrs.forEach(lsnr -> executeSafe(() -> lsnr.onIndexDropped(schemaName, tblName, idxName)));
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onIndexRebuildStarted(String schemaName, String tblName) {
+            lsnrs.forEach(lsnr -> executeSafe(() -> lsnr.onIndexRebuildStarted(schemaName, tblName)));
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onIndexRebuildFinished(String schemaName, String tblName) {
+            lsnrs.forEach(lsnr -> executeSafe(() -> lsnr.onIndexRebuildFinished(schemaName, tblName)));
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onFunctionCreated(String schemaName, String name, boolean deterministic, Method method) {
+            lsnrs.forEach(lsnr -> executeSafe(() -> lsnr.onFunctionCreated(schemaName, name, deterministic, method)));
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onSystemViewCreated(String schemaName, SystemView<?> sysView) {
+            lsnrs.forEach(lsnr -> executeSafe(() -> lsnr.onSystemViewCreated(schemaName, sysView)));
+        }
+
+        /** */
+        private void executeSafe(Runnable r) {
+            try {
+                r.run();
+            }
+            catch (Exception e) {
+                log.warning("Failed to notify listener (will ignore): " + e.getMessage(), e);
+            }
+        }
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/management/SortedIndexDescriptorFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/management/SortedIndexDescriptorFactory.java
new file mode 100644
index 00000000000..0bbeda52a0b
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/management/SortedIndexDescriptorFactory.java
@@ -0,0 +1,181 @@
+/*
+ * 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.schema.management;
+
+import java.util.LinkedHashMap;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.cache.query.index.Index;
+import org.apache.ignite.internal.cache.query.index.IndexName;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyDefinition;
+import org.apache.ignite.internal.cache.query.index.sorted.QueryIndexDefinition;
+import org.apache.ignite.internal.cache.query.index.sorted.client.ClientIndexDefinition;
+import org.apache.ignite.internal.cache.query.index.sorted.client.ClientIndexFactory;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.InlineIndex;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.InlineIndexFactory;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.GridCacheContextInfo;
+import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree;
+import org.apache.ignite.internal.processors.query.GridQueryIndexDescriptor;
+import org.apache.ignite.internal.processors.query.GridQueryProperty;
+import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
+import org.apache.ignite.internal.processors.query.QueryUtils;
+import org.apache.ignite.internal.processors.query.schema.SchemaIndexCacheVisitor;
+import org.apache.ignite.internal.util.typedef.F;
+import org.jetbrains.annotations.Nullable;
+
+/** Factory to create sorted index descriptors. */
+public class SortedIndexDescriptorFactory extends AbstractIndexDescriptorFactory {
+    /** */
+    private static final InlineIndexFactory SORTED_IDX_FACTORY = InlineIndexFactory.INSTANCE;
+
+    /** */
+    private final IgniteLogger log;
+
+    /** */
+    public SortedIndexDescriptorFactory(IgniteLogger log) {
+        this.log = log;
+    }
+
+    /** {@inheritDoc} */
+    @Override public IndexDescriptor create(
+        GridKernalContext ctx,
+        GridQueryIndexDescriptor idxDesc,
+        TableDescriptor tbl,
+        @Nullable SchemaIndexCacheVisitor cacheVisitor
+    ) {
+        GridCacheContextInfo<?, ?> cacheInfo = tbl.cacheInfo();
+        GridQueryTypeDescriptor typeDesc = tbl.type();
+        String idxName = idxDesc.name();
+        boolean isPk = QueryUtils.PRIMARY_KEY_INDEX.equals(idxName);
+        boolean isAff = QueryUtils.AFFINITY_KEY_INDEX.equals(idxName);
+
+        if (log.isDebugEnabled())
+            log.debug("Creating cache index [cacheId=" + cacheInfo.cacheId() + ", idxName=" + idxName + ']');
+
+        LinkedHashMap<String, IndexKeyDefinition> originalIdxCols = indexDescriptorToKeysDefinition(idxDesc, typeDesc);
+        LinkedHashMap<String, IndexKeyDefinition> wrappedCols = new LinkedHashMap<>(originalIdxCols);
+
+        // Columns conditions below is caused by legacy implementation, to maintain backward compatibility.
+
+        // Enrich wrapped columns collection with key and affinity key.
+        if (isAff || F.isEmpty(tbl.type().keyFieldName()) || !wrappedCols.containsKey(tbl.type().keyFieldAlias()))
+            addKeyColumn(wrappedCols, tbl);
+
+        if (!(isPk && QueryUtils.KEY_FIELD_NAME.equals(tbl.affinityKey())))
+            addAffinityColumn(wrappedCols, tbl);
+
+        LinkedHashMap<String, IndexKeyDefinition> unwrappedCols = new LinkedHashMap<>(originalIdxCols);
+
+        // Enrich unwrapped columns collection with unwrapped key fields and affinity key.
+        addUnwrappedKeyColumns(unwrappedCols, tbl);
+
+        if (!(isPk && QueryUtils.KEY_FIELD_NAME.equals(tbl.affinityKey())))
+            addAffinityColumn(unwrappedCols, tbl);
+
+        LinkedHashMap<String, IndexKeyDefinition> idxCols = unwrappedCols;
+
+        Index idx;
+
+        if (cacheInfo.affinityNode()) {
+            GridCacheContext<?, ?> cctx = cacheInfo.cacheContext();
+
+            int typeId = cctx.binaryMarshaller() ? typeDesc.typeId() : typeDesc.valueClass().hashCode();
+
+            String treeName = BPlusTree.treeName(typeId + "_" + idxName, "H2Tree");
+
+            if (!ctx.indexProcessor().useUnwrappedPk(cctx, treeName))
+                idxCols = wrappedCols;
+
+            QueryIndexDefinition idxDef = new QueryIndexDefinition(
+                typeDesc,
+                cacheInfo,
+                new IndexName(cacheInfo.name(), typeDesc.schemaName(), typeDesc.tableName(), idxName),
+                treeName,
+                ctx.indexProcessor().rowCacheCleaner(cacheInfo.groupId()),
+                isPk,
+                isAff,
+                idxCols,
+                idxDesc.inlineSize(),
+                ctx.indexProcessor().keyTypeSettings()
+            );
+
+            if (cacheVisitor != null)
+                idx = ctx.indexProcessor().createIndexDynamically(cctx, SORTED_IDX_FACTORY, idxDef, cacheVisitor);
+            else
+                idx = ctx.indexProcessor().createIndex(cctx, SORTED_IDX_FACTORY, idxDef);
+        }
+        else {
+            ClientIndexDefinition d = new ClientIndexDefinition(
+                new IndexName(tbl.cacheInfo().name(), tbl.type().schemaName(), tbl.type().tableName(), idxName),
+                idxCols,
+                idxDesc.inlineSize(),
+                tbl.cacheInfo().config().getSqlIndexMaxInlineSize());
+
+            idx = ctx.indexProcessor().createIndex(tbl.cacheInfo().cacheContext(), new ClientIndexFactory(log), d);
+        }
+
+        assert idx instanceof InlineIndex : idx;
+
+        return new IndexDescriptor(tbl, idxName, idxDesc.type(), idxCols, isPk, isAff,
+            ((InlineIndex)idx).inlineSize(), idx);
+    }
+
+    /** Split key into simple components and add to columns list. */
+    private static void addUnwrappedKeyColumns(LinkedHashMap<String, IndexKeyDefinition> cols, TableDescriptor tbl) {
+        // Key unwrapping possible only for SQL created tables.
+        if (!tbl.isSql() || QueryUtils.isSqlType(tbl.type().keyClass())) {
+            addKeyColumn(cols, tbl);
+
+            return;
+        }
+
+        if (!tbl.type().primaryKeyFields().isEmpty()) {
+            for (String keyName : tbl.type().primaryKeyFields())
+                cols.putIfAbsent(keyName, keyDefinition(tbl.type(), keyName, true));
+        }
+        else {
+            boolean haveKeyFields = false;
+
+            for (String propName : tbl.type().fields().keySet()) {
+                GridQueryProperty prop = tbl.type().property(propName);
+
+                if (prop.key()) {
+                    cols.putIfAbsent(propName, keyDefinition(tbl.type(), propName, true));
+                    haveKeyFields = true;
+                }
+            }
+
+            // If key is object but the user has not specified any particular columns,
+            // we have to fall back to whole-key index.
+            if (!haveKeyFields)
+                addKeyColumn(cols, tbl);
+        }
+    }
+
+    /** Add key column, if it (or it's alias) wasn't added before. */
+    private static void addKeyColumn(LinkedHashMap<String, IndexKeyDefinition> cols, TableDescriptor tbl) {
+        cols.putIfAbsent(QueryUtils.KEY_FIELD_NAME, keyDefinition(tbl.type(), QueryUtils.KEY_FIELD_NAME, true));
+    }
+
+    /** Add affinity column, if it wasn't added before. */
+    private static void addAffinityColumn(LinkedHashMap<String, IndexKeyDefinition> cols, TableDescriptor tbl) {
+        if (tbl.affinityKey() != null)
+            cols.putIfAbsent(tbl.affinityKey(), keyDefinition(tbl.type(), tbl.affinityKey(), true));
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/management/TableDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/management/TableDescriptor.java
new file mode 100644
index 00000000000..a2a17f9eca5
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/management/TableDescriptor.java
@@ -0,0 +1,113 @@
+/*
+ * 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.schema.management;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.ignite.internal.processors.cache.GridCacheContextInfo;
+import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
+import org.apache.ignite.internal.processors.query.QueryUtils;
+import org.apache.ignite.internal.util.typedef.F;
+
+/**
+ * Local database table object.
+ */
+public class TableDescriptor {
+    /** */
+    private final GridCacheContextInfo<?, ?> cacheInfo;
+
+    /** */
+    private final GridQueryTypeDescriptor typeDesc;
+
+    /** */
+    private final boolean isSql;
+
+    /** */
+    private final Map<String, IndexDescriptor> idxs = new ConcurrentHashMap<>();
+
+    /** */
+    private final String affKey;
+
+    /** */
+    private volatile boolean idxRebuildInProgress;
+
+    /**
+     * Ctor.
+     *
+     * @param cacheInfo Cache cacheInfo context.
+     * @param typeDesc Descriptor.
+     */
+    public TableDescriptor(GridCacheContextInfo<?, ?> cacheInfo, GridQueryTypeDescriptor typeDesc, boolean isSql) {
+        this.cacheInfo = cacheInfo;
+        this.typeDesc = typeDesc;
+        this.isSql = isSql;
+
+        if (F.isEmpty(typeDesc.affinityKey()) || F.eq(typeDesc.affinityKey(), typeDesc.keyFieldName()))
+            affKey = QueryUtils.KEY_FIELD_NAME;
+        else if (typeDesc.customAffinityKeyMapper() || !typeDesc.fields().containsKey(typeDesc.affinityKey()))
+            affKey = null;
+        else
+            affKey = typeDesc.affinityKey();
+    }
+
+    /** */
+    public GridCacheContextInfo<?, ?> cacheInfo() {
+        return cacheInfo;
+    }
+
+    /** */
+    public GridQueryTypeDescriptor type() {
+        return typeDesc;
+    }
+
+    /** */
+    public boolean isSql() {
+        return isSql;
+    }
+
+    /** */
+    void addIndex(String idxName, IndexDescriptor idx) {
+        idxs.put(idxName, idx);
+    }
+
+    /** */
+    IndexDescriptor dropIndex(String idxName) {
+        return idxs.remove(idxName);
+    }
+
+    /** */
+    Map<String, IndexDescriptor> indexes() {
+        return Collections.unmodifiableMap(idxs);
+    }
+
+    /** */
+    public String affinityKey() {
+        return affKey;
+    }
+
+    /** */
+    public boolean isIndexRebuildInProgress() {
+        return idxRebuildInProgress;
+    }
+
+    /** */
+    void markIndexRebuildInProgress(boolean idxRebuildInProgress) {
+        this.idxRebuildInProgress = idxRebuildInProgress;
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsConfigurationManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsConfigurationManager.java
index 2b4835f2811..b4cdbc38e07 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsConfigurationManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsConfigurationManager.java
@@ -46,12 +46,13 @@ import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
 import org.apache.ignite.internal.processors.cluster.GridClusterStateProcessor;
 import org.apache.ignite.internal.processors.metastorage.DistributedMetaStorage;
 import org.apache.ignite.internal.processors.metastorage.DistributedMetastorageLifecycleListener;
-import org.apache.ignite.internal.processors.query.GridQuerySchemaManager;
 import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
 import org.apache.ignite.internal.processors.query.IgniteSQLException;
 import org.apache.ignite.internal.processors.query.QueryUtils;
 import org.apache.ignite.internal.processors.query.schema.AbstractSchemaChangeListener;
 import org.apache.ignite.internal.processors.query.schema.SchemaChangeListener;
+import org.apache.ignite.internal.processors.query.schema.management.SchemaManager;
+import org.apache.ignite.internal.processors.query.schema.management.TableDescriptor;
 import org.apache.ignite.internal.processors.query.stat.config.StatisticsColumnConfiguration;
 import org.apache.ignite.internal.processors.query.stat.config.StatisticsObjectConfiguration;
 import org.apache.ignite.internal.processors.query.stat.view.ColumnConfigurationViewSupplier;
@@ -79,7 +80,7 @@ public class IgniteStatisticsConfigurationManager {
     public static final String[] EMPTY_STRINGS = new String[0];
 
     /** Schema manager. */
-    private final GridQuerySchemaManager schemaMgr;
+    private final SchemaManager schemaMgr;
 
     /** Distributed metastore. */
     private volatile DistributedMetaStorage distrMetaStorage;
@@ -201,7 +202,7 @@ public class IgniteStatisticsConfigurationManager {
      * @param isServerNode Server node flag.
      */
     public IgniteStatisticsConfigurationManager(
-        GridQuerySchemaManager schemaMgr,
+        SchemaManager schemaMgr,
         GridInternalSubscriptionProcessor subscriptionProcessor,
         GridSystemViewManager sysViewMgr,
         GridClusterStateProcessor cluster,
@@ -266,8 +267,9 @@ public class IgniteStatisticsConfigurationManager {
      * @param cfg Statistics object configuration to update statistics by.
      */
     private void updateLocalStatistics(StatisticsObjectConfiguration cfg) {
-        GridQueryTypeDescriptor tbl = schemaMgr.typeDescriptorForTable(cfg.key().schema(), cfg.key().obj());
-        GridCacheContextInfo<?, ?> cacheInfo = schemaMgr.cacheInfoForTable(cfg.key().schema(), cfg.key().obj());
+        TableDescriptor tbl = schemaMgr.table(cfg.key().schema(), cfg.key().obj());
+        GridQueryTypeDescriptor typeDesc = tbl != null ? tbl.type() : null;
+        GridCacheContextInfo<?, ?> cacheInfo = tbl != null ? tbl.cacheInfo() : null;
         GridCacheContext<?, ?> cctx = cacheInfo != null ? cacheInfo.cacheContext() : null;
 
         if (tbl == null || cfg.columns().isEmpty()) {
@@ -280,8 +282,8 @@ public class IgniteStatisticsConfigurationManager {
             }
 
             // Ensure to clean local metastorage.
-            LocalStatisticsGatheringContext ctx = new LocalStatisticsGatheringContext(false, tbl, cacheInfo,
-                cfg, Collections.emptySet(), topVer);
+            LocalStatisticsGatheringContext ctx = new LocalStatisticsGatheringContext(false, typeDesc,
+                cacheInfo, cfg, Collections.emptySet(), topVer);
 
             statProc.updateLocalStatistics(ctx);
 
@@ -307,8 +309,9 @@ public class IgniteStatisticsConfigurationManager {
 
             final Set<Integer> primParts = cctx.affinity().primaryPartitions(cctx.localNodeId(), topVer0);
 
-            LocalStatisticsGatheringContext ctx = new LocalStatisticsGatheringContext(false, tbl, cacheInfo,
-                cfg, primParts, topVer0);
+            LocalStatisticsGatheringContext ctx = new LocalStatisticsGatheringContext(false, typeDesc,
+                cacheInfo, cfg, primParts, topVer0);
+
             statProc.updateLocalStatistics(ctx);
         }
         catch (IgniteCheckedException e) {
@@ -391,15 +394,14 @@ public class IgniteStatisticsConfigurationManager {
             log.debug("Update statistics [targets=" + targets + ']');
 
         for (StatisticsObjectConfiguration target : targets) {
-
-            GridQueryTypeDescriptor tbl = schemaMgr.typeDescriptorForTable(target.key().schema(), target.key().obj());
+            TableDescriptor tbl = schemaMgr.table(target.key().schema(), target.key().obj());
 
             validate(target, tbl);
 
             List<StatisticsColumnConfiguration> colCfgs;
 
             if (F.isEmpty(target.columns()))
-                colCfgs = tbl.fields().keySet().stream()
+                colCfgs = tbl.type().fields().keySet().stream()
                     .filter(col -> !QueryUtils.KEY_FIELD_NAME.equals(col) && !QueryUtils.VAL_FIELD_NAME.equals(col))
                     .map(col -> new StatisticsColumnConfiguration(col, null))
                     .collect(Collectors.toList());
@@ -581,7 +583,7 @@ public class IgniteStatisticsConfigurationManager {
      * @param cfg Statistics object configuration to check.
      * @param tbl Corresponding table (if exists).
      */
-    private void validate(StatisticsObjectConfiguration cfg, GridQueryTypeDescriptor tbl) {
+    private void validate(StatisticsObjectConfiguration cfg, TableDescriptor tbl) {
         if (tbl == null) {
             throw new IgniteSQLException(
                 "Table doesn't exist [schema=" + cfg.key().schema() + ", table=" + cfg.key().obj() + ']',
@@ -590,7 +592,7 @@ public class IgniteStatisticsConfigurationManager {
 
         if (!F.isEmpty(cfg.columns())) {
             for (String col : cfg.columns().keySet()) {
-                if (!tbl.fields().containsKey(col)) {
+                if (!tbl.type().fields().containsKey(col)) {
                     throw new IgniteSQLException(
                         "Column doesn't exist [schema=" + cfg.key().schema() +
                             ", table=" + cfg.key().obj() +
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsHelper.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsHelper.java
index 3dd953915cd..ce7cd5ae68e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsHelper.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsHelper.java
@@ -39,10 +39,10 @@ import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.CacheGroupContext;
-import org.apache.ignite.internal.processors.cache.GridCacheContextInfo;
-import org.apache.ignite.internal.processors.query.GridQuerySchemaManager;
 import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
 import org.apache.ignite.internal.processors.query.QueryUtils;
+import org.apache.ignite.internal.processors.query.schema.management.SchemaManager;
+import org.apache.ignite.internal.processors.query.schema.management.TableDescriptor;
 import org.apache.ignite.internal.processors.query.stat.config.StatisticsColumnConfiguration;
 import org.apache.ignite.internal.processors.query.stat.config.StatisticsObjectConfiguration;
 import org.apache.ignite.internal.processors.query.stat.messages.StatisticsKeyMessage;
@@ -59,7 +59,7 @@ public class IgniteStatisticsHelper {
     private final IgniteLogger log;
 
     /** Schema manager. */
-    private final GridQuerySchemaManager schemaMgr;
+    private final SchemaManager schemaMgr;
 
     /**
      * Constructor.
@@ -70,7 +70,7 @@ public class IgniteStatisticsHelper {
      */
     public IgniteStatisticsHelper(
         UUID locNodeId,
-        GridQuerySchemaManager schemaMgr,
+        SchemaManager schemaMgr,
         Function<Class<?>, IgniteLogger> logSupplier
     ) {
         this.schemaMgr = schemaMgr;
@@ -85,12 +85,12 @@ public class IgniteStatisticsHelper {
      * @throws IgniteCheckedException If unable to find table by specified key.
      */
     public CacheGroupContext groupContext(StatisticsKey key) throws IgniteCheckedException {
-        GridCacheContextInfo<?, ?> tbl = schemaMgr.cacheInfoForTable(key.schema(), key.obj());
+        TableDescriptor tbl = schemaMgr.table(key.schema(), key.obj());
 
         if (tbl == null)
             throw new IgniteCheckedException(String.format("Can't find object %s.%s", key.schema(), key.obj()));
 
-        return tbl.cacheContext().group();
+        return tbl.cacheInfo().cacheContext().group();
     }
 
     /**
@@ -150,7 +150,7 @@ public class IgniteStatisticsHelper {
         );
 
         // For now there can be only tables
-        GridQueryTypeDescriptor tbl = schemaMgr.typeDescriptorForTable(keyMsg.schema(), keyMsg.obj());
+        TableDescriptor tbl = schemaMgr.table(keyMsg.schema(), keyMsg.obj());
 
         if (tbl == null) {
             // remove all loaded statistics.
@@ -161,7 +161,7 @@ public class IgniteStatisticsHelper {
             return null;
         }
 
-        return aggregateLocalStatistics(tbl, cfg, stats, log);
+        return aggregateLocalStatistics(tbl.type(), cfg, stats, log);
     }
 
     /**
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsManagerImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsManagerImpl.java
index f63d625bec9..9b503298b2c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsManagerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsManagerImpl.java
@@ -37,8 +37,8 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.Gri
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.PartitionsExchangeAware;
 import org.apache.ignite.internal.processors.cache.persistence.IgniteCacheDatabaseSharedManager;
 import org.apache.ignite.internal.processors.configuration.distributed.DistributedEnumProperty;
-import org.apache.ignite.internal.processors.query.GridQuerySchemaManager;
-import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
+import org.apache.ignite.internal.processors.query.schema.management.SchemaManager;
+import org.apache.ignite.internal.processors.query.schema.management.TableDescriptor;
 import org.apache.ignite.internal.processors.query.stat.config.StatisticsObjectConfiguration;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor;
 import org.apache.ignite.internal.util.collection.IntMap;
@@ -69,7 +69,7 @@ public class IgniteStatisticsManagerImpl implements IgniteStatisticsManager {
     private final GridKernalContext ctx;
 
     /** SchemaManager */
-    private final GridQuerySchemaManager schemaMgr;
+    private final SchemaManager schemaMgr;
 
     /** Statistics repository. */
     private final IgniteStatisticsRepository statsRepos;
@@ -134,14 +134,10 @@ public class IgniteStatisticsManagerImpl implements IgniteStatisticsManager {
      * Constructor.
      *
      * @param ctx Kernal context.
-     * @param schemaMgr Schema manager.
      */
-    public IgniteStatisticsManagerImpl(
-        GridKernalContext ctx,
-        GridQuerySchemaManager schemaMgr
-    ) {
+    public IgniteStatisticsManagerImpl(GridKernalContext ctx) {
         this.ctx = ctx;
-        this.schemaMgr = schemaMgr;
+        schemaMgr = ctx.query().schemaManager();
 
         boolean serverNode = !(ctx.config().isClientMode() || ctx.isDaemon());
 
@@ -485,8 +481,8 @@ public class IgniteStatisticsManagerImpl implements IgniteStatisticsManager {
 
             Set<Integer> tasksParts = calculateObsolescencedPartitions(cfg, statsRepos.getObsolescence(key));
 
-            GridQueryTypeDescriptor tbl = schemaMgr.typeDescriptorForTable(key.schema(), key.obj());
-            GridCacheContextInfo<?, ?> cacheInfo = schemaMgr.cacheInfoForTable(key.schema(), key.obj());
+            TableDescriptor tbl = schemaMgr.table(key.schema(), key.obj());
+            GridCacheContextInfo<?, ?> cacheInfo = tbl.cacheInfo();
 
             if (tbl == null) {
                 // Table can be removed earlier, but not already processed. Or somethink goes wrong. Try to reschedule.
@@ -494,8 +490,9 @@ public class IgniteStatisticsManagerImpl implements IgniteStatisticsManager {
                     log.debug(String.format("Got obsolescence statistics for unknown table %s", key));
             }
 
-            LocalStatisticsGatheringContext ctx = new LocalStatisticsGatheringContext(true, tbl, cacheInfo, cfg,
-                tasksParts, null);
+            LocalStatisticsGatheringContext ctx = new LocalStatisticsGatheringContext(true,
+                tbl.type(), cacheInfo, cfg, tasksParts, null);
+
             statProc.updateLocalStatistics(ctx);
         }
     }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlCommandProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlCommandProcessor.java
index 43541370511..c6f28765310 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlCommandProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/sql/SqlCommandProcessor.java
@@ -35,12 +35,13 @@ import org.apache.ignite.internal.processors.cache.GridCacheContextInfo;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal;
 import org.apache.ignite.internal.processors.cache.mvcc.MvccUtils;
 import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
-import org.apache.ignite.internal.processors.query.GridQueryIndexing;
 import org.apache.ignite.internal.processors.query.GridQueryProperty;
-import org.apache.ignite.internal.processors.query.GridQuerySchemaManager;
 import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
 import org.apache.ignite.internal.processors.query.IgniteSQLException;
 import org.apache.ignite.internal.processors.query.schema.SchemaOperationException;
+import org.apache.ignite.internal.processors.query.schema.management.IndexDescriptor;
+import org.apache.ignite.internal.processors.query.schema.management.SchemaManager;
+import org.apache.ignite.internal.processors.query.schema.management.TableDescriptor;
 import org.apache.ignite.internal.processors.query.stat.StatisticsKey;
 import org.apache.ignite.internal.processors.query.stat.StatisticsTarget;
 import org.apache.ignite.internal.processors.query.stat.config.StatisticsObjectConfiguration;
@@ -65,6 +66,7 @@ import org.apache.ignite.internal.sql.command.SqlStatisticsCommands;
 import org.apache.ignite.internal.util.future.GridFinishedFuture;
 import org.apache.ignite.plugin.security.SecurityPermission;
 import org.jetbrains.annotations.Nullable;
+
 import static org.apache.ignite.internal.processors.query.QueryUtils.convert;
 import static org.apache.ignite.internal.processors.query.QueryUtils.isDdlOnSchemaSupported;
 
@@ -79,17 +81,16 @@ public class SqlCommandProcessor {
     protected final IgniteLogger log;
 
     /** Schema manager. */
-    protected final GridQuerySchemaManager schemaMgr;
+    protected final SchemaManager schemaMgr;
 
     /**
      * Constructor.
      *
      * @param ctx Kernal context.
-     * @param schemaMgr Schema manager.
      */
-    public SqlCommandProcessor(GridKernalContext ctx, GridQuerySchemaManager schemaMgr) {
+    public SqlCommandProcessor(GridKernalContext ctx) {
         this.ctx = ctx;
-        this.schemaMgr = schemaMgr;
+        this.schemaMgr = ctx.query().schemaManager();
         log = ctx.log(getClass());
     }
 
@@ -221,8 +222,6 @@ public class SqlCommandProcessor {
     private void processAnalyzeCommand(SqlAnalyzeCommand cmd) {
         ctx.security().authorize(SecurityPermission.CHANGE_STATISTICS);
 
-        GridQueryIndexing indexing = ctx.query().getIndexing();
-
         StatisticsObjectConfiguration objCfgs[] = cmd.configurations().stream()
             .map(t -> {
                 if (t.key().schema() == null) {
@@ -236,7 +235,7 @@ public class SqlCommandProcessor {
             }).toArray(StatisticsObjectConfiguration[]::new);
 
         try {
-            indexing.statsManager().collectStatistics(objCfgs);
+            ctx.query().statsManager().collectStatistics(objCfgs);
         }
         catch (IgniteCheckedException e) {
             throw new IgniteSQLException(e.getMessage(), e);
@@ -251,14 +250,12 @@ public class SqlCommandProcessor {
     private void processRefreshStatisticsCommand(SqlRefreshStatitsicsCommand cmd) {
         ctx.security().authorize(SecurityPermission.REFRESH_STATISTICS);
 
-        GridQueryIndexing indexing = ctx.query().getIndexing();
-
         StatisticsTarget[] targets = cmd.targets().stream()
             .map(t -> (t.schema() == null) ? new StatisticsTarget(cmd.schemaName(), t.obj(), t.columns()) : t)
             .toArray(StatisticsTarget[]::new);
 
         try {
-            indexing.statsManager().refreshStatistics(targets);
+            ctx.query().statsManager().refreshStatistics(targets);
         }
         catch (IgniteCheckedException e) {
             throw new IgniteSQLException(e.getMessage(), e);
@@ -273,14 +270,12 @@ public class SqlCommandProcessor {
     private void processDropStatisticsCommand(SqlDropStatisticsCommand cmd) {
         ctx.security().authorize(SecurityPermission.CHANGE_STATISTICS);
 
-        GridQueryIndexing indexing = ctx.query().getIndexing();
-
         StatisticsTarget[] targets = cmd.targets().stream()
             .map(t -> (t.schema() == null) ? new StatisticsTarget(cmd.schemaName(), t.obj(), t.columns()) : t)
             .toArray(StatisticsTarget[]::new);
 
         try {
-            indexing.statsManager().dropStatistics(targets);
+            ctx.query().statsManager().dropStatistics(targets);
         }
         catch (IgniteCheckedException e) {
             throw new IgniteSQLException(e.getMessage(), e);
@@ -303,12 +298,14 @@ public class SqlCommandProcessor {
             if (cmd instanceof SqlCreateIndexCommand) {
                 SqlCreateIndexCommand cmd0 = (SqlCreateIndexCommand)cmd;
 
-                GridQueryTypeDescriptor typeDesc = schemaMgr.typeDescriptorForTable(cmd0.schemaName(), cmd0.tableName());
-                GridCacheContextInfo<?, ?> cacheInfo = schemaMgr.cacheInfoForTable(cmd0.schemaName(), cmd0.tableName());
+                TableDescriptor tbl = schemaMgr.table(cmd0.schemaName(), cmd0.tableName());
 
-                if (typeDesc == null)
+                if (tbl == null)
                     throw new SchemaOperationException(SchemaOperationException.CODE_TABLE_NOT_FOUND, cmd0.tableName());
 
+                GridQueryTypeDescriptor typeDesc = tbl.type();
+                GridCacheContextInfo<?, ?> cacheInfo = tbl.cacheInfo();
+
                 QueryIndex newIdx = new QueryIndex();
 
                 newIdx.setName(cmd0.indexName());
@@ -335,11 +332,11 @@ public class SqlCommandProcessor {
             else if (cmd instanceof SqlDropIndexCommand) {
                 SqlDropIndexCommand cmd0 = (SqlDropIndexCommand)cmd;
 
-                GridQueryTypeDescriptor typeDesc = schemaMgr.typeDescriptorForIndex(cmd0.schemaName(), cmd0.indexName());
+                IndexDescriptor idxDesc = schemaMgr.index(cmd0.schemaName(), cmd0.indexName());
 
-                if (typeDesc != null) {
-                    GridCacheContextInfo<?, ?> cacheInfo = schemaMgr.cacheInfoForTable(typeDesc.schemaName(),
-                        typeDesc.tableName());
+                // Do not allow to drop system indexes.
+                if (idxDesc != null && !idxDesc.isPk() && !idxDesc.isAffinity() && !idxDesc.isProxy()) {
+                    GridCacheContextInfo<?, ?> cacheInfo = idxDesc.table().cacheInfo();
 
                     fut = ctx.query().dynamicIndexDrop(cacheInfo.name(), cmd0.schemaName(), cmd0.indexName(),
                         cmd0.ifExists());
@@ -355,13 +352,15 @@ public class SqlCommandProcessor {
             else if (cmd instanceof SqlAlterTableCommand) {
                 SqlAlterTableCommand cmd0 = (SqlAlterTableCommand)cmd;
 
-                GridCacheContextInfo<?, ?> cacheInfo = schemaMgr.cacheInfoForTable(cmd0.schemaName(), cmd0.tableName());
+                TableDescriptor tbl = schemaMgr.table(cmd0.schemaName(), cmd0.tableName());
 
-                if (cacheInfo == null) {
+                if (tbl == null) {
                     throw new SchemaOperationException(SchemaOperationException.CODE_TABLE_NOT_FOUND,
                         cmd0.tableName());
                 }
 
+                GridCacheContextInfo<?, ?> cacheInfo = tbl.cacheInfo();
+
                 Boolean logging = cmd0.logging();
 
                 assert logging != null : "Only LOGGING/NOLOGGING are supported at the moment.";
diff --git a/modules/indexing/src/main/java/org/apache/ignite/spi/systemview/view/SqlIndexView.java b/modules/core/src/main/java/org/apache/ignite/spi/systemview/view/sql/SqlIndexView.java
similarity index 73%
rename from modules/indexing/src/main/java/org/apache/ignite/spi/systemview/view/SqlIndexView.java
rename to modules/core/src/main/java/org/apache/ignite/spi/systemview/view/sql/SqlIndexView.java
index aa6119c7eb6..7b0ff196ad2 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/spi/systemview/view/SqlIndexView.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/systemview/view/sql/SqlIndexView.java
@@ -15,27 +15,30 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.spi.systemview.view;
+package org.apache.ignite.spi.systemview.view.sql;
 
+import java.util.stream.Collectors;
+import org.apache.ignite.internal.cache.query.index.SortOrder;
 import org.apache.ignite.internal.managers.systemview.walker.Order;
-import org.apache.ignite.internal.processors.query.h2.database.IndexInformation;
-import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table;
+import org.apache.ignite.internal.processors.query.schema.management.IndexDescriptor;
+import org.apache.ignite.internal.processors.query.schema.management.TableDescriptor;
+import org.apache.ignite.spi.systemview.view.SystemView;
 
 /**
  * Sql index representation for a {@link SystemView}.
  */
 public class SqlIndexView {
     /** Table. */
-    private final GridH2Table tbl;
+    private final TableDescriptor tbl;
 
     /** Index. */
-    private final IndexInformation idx;
+    private final IndexDescriptor idx;
 
     /**
-     * @param tbl H2 table implementation.
+     * @param tbl Table information.
      * @param idx Index information.
      */
-    public SqlIndexView(GridH2Table tbl, IndexInformation idx) {
+    public SqlIndexView(TableDescriptor tbl, IndexDescriptor idx) {
         this.tbl = tbl;
         this.idx = idx;
     }
@@ -66,7 +69,7 @@ public class SqlIndexView {
      */
     @Order(2)
     public int cacheId() {
-        return tbl.cacheId();
+        return tbl.cacheInfo().cacheId();
     }
 
     /**
@@ -76,7 +79,7 @@ public class SqlIndexView {
      */
     @Order(3)
     public String cacheName() {
-        return tbl.cacheName();
+        return tbl.cacheInfo().name();
     }
 
     /**
@@ -86,7 +89,7 @@ public class SqlIndexView {
      */
     @Order(4)
     public String schemaName() {
-        return tbl.getSchema().getName();
+        return tbl.type().schemaName();
     }
 
     /**
@@ -96,7 +99,7 @@ public class SqlIndexView {
      */
     @Order(5)
     public String tableName() {
-        return tbl.identifier().table();
+        return tbl.type().tableName();
     }
 
     /**
@@ -116,7 +119,7 @@ public class SqlIndexView {
      */
     @Order(7)
     public String indexType() {
-        return idx.type();
+        return idx.type().name();
     }
 
     /**
@@ -126,7 +129,10 @@ public class SqlIndexView {
      */
     @Order(8)
     public String columns() {
-        return idx.keySql();
+        return idx.keyDefinitions().entrySet().stream()
+            .map(fld -> '"' + fld.getKey() + '"' +
+                (fld.getValue().order().sortOrder() == SortOrder.DESC ? " DESC" : " ASC"))
+            .collect(Collectors.joining(", "));
     }
 
     /**
@@ -136,7 +142,7 @@ public class SqlIndexView {
      */
     @Order(9)
     public boolean isPk() {
-        return idx.pk();
+        return idx.isPk();
     }
 
     /**
@@ -146,7 +152,7 @@ public class SqlIndexView {
      */
     @Order(10)
     public boolean isUnique() {
-        return idx.unique();
+        return idx.isPk() || (idx.isProxy() && idx.targetIdx().isPk());
     }
 
     /**
diff --git a/modules/indexing/src/main/java/org/apache/ignite/spi/systemview/view/SqlSchemaView.java b/modules/core/src/main/java/org/apache/ignite/spi/systemview/view/sql/SqlSchemaView.java
similarity index 82%
rename from modules/indexing/src/main/java/org/apache/ignite/spi/systemview/view/SqlSchemaView.java
rename to modules/core/src/main/java/org/apache/ignite/spi/systemview/view/sql/SqlSchemaView.java
index 7a1ecbd415f..b1ca9e6715f 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/spi/systemview/view/SqlSchemaView.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/systemview/view/sql/SqlSchemaView.java
@@ -15,22 +15,22 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.spi.systemview.view;
+package org.apache.ignite.spi.systemview.view.sql;
 
 import org.apache.ignite.internal.managers.systemview.walker.Order;
-import org.apache.ignite.internal.processors.query.h2.H2Schema;
+import org.apache.ignite.internal.processors.query.schema.management.SchemaDescriptor;
 
 /**
  * Sql schema system view representation.
  */
 public class SqlSchemaView {
-    /** H2 schema. */
-    private final H2Schema schema;
+    /** Schema. */
+    private final SchemaDescriptor schema;
 
     /**
-     * @param schema H2 schema.
+     * @param schema Schema.
      */
-    public SqlSchemaView(H2Schema schema) {
+    public SqlSchemaView(SchemaDescriptor schema) {
         this.schema = schema;
     }
 
diff --git a/modules/indexing/src/main/java/org/apache/ignite/spi/systemview/view/SqlTableColumnView.java b/modules/core/src/main/java/org/apache/ignite/spi/systemview/view/sql/SqlTableColumnView.java
similarity index 63%
rename from modules/indexing/src/main/java/org/apache/ignite/spi/systemview/view/SqlTableColumnView.java
rename to modules/core/src/main/java/org/apache/ignite/spi/systemview/view/sql/SqlTableColumnView.java
index 77d2965fd90..18c8c7538c0 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/spi/systemview/view/SqlTableColumnView.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/systemview/view/sql/SqlTableColumnView.java
@@ -15,108 +15,89 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.spi.systemview.view;
+package org.apache.ignite.spi.systemview.view.sql;
 
 import org.apache.ignite.internal.managers.systemview.walker.Order;
 import org.apache.ignite.internal.processors.query.GridQueryProperty;
-import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table;
-import org.h2.table.Column;
-import org.h2.table.IndexColumn;
+import org.apache.ignite.internal.processors.query.schema.management.TableDescriptor;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.spi.systemview.view.SystemView;
 
 /**
  * Sql table column representation for a {@link SystemView}.
  */
 public class SqlTableColumnView {
     /** Table. */
-    private final GridH2Table tbl;
-
-    /** Column. */
-    private final Column col;
+    private final TableDescriptor tbl;
 
     /** Query property. */
     private final GridQueryProperty prop;
 
-    /** Affinity column. */
-    private final IndexColumn affCol;
-
     /**
      * @param tbl Table.
-     * @param col Column.
+     * @param prop Column.
      */
-    public SqlTableColumnView(GridH2Table tbl, Column col) {
+    public SqlTableColumnView(TableDescriptor tbl, GridQueryProperty prop) {
         this.tbl = tbl;
-        this.col = col;
-        this.prop = tbl.rowDescriptor().type().property(col.getName());
-        this.affCol = tbl.getAffinityKeyColumn();
+        this.prop = prop;
     }
 
     /** @return Column name. */
     @Order
     public String columnName() {
-        return col.getName();
+        return prop.name();
     }
 
     /** @return Schema name. */
     @Order(2)
     public String schemaName() {
-        return tbl.getSchema().getName();
+        return tbl.type().schemaName();
     }
 
     /** @return Table name. */
     @Order(1)
     public String tableName() {
-        return tbl.identifier().table();
+        return tbl.type().tableName();
     }
 
     /** @return Field data type. */
     public Class<?> type() {
-        if (prop == null)
-            return null;
-
         return prop.type();
     }
 
     /** @return Field default. */
     public String defaultValue() {
-        if (prop == null)
-            return null;
-
-        return String.valueOf(prop.defaultValue());
+        return prop.defaultValue() == null ? null : prop.defaultValue().toString();
     }
 
     /** @return Precision. */
     public int precision() {
-        if (prop == null)
-            return -1;
-
         return prop.precision();
     }
 
     /** @return Scale. */
     public int scale() {
-        if (prop == null)
-            return -1;
-
         return prop.scale();
     }
 
     /** @return {@code True} if nullable field. */
     public boolean nullable() {
-        return col.isNullable();
+        return !prop.notNull();
     }
 
     /** @return {@code True} if primary key. */
     public boolean pk() {
-        return tbl.rowDescriptor().isKeyColumn(col.getColumnId());
+        return F.eq(prop.name(), tbl.type().keyFieldName()) || prop.key();
     }
 
     /** @return {@code True} if autoincremented field. */
     public boolean autoIncrement() {
-        return col.isAutoIncrement();
+        return false;
     }
 
-    /** @return {@code True} if autoincremented field. */
+    /** @return {@code True} if affinity column. */
     public boolean affinityColumn() {
-        return affCol != null && col.getColumnId() == affCol.column.getColumnId();
+        return !tbl.type().customAffinityKeyMapper() &&
+            (F.eq(prop.name(), tbl.type().affinityKey()) || (F.isEmpty(tbl.type().affinityKey()) && pk()));
     }
 }
diff --git a/modules/indexing/src/main/java/org/apache/ignite/spi/systemview/view/SqlTableView.java b/modules/core/src/main/java/org/apache/ignite/spi/systemview/view/sql/SqlTableView.java
similarity index 77%
rename from modules/indexing/src/main/java/org/apache/ignite/spi/systemview/view/SqlTableView.java
rename to modules/core/src/main/java/org/apache/ignite/spi/systemview/view/sql/SqlTableView.java
index c2b8a896a69..1484b0f91b7 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/spi/systemview/view/SqlTableView.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/systemview/view/sql/SqlTableView.java
@@ -15,18 +15,19 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.spi.systemview.view;
+package org.apache.ignite.spi.systemview.view.sql;
 
 import org.apache.ignite.internal.managers.systemview.walker.Order;
-import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table;
-import org.h2.table.IndexColumn;
+import org.apache.ignite.internal.processors.query.schema.management.TableDescriptor;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.spi.systemview.view.SystemView;
 
 /**
  * Sql table representation for a {@link SystemView}.
  */
 public class SqlTableView {
     /** Table. */
-    private final GridH2Table tbl;
+    private final TableDescriptor tbl;
 
     /** Affinity column name. */
     private String affColName;
@@ -34,16 +35,15 @@ public class SqlTableView {
     /**
      * @param tbl Table.
      */
-    public SqlTableView(GridH2Table tbl) {
+    public SqlTableView(TableDescriptor tbl) {
         this.tbl = tbl;
 
-        IndexColumn affCol = tbl.getAffinityKeyColumn();
+        String affCol = tbl.type().affinityKey();
 
         if (affCol != null) {
             // Only explicit affinity column should be shown. Do not do this for _KEY or it's alias.
-            if (!tbl.rowDescriptor().isKeyColumn(affCol.column.getColumnId())) {
-                affColName = affCol.columnName;
-            }
+            if (!F.eq(tbl.type().keyFieldName(), affCol) && !F.eq(tbl.type().keyFieldAlias(), affCol))
+                affColName = affCol;
         }
     }
 
@@ -74,7 +74,7 @@ public class SqlTableView {
      */
     @Order(2)
     public int cacheId() {
-        return tbl.cacheId();
+        return tbl.cacheInfo().cacheId();
     }
 
     /**
@@ -84,7 +84,7 @@ public class SqlTableView {
      */
     @Order(3)
     public String cacheName() {
-        return tbl.cacheName();
+        return tbl.cacheInfo().name();
     }
 
     /**
@@ -94,7 +94,7 @@ public class SqlTableView {
      */
     @Order(4)
     public String schemaName() {
-        return tbl.getSchema().getName();
+        return tbl.type().schemaName();
     }
 
     /**
@@ -104,7 +104,7 @@ public class SqlTableView {
      */
     @Order(5)
     public String tableName() {
-        return tbl.identifier().table();
+        return tbl.type().tableName();
     }
 
     /**
@@ -124,7 +124,7 @@ public class SqlTableView {
      */
     @Order(7)
     public String keyAlias() {
-        return tbl.rowDescriptor().type().keyFieldAlias();
+        return tbl.type().keyFieldAlias();
     }
 
     /**
@@ -134,7 +134,7 @@ public class SqlTableView {
      */
     @Order(8)
     public String valueAlias() {
-        return tbl.rowDescriptor().type().valueFieldAlias();
+        return tbl.type().valueFieldAlias();
     }
 
     /**
@@ -144,7 +144,7 @@ public class SqlTableView {
      */
     @Order(9)
     public String keyTypeName() {
-        return tbl.rowDescriptor().type().keyTypeName();
+        return tbl.type().keyTypeName();
     }
 
     /**
@@ -154,11 +154,11 @@ public class SqlTableView {
      */
     @Order(10)
     public String valueTypeName() {
-        return tbl.rowDescriptor().type().valueTypeName();
+        return tbl.type().valueTypeName();
     }
 
     /** @return {@code True} if index rebuild is in progress. */
     public boolean isIndexRebuildInProgress() {
-        return tbl.rebuildFromHashInProgress();
+        return tbl.isIndexRebuildInProgress();
     }
 }
diff --git a/modules/indexing/src/main/java/org/apache/ignite/spi/systemview/view/SqlViewColumnView.java b/modules/core/src/main/java/org/apache/ignite/spi/systemview/view/sql/SqlViewColumnView.java
similarity index 73%
rename from modules/indexing/src/main/java/org/apache/ignite/spi/systemview/view/SqlViewColumnView.java
rename to modules/core/src/main/java/org/apache/ignite/spi/systemview/view/sql/SqlViewColumnView.java
index cacc0454a01..be8a78d0f37 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/spi/systemview/view/SqlViewColumnView.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/systemview/view/sql/SqlViewColumnView.java
@@ -15,29 +15,29 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.spi.systemview.view;
+package org.apache.ignite.spi.systemview.view.sql;
 
+import java.util.Map;
 import org.apache.ignite.internal.managers.systemview.walker.Order;
+import org.apache.ignite.internal.processors.metric.impl.MetricUtils;
 import org.apache.ignite.internal.processors.query.QueryUtils;
-import org.apache.ignite.internal.processors.query.h2.sys.view.SqlSystemView;
-import org.h2.table.Column;
-import org.h2.value.DataType;
+import org.apache.ignite.spi.systemview.view.SystemView;
 
 /**
  * Sql view column representation for a {@link SystemView}.
  */
 public class SqlViewColumnView {
     /** System view. */
-    private final SqlSystemView view;
+    private final SystemView<?> view;
 
     /** Column. */
-    private final Column col;
+    private final Map.Entry<String, Class<?>> col;
 
     /**
      * @param view View.
      * @param col Column.
      */
-    public SqlViewColumnView(SqlSystemView view, Column col) {
+    public SqlViewColumnView(SystemView<?> view, Map.Entry<String, Class<?>> col) {
         this.view = view;
         this.col = col;
     }
@@ -45,7 +45,7 @@ public class SqlViewColumnView {
     /** @return Column name. */
     @Order
     public String columnName() {
-        return col.getName();
+        return MetricUtils.toSqlName(col.getKey());
     }
 
     /** @return Schema name. */
@@ -57,31 +57,31 @@ public class SqlViewColumnView {
     /** @return View name. */
     @Order(1)
     public String viewName() {
-        return view.getTableName();
+        return MetricUtils.toSqlName(view.name());
     }
 
     /** @return Field data type. */
     public String type() {
-        return DataType.getTypeClassName(col.getType());
+        return col.getValue().getName();
     }
 
     /** @return Field default. */
     public String defaultValue() {
-        return String.valueOf(col.getDefaultExpression());
+        return null;
     }
 
     /** @return Precision. */
-    public long precision() {
-        return col.getPrecision();
+    public int precision() {
+        return -1;
     }
 
     /** @return Scale. */
     public int scale() {
-        return col.getScale();
+        return -1;
     }
 
     /** @return {@code True} if nullable field. */
     public boolean nullable() {
-        return col.isNullable();
+        return !col.getValue().isPrimitive();
     }
 }
diff --git a/modules/indexing/src/main/java/org/apache/ignite/spi/systemview/view/SqlViewView.java b/modules/core/src/main/java/org/apache/ignite/spi/systemview/view/sql/SqlViewView.java
similarity index 76%
rename from modules/indexing/src/main/java/org/apache/ignite/spi/systemview/view/SqlViewView.java
rename to modules/core/src/main/java/org/apache/ignite/spi/systemview/view/sql/SqlViewView.java
index aceac67042c..46f0efdc947 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/spi/systemview/view/SqlViewView.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/systemview/view/sql/SqlViewView.java
@@ -15,34 +15,35 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.spi.systemview.view;
+package org.apache.ignite.spi.systemview.view.sql;
 
 import org.apache.ignite.internal.managers.systemview.walker.Order;
+import org.apache.ignite.internal.processors.metric.impl.MetricUtils;
 import org.apache.ignite.internal.processors.query.QueryUtils;
-import org.apache.ignite.internal.processors.query.h2.sys.view.SqlSystemView;
+import org.apache.ignite.spi.systemview.view.SystemView;
 
 /**
  * Sql view representation for a {@link SystemView}.
  */
 public class SqlViewView {
-    /** Sql system view. */
-    private final SqlSystemView view;
+    /** System view. */
+    private final SystemView<?> view;
 
-    /** @param view Sql system view. */
-    public SqlViewView(SqlSystemView view) {
+    /** @param view System view. */
+    public SqlViewView(SystemView<?> view) {
         this.view = view;
     }
 
     /** @return View name. */
     @Order
     public String name() {
-        return view.getTableName();
+        return MetricUtils.toSqlName(view.name());
     }
 
     /** @return View description. */
     @Order(2)
     public String description() {
-        return view.getDescription();
+        return view.description();
     }
 
     /** @return View schema. */
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientCacheInitializationFailTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientCacheInitializationFailTest.java
index da0cc5f9c59..7e171a96554 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientCacheInitializationFailTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientCacheInitializationFailTest.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.internal.processors.cache;
 
-import java.util.Collection;
 import java.util.Collections;
 import java.util.HashSet;
 import java.util.Random;
@@ -32,10 +31,8 @@ import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.configuration.NearCacheConfiguration;
 import org.apache.ignite.internal.IgniteKernal;
-import org.apache.ignite.internal.processors.query.ColumnInformation;
 import org.apache.ignite.internal.processors.query.DummyQueryIndexing;
 import org.apache.ignite.internal.processors.query.GridQueryProcessor;
-import org.apache.ignite.internal.processors.query.TableInformation;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.junit.Ignore;
@@ -241,25 +238,5 @@ public class IgniteClientCacheInitializationFailTest extends GridCommonAbstractT
             if (FAILED_CACHES.contains(cacheInfo.name()) && cacheInfo.cacheContext().kernalContext().clientNode())
                 throw new IgniteCheckedException("Test query exception " + cacheInfo.name() + " " + new Random().nextInt());
         }
-
-        /** {@inheritDoc} */
-        @Override public boolean initCacheContext(GridCacheContext ctx) throws IgniteCheckedException {
-            if (FAILED_CACHES.contains(ctx.name()) && ctx.kernalContext().clientNode())
-                throw new IgniteCheckedException("Test query exception " + ctx.name() + " " + new Random().nextInt());
-
-            return true;
-        }
-
-        /** {@inheritDoc} */
-        @Override public Collection<TableInformation> tablesInformation(String schemaNamePtrn, String tblNamePtrn,
-            String[] tblTypes) {
-            return null;
-        }
-
-        /** {@inheritDoc} */
-        @Override public Collection<ColumnInformation> columnsInformation(String schemaNamePtrn, String tblNamePtrn,
-            String colNamePtrn) {
-            return null;
-        }
     }
 }
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/query/DummyQueryIndexing.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/query/DummyQueryIndexing.java
index 3cbf8900e66..f1206050285 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/query/DummyQueryIndexing.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/query/DummyQueryIndexing.java
@@ -17,9 +17,7 @@
 
 package org.apache.ignite.internal.processors.query;
 
-import java.util.Collection;
 import java.util.List;
-import java.util.Set;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteDataStreamer;
 import org.apache.ignite.cache.query.FieldsQueryCursor;
@@ -33,8 +31,6 @@ import org.apache.ignite.internal.processors.cache.GridCacheContextInfo;
 import org.apache.ignite.internal.processors.cache.mvcc.MvccSnapshot;
 import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
 import org.apache.ignite.internal.processors.odbc.jdbc.JdbcParameterMeta;
-import org.apache.ignite.internal.processors.query.schema.SchemaIndexCacheVisitor;
-import org.apache.ignite.internal.processors.query.stat.IgniteStatisticsManager;
 import org.apache.ignite.internal.util.GridSpinBusyLock;
 import org.apache.ignite.internal.util.lang.GridCloseableIterator;
 import org.apache.ignite.lang.IgniteBiTuple;
@@ -113,48 +109,6 @@ public class DummyQueryIndexing implements GridQueryIndexing {
         return null;
     }
 
-    /** {@inheritDoc} */
-    @Override public void dynamicIndexCreate(
-        String schemaName,
-        String tblName,
-        QueryIndexDescriptorImpl idxDesc,
-        boolean ifNotExists,
-        SchemaIndexCacheVisitor cacheVisitor
-    ) throws IgniteCheckedException {
-
-    }
-
-    /** {@inheritDoc} */
-    @Override public void dynamicIndexDrop(
-        String schemaName,
-        String idxName,
-        boolean ifExists
-    ) throws IgniteCheckedException {
-
-    }
-
-    /** {@inheritDoc} */
-    @Override public void dynamicAddColumn(
-        String schemaName,
-        String tblName,
-        List<QueryField> cols,
-        boolean ifTblExists,
-        boolean ifColNotExists
-    ) throws IgniteCheckedException {
-
-    }
-
-    /** {@inheritDoc} */
-    @Override public void dynamicDropColumn(
-        String schemaName,
-        String tblName,
-        List<String> cols,
-        boolean ifTblExists,
-        boolean ifColExists
-    ) throws IgniteCheckedException {
-
-    }
-
     /** {@inheritDoc} */
     @Override public void registerCache(
         String cacheName,
@@ -165,7 +119,7 @@ public class DummyQueryIndexing implements GridQueryIndexing {
     }
 
     /** {@inheritDoc} */
-    @Override public void unregisterCache(GridCacheContextInfo cacheInfo, boolean rmvIdx, boolean clearIdx) throws IgniteCheckedException {
+    @Override public void unregisterCache(GridCacheContextInfo<?, ?> cacheInfo) {
 
     }
 
@@ -187,15 +141,6 @@ public class DummyQueryIndexing implements GridQueryIndexing {
         return null;
     }
 
-    /** {@inheritDoc} */
-    @Override public boolean registerType(
-        GridCacheContextInfo cacheInfo,
-        GridQueryTypeDescriptor desc,
-        boolean isSql
-    ) throws IgniteCheckedException {
-        return false;
-    }
-
     /** {@inheritDoc} */
     @Override public List<JdbcParameterMeta> parameterMetaData(
         String schemaName,
@@ -228,11 +173,6 @@ public class DummyQueryIndexing implements GridQueryIndexing {
 
     }
 
-    /** {@inheritDoc} */
-    @Override public void markAsRebuildNeeded(GridCacheContext cctx, boolean val) {
-
-    }
-
     /** {@inheritDoc} */
     @Override public IndexingQueryFilter backupFilter(AffinityTopologyVersion topVer, int[] parts) {
         return null;
@@ -253,64 +193,18 @@ public class DummyQueryIndexing implements GridQueryIndexing {
 
     }
 
-    /** {@inheritDoc} */
-    @Override public String schema(String cacheName) {
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public Set<String> schemasNames() {
-        return null;
-    }
-
     /** {@inheritDoc} */
     @Override public boolean isStreamableInsertStatement(String schemaName, SqlFieldsQuery sql) {
         return false;
     }
 
-    /** {@inheritDoc} */
-    @Override public @Nullable GridCacheContextInfo registeredCacheInfo(String cacheName) {
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void closeCacheOnClient(String cacheName) {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean initCacheContext(GridCacheContext ctx) throws IgniteCheckedException {
-        return false;
-    }
-
     /** {@inheritDoc} */
     @Override public void registerMxBeans(IgniteMBeansManager mbMgr) {
 
     }
 
-    /** {@inheritDoc} */
-    @Override public Collection<TableInformation> tablesInformation(
-        String schemaNamePtrn,
-        String tblNamePtrn,
-        String... tblTypes) {
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public Collection<ColumnInformation> columnsInformation(
-        String schemaNamePtrn,
-        String tblNamePtrn,
-        String colNamePtrn) {
-        return null;
-    }
-
     /** {@inheritDoc} */
     @Override public boolean isConvertibleToColumnType(String schemaName, String tblName, String colName, Class<?> cls) {
         return false;
     }
-
-    /** {@inheritDoc} */
-    @Override public IgniteStatisticsManager statsManager() {
-        return null;
-    }
 }
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 3392d2f74e8..49278ed2a89 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
@@ -21,17 +21,17 @@ import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
-import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
 import java.util.Set;
 import java.util.UUID;
+import java.util.stream.Collectors;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteDataStreamer;
 import org.apache.ignite.IgniteSystemProperties;
 import org.apache.ignite.cache.QueryEntity;
-import org.apache.ignite.cache.QueryIndex;
+import org.apache.ignite.cache.QueryIndexType;
 import org.apache.ignite.cache.query.BulkLoadContextCursor;
 import org.apache.ignite.cache.query.FieldsQueryCursor;
 import org.apache.ignite.configuration.CacheConfiguration;
@@ -45,7 +45,6 @@ import org.apache.ignite.internal.processors.bulkload.BulkLoadStreamerWriter;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal;
 import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
-import org.apache.ignite.internal.processors.query.GridQueryProperty;
 import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
 import org.apache.ignite.internal.processors.query.IgniteSQLException;
 import org.apache.ignite.internal.processors.query.NestedTxMode;
@@ -71,9 +70,11 @@ 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.SqlDropIndexCommand;
+import org.apache.ignite.internal.sql.command.SqlIndexColumn;
 import org.apache.ignite.internal.sql.command.SqlRollbackTransactionCommand;
 import org.apache.ignite.internal.sql.command.SqlSetStreamingCommand;
-import org.apache.ignite.internal.util.future.GridFinishedFuture;
 import org.apache.ignite.internal.util.lang.IgniteClosureX;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.U;
@@ -104,7 +105,7 @@ import static org.apache.ignite.internal.processors.query.h2.sql.GridSqlQueryPar
  */
 public class CommandProcessor extends SqlCommandProcessor {
     /** Schema manager. */
-    private final SchemaManager schemaMgr;
+    private final H2SchemaManager schemaMgr;
 
     /** H2 Indexing. */
     private final IgniteH2Indexing idx;
@@ -119,8 +120,8 @@ public class CommandProcessor extends SqlCommandProcessor {
      * @param ctx Kernal context.
      * @param schemaMgr Schema manager.
      */
-    public CommandProcessor(GridKernalContext ctx, SchemaManager schemaMgr, IgniteH2Indexing idx) {
-        super(ctx, schemaMgr);
+    public CommandProcessor(GridKernalContext ctx, H2SchemaManager schemaMgr, IgniteH2Indexing idx) {
+        super(ctx);
 
         this.schemaMgr = schemaMgr;
         this.idx = idx;
@@ -145,9 +146,14 @@ public class CommandProcessor extends SqlCommandProcessor {
         FieldsQueryCursor<List<?>> res = H2Utils.zeroCursor();
         boolean unregister = true;
 
-        if (cmdNative != null) {
-            assert cmdH2 == null;
+        if (cmdH2 != null) {
+            assert cmdNative == null;
+
+            // Some commands are duplicated.
+            cmdNative = convertH2Command(cmdH2);
+        }
 
+        if (cmdNative != null) {
             if (isCommandSupported(cmdNative)) {
                 FieldsQueryCursor<List<?>> resNative = runNativeCommand(sql, cmdNative, params, cliCtx, qryId);
 
@@ -203,6 +209,41 @@ public class CommandProcessor extends SqlCommandProcessor {
             || cmd instanceof SqlSetStreamingCommand;
     }
 
+    /**
+     * Converts H2 command to corresponding native command if possible.
+     *
+     * @param cmdH2 H2 command.
+     * @return Native command.
+     **/
+    private @Nullable SqlCommand convertH2Command(GridSqlStatement cmdH2) {
+        if (cmdH2 instanceof GridSqlCreateIndex) {
+            GridSqlCreateIndex cmd = (GridSqlCreateIndex)cmdH2;
+
+            return new SqlCreateIndexCommand(
+                cmd.schemaName(),
+                cmd.tableName(),
+                cmd.index().getName(),
+                cmd.ifNotExists(),
+                cmd.index().getFields().entrySet().stream().map(e -> new SqlIndexColumn(e.getKey(), !e.getValue()))
+                    .collect(Collectors.toList()),
+                cmd.index().getIndexType() == QueryIndexType.GEOSPATIAL,
+                0,
+                -1
+            );
+        }
+        else if (cmdH2 instanceof GridSqlDropIndex) {
+            GridSqlDropIndex cmd = (GridSqlDropIndex)cmdH2;
+
+            return new SqlDropIndexCommand(
+                cmd.schemaName(),
+                cmd.indexName(),
+                cmd.ifExists()
+            );
+        }
+
+        return null;
+    }
+
     /**
      * Execute DDL statement.
      *
@@ -215,63 +256,7 @@ public class CommandProcessor extends SqlCommandProcessor {
         try {
             finishActiveTxIfNecessary();
 
-            if (cmdH2 instanceof GridSqlCreateIndex) {
-                GridSqlCreateIndex cmd = (GridSqlCreateIndex)cmdH2;
-
-                isDdlOnSchemaSupported(cmd.schemaName());
-
-                GridH2Table tbl = schemaMgr.dataTable(cmd.schemaName(), cmd.tableName());
-
-                if (tbl == null)
-                    throw new SchemaOperationException(SchemaOperationException.CODE_TABLE_NOT_FOUND, cmd.tableName());
-
-                assert tbl.rowDescriptor() != null;
-
-                QueryIndex newIdx = new QueryIndex();
-
-                newIdx.setName(cmd.index().getName());
-
-                newIdx.setIndexType(cmd.index().getIndexType());
-
-                LinkedHashMap<String, Boolean> flds = new LinkedHashMap<>();
-
-                // Let's replace H2's table and property names by those operated by GridQueryProcessor.
-                GridQueryTypeDescriptor typeDesc = tbl.rowDescriptor().type();
-
-                for (Map.Entry<String, Boolean> e : cmd.index().getFields().entrySet()) {
-                    GridQueryProperty prop = typeDesc.property(e.getKey());
-
-                    if (prop == null)
-                        throw new SchemaOperationException(SchemaOperationException.CODE_COLUMN_NOT_FOUND, e.getKey());
-
-                    flds.put(prop.name(), e.getValue());
-                }
-
-                newIdx.setFields(flds);
-
-                fut = ctx.query().dynamicIndexCreate(tbl.cacheName(), cmd.schemaName(), typeDesc.tableName(),
-                    newIdx, cmd.ifNotExists(), 0);
-            }
-            else if (cmdH2 instanceof GridSqlDropIndex) {
-                GridSqlDropIndex cmd = (GridSqlDropIndex)cmdH2;
-
-                isDdlOnSchemaSupported(cmd.schemaName());
-
-                GridH2Table tbl = schemaMgr.dataTableForIndex(cmd.schemaName(), cmd.indexName());
-
-                if (tbl != null) {
-                    fut = ctx.query().dynamicIndexDrop(tbl.cacheName(), cmd.schemaName(), cmd.indexName(),
-                        cmd.ifExists());
-                }
-                else {
-                    if (cmd.ifExists())
-                        fut = new GridFinishedFuture();
-                    else
-                        throw new SchemaOperationException(SchemaOperationException.CODE_INDEX_NOT_FOUND,
-                            cmd.indexName());
-                }
-            }
-            else if (cmdH2 instanceof GridSqlCreateTable) {
+            if (cmdH2 instanceof GridSqlCreateTable) {
                 GridSqlCreateTable cmd = (GridSqlCreateTable)cmdH2;
 
                 ctx.security().authorize(cmd.cacheName(), SecurityPermission.CACHE_CREATE);
@@ -779,7 +764,7 @@ public class CommandProcessor extends SqlCommandProcessor {
                 IgniteQueryErrorCode.TABLE_NOT_FOUND);
         }
 
-        H2Utils.checkAndStartNotStartedCache(ctx, tbl);
+        H2Utils.checkAndStartNotStartedCache(ctx, tbl.cacheInfo());
 
         UpdatePlan plan = UpdatePlanBuilder.planForBulkLoad(cmd, tbl);
 
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 26b496324b0..098e552d21c 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
@@ -32,6 +32,7 @@ import org.apache.ignite.internal.processors.query.QueryUtils;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2DefaultTableEngine;
 import org.apache.ignite.internal.processors.query.h2.opt.H2PlainRowFactory;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor;
+import org.apache.ignite.internal.util.GridBusyLock;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.h2.api.JavaObjectSerializer;
 import org.h2.engine.Database;
@@ -101,11 +102,14 @@ public class ConnectionManager {
     private final ConcurrentStripedPool<H2Connection> connPool;
 
     /** H2 connection for INFORMATION_SCHEMA. Holds H2 open until node is stopped. */
-    private volatile Connection sysConn;
+    private final Connection sysConn;
 
     /** H2 data handler. Primarily used for serialization. */
     private final DataHandler dataNhd;
 
+    /** Busy lock. */
+    private final GridBusyLock busyLock = new GridBusyLock();
+
     /**
      * Constructor.
      *
@@ -125,11 +129,7 @@ public class ConnectionManager {
 
             sysConn.setSchema(QueryUtils.SCHEMA_INFORMATION);
 
-            assert sysConn instanceof JdbcConnection : sysConn;
-
-            JdbcConnection conn = (JdbcConnection)sysConn;
-
-            dataNhd = conn.getSession().getDataHandler();
+            dataNhd = sysConn.unwrap(JdbcConnection.class).getSession().getDataHandler();
         }
         catch (SQLException e) {
             throw new IgniteSQLException("Failed to initialize DB connection: " + dbUrl, e);
@@ -193,6 +193,8 @@ public class ConnectionManager {
      * Close all connections.
      */
     private void closeConnections() {
+        busyLock.block();
+
         connPool.forEach(c -> U.close(c.connection(), log));
         connPool.clear();
 
@@ -272,6 +274,9 @@ public class ConnectionManager {
      * @return H2 connection wrapper.
      */
     public H2PooledConnection connection() {
+        if (!busyLock.enterBusy())
+            throw new IllegalStateException("Failed to initialize DB connection (grid is stopping)");
+
         try {
             H2Connection conn = connPool.borrow();
 
@@ -289,6 +294,9 @@ public class ConnectionManager {
         catch (SQLException e) {
             throw new IgniteSQLException("Failed to initialize DB connection: " + dbUrl, e);
         }
+        finally {
+            busyLock.leaveBusy();
+        }
     }
 
     /**
@@ -311,12 +319,20 @@ public class ConnectionManager {
      * @param conn Connection.
      */
     void recycle(H2Connection conn) {
-        boolean rmv = usedConns.remove(conn);
+        if (!busyLock.enterBusy())
+            return;
+
+        try {
+            boolean rmv = usedConns.remove(conn);
 
-        assert rmv : "Connection isn't tracked [conn=" + conn + ']';
+            assert rmv : "Connection isn't tracked [conn=" + conn + ']';
 
-        if (!connPool.recycle(conn))
-            conn.close();
+            if (!connPool.recycle(conn))
+                conn.close();
+        }
+        finally {
+            busyLock.leaveBusy();
+        }
     }
 
     /**
@@ -335,11 +351,4 @@ public class ConnectionManager {
         if (dataNhd != null && dataNhd instanceof Database)
             DB_JOBJ_SERIALIZER.set((Database)dataNhd, serializer);
     }
-
-    /**
-     * @return H2 connection.
-     */
-    public JdbcConnection jdbcConnection() {
-        return (JdbcConnection)sysConn;
-    }
 }
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2IndexFactory.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2IndexFactory.java
new file mode 100644
index 00000000000..14250c84972
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2IndexFactory.java
@@ -0,0 +1,115 @@
+/*
+ * 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.ArrayList;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cache.QueryIndexType;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.cache.query.index.SortOrder;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyDefinition;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.InlineIndex;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.InlineIndexImpl;
+import org.apache.ignite.internal.processors.cache.GridCacheContextInfo;
+import org.apache.ignite.internal.processors.query.h2.database.H2TreeClientIndex;
+import org.apache.ignite.internal.processors.query.h2.database.H2TreeIndex;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2ProxyIndex;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2ProxySpatialIndex;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table;
+import org.apache.ignite.internal.processors.query.schema.management.IndexDescriptor;
+import org.h2.index.Index;
+import org.h2.index.IndexType;
+import org.h2.table.Column;
+import org.h2.table.IndexColumn;
+
+/**
+ * Index factory for H2 indexes.
+ */
+class H2IndexFactory {
+    /** */
+    private final IgniteLogger log;
+
+    /** */
+    H2IndexFactory(GridKernalContext ctx) {
+        log = ctx.log(H2IndexFactory.class);
+    }
+
+    /**
+     * Create H2 index.
+     */
+    Index createIndex(GridH2Table tbl, IndexDescriptor idxDesc) {
+        GridCacheContextInfo<?, ?> cacheInfo = tbl.cacheInfo();
+
+        if (log.isDebugEnabled())
+            log.debug("Creating H2 index [cacheId=" + cacheInfo.cacheId() + ", idxName=" + idxDesc.name() + ']');
+
+        // Convert key definitions to list of IndexColumns.
+        LinkedHashMap<String, IndexKeyDefinition> keyDefs = idxDesc.keyDefinitions();
+        List<IndexColumn> idxCols = new ArrayList<>(keyDefs.size());
+
+        for (Map.Entry<String, IndexKeyDefinition> keyDef : keyDefs.entrySet()) {
+            Column col = tbl.getColumn(keyDef.getKey());
+
+            idxCols.add(tbl.indexColumn(col.getColumnId(),
+                keyDef.getValue().order().sortOrder() == SortOrder.ASC ? org.h2.result.SortOrder.ASCENDING
+                    : org.h2.result.SortOrder.DESCENDING));
+        }
+
+        IndexColumn[] idxColsArr = idxCols.toArray(new IndexColumn[idxCols.size()]);
+
+        if (idxDesc.type() == QueryIndexType.SORTED) {
+            if (idxDesc.isProxy()) {
+                Index targetIdx = tbl.getIndex(idxDesc.targetIdx().name());
+
+                assert targetIdx != null;
+
+                return new GridH2ProxyIndex(tbl, idxDesc.name(), idxCols, targetIdx);
+            }
+
+            if (cacheInfo.affinityNode()) {
+                InlineIndexImpl qryIdx = idxDesc.index().unwrap(InlineIndexImpl.class);
+
+                return new H2TreeIndex(qryIdx, tbl, idxColsArr, idxDesc.isPk(), log);
+            }
+            else {
+                InlineIndex qryIdx = idxDesc.index().unwrap(InlineIndex.class);
+
+                IndexType idxType = idxDesc.isPk() ? IndexType.createPrimaryKey(false, false) :
+                    IndexType.createNonUnique(false, false, false);
+
+                return new H2TreeClientIndex(qryIdx, tbl, idxDesc.name(), idxColsArr, idxType);
+            }
+        }
+        else if (idxDesc.type() == QueryIndexType.GEOSPATIAL) {
+            if (idxDesc.isProxy()) {
+                Index targetIdx = tbl.getIndex(idxDesc.targetIdx().name());
+
+                assert targetIdx != null;
+
+                return new GridH2ProxySpatialIndex(tbl, idxDesc.name(), idxCols, targetIdx);
+            }
+
+            return H2Utils.createSpatialIndex(tbl, idxDesc, idxCols);
+        }
+
+        throw new IllegalStateException("Index type: " + idxDesc.type());
+    }
+}
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2SchemaManager.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2SchemaManager.java
new file mode 100644
index 00000000000..b93d6ce21f4
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2SchemaManager.java
@@ -0,0 +1,417 @@
+/*
+ * 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.lang.reflect.Method;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.processors.cache.GridCacheContextInfo;
+import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
+import org.apache.ignite.internal.processors.query.GridQueryRowDescriptorImpl;
+import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
+import org.apache.ignite.internal.processors.query.IgniteSQLException;
+import org.apache.ignite.internal.processors.query.QueryField;
+import org.apache.ignite.internal.processors.query.QueryUtils;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2ProxyIndex;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2RowDescriptor;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table;
+import org.apache.ignite.internal.processors.query.h2.sys.SqlSystemTableEngine;
+import org.apache.ignite.internal.processors.query.h2.sys.view.FiltrableSystemViewLocal;
+import org.apache.ignite.internal.processors.query.h2.sys.view.SystemViewLocal;
+import org.apache.ignite.internal.processors.query.schema.SchemaChangeListener;
+import org.apache.ignite.internal.processors.query.schema.management.IndexDescriptor;
+import org.apache.ignite.internal.processors.query.schema.management.SchemaManager;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.spi.systemview.view.FiltrableSystemView;
+import org.apache.ignite.spi.systemview.view.SystemView;
+import org.h2.index.Index;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * H2 schema manager. Responsible for reflecting to H2 database all schema changes.
+ */
+public class H2SchemaManager implements SchemaChangeListener {
+    /** Connection manager. */
+    private final ConnectionManager connMgr;
+
+    /** Data tables. */
+    private final ConcurrentMap<QueryTable, GridH2Table> dataTables = new ConcurrentHashMap<>();
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Indexing. */
+    private final IgniteH2Indexing idx;
+
+    /** H2 index factory. */
+    private final H2IndexFactory idxFactory;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Underlying core schema manager. */
+    private volatile SchemaManager schemaMgr;
+
+    /**
+     * Constructor.
+     *
+     * @param ctx Kernal context.
+     * @param idx Indexing.
+     * @param connMgr Connection manager.
+     */
+    public H2SchemaManager(GridKernalContext ctx, IgniteH2Indexing idx, ConnectionManager connMgr) {
+        this.ctx = ctx;
+        this.idx = idx;
+        this.connMgr = connMgr;
+
+        idxFactory = new H2IndexFactory(ctx);
+        log = ctx.log(H2SchemaManager.class);
+    }
+
+    /**
+     * Handle node start.
+     */
+    public void start() throws IgniteCheckedException {
+        schemaMgr = ctx.query().schemaManager();
+
+        ctx.internalSubscriptionProcessor().registerSchemaChangeListener(this);
+
+        // Register predefined system functions.
+        createSqlFunction(QueryUtils.DFLT_SCHEMA, "QUERY_ENGINE", true,
+            H2Utils.class.getName() + ".queryEngine");
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onSchemaCreated(String schema) {
+        try {
+            connMgr.executeSystemStatement("CREATE SCHEMA IF NOT EXISTS " + H2Utils.withQuotes(schema));
+
+            if (log.isDebugEnabled())
+                log.debug("Created H2 schema for index database: " + schema);
+        }
+        catch (IgniteCheckedException e) {
+            throw new IgniteException("Failed to create database schema: " + schema, e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onSchemaDropped(String schema) {
+        try {
+            connMgr.executeSystemStatement("DROP SCHEMA IF EXISTS " + H2Utils.withQuotes(schema));
+
+            if (log.isDebugEnabled())
+                log.debug("Dropped H2 schema for index database: " + schema);
+        }
+        catch (IgniteCheckedException e) {
+            throw new IgniteException("Failed to drop database schema: " + schema, e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onSqlTypeCreated(
+        String schemaName,
+        GridQueryTypeDescriptor typeDesc,
+        GridCacheContextInfo<?, ?> cacheInfo
+    ) {
+        H2TableDescriptor tblDesc = new H2TableDescriptor(idx, schemaName, typeDesc, cacheInfo);
+
+        try (H2PooledConnection conn = connMgr.connection(schemaName)) {
+            GridH2Table h2tbl = createTable(tblDesc, conn);
+
+            if (dataTables.putIfAbsent(h2tbl.identifier(), h2tbl) != null)
+                throw new IllegalStateException("Table already exists: " + h2tbl.identifierString());
+        }
+        catch (SQLException e) {
+            throw new IgniteException("Failed to register query type: " + tblDesc, e);
+        }
+    }
+
+    /**
+     * Create db table by using given table descriptor.
+     *
+     * @param tbl Table descriptor.
+     * @param conn Connection.
+     * @throws SQLException If failed to create db table.
+     */
+    private GridH2Table createTable(H2TableDescriptor tbl, H2PooledConnection conn) throws SQLException {
+        assert tbl != null;
+
+        String sql = H2Utils.tableCreateSql(tbl);
+
+        if (log.isDebugEnabled())
+            log.debug("Creating DB table with SQL: " + sql);
+
+        GridH2RowDescriptor rowDesc = new GridH2RowDescriptor(
+            new GridQueryRowDescriptorImpl(tbl.cacheInfo(), tbl.type()));
+
+        return H2TableEngine.createTable(conn.connection(), sql, rowDesc, tbl, ctx.indexProcessor());
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onSqlTypeDropped(String schemaName, GridQueryTypeDescriptor typeDesc, boolean destroy) {
+        GridH2Table tbl = dataTable(schemaName, typeDesc.tableName());
+
+        if (tbl == null) {
+            throw new IgniteException("Failed to drop database table (table not found) [schemaName=" + schemaName +
+                ", tblName=" + typeDesc.tableName() + ']');
+        }
+
+        dropTable(tbl.tableDescriptor());
+        dataTables.remove(tbl.identifier(), tbl);
+    }
+
+    /**
+     * Drops table form h2 database and clear all related indexes (h2 text, lucene).
+     *
+     * @param tbl Table to unregister.
+     */
+    private void dropTable(H2TableDescriptor tbl) {
+        assert tbl != null;
+
+        if (log.isDebugEnabled())
+            log.debug("Removing query index table: " + tbl.fullTableName());
+
+        try (H2PooledConnection c = connMgr.connection(tbl.schemaName())) {
+            Statement stmt = null;
+
+            try {
+                stmt = c.connection().createStatement();
+
+                String sql = "DROP TABLE IF EXISTS " + tbl.fullTableName();
+
+                if (log.isDebugEnabled())
+                    log.debug("Dropping database index table with SQL: " + sql);
+
+                stmt.executeUpdate(sql);
+            }
+            catch (SQLException e) {
+                throw new IgniteSQLException("Failed to drop database table [type=" + tbl.type().name() +
+                    ", table=" + tbl.fullTableName() + "]", IgniteQueryErrorCode.TABLE_DROP_FAILED, e);
+            }
+            finally {
+                U.close(stmt, log);
+            }
+        }
+
+        tbl.onDrop();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onFunctionCreated(String schema, String name, boolean deterministic, Method method) {
+        try {
+            createSqlFunction(schema, name, deterministic,
+                method.getDeclaringClass().getName() + '.' + method.getName());
+        }
+        catch (IgniteCheckedException e) {
+            throw new IgniteException("Failed to create database function: " + name, e);
+        }
+    }
+
+    /**
+     * Registers SQL function.
+     *
+     * @param schema Schema.
+     * @param alias Function alias.
+     * @param deterministic Deterministic flag.
+     * @param methodName Public static method name (including class full name).
+     */
+    private void createSqlFunction(String schema, String alias, boolean deterministic, String methodName)
+        throws IgniteCheckedException {
+        String clause = "CREATE ALIAS IF NOT EXISTS " + alias + (deterministic ?
+            " DETERMINISTIC FOR \"" :
+            " FOR \"") +
+            methodName + '"';
+
+        connMgr.executeStatement(schema, clause);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onSystemViewCreated(String schema, SystemView<?> view) {
+        SystemViewLocal<?> sysView = view instanceof FiltrableSystemView ?
+            new FiltrableSystemViewLocal<>(ctx, view) : new SystemViewLocal<>(ctx, view);
+
+        try {
+            try (H2PooledConnection c = connMgr.connection(schema)) {
+                SqlSystemTableEngine.registerView(c.connection(), sysView);
+            }
+        }
+        catch (SQLException e) {
+            throw new IgniteException("Failed to register system view: " + view.name(), e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onIndexCreated(String schemaName, String tblName, String idxName, IndexDescriptor idxDesc) {
+        GridH2Table tbl = dataTable(schemaName, tblName);
+
+        if (tbl == null)
+            return;
+
+        try {
+            Index h2Idx = idxFactory.createIndex(tbl, idxDesc);
+
+            // Do not register system indexes as DB objects.
+            if (isSystemIndex(h2Idx)) {
+                tbl.addSystemIndex(h2Idx);
+
+                return;
+            }
+
+            tbl.proposeUserIndex(h2Idx);
+
+            try {
+                // At this point index is in consistent state, promote it through H2 SQL statement, so that cached
+                // prepared statements are re-built.
+                String sql = H2Utils.indexCreateSql(tbl.tableDescriptor().fullTableName(), h2Idx, true);
+
+                connMgr.executeStatement(schemaName, sql);
+            }
+            catch (Exception e) {
+                // Rollback and re-throw.
+                tbl.rollbackUserIndex(h2Idx.getName());
+
+                throw e;
+            }
+        }
+        catch (Exception e) {
+            throw new IgniteException("Failed to register index in internal H2 database: " + idxName, e);
+        }
+    }
+
+    /** */
+    private static boolean isSystemIndex(Index idx) {
+        return QueryUtils.PRIMARY_KEY_INDEX.equals(idx.getName())
+            || QueryUtils.AFFINITY_KEY_INDEX.equals(idx.getName())
+            || (idx instanceof GridH2ProxyIndex && isSystemIndex(((GridH2ProxyIndex)idx).underlyingIndex()));
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onIndexDropped(String schemaName, String tblName, String idxName) {
+        GridH2Table tbl = dataTable(schemaName, tblName);
+
+        if (tbl == null)
+            return;
+
+        Index idx = F.find(tbl.getIndexes(), null, i -> idxName.equals(i.getName()));
+
+        if (idx == null)
+            return;
+
+        // System indexes are not registred as DB objects.
+        if (isSystemIndex(idx)) {
+            tbl.removeIndex(idx);
+
+            return;
+        }
+
+        String sql = H2Utils.indexDropSql(schemaName, idxName, true);
+
+        try {
+            connMgr.executeStatement(schemaName, sql);
+        }
+        catch (IgniteCheckedException e) {
+            throw new IgniteException("Failed to unregister index: " + idxName, e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onColumnsAdded(
+        String schemaName,
+        GridQueryTypeDescriptor typeDesc,
+        GridCacheContextInfo<?, ?> cacheInfo,
+        List<QueryField> cols
+    ) {
+        GridH2Table tbl = dataTable(schemaName, typeDesc.tableName());
+
+        if (tbl == null)
+            return;
+
+        tbl.addColumns(cols);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onColumnsDropped(
+        String schemaName,
+        GridQueryTypeDescriptor typeDesc,
+        GridCacheContextInfo<?, ?> cacheInfo,
+        List<String> cols
+    ) {
+        GridH2Table tbl = dataTable(schemaName, typeDesc.tableName());
+
+        if (tbl == null)
+            return;
+
+        tbl.dropColumns(cols);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onIndexRebuildStarted(String schemaName, String tblName) {
+        GridH2Table tbl = dataTable(schemaName, tblName);
+
+        if (tbl != null)
+            tbl.markRebuildFromHashInProgress(true);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onIndexRebuildFinished(String schemaName, String tblName) {
+        GridH2Table tbl = dataTable(schemaName, tblName);
+
+        if (tbl != null)
+            tbl.markRebuildFromHashInProgress(false);
+    }
+
+    /**
+     * Get table descriptor.
+     *
+     * @param schemaName Schema name.
+     * @param cacheName Cache name.
+     * @param type Type name.
+     * @return Descriptor.
+     */
+    @Nullable public H2TableDescriptor tableForType(String schemaName, String cacheName, String type) {
+        GridQueryTypeDescriptor typeDesc = schemaMgr.typeDescriptorForType(schemaName, cacheName, type);
+
+        if (typeDesc != null) {
+            GridH2Table tbl = dataTable(schemaName, typeDesc.tableName());
+
+            if (tbl != null)
+                return tbl.tableDescriptor();
+        }
+
+        return null;
+    }
+
+    /**
+     * Find H2 table by it's identifier.
+     *
+     * @param schemaName Schema name.
+     * @param tblName Table name.
+     * @return Table or {@code null} if none found.
+     */
+    public GridH2Table dataTable(String schemaName, String tblName) {
+        return dataTables.get(new QueryTable(schemaName, tblName));
+    }
+}
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2TableDescriptor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2TableDescriptor.java
index 288b8e91a16..e8206a48835 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2TableDescriptor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2TableDescriptor.java
@@ -17,34 +17,23 @@
 
 package org.apache.ignite.internal.processors.query.h2;
 
-import java.util.ArrayList;
-import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
-import org.apache.ignite.cache.QueryIndexType;
-import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.GridCacheContextInfo;
 import org.apache.ignite.internal.processors.query.GridQueryIndexDescriptor;
-import org.apache.ignite.internal.processors.query.GridQueryProperty;
-import org.apache.ignite.internal.processors.query.GridQueryRowDescriptor;
 import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
 import org.apache.ignite.internal.processors.query.QueryUtils;
 import org.apache.ignite.internal.processors.query.h2.database.H2PkHashIndex;
-import org.apache.ignite.internal.processors.query.h2.database.H2TreeIndexBase;
-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.opt.GridLuceneIndex;
-import org.apache.ignite.internal.processors.query.schema.SchemaIndexCacheVisitor;
+import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.h2.index.Index;
 import org.h2.result.SortOrder;
-import org.h2.table.Column;
 import org.h2.table.IndexColumn;
-import org.jetbrains.annotations.NotNull;
-import org.jetbrains.annotations.Nullable;
 
 /**
  * Information about table in database.
@@ -57,7 +46,7 @@ public class H2TableDescriptor {
     public static final String PK_HASH_IDX_NAME = "_key_PK_hash";
 
     /** Affinity key index name. */
-    public static final String AFFINITY_KEY_IDX_NAME = "AFFINITY_KEY";
+    public static final String AFFINITY_KEY_IDX_NAME = QueryUtils.AFFINITY_KEY_INDEX;
 
     /** Indexing. */
     private final IgniteH2Indexing idx;
@@ -72,7 +61,7 @@ public class H2TableDescriptor {
     private final String schemaName;
 
     /** Cache context info. */
-    private final GridCacheContextInfo cacheInfo;
+    private final GridCacheContextInfo<?, ?> cacheInfo;
 
     /** */
     private GridH2Table tbl;
@@ -83,9 +72,6 @@ public class H2TableDescriptor {
     /** */
     private H2PkHashIndex pkHashIdx;
 
-    /** Flag of table has been created from SQL*/
-    private boolean isSql;
-
     /**
      * Constructor.
      *
@@ -93,26 +79,21 @@ public class H2TableDescriptor {
      * @param schemaName Schema name.
      * @param type Type descriptor.
      * @param cacheInfo Cache context info.
-     * @param isSql {@code true} in case table has been created from SQL.
      */
-    public H2TableDescriptor(IgniteH2Indexing idx, String schemaName, GridQueryTypeDescriptor type,
-        GridCacheContextInfo cacheInfo, boolean isSql) {
+    public H2TableDescriptor(
+        IgniteH2Indexing idx,
+        String schemaName,
+        GridQueryTypeDescriptor type,
+        GridCacheContextInfo<?, ?> cacheInfo
+    ) {
         this.idx = idx;
         this.type = type;
         this.schemaName = schemaName;
         this.cacheInfo = cacheInfo;
-        this.isSql = isSql;
 
         fullTblName = H2Utils.withQuotes(schemaName) + "." + H2Utils.withQuotes(type.tableName());
     }
 
-    /**
-     * @return {@code true} In case table was created from SQL.
-     */
-    public boolean sql() {
-        return isSql;
-    }
-
     /**
      * @return Indexing.
      */
@@ -172,17 +153,10 @@ public class H2TableDescriptor {
     /**
      * @return Cache context info.
      */
-    public GridCacheContextInfo cacheInfo() {
+    public GridCacheContextInfo<?, ?> cacheInfo() {
         return cacheInfo;
     }
 
-    /**
-     * @return Cache context.
-     */
-    public GridCacheContext cache() {
-        return cacheInfo.cacheContext();
-    }
-
     /**
      * @return Type.
      */
@@ -197,54 +171,42 @@ public class H2TableDescriptor {
         return luceneIdx;
     }
 
+    /**
+     * @return Hash index.
+     */
+    public Index hashIndex() {
+        return pkHashIdx;
+    }
+
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(H2TableDescriptor.class, this);
     }
 
     /**
-     * Create list of indexes. First must be primary key, after that all unique indexes and only then non-unique
-     * indexes. All indexes must be subtypes of {@link H2TreeIndexBase}.
-     *
-     * @param tbl Table to create indexes for.
-     * @return List of indexes.
+     * Create hash index if needed.
      */
-    public ArrayList<Index> createSystemIndexes(GridH2Table tbl) {
-        ArrayList<Index> idxs = new ArrayList<>();
-
-        IndexColumn keyCol = tbl.indexColumn(QueryUtils.KEY_COL, SortOrder.ASCENDING);
-        IndexColumn affCol = tbl.getAffinityKeyColumn();
-
-        if (affCol != null && H2Utils.equals(affCol, keyCol))
-            affCol = null;
-
-        List<IndexColumn> unwrappedKeyAndAffinityCols = extractKeyColumns(tbl, keyCol, affCol);
-
-        List<IndexColumn> wrappedKeyCols = H2Utils.treeIndexColumns(tbl.rowDescriptor(),
-            new ArrayList<>(2), keyCol, affCol);
+    public void createHashIndex(GridH2Table tbl) {
+        if (cacheInfo.affinityNode()) {
+            IndexColumn keyCol = tbl.indexColumn(QueryUtils.KEY_COL, SortOrder.ASCENDING);
+            IndexColumn affCol = tbl.getAffinityKeyColumn();
 
-        Index hashIdx = createHashIndex(
-            tbl,
-            wrappedKeyCols
-        );
+            if (affCol != null && H2Utils.equals(affCol, keyCol))
+                affCol = null;
 
-        if (hashIdx != null)
-            idxs.add(hashIdx);
+            List<IndexColumn> cols = affCol == null ? Collections.singletonList(keyCol) : F.asList(keyCol, affCol);
 
-        // Add primary key index.
-        Index pkIdx = idx.createSortedIndex(
-            PK_IDX_NAME,
-            tbl,
-            true,
-            false,
-            unwrappedKeyAndAffinityCols,
-            wrappedKeyCols,
-            tbl.rowDescriptor().type().primaryKeyInlineSize(),
-            null
-        );
+            assert pkHashIdx == null : pkHashIdx;
 
-        idxs.add(pkIdx);
+            pkHashIdx = new H2PkHashIndex(cacheInfo.cacheContext(), tbl, PK_HASH_IDX_NAME, cols,
+                tbl.rowDescriptor().context().config().getQueryParallelism());
+        }
+    }
 
+    /**
+     * Create text (lucene) index if needed.
+     */
+    public void createTextIndex(GridH2Table tbl) {
         if (type().valueClass() == String.class
             && !idx.distributedConfiguration().isDisableCreateLuceneIndexForStringValueType()) {
             try {
@@ -265,202 +227,6 @@ public class H2TableDescriptor {
                 throw new IgniteException(e1);
             }
         }
-
-        // Locate index where affinity column is first (if any).
-        if (affCol != null) {
-            boolean affIdxFound = false;
-
-            for (GridQueryIndexDescriptor idxDesc : type.indexes().values()) {
-                if (idxDesc.type() != QueryIndexType.SORTED)
-                    continue;
-
-                String firstField = idxDesc.fields().iterator().next();
-
-                Column col = tbl.getColumn(firstField);
-
-                IndexColumn idxCol = tbl.indexColumn(col.getColumnId(),
-                    idxDesc.descending(firstField) ? SortOrder.DESCENDING : SortOrder.ASCENDING);
-
-                affIdxFound |= H2Utils.equals(idxCol, affCol);
-            }
-
-            // Add explicit affinity key index if nothing alike was found.
-            if (!affIdxFound) {
-                List<IndexColumn> unwrappedKeyCols = extractKeyColumns(tbl, keyCol, null);
-
-                ArrayList<IndexColumn> colsWithUnwrappedKey = new ArrayList<>(unwrappedKeyCols.size());
-
-                colsWithUnwrappedKey.add(affCol);
-
-                //We need to reorder PK columns to have affinity key as first column, that's why we can't use simple PK columns
-                H2Utils.addUniqueColumns(colsWithUnwrappedKey, unwrappedKeyCols);
-
-                List<IndexColumn> cols = H2Utils.treeIndexColumns(tbl.rowDescriptor(), new ArrayList<>(2), affCol, keyCol);
-
-                idxs.add(idx.createSortedIndex(
-                    AFFINITY_KEY_IDX_NAME,
-                    tbl,
-                    false,
-                    true,
-                    colsWithUnwrappedKey,
-                    cols,
-                    tbl.rowDescriptor().type().affinityFieldInlineSize(),
-                    null)
-                );
-            }
-        }
-
-        return idxs;
-    }
-
-    /**
-     * Create list of affinity and key index columns. Key, if it possible, partitions into simple components.
-     *
-     * @param tbl GridH2Table instance
-     * @param keyCol Key index column.
-     * @param affCol Affinity index column.
-     *
-     * @return List of key and affinity columns. Key's, if it possible, splitted into simple components.
-     */
-    @NotNull private List<IndexColumn> extractKeyColumns(GridH2Table tbl, IndexColumn keyCol, IndexColumn affCol) {
-        ArrayList<IndexColumn> keyCols;
-
-        if (isSql) {
-            keyCols = new ArrayList<>(type.fields().size() + 1);
-
-            // Check if key is simple type.
-            if (QueryUtils.isSqlType(type.keyClass()))
-                keyCols.add(keyCol);
-            else {
-                if (!type.primaryKeyFields().isEmpty()) {
-                    for (String keyName : type.primaryKeyFields()) {
-                        GridQueryProperty prop = type.property(keyName);
-
-                        assert prop.key() : keyName + " is not a key field";
-
-                        Column col = tbl.getColumn(prop.name());
-
-                        keyCols.add(tbl.indexColumn(col.getColumnId(), SortOrder.ASCENDING));
-                    }
-                }
-                else {
-                    for (String propName : type.fields().keySet()) {
-                        GridQueryProperty prop = type.property(propName);
-
-                        if (prop.key()) {
-                            Column col = tbl.getColumn(propName);
-
-                            keyCols.add(tbl.indexColumn(col.getColumnId(), SortOrder.ASCENDING));
-                        }
-                    }
-                }
-
-                // If key is object but the user has not specified any particular columns,
-                // we have to fall back to whole-key index.
-                if (keyCols.isEmpty())
-                    keyCols.add(keyCol);
-            }
-
-        }
-        else {
-            keyCols = new ArrayList<>(2);
-
-            keyCols.add(keyCol);
-        }
-
-        if (affCol != null && !H2Utils.containsColumn(keyCols, affCol))
-            keyCols.add(affCol);
-        else
-            keyCols.trimToSize();
-
-        return Collections.unmodifiableList(keyCols);
-    }
-
-    /**
-     * Get collection of user indexes.
-     *
-     * @return User indexes.
-     */
-    public Collection<GridH2IndexBase> createUserIndexes() {
-        assert tbl != null;
-
-        ArrayList<GridH2IndexBase> res = new ArrayList<>();
-
-        for (GridQueryIndexDescriptor idxDesc : type.indexes().values()) {
-            GridH2IndexBase idx = createUserIndex(idxDesc, null);
-
-            res.add(idx);
-        }
-
-        return res;
-    }
-
-    /**
-     * Create user index.
-     *
-     * @param idxDesc Index descriptor.
-     * @param cacheVisitor Cache visitor.
-     * @return Index.
-     */
-    public GridH2IndexBase createUserIndex(GridQueryIndexDescriptor idxDesc, @Nullable SchemaIndexCacheVisitor cacheVisitor) {
-        IndexColumn keyCol = tbl.indexColumn(QueryUtils.KEY_COL, SortOrder.ASCENDING);
-        IndexColumn affCol = tbl.getAffinityKeyColumn();
-
-        List<IndexColumn> cols = new ArrayList<>(idxDesc.fields().size() + 2);
-
-        for (String field : idxDesc.fields()) {
-            Column col = tbl.getColumn(field);
-
-            cols.add(tbl.indexColumn(col.getColumnId(),
-                idxDesc.descending(field) ? SortOrder.DESCENDING : SortOrder.ASCENDING));
-        }
-
-        GridQueryRowDescriptor desc = tbl.rowDescriptor();
-
-        if (idxDesc.type() == QueryIndexType.SORTED) {
-            List<IndexColumn> unwrappedKeyCols = extractKeyColumns(tbl, keyCol, affCol);
-
-            List<IndexColumn> colsWithUnwrappedKey = new ArrayList<>(cols);
-
-            H2Utils.addUniqueColumns(colsWithUnwrappedKey, unwrappedKeyCols);
-
-            cols = H2Utils.treeIndexColumns(desc, cols, keyCol, affCol);
-
-            return idx.createSortedIndex(
-                idxDesc.name(),
-                tbl,
-                false,
-                false,
-                colsWithUnwrappedKey,
-                cols,
-                idxDesc.inlineSize(),
-                cacheVisitor
-            );
-        }
-        else if (idxDesc.type() == QueryIndexType.GEOSPATIAL)
-            return H2Utils.createSpatialIndex(tbl, idxDesc.name(), cols);
-
-        throw new IllegalStateException("Index type: " + idxDesc.type());
-    }
-
-    /**
-     * Create hash index.
-     *
-     * @param tbl Table.
-     * @param cols Columns.
-     * @return Index.
-     */
-    private Index createHashIndex(GridH2Table tbl, List<IndexColumn> cols) {
-        if (cacheInfo.affinityNode()) {
-            assert pkHashIdx == null : pkHashIdx;
-
-            pkHashIdx = new H2PkHashIndex(cacheInfo.cacheContext(), tbl, PK_HASH_IDX_NAME, cols,
-                tbl.rowDescriptor().context().config().getQueryParallelism());
-
-            return pkHashIdx;
-        }
-
-        return null;
     }
 
     /**
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2TableEngine.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2TableEngine.java
index f5bfbabd43f..070684e67c1 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2TableEngine.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2TableEngine.java
@@ -64,7 +64,6 @@ public class H2TableEngine implements TableEngine {
         throws SQLException {
         rowDesc0 = rowDesc;
         tblDesc0 = tblDesc;
-        idxMgr0 = idxMgr;
 
         try {
             try (Statement s = conn.createStatement()) {
@@ -79,13 +78,12 @@ public class H2TableEngine implements TableEngine {
             resTbl0 = null;
             tblDesc0 = null;
             rowDesc0 = null;
-            idxMgr0 = idxMgr;
         }
     }
 
     /** {@inheritDoc} */
     @Override public TableBase createTable(CreateTableData createTblData) {
-        resTbl0 = new GridH2Table(createTblData, rowDesc0, tblDesc0, tblDesc0.cacheInfo(), idxMgr0);
+        resTbl0 = new GridH2Table(createTblData, rowDesc0, tblDesc0, tblDesc0.cacheInfo());
 
         return resTbl0;
     }
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2Utils.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2Utils.java
index d5dffc92ebf..d5144301e5c 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2Utils.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2Utils.java
@@ -30,7 +30,6 @@ import java.sql.SQLException;
 import java.sql.Time;
 import java.sql.Timestamp;
 import java.sql.Types;
-import java.text.MessageFormat;
 import java.time.LocalDate;
 import java.time.LocalDateTime;
 import java.time.LocalTime;
@@ -40,7 +39,6 @@ import java.util.Collections;
 import java.util.Comparator;
 import java.util.HashMap;
 import java.util.HashSet;
-import java.util.LinkedHashMap;
 import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Map;
@@ -54,12 +52,10 @@ import org.apache.ignite.indexing.IndexingQueryEngineConfiguration;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.binary.BinaryArray;
 import org.apache.ignite.internal.binary.BinaryUtils;
-import org.apache.ignite.internal.cache.query.index.NullsOrder;
-import org.apache.ignite.internal.cache.query.index.Order;
-import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyDefinition;
 import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.CacheObjectValueContext;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.GridCacheContextInfo;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
 import org.apache.ignite.internal.processors.cache.QueryCursorImpl;
 import org.apache.ignite.internal.processors.cache.mvcc.MvccUtils;
@@ -67,8 +63,6 @@ import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
 import org.apache.ignite.internal.processors.odbc.jdbc.JdbcParameterMeta;
 import org.apache.ignite.internal.processors.query.GridQueryFieldMetadata;
 import org.apache.ignite.internal.processors.query.GridQueryProperty;
-import org.apache.ignite.internal.processors.query.GridQueryRowDescriptor;
-import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
 import org.apache.ignite.internal.processors.query.IgniteSQLException;
 import org.apache.ignite.internal.processors.query.QueryUtils;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2IndexBase;
@@ -79,15 +73,17 @@ import org.apache.ignite.internal.processors.query.h2.opt.QueryContext;
 import org.apache.ignite.internal.processors.query.h2.twostep.msg.GridH2RowMessage;
 import org.apache.ignite.internal.processors.query.h2.twostep.msg.GridH2ValueMessage;
 import org.apache.ignite.internal.processors.query.h2.twostep.msg.GridH2ValueMessageFactory;
+import org.apache.ignite.internal.processors.query.schema.management.IndexDescriptor;
+import org.apache.ignite.internal.processors.query.schema.management.TableDescriptor;
 import org.apache.ignite.internal.util.GridStringBuilder;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.SB;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.h2.engine.Session;
+import org.h2.index.Index;
 import org.h2.jdbc.JdbcConnection;
 import org.h2.result.Row;
 import org.h2.result.SortOrder;
-import org.h2.table.Column;
 import org.h2.table.IndexColumn;
 import org.h2.util.LocalDateTimeUtils;
 import org.h2.value.CompareMode;
@@ -111,10 +107,9 @@ import org.h2.value.ValueString;
 import org.h2.value.ValueTime;
 import org.h2.value.ValueTimestamp;
 import org.h2.value.ValueUuid;
-import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
+
 import static java.sql.ResultSetMetaData.columnNullableUnknown;
-import static org.apache.ignite.internal.processors.query.QueryUtils.KEY_COL;
 import static org.apache.ignite.internal.processors.query.QueryUtils.KEY_FIELD_NAME;
 import static org.apache.ignite.internal.processors.query.QueryUtils.VAL_FIELD_NAME;
 
@@ -183,37 +178,6 @@ public class H2Utils {
         return c1.column.getColumnId() == c2.column.getColumnId();
     }
 
-    /**
-     * @param cols Columns list.
-     * @param col Column to find.
-     * @return {@code true} If found.
-     */
-    @SuppressWarnings("BooleanMethodIsAlwaysInverted")
-    public static boolean containsColumn(List<IndexColumn> cols, IndexColumn col) {
-        for (int i = cols.size() - 1; i >= 0; i--) {
-            if (equals(cols.get(i), col))
-                return true;
-        }
-
-        return false;
-    }
-
-    /**
-     * Check whether columns list contains key or key alias column.
-     *
-     * @param desc Row descriptor.
-     * @param cols Columns list.
-     * @return Result.
-     */
-    public static boolean containsKeyColumn(GridQueryRowDescriptor desc, List<IndexColumn> cols) {
-        for (int i = cols.size() - 1; i >= 0; i--) {
-            if (desc.isKeyColumn(cols.get(i).column.getColumnId()))
-                return true;
-        }
-
-        return false;
-    }
-
     /**
      * Prepare SQL statement for CREATE TABLE command.
      *
@@ -266,7 +230,7 @@ public class H2Utils {
      * @param ifNotExists Quietly skip index creation if it exists.
      * @return Statement string.
      */
-    public static String indexCreateSql(String fullTblName, GridH2IndexBase h2Idx, boolean ifNotExists) {
+    public static String indexCreateSql(String fullTblName, Index h2Idx, boolean ifNotExists) {
         boolean spatial = F.eq(SPATIAL_IDX_CLS, h2Idx.getClass().getName());
 
         GridStringBuilder sb = new SB("CREATE ")
@@ -319,47 +283,41 @@ public class H2Utils {
         return "DROP INDEX " + (ifExists ? "IF EXISTS " : "") + withQuotes(schemaName) + '.' + withQuotes(idxName);
     }
 
-    /**
-     * @param desc Row descriptor.
-     * @param cols Columns list.
-     * @param keyCol Primary key column.
-     * @param affCol Affinity key column.
-     * @return The same list back.
-     */
-    public static List<IndexColumn> treeIndexColumns(GridQueryRowDescriptor desc, List<IndexColumn> cols,
-        IndexColumn keyCol, IndexColumn affCol) {
-        assert keyCol != null;
-
-        if (!containsKeyColumn(desc, cols))
-            cols.add(keyCol);
-
-        if (affCol != null && !containsColumn(cols, affCol))
-            cols.add(affCol);
-
-        return cols;
-    }
-
     /**
      * Create spatial index.
      *
      * @param tbl Table.
-     * @param idxName Index name.
+     * @param idxDesc Index descriptor.
      * @param cols Columns.
      */
     @SuppressWarnings("ConstantConditions")
-    public static GridH2IndexBase createSpatialIndex(GridH2Table tbl, String idxName, List<IndexColumn> cols) {
+    public static GridH2IndexBase createSpatialIndex(GridH2Table tbl, IndexDescriptor idxDesc, List<IndexColumn> cols) {
         try {
             Class<?> fctCls = Class.forName(SPATIAL_IDX_FACTORY_CLS);
 
-            Method fctMethod = fctCls.getMethod("createIndex", GridH2Table.class, String.class, List.class);
+            Method fctMethod = fctCls.getMethod("createIndex", GridH2Table.class, IndexDescriptor.class, List.class);
 
-            return (GridH2IndexBase)fctMethod.invoke(null, tbl, idxName, cols);
+            return (GridH2IndexBase)fctMethod.invoke(null, tbl, idxDesc, cols);
         }
         catch (Exception e) {
             throw new IgniteException("Failed to instantiate: " + SPATIAL_IDX_CLS, e);
         }
     }
 
+    /**
... 3884 lines suppressed ...