You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by iv...@apache.org on 2022/08/19 18:54:23 UTC

[ignite] branch ignite-2.14 updated: IGNITE-17190 Move ignite query statistics to core, fix SQL ANALYZE command (#10175)

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

ivandasch pushed a commit to branch ignite-2.14
in repository https://gitbox.apache.org/repos/asf/ignite.git


The following commit(s) were added to refs/heads/ignite-2.14 by this push:
     new 728c2df2d5d IGNITE-17190 Move ignite query statistics to core, fix SQL ANALYZE command (#10175)
728c2df2d5d is described below

commit 728c2df2d5d341f898bea98caaf74e24263f1967
Author: Ivan Daschinskiy <iv...@apache.org>
AuthorDate: Fri Aug 19 21:31:32 2022 +0300

    IGNITE-17190 Move ignite query statistics to core, fix SQL ANALYZE command (#10175)
    
    (cherry picked from commit 5b62a8beeaa1cc1e29efb865c12937c37b31c92b)
---
 .../calcite/metadata/IgniteMdSelectivity.java      |  91 +-----
 .../query/calcite/prepare/QueryPlanCacheImpl.java  |  26 +-
 .../query/calcite/schema/SchemaHolderImpl.java     |  19 +-
 .../calcite/planner/StatisticsPlannerTest.java     |  49 +--
 .../SystemViewRowAttributeWalkerGenerator.java     |   8 +-
 .../index/sorted/inline/types/DateValueUtils.java  |  37 +++
 .../communication/GridIoMessageFactory.java        |  16 +-
 .../StatisticsColumnConfigurationViewWalker.java   |   0
 .../StatisticsColumnGlobalDataViewWalker.java      |   0
 .../StatisticsColumnLocalDataViewWalker.java       |   0
 .../StatisticsColumnPartitionDataViewWalker.java   |   0
 .../processors/query/GridQueryIndexing.java        |   6 +
 .../query/schema/AbstractSchemaChangeListener.java | 116 +++++++
 .../query/schema/SchemaChangeListener.java         |  38 ++-
 .../processors/query/stat/BusyExecutor.java        |   0
 .../processors/query/stat/CancellableTask.java     |   0
 .../processors/query/stat/ColumnStatistics.java    |  19 +-
 .../query/stat/ColumnStatisticsCollector.java      | 345 +++++++++++++++++++++
 .../query/stat/GatherStatisticCancelException.java |   4 +-
 .../internal/processors/query/stat/Hasher.java     |   0
 .../query/stat/IgniteGlobalStatisticsManager.java  |   1 -
 .../stat/IgniteStatisticsConfigurationManager.java | 156 +++++-----
 .../query/stat/IgniteStatisticsDummyStoreImpl.java |   1 -
 .../query/stat/IgniteStatisticsHelper.java         |  83 +++--
 .../stat/IgniteStatisticsInMemoryStoreImpl.java    |   0
 .../query/stat/IgniteStatisticsManagerImpl.java    |  20 +-
 .../stat/IgniteStatisticsPersistenceStoreImpl.java |   3 +-
 .../query/stat/IgniteStatisticsRepository.java     |   1 -
 .../query/stat/IgniteStatisticsStore.java          |   1 -
 .../stat/LocalStatisticsGatheringContext.java      |  23 +-
 .../query/stat/ObjectPartitionStatisticsImpl.java  |   1 -
 .../ObjectPartitionStatisticsObsolescence.java     |   1 -
 .../query/stat/ObjectStatisticsEvent.java          |   0
 .../query/stat/ObjectStatisticsImpl.java           |   1 -
 .../query/stat/StatisticsAddressedRequest.java     |   0
 .../processors/query/stat/StatisticsProcessor.java |   0
 .../processors/query/stat/StatisticsType.java      |   0
 .../processors/query/stat/StatisticsUtils.java     |  76 +++--
 .../stat/config/StatisticsColumnConfiguration.java |   1 -
 .../stat/config/StatisticsColumnOverrides.java     |   1 -
 .../stat/config/StatisticsObjectConfiguration.java |   1 -
 .../internal/processors/query/stat/hll/HLL.java    |   3 +-
 .../processors/query/stat/hll/HLLType.java         |   0
 .../BigEndianAscendingWordDeserializer.java        |   0
 .../BigEndianAscendingWordSerializer.java          |   0
 .../query/stat/hll/serialization/HLLMetadata.java  |   0
 .../query/stat/hll/serialization/IHLLMetadata.java |   0
 .../stat/hll/serialization/ISchemaVersion.java     |   0
 .../stat/hll/serialization/IWordDeserializer.java  |   0
 .../stat/hll/serialization/IWordSerializer.java    |   0
 .../stat/hll/serialization/SchemaVersionOne.java   |   0
 .../stat/hll/serialization/SerializationUtil.java  |   0
 .../processors/query/stat/hll/util/BitUtil.java    |   0
 .../processors/query/stat/hll/util/BitVector.java  |   6 +-
 .../processors/query/stat/hll/util/HLLUtil.java    |   0
 .../query/stat/hll/util/LongIterator.java          |   0
 .../processors/query/stat/hll/util/NumberUtil.java |   0
 .../query/stat/messages/StatisticsColumnData.java  |  17 +-
 .../stat/messages/StatisticsDecimalMessage.java}   |  78 +++--
 .../query/stat/messages/StatisticsKeyMessage.java  |   1 -
 .../query/stat/messages/StatisticsObjectData.java  |   3 +-
 .../query/stat/messages/StatisticsRequest.java     |   2 +-
 .../query/stat/messages/StatisticsResponse.java    |   2 +-
 .../query/stat/task/GatherPartitionStatistics.java |  95 ++++--
 .../stat/view/ColumnConfigurationViewSupplier.java |   0
 .../stat/view/ColumnLocalDataViewSupplier.java     |   0
 .../stat/view/ColumnPartitionDataViewSupplier.java |   0
 .../view/StatisticsColumnConfigurationView.java    |   0
 .../stat/view/StatisticsColumnGlobalDataView.java  |   0
 .../stat/view/StatisticsColumnLocalDataView.java   |   0
 .../view/StatisticsColumnPartitionDataView.java    |   0
 .../ignite/internal/sql/SqlCommandProcessor.java   |  92 +++++-
 .../processors/query/DummyQueryIndexing.java       |   6 +
 .../processors/query/h2/CommandProcessor.java      |  81 -----
 .../processors/query/h2/IgniteH2Indexing.java      |  10 +-
 .../processors/query/h2/SchemaManager.java         | 151 ++-------
 .../processors/query/h2/opt/H2IndexCostedBase.java |  11 +-
 .../h2/twostep/msg/GridH2ValueMessageFactory.java  |  11 -
 .../query/stat/ColumnStatisticsCollector.java      | 274 ----------------
 .../ColumnStatisticsCollectorAggregationTest.java  |  26 +-
 .../query/stat/ColumnStatisticsCollectorTest.java  | 127 +++-----
 .../stat/IgniteStatisticsRepositoryStaticTest.java |   7 +-
 .../query/stat/IgniteStatisticsRepositoryTest.java |   7 +-
 .../query/stat/ManagerStatisticsTypesTest.java     | 114 ++++---
 .../query/stat/SqlStatisticsCommandTests.java      |   3 -
 .../query/stat/StatisticsAbstractTest.java         |   2 +-
 .../query/stat/StatisticsConfigurationTest.java    |   3 +-
 .../query/stat/StatisticsGatheringTest.java        |   5 +-
 .../query/stat/StatisticsObsolescenceTest.java     |   2 +-
 .../processors/query/stat/StatisticsViewsTest.java |  52 +---
 90 files changed, 1261 insertions(+), 1075 deletions(-)

diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/IgniteMdSelectivity.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/IgniteMdSelectivity.java
index 7abc2497067..85ceba2c16a 100644
--- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/IgniteMdSelectivity.java
+++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/metadata/IgniteMdSelectivity.java
@@ -18,12 +18,10 @@
 package org.apache.ignite.internal.processors.query.calcite.metadata;
 
 import java.math.BigDecimal;
-import java.math.BigInteger;
 import java.math.MathContext;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-
 import org.apache.calcite.plan.RelOptUtil;
 import org.apache.calcite.plan.volcano.RelSubset;
 import org.apache.calcite.rel.RelNode;
@@ -60,7 +58,6 @@ import org.apache.ignite.internal.processors.query.calcite.schema.IgniteStatisti
 import org.apache.ignite.internal.processors.query.calcite.schema.IgniteTable;
 import org.apache.ignite.internal.processors.query.calcite.util.RexUtils;
 import org.apache.ignite.internal.processors.query.stat.ColumnStatistics;
-import org.h2.value.Value;
 import org.jetbrains.annotations.Nullable;
 
 /** */
@@ -167,74 +164,6 @@ public class IgniteMdSelectivity extends RelMdSelectivity {
         return null;
     }
 
-    /**
-     * Convert specified value into comparable type: BigDecimal,
-     *
-     * @param val Value to convert to comparable form.
-     * @return Comparable form of value.
-     */
-    private BigDecimal toComparableValue(Value val) {
-        if (val == null)
-            return null;
-
-        switch (val.getType()) {
-            case Value.NULL:
-                throw new IllegalArgumentException("Can't compare null values");
-
-            case Value.BOOLEAN:
-                return (val.getBoolean()) ? BigDecimal.ONE : BigDecimal.ZERO;
-
-            case Value.BYTE:
-                return new BigDecimal(val.getByte());
-
-            case Value.SHORT:
-                return new BigDecimal(val.getShort());
-
-            case Value.INT:
-                return new BigDecimal(val.getInt());
-
-            case Value.LONG:
-                return new BigDecimal(val.getLong());
-
-            case Value.DECIMAL:
-                return val.getBigDecimal();
-
-            case Value.DOUBLE:
-                return BigDecimal.valueOf(val.getDouble());
-
-            case Value.FLOAT:
-                return BigDecimal.valueOf(val.getFloat());
-
-            case Value.DATE:
-                return BigDecimal.valueOf(val.getDate().getTime());
-
-            case Value.TIME:
-                return BigDecimal.valueOf(val.getTime().getTime());
-
-            case Value.TIMESTAMP:
-                return BigDecimal.valueOf(val.getTimestamp().getTime());
-
-            case Value.BYTES:
-                BigInteger bigInteger = new BigInteger(1, val.getBytes());
-                return new BigDecimal(bigInteger);
-
-            case Value.STRING:
-            case Value.STRING_FIXED:
-            case Value.STRING_IGNORECASE:
-            case Value.ARRAY:
-            case Value.JAVA_OBJECT:
-            case Value.GEOMETRY:
-                return null;
-
-            case Value.UUID:
-                BigInteger bigInt = new BigInteger(1, val.getBytes());
-                return new BigDecimal(bigInt);
-
-            default:
-                throw new IllegalStateException("Unsupported H2 type: " + val.getType());
-        }
-    }
-
     /**
      * Predicate based selectivity for table. Estimate condition on each column taking in comparison it's statistics.
      *
@@ -395,16 +324,20 @@ public class IgniteMdSelectivity extends RelMdSelectivity {
         ColumnStatistics colStat = getColumnStatistics(mq, rel, ref);
         double res = 0.33;
 
-        if (colStat == null) {
+        if (colStat == null || colStat.max() == null || colStat.min() == null) {
             // true, false and null with equivalent probability
             return res;
         }
 
-        if (colStat.max() == null || colStat.max().getType() != Value.BOOLEAN)
+        // Check whether it can be considered as BOOL.
+        boolean isBool = (colStat.max().compareTo(BigDecimal.ONE) == 0 || colStat.max().compareTo(BigDecimal.ZERO) == 0)
+            && (colStat.min().compareTo(BigDecimal.ONE) == 0 || colStat.min().compareTo(BigDecimal.ZERO) == 0);
+
+        if (!isBool)
             return res;
 
-        Boolean min = colStat.min().getBoolean();
-        Boolean max = colStat.max().getBoolean();
+        Boolean min = colStat.min().compareTo(BigDecimal.ONE) == 0;
+        Boolean max = colStat.max().compareTo(BigDecimal.ONE) == 0;
 
         if (!max)
             return 0;
@@ -450,8 +383,8 @@ public class IgniteMdSelectivity extends RelMdSelectivity {
 
         SqlOperator op = ((RexCall)pred).op;
 
-        BigDecimal min = toComparableValue(colStat.min());
-        BigDecimal max = toComparableValue(colStat.max());
+        BigDecimal min = colStat.min();
+        BigDecimal max = colStat.max();
         BigDecimal total = (min == null || max == null) ? null : max.subtract(min).abs();
 
         if (total == null)
@@ -537,13 +470,13 @@ public class IgniteMdSelectivity extends RelMdSelectivity {
             return guessSelectivity(pred);
 
         if (colStat.min() != null) {
-            BigDecimal minComparable = toComparableValue(colStat.min());
+            BigDecimal minComparable = colStat.min();
             if (minComparable != null && minComparable.compareTo(comparableVal) > 0)
                 return 0.;
         }
 
         if (colStat.max() != null) {
-            BigDecimal maxComparable = toComparableValue(colStat.max());
+            BigDecimal maxComparable = colStat.max();
             if (maxComparable != null && maxComparable.compareTo(comparableVal) < 0)
                 return 0.;
         }
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 37d82962b06..74b992cec39 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
@@ -18,6 +18,7 @@
 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;
@@ -25,6 +26,7 @@ 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.subscription.GridInternalSubscriptionProcessor;
@@ -100,7 +102,11 @@ public class QueryPlanCacheImpl extends AbstractService implements QueryPlanCach
     }
 
     /** {@inheritDoc} */
-    @Override public void onSqlTypeDropped(String schemaName, GridQueryTypeDescriptor typeDescriptor) {
+    @Override public void onSqlTypeDropped(
+        String schemaName,
+        GridQueryTypeDescriptor typeDescriptor,
+        boolean destroy
+    ) {
         clear();
     }
 
@@ -140,8 +146,22 @@ public class QueryPlanCacheImpl extends AbstractService implements QueryPlanCach
     }
 
     /** {@inheritDoc} */
-    @Override public void onSqlTypeUpdated(String schemaName, GridQueryTypeDescriptor typeDesc,
-        GridCacheContextInfo<?, ?> cacheInfo) {
+    @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/SchemaHolderImpl.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/SchemaHolderImpl.java
index 2e3486e7dad..9ea703a577b 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
@@ -37,6 +37,7 @@ 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.exec.exp.IgniteScalarFunction;
 import org.apache.ignite.internal.processors.query.calcite.trait.TraitUtils;
 import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactory;
@@ -188,10 +189,21 @@ public class SchemaHolderImpl extends AbstractService implements SchemaHolder, S
     }
 
     /** {@inheritDoc} */
-    @Override public void onSqlTypeUpdated(
+    @Override public void onColumnsAdded(
         String schemaName,
         GridQueryTypeDescriptor typeDesc,
-        GridCacheContextInfo<?, ?> cacheInfo
+        GridCacheContextInfo<?, ?> cacheInfo,
+        List<QueryField> cols
+    ) {
+        onSqlTypeCreated(schemaName, typeDesc, cacheInfo);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onColumnsDropped(
+        String schemaName,
+        GridQueryTypeDescriptor typeDesc,
+        GridCacheContextInfo<?, ?> cacheInfo,
+        List<String> cols
     ) {
         onSqlTypeCreated(schemaName, typeDesc, cacheInfo);
     }
@@ -206,7 +218,8 @@ public class SchemaHolderImpl extends AbstractService implements SchemaHolder, S
     /** {@inheritDoc} */
     @Override public synchronized void onSqlTypeDropped(
         String schemaName,
-        GridQueryTypeDescriptor typeDesc
+        GridQueryTypeDescriptor typeDesc,
+        boolean destroy
     ) {
         IgniteSchema schema = igniteSchemas.computeIfAbsent(schemaName, IgniteSchema::new);
 
diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/StatisticsPlannerTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/StatisticsPlannerTest.java
index 15e048e01ab..5c5de98f8bf 100644
--- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/StatisticsPlannerTest.java
+++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/StatisticsPlannerTest.java
@@ -37,20 +37,11 @@ import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeFactor
 import org.apache.ignite.internal.processors.query.calcite.type.IgniteTypeSystem;
 import org.apache.ignite.internal.processors.query.stat.ColumnStatistics;
 import org.apache.ignite.internal.processors.query.stat.ObjectStatisticsImpl;
-import org.h2.value.ValueBoolean;
-import org.h2.value.ValueByte;
-import org.h2.value.ValueDate;
-import org.h2.value.ValueDouble;
-import org.h2.value.ValueFloat;
-import org.h2.value.ValueInt;
-import org.h2.value.ValueLong;
-import org.h2.value.ValueShort;
-import org.h2.value.ValueString;
-import org.h2.value.ValueTime;
-import org.h2.value.ValueTimestamp;
 import org.junit.Before;
 import org.junit.Test;
 
+import static org.apache.ignite.internal.processors.query.stat.StatisticsUtils.toDecimal;
+
 /**
  * Statistic related simple tests.
  */
@@ -150,40 +141,35 @@ public class StatisticsPlannerTest extends AbstractPlannerTest {
         tbl4.addIndex("TBL4_SHORT_LONG", 6, 7);
 
         HashMap<String, ColumnStatistics> colStat1 = new HashMap<>();
-        colStat1.put("T1C1INT", new ColumnStatistics(ValueInt.get(1), ValueInt.get(1000),
-            0, 1000, t1rc, 4, null, 1, 0));
+        colStat1.put("T1C1INT", new ColumnStatistics(toDecimal(1), toDecimal(1000), 0, 1000, t1rc, 4, null, 1, 0));
 
-        colStat1.put("T1C2STR", new ColumnStatistics(ValueString.get("A1"), ValueString.get("Z9"),
-            100, 20, t1rc, 2, null, 1, 0));
+        colStat1.put("T1C2STR", new ColumnStatistics(null, null, 100, 20, t1rc, 2, null, 1, 0));
 
-        colStat1.put("T1C3DBL", new ColumnStatistics(ValueDouble.get(0.01), ValueDouble.get(0.99),
-            10, 1000, t1rc, 8, null, 1, 0));
+        colStat1.put("T1C3DBL", new ColumnStatistics(toDecimal(0.01), toDecimal(0.99), 10, 1000, t1rc, 8, null, 1, 0));
 
-        colStat1.put("T1C4BYTE", new ColumnStatistics(ValueByte.get((byte)0), ValueByte.get((byte)255),
-            10, 1000, t1rc, 8, null, 1, 0));
+        colStat1.put("T1C4BYTE", new ColumnStatistics(toDecimal((byte)0), toDecimal((byte)255), 10, 1000, t1rc, 8, null,
+            1, 0));
 
-        colStat1.put("T1C5BOOLEAN", new ColumnStatistics(ValueBoolean.get(false), ValueBoolean.get(true),
-            0, 2, t1rc, 1, null, 1, 0));
+        colStat1.put("T1C5BOOLEAN", new ColumnStatistics(toDecimal(false), toDecimal(true), 0, 2, t1rc, 1, null, 1, 0));
 
-        colStat1.put("T1C6CHARACTER", new ColumnStatistics(ValueString.get("A"), ValueString.get("Z"),
-            10, 10, t1rc, 1, null, 1, 0));
+        colStat1.put("T1C6CHARACTER", new ColumnStatistics(null, null, 10, 10, t1rc, 1, null, 1, 0));
 
-        colStat1.put("T1C7SHORT", new ColumnStatistics(ValueShort.get((short)1), ValueShort.get((short)5000),
+        colStat1.put("T1C7SHORT", new ColumnStatistics(toDecimal((short)1), toDecimal((short)5000),
             110, 500, t1rc, 2, null, 1, 0));
 
-        colStat1.put("T1C8LONG", new ColumnStatistics(ValueLong.get(1L), ValueLong.get(100000L),
+        colStat1.put("T1C8LONG", new ColumnStatistics(toDecimal(1L), toDecimal(100000L),
             10, 100000, t1rc, 8, null, 1, 0));
 
-        colStat1.put("T1C9FLOAT", new ColumnStatistics(ValueFloat.get((float)0.1), ValueFloat.get((float)0.9),
+        colStat1.put("T1C9FLOAT", new ColumnStatistics(toDecimal((float)0.1), toDecimal((float)0.9),
             10, 1000, t1rc, 8, null, 1, 0));
 
-        colStat1.put("T1C10DATE", new ColumnStatistics(ValueDate.get(MIN_DATE), ValueDate.get(MAX_DATE),
+        colStat1.put("T1C10DATE", new ColumnStatistics(toDecimal(MIN_DATE), toDecimal(MAX_DATE),
             20, 1000, t1rc, 8, null, 1, 0));
 
-        colStat1.put("T1C11TIME", new ColumnStatistics(ValueTime.get(MIN_TIME), ValueTime.get(MAX_TIME),
+        colStat1.put("T1C11TIME", new ColumnStatistics(toDecimal(MIN_TIME), toDecimal(MAX_TIME),
             10, 1000, t1rc, 8, null, 1, 0));
 
-        colStat1.put("T1C12TIMESTAMP", new ColumnStatistics(ValueTimestamp.get(MIN_TIMESTAMP), ValueTimestamp.get(MAX_TIMESTAMP),
+        colStat1.put("T1C12TIMESTAMP", new ColumnStatistics(toDecimal(MIN_TIMESTAMP), toDecimal(MAX_TIMESTAMP),
             20, 1000, t1rc, 8, null, 1, 0));
 
         tbl1stat = new IgniteStatisticsImpl(new ObjectStatisticsImpl(1000, colStat1));
@@ -437,10 +423,9 @@ public class StatisticsPlannerTest extends AbstractPlannerTest {
         int rowCnt = 10_000;
 
         HashMap<String, ColumnStatistics> colStat1 = new HashMap<>();
-        colStat1.put("T1C2STR", new ColumnStatistics(ValueString.get("A1"), ValueString.get("Z9"),
-            0, 1, rowCnt, 2, null, 1, 0));
+        colStat1.put("T1C2STR", new ColumnStatistics(null, null, 0, 1, rowCnt, 2, null, 1, 0));
 
-        colStat1.put("T1C7SHORT", new ColumnStatistics(ValueShort.get((short)1), ValueShort.get((short)5000),
+        colStat1.put("T1C7SHORT", new ColumnStatistics(toDecimal((short)1), toDecimal((short)5000),
             0, rowCnt, rowCnt, 2, null, 1, 0));
 
         IgniteStatisticsImpl stat = new IgniteStatisticsImpl(new ObjectStatisticsImpl(1000, colStat1));
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 98fb664843c..be028aaccfb 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
@@ -153,10 +153,10 @@ public class SystemViewRowAttributeWalkerGenerator {
         gen.generateAndWrite(SqlTableColumnView.class, INDEXING_SRC_DIR);
         gen.generateAndWrite(SqlViewColumnView.class, INDEXING_SRC_DIR);
 
-        gen.generateAndWrite(StatisticsColumnConfigurationView.class, INDEXING_SRC_DIR);
-        gen.generateAndWrite(StatisticsColumnLocalDataView.class, INDEXING_SRC_DIR);
-        gen.generateAndWrite(StatisticsColumnGlobalDataView.class, INDEXING_SRC_DIR);
-        gen.generateAndWrite(StatisticsColumnPartitionDataView.class, INDEXING_SRC_DIR);
+        gen.generateAndWrite(StatisticsColumnConfigurationView.class, DFLT_SRC_DIR);
+        gen.generateAndWrite(StatisticsColumnLocalDataView.class, DFLT_SRC_DIR);
+        gen.generateAndWrite(StatisticsColumnGlobalDataView.class, DFLT_SRC_DIR);
+        gen.generateAndWrite(StatisticsColumnPartitionDataView.class, DFLT_SRC_DIR);
     }
 
     /**
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/types/DateValueUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/types/DateValueUtils.java
index 66a52cc644f..49d5a87a81d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/types/DateValueUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/types/DateValueUtils.java
@@ -17,9 +17,16 @@
 
 package org.apache.ignite.internal.cache.query.index.sorted.inline.types;
 
+import java.sql.Date;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
 import java.util.Calendar;
 import java.util.GregorianCalendar;
 import java.util.TimeZone;
+import java.util.concurrent.TimeUnit;
 
 /**
  * DateValue is a representation of a date in bit form:
@@ -132,4 +139,34 @@ public class DateValueUtils {
         return utcMillis - DEFAULT_TZ.getOffset(utcMillis - DEFAULT_TZ.getOffset(utcMillis));
 
     }
+
+    /** */
+    public static Timestamp convertToTimestamp(LocalDateTime locDateTime) {
+        LocalDate locDate = locDateTime.toLocalDate();
+        LocalTime locTime = locDateTime.toLocalTime();
+
+        long dateVal = dateValue(locDate.getYear(), locDate.getMonthValue(), locDate.getDayOfMonth());
+        long millis = millisFromDateValue(dateVal) + TimeUnit.NANOSECONDS.toMillis(locTime.toNanoOfDay());
+        long nanos = locTime.toNanoOfDay() % 1_000_000_000L;
+
+        Timestamp res = new Timestamp(defaultTzMillisFromUtc(millis));
+        res.setNanos((int)nanos);
+
+        return res;
+    }
+
+    /** */
+    public static Time convertToSqlTime(LocalTime locTime) {
+        long millis = TimeUnit.NANOSECONDS.toMillis(locTime.toNanoOfDay());
+
+        return new Time(defaultTzMillisFromUtc(millis));
+    }
+
+    /** */
+    public static Date convertToSqlDate(LocalDate locDate) {
+        long dateVal = dateValue(locDate.getYear(), locDate.getMonthValue(), locDate.getDayOfMonth());
+        long millis = millisFromDateValue(dateVal);
+
+        return new Date(defaultTzMillisFromUtc(millis));
+    }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
index 567cf7fe81e..51aade1e347 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
@@ -168,6 +168,12 @@ import org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQuery
 import org.apache.ignite.internal.processors.query.messages.GridQueryKillRequest;
 import org.apache.ignite.internal.processors.query.messages.GridQueryKillResponse;
 import org.apache.ignite.internal.processors.query.schema.message.SchemaOperationStatusMessage;
+import org.apache.ignite.internal.processors.query.stat.messages.StatisticsColumnData;
+import org.apache.ignite.internal.processors.query.stat.messages.StatisticsDecimalMessage;
+import org.apache.ignite.internal.processors.query.stat.messages.StatisticsKeyMessage;
+import org.apache.ignite.internal.processors.query.stat.messages.StatisticsObjectData;
+import org.apache.ignite.internal.processors.query.stat.messages.StatisticsRequest;
+import org.apache.ignite.internal.processors.query.stat.messages.StatisticsResponse;
 import org.apache.ignite.internal.processors.rest.handlers.task.GridTaskResultRequest;
 import org.apache.ignite.internal.processors.rest.handlers.task.GridTaskResultResponse;
 import org.apache.ignite.internal.processors.service.ServiceDeploymentProcessId;
@@ -376,9 +382,17 @@ public class GridIoMessageFactory implements MessageFactoryProvider {
         factory.register(SnapshotFilesRequestMessage.TYPE_CODE, SnapshotFilesRequestMessage::new);
         factory.register(SnapshotFilesFailureMessage.TYPE_CODE, SnapshotFilesFailureMessage::new);
 
+        // Index statistics.
+        factory.register(StatisticsKeyMessage.TYPE_CODE, StatisticsKeyMessage::new);
+        factory.register(StatisticsDecimalMessage.TYPE_CODE, StatisticsDecimalMessage::new);
+        factory.register(StatisticsObjectData.TYPE_CODE, StatisticsObjectData::new);
+        factory.register(StatisticsColumnData.TYPE_CODE, StatisticsColumnData::new);
+        factory.register(StatisticsRequest.TYPE_CODE, StatisticsRequest::new);
+        factory.register(StatisticsResponse.TYPE_CODE, StatisticsResponse::new);
+
         // [-3..119] [124..129] [-23..-28] [-36..-55] - this
         // [120..123] - DR
-        // [-4..-22, -30..-35] - SQL
+        // [-4..-22, -30..-35, -54..-57] - SQL
         // [2048..2053] - Snapshots
         // [-42..-37] - former hadoop.
         // [64..71] - former IGFS.
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/managers/systemview/walker/StatisticsColumnConfigurationViewWalker.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/systemview/walker/StatisticsColumnConfigurationViewWalker.java
similarity index 100%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/managers/systemview/walker/StatisticsColumnConfigurationViewWalker.java
rename to modules/core/src/main/java/org/apache/ignite/internal/managers/systemview/walker/StatisticsColumnConfigurationViewWalker.java
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/managers/systemview/walker/StatisticsColumnGlobalDataViewWalker.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/systemview/walker/StatisticsColumnGlobalDataViewWalker.java
similarity index 100%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/managers/systemview/walker/StatisticsColumnGlobalDataViewWalker.java
rename to modules/core/src/main/java/org/apache/ignite/internal/managers/systemview/walker/StatisticsColumnGlobalDataViewWalker.java
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/managers/systemview/walker/StatisticsColumnLocalDataViewWalker.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/systemview/walker/StatisticsColumnLocalDataViewWalker.java
similarity index 100%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/managers/systemview/walker/StatisticsColumnLocalDataViewWalker.java
rename to modules/core/src/main/java/org/apache/ignite/internal/managers/systemview/walker/StatisticsColumnLocalDataViewWalker.java
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/managers/systemview/walker/StatisticsColumnPartitionDataViewWalker.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/systemview/walker/StatisticsColumnPartitionDataViewWalker.java
similarity index 100%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/managers/systemview/walker/StatisticsColumnPartitionDataViewWalker.java
rename to modules/core/src/main/java/org/apache/ignite/internal/managers/systemview/walker/StatisticsColumnPartitionDataViewWalker.java
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 a554802e075..c1114691f3f 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
@@ -37,6 +37,7 @@ 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;
@@ -451,4 +452,9 @@ public interface GridQueryIndexing {
      * @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();
 }
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
new file mode 100644
index 00000000000..c3d52136a7a
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/AbstractSchemaChangeListener.java
@@ -0,0 +1,116 @@
+/*
+ * 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;
+
+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.spi.systemview.view.SystemView;
+
+/**
+ * Abstract schema change listener with no-op implementation for all calbacks.
+ */
+public abstract class AbstractSchemaChangeListener implements SchemaChangeListener {
+    /** {@inheritDoc} */
+    @Override public void onSchemaCreated(String schemaName) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onSchemaDropped(String schemaName) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onIndexCreated(
+        String schemaName,
+        String tblName,
+        String idxName,
+        GridQueryIndexDescriptor idxDesc,
+        Index idx
+    ) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onIndexDropped(String schemaName, String tblName, String idxName) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onIndexRebuildStarted(String schemaName, String tblName) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onIndexRebuildFinished(String schemaName, String tblName) {
+        // 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
+    ) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onColumnsDropped(
+        String schemaName,
+        GridQueryTypeDescriptor typeDesc,
+        GridCacheContextInfo<?, ?> cacheInfo,
+        List<String> cols
+    ){
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onSqlTypeDropped(
+        String schemaName,
+        GridQueryTypeDescriptor typeDescriptor,
+        boolean destroy
+    ) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onFunctionCreated(String schemaName, String name, Method method) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onSystemViewCreated(String schemaName, SystemView<?> sysView) {
+        // 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 278309418dd..47ef2c575d2 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
@@ -18,10 +18,12 @@
 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.spi.systemview.view.SystemView;
 import org.jetbrains.annotations.Nullable;
 
@@ -50,26 +52,50 @@ public interface SchemaChangeListener {
      * @param typeDesc Type descriptor.
      * @param cacheInfo Cache info.
      */
-    public void onSqlTypeCreated(String schemaName, GridQueryTypeDescriptor typeDesc,
-        GridCacheContextInfo<?, ?> cacheInfo);
+    public void onSqlTypeCreated(
+        String schemaName,
+        GridQueryTypeDescriptor typeDesc,
+        GridCacheContextInfo<?, ?> cacheInfo
+    );
 
     /**
-     * Callback method.
+     * Callback on columns added.
+     *
+     * @param schemaName Schema name.
+     * @param typeDesc Type descriptor.
+     * @param cacheInfo Cache info.
+     * @param cols Added columns' names.
+     */
+    public void onColumnsAdded(
+        String schemaName,
+        GridQueryTypeDescriptor typeDesc,
+        GridCacheContextInfo<?, ?> cacheInfo,
+        List<QueryField> cols
+    );
+
+    /**
+     * Callback on columns dropped.
      *
      * @param schemaName Schema name.
      * @param typeDesc Type descriptor.
      * @param cacheInfo Cache info.
+     * @param cols Dropped columns' names.
      */
-    public void onSqlTypeUpdated(String schemaName, GridQueryTypeDescriptor typeDesc,
-        GridCacheContextInfo<?, ?> cacheInfo);
+    public void onColumnsDropped(
+        String schemaName,
+        GridQueryTypeDescriptor typeDesc,
+        GridCacheContextInfo<?, ?> cacheInfo,
+        List<String> cols
+    );
 
     /**
      * Callback method.
      *
      * @param schemaName Schema name.
      * @param typeDesc Type descriptor.
+     * @param destroy Cache destroy flag.
      */
-    public void onSqlTypeDropped(String schemaName, GridQueryTypeDescriptor typeDesc);
+    public void onSqlTypeDropped(String schemaName, GridQueryTypeDescriptor typeDesc, boolean destroy);
 
     /**
      * Callback on index creation.
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/BusyExecutor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/BusyExecutor.java
similarity index 100%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/BusyExecutor.java
rename to modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/BusyExecutor.java
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/CancellableTask.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/CancellableTask.java
similarity index 100%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/CancellableTask.java
rename to modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/CancellableTask.java
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/ColumnStatistics.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/ColumnStatistics.java
similarity index 92%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/ColumnStatistics.java
rename to modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/ColumnStatistics.java
index 1ec1a40f090..8c81b28bbcc 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/ColumnStatistics.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/ColumnStatistics.java
@@ -17,21 +17,20 @@
 
 package org.apache.ignite.internal.processors.query.stat;
 
+import java.math.BigDecimal;
 import java.util.Arrays;
 import java.util.Objects;
-
 import org.apache.ignite.internal.util.typedef.internal.S;
-import org.h2.value.Value;
 
 /**
  * Values statistic in particular column.
  */
 public class ColumnStatistics {
     /** Minimum value in column or {@code null} if there are no non null values in the column. */
-    private final Value min;
+    private final BigDecimal min;
 
     /** Maximum value in column or {@code null} if there are no non null values in the column. */
-    private final Value max;
+    private final BigDecimal max;
 
     /** Number of null values in column. */
     private final long nulls;
@@ -68,8 +67,8 @@ public class ColumnStatistics {
      * @param createdAt Created at time, milliseconds.
      */
     public ColumnStatistics(
-        Value min,
-        Value max,
+        BigDecimal min,
+        BigDecimal max,
         long nulls,
         long distinct,
         long total,
@@ -92,14 +91,14 @@ public class ColumnStatistics {
     /**
      * @return Min value in column.
      */
-    public Value min() {
+    public BigDecimal min() {
         return min;
     }
 
     /**
      * @return Max value in column.
      */
-    public Value max() {
+    public BigDecimal max() {
         return max;
     }
 
@@ -163,8 +162,8 @@ public class ColumnStatistics {
             size == that.size &&
             ver == that.ver &&
             createdAt == that.createdAt &&
-            Objects.equals(min, that.min) &&
-            Objects.equals(max, that.max) &&
+            (min == null ? that.min == null : min.compareTo(that.min) == 0) &&
+            (max == null ? that.max == null : max.compareTo(that.max) == 0) &&
             Arrays.equals(raw, that.raw);
     }
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/ColumnStatisticsCollector.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/ColumnStatisticsCollector.java
new file mode 100644
index 00000000000..4967f882e56
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/ColumnStatisticsCollector.java
@@ -0,0 +1,345 @@
+/*
+ * 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.stat;
+
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.charset.StandardCharsets;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.Arrays;
+import java.util.Date;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.binary.BinaryObjectImpl;
+import org.apache.ignite.internal.cache.query.index.IndexProcessor;
+import org.apache.ignite.internal.processors.query.stat.config.StatisticsColumnOverrides;
+import org.apache.ignite.internal.processors.query.stat.hll.HLL;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+import static org.apache.ignite.internal.cache.query.index.sorted.inline.types.DateValueUtils.convertToSqlDate;
+import static org.apache.ignite.internal.cache.query.index.sorted.inline.types.DateValueUtils.convertToSqlTime;
+import static org.apache.ignite.internal.cache.query.index.sorted.inline.types.DateValueUtils.convertToTimestamp;
+import static org.apache.ignite.internal.processors.query.stat.StatisticsUtils.toDecimal;
+
+/**
+ * Collector to compute statistic by single column.
+ */
+public class ColumnStatisticsCollector {
+    /** */
+    private static final Set<Class<?>> comparableCls = new HashSet<>(Arrays.<Class<?>>asList(
+        Boolean.class,
+        Byte.class,
+        Short.class,
+        Integer.class,
+        Long.class,
+        BigDecimal.class,
+        Double.class,
+        Float.class,
+        Time.class,
+        Timestamp.class,
+        Date.class,
+        java.sql.Date.class,
+        LocalTime.class,
+        LocalDate.class,
+        LocalDateTime.class,
+        UUID.class
+    ));
+
+    /** Column name. */
+    private final String colName;
+
+    /** Column id. */
+    private final int colId;
+
+    /** Hyper Log Log structure */
+    private final HLL hll = buildHll();
+
+    /** Minimum value. */
+    private BigDecimal min;
+
+    /** Maximum value. */
+    private BigDecimal max;
+
+    /** Total values in column. */
+    private long total;
+
+    /** Total size of all non nulls values (in bytes).*/
+    private long size;
+
+    /** Null values counter. */
+    private long nullsCnt;
+
+    /** Is column has complex type. */
+    private final boolean isComparable;
+
+    /** Hasher. */
+    private final Hasher hash = new Hasher();
+
+    /** Version. */
+    private final long ver;
+
+    /** Column type. */
+    private final Class<?> colType;
+
+    /**
+     * Constructor.
+     */
+    public ColumnStatisticsCollector(int colId, String colName, Class<?> colType) {
+        this(colId, colName, colType, 0);
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param ver Target statistic version.
+     */
+    public ColumnStatisticsCollector(int colId, String colName, Class<?> colType, long ver) {
+        this.colId = colId;
+        this.colName = colName;
+        this.ver = ver;
+        this.colType = colType;
+
+        isComparable = colType != null && comparableCls.contains(colType);
+    }
+
+    /**
+     * Add value to statistics.
+     *
+     * @param val Value to add to statistics.
+     */
+    public void add(Object val) throws IgniteCheckedException {
+        total++;
+
+        if (val == null) {
+            nullsCnt++;
+
+            return;
+        }
+
+        addToHll(val);
+
+        if (isComparable) {
+            BigDecimal decVal = toDecimal(val);
+
+            if (null == min || min.compareTo(decVal) > 0)
+                min = decVal;
+
+            if (null == max || max.compareTo(decVal) < 0)
+                max = decVal;
+        }
+    }
+
+    /**
+     * Get total column statistics.
+     *
+     * @return Aggregated column statistics.
+     */
+    public ColumnStatistics finish() {
+        int averageSize = averageSize(size, total, nullsCnt);
+
+        return new ColumnStatistics(toDecimal(min), toDecimal(max), nullsCnt, hll.cardinality(), total, averageSize,
+            hll.toBytes(), ver, U.currentTimeMillis());
+    }
+
+    /**
+     * Calculate average record size in bytes.
+     *
+     * @param size Total size of all records.
+     * @param total Total number of all records.
+     * @param nullsCnt Number of nulls record.
+     * @return Average size of not null record in byte.
+     */
+    private static int averageSize(long size, long total, long nullsCnt) {
+        long averageSizeLong = (total - nullsCnt > 0) ? (size / (total - nullsCnt)) : 0;
+        return (averageSizeLong > Integer.MAX_VALUE) ? Integer.MAX_VALUE : (int)averageSizeLong;
+    }
+
+    /**
+     * @return Column id.
+     */
+    public int columnId() {
+        return colId;
+    }
+
+    /**
+     * @return Column name.
+     */
+    public String columnName() {
+        return colName;
+    }
+
+    /**
+     * @return Column type.
+     */
+    public Class<?> columnType() {
+        return colType;
+    }
+
+    /**
+     * Aggregate specified (partition or local) column statistics into (local or global) single one.
+     *
+     * @param partStats Column statistics by partitions.
+     * @param overrides Overrides or {@code null} to keep calculated values.
+     * @return Column statistics for all partitions.
+     */
+    public static ColumnStatistics aggregate(
+        List<ColumnStatistics> partStats,
+        StatisticsColumnOverrides overrides
+    ) {
+        assert !F.isEmpty(partStats);
+
+        Long overrideDistinct = (overrides == null) ? null : overrides.distinct();
+        HLL hll = buildHll();
+
+        BigDecimal min = null;
+        BigDecimal max = null;
+
+        // Total number of nulls
+        long nullsCnt = 0;
+
+        // Total values (null and not null) counter)
+        long total = 0;
+
+        // Total size in bytes
+        long totalSize = 0;
+
+        ColumnStatistics firstStat = F.first(partStats);
+        long ver = firstStat.version();
+        long createdAt = firstStat.createdAt();
+
+        for (ColumnStatistics partStat : partStats) {
+            assert ver == partStat.version() : "Aggregate statistics with different version [stats=" + partStats + ']';
+
+            if (overrideDistinct == null) {
+                HLL partHll = HLL.fromBytes(partStat.raw());
+                hll.union(partHll);
+            }
+
+            total += partStat.total();
+            nullsCnt += partStat.nulls();
+            totalSize += (long)partStat.size() * (partStat.total() - partStat.nulls());
+
+            if (min == null || (partStat.min() != null && partStat.min().compareTo(min) < 0))
+                min = partStat.min();
+
+            if (max == null || (partStat.max() != null && partStat.max().compareTo(max) > 0))
+                max = partStat.max();
+
+            if (createdAt < partStat.createdAt())
+                createdAt = partStat.createdAt();
+        }
+
+        Integer overrideSize = (overrides == null) ? null : overrides.size();
+        int averageSize = (overrideSize == null) ? averageSize(totalSize, total, nullsCnt) : overrideSize;
+
+        long distinct = (overrideDistinct == null) ? hll.cardinality() : overrideDistinct;
+
+        Long overrideNulls = (overrides == null) ? null : overrides.nulls();
+        long nulls = (overrideNulls == null) ? nullsCnt : overrideNulls;
+
+        Long overrideTotal = (overrides == null) ? null : overrides.total();
+        total = (overrideTotal == null) ? total : overrideTotal;
+
+        return new ColumnStatistics(min, max, nulls, distinct, total, averageSize, hll.toBytes(), ver, createdAt);
+    }
+
+    /**
+     * Get HLL with default params.
+     *
+     * @return Empty hll structure.
+     */
+    private static HLL buildHll() {
+        return new HLL(13, 5);
+    }
+
+    /** */
+    private void addToHll(Object obj) {
+        assert obj != null;
+
+        Class<?> cls = U.box(obj.getClass());
+
+        byte[] buf;
+        if (Boolean.class.isAssignableFrom(cls))
+            buf = new byte[]{(Boolean)obj ? (byte)1 : (byte)0};
+        else if (Byte.class.isAssignableFrom(cls))
+            buf = new byte[] {(Byte)obj};
+        else if (Short.class.isAssignableFrom(cls))
+            buf = U.shortToBytes((Short)obj);
+        else if (Integer.class.isAssignableFrom(cls))
+            buf = U.intToBytes((Integer)obj);
+        else if (Long.class.isAssignableFrom(cls))
+            buf = U.longToBytes((Long)obj);
+        else if (Float.class.isAssignableFrom(cls))
+            buf = U.intToBytes(Float.floatToIntBits((Float)obj));
+        else if (Double.class.isAssignableFrom(cls))
+            buf = U.longToBytes(Double.doubleToLongBits((Double)obj));
+        else if (BigDecimal.class.isAssignableFrom(cls)) {
+            BigInteger unscaledVal = ((BigDecimal)obj).unscaledValue();
+            int scale = ((BigDecimal)obj).scale();
+            buf = U.join(unscaledVal.toByteArray(), U.intToBytes(scale));
+        }
+        else if (UUID.class.isAssignableFrom(cls))
+            buf = U.uuidToBytes((UUID)obj);
+        else if (LocalDate.class.isAssignableFrom(cls))
+            buf = U.longToBytes(convertToSqlDate((LocalDate)obj).getTime());
+        else if (LocalTime.class.isAssignableFrom(cls))
+            buf = U.longToBytes(convertToSqlTime((LocalTime)obj).getTime());
+        else if (LocalDateTime.class.isAssignableFrom(cls))
+            buf = timestampToBytes(convertToTimestamp((LocalDateTime)obj));
+        else if (Timestamp.class.isAssignableFrom(cls))
+            buf = timestampToBytes((Timestamp)obj);
+        else if (java.util.Date.class.isAssignableFrom(cls))
+            buf = U.longToBytes(((Date)obj).getTime());
+        else if (cls.isAssignableFrom(byte[].class))
+            buf = (byte[])obj;
+        else if (cls.isAssignableFrom(String.class))
+            buf = ((String)obj).getBytes(StandardCharsets.UTF_8);
+        else if (obj instanceof BinaryObjectImpl)
+            buf = ((BinaryObjectImpl)obj).array();
+        else {
+            try {
+                buf = IndexProcessor.serializer.serialize(obj);
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException(e);
+            }
+        }
+
+        size += buf.length;
+        hll.addRaw(hash.fastHash(buf));
+    }
+
+    /** */
+    private static byte[] timestampToBytes(java.sql.Timestamp ts) {
+        byte[] buf = new byte[12];
+
+        U.longToBytes(ts.getTime(), buf, 0);
+        U.intToBytes(ts.getNanos(), buf, 8);
+
+        return buf;
+    }
+}
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/GatherStatisticCancelException.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/GatherStatisticCancelException.java
similarity index 93%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/GatherStatisticCancelException.java
rename to modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/GatherStatisticCancelException.java
index 819d71ff676..d815cbcda62 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/GatherStatisticCancelException.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/GatherStatisticCancelException.java
@@ -22,7 +22,5 @@ import org.apache.ignite.IgniteException;
 /** */
 public class GatherStatisticCancelException extends IgniteException {
     /** */
-    public GatherStatisticCancelException() {
-        // No-op.
-    }
+    private static final long serialVersionUID = 0L;
 }
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/Hasher.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/Hasher.java
similarity index 100%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/Hasher.java
rename to modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/Hasher.java
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteGlobalStatisticsManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteGlobalStatisticsManager.java
similarity index 99%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteGlobalStatisticsManager.java
rename to modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteGlobalStatisticsManager.java
index d4071e8193f..3ab61314d2a 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteGlobalStatisticsManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteGlobalStatisticsManager.java
@@ -28,7 +28,6 @@ import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.function.Function;
 import java.util.stream.Collectors;
-
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.cluster.ClusterNode;
diff --git a/modules/indexing/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
similarity index 88%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsConfigurationManager.java
rename to modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsConfigurationManager.java
index d6bc0c5de8a..54259b56e63 100644
--- a/modules/indexing/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
@@ -24,7 +24,6 @@ import java.util.Collections;
 import java.util.List;
 import java.util.Set;
 import java.util.concurrent.CopyOnWriteArrayList;
-import java.util.function.BiConsumer;
 import java.util.function.Consumer;
 import java.util.function.Function;
 import java.util.function.Supplier;
@@ -41,15 +40,18 @@ import org.apache.ignite.internal.managers.systemview.walker.StatisticsColumnCon
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.DynamicCacheChangeBatch;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.GridCacheContextInfo;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture;
 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.h2.SchemaManager;
-import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table;
+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.stat.config.StatisticsColumnConfiguration;
 import org.apache.ignite.internal.processors.query.stat.config.StatisticsObjectConfiguration;
 import org.apache.ignite.internal.processors.query.stat.view.ColumnConfigurationViewSupplier;
@@ -77,7 +79,7 @@ public class IgniteStatisticsConfigurationManager {
     public static final String[] EMPTY_STRINGS = new String[0];
 
     /** Schema manager. */
-    private final SchemaManager schemaMgr;
+    private final GridQuerySchemaManager schemaMgr;
 
     /** Distributed metastore. */
     private volatile DistributedMetaStorage distrMetaStorage;
@@ -103,8 +105,11 @@ public class IgniteStatisticsConfigurationManager {
     /** Is server node flag. */
     private final boolean isServerNode;
 
+    /** Active flag. */
+    private volatile boolean active;
+
     /** Configuration change subscribers. */
-    private List<Consumer<StatisticsObjectConfiguration>> subscribers = new CopyOnWriteArrayList<>();
+    private final List<Consumer<StatisticsObjectConfiguration>> subscribers = new CopyOnWriteArrayList<>();
 
     /** Change statistics configuration listener to update particular object statistics. */
     private final DistributedMetastorageLifecycleListener distrMetaStoreLsnr =
@@ -132,6 +137,55 @@ public class IgniteStatisticsConfigurationManager {
             }
         };
 
+    /** Schema change listener */
+    private final SchemaChangeListener schemaLsnr = new AbstractSchemaChangeListener() {
+        @Override public void onColumnsDropped(
+            String schemaName,
+            GridQueryTypeDescriptor typeDesc,
+            GridCacheContextInfo<?, ?> cacheInfo,
+            List<String> cols
+        ) {
+            if (!active)
+                return;
+
+            assert !F.isEmpty(cols);
+
+            // Drop statistics after columns dropped.
+            dropStatistics(
+                Collections.singletonList(
+                    new StatisticsTarget(schemaName, typeDesc.tableName(), cols.toArray(EMPTY_STRINGS))
+                ),
+                false
+            );
+        }
+
+        @Override public void onSqlTypeDropped(
+            String schemaName,
+            GridQueryTypeDescriptor typeDesc,
+            boolean destroy
+        ) {
+            if (!active || !destroy)
+                return;
+
+            String name = typeDesc.tableName();
+
+            assert !F.isEmpty(schemaName) && !F.isEmpty(name) : schemaName + ":" + name;
+
+            StatisticsKey key = new StatisticsKey(schemaName, name);
+
+            try {
+                StatisticsObjectConfiguration cfg = config(key);
+
+                if (cfg != null && !F.isEmpty(cfg.columns()))
+                    dropStatistics(Collections.singletonList(new StatisticsTarget(schemaName, name)), false);
+            }
+            catch (Throwable e) {
+                if (!X.hasCause(e, NodeStoppingException.class))
+                    throw new IgniteSQLException("Error on drop statistics for dropped table [key=" + key + ']', e);
+            }
+        }
+    };
+
     /**
      * Constructor.
      *
@@ -147,7 +201,7 @@ public class IgniteStatisticsConfigurationManager {
      * @param isServerNode Server node flag.
      */
     public IgniteStatisticsConfigurationManager(
-        SchemaManager schemaMgr,
+        GridQuerySchemaManager schemaMgr,
         GridInternalSubscriptionProcessor subscriptionProcessor,
         GridSystemViewManager sysViewMgr,
         GridClusterStateProcessor cluster,
@@ -168,6 +222,9 @@ public class IgniteStatisticsConfigurationManager {
 
         subscriptionProcessor.registerDistributedMetastorageListener(distrMetaStoreLsnr);
 
+        if (isServerNode)
+            subscriptionProcessor.registerSchemaChangeListener(schemaLsnr);
+
         ColumnConfigurationViewSupplier colCfgViewSupplier = new ColumnConfigurationViewSupplier(this,
             logSupplier);
 
@@ -203,60 +260,15 @@ public class IgniteStatisticsConfigurationManager {
         mgmtBusyExecutor.execute(this::updateAllLocalStatistics);
     }
 
-    /** Drop columns listener to clean its statistics configuration. */
-    private final BiConsumer<GridH2Table, List<String>> dropColsLsnr = new BiConsumer<GridH2Table, List<String>>() {
-        /**
-         * Drop statistics after columns dropped.
-         *
-         * @param tbl Table.
-         * @param cols Dropped columns.
-         */
-        @Override public void accept(GridH2Table tbl, List<String> cols) {
-            assert !F.isEmpty(cols);
-            dropStatistics(Collections.singletonList(
-                    new StatisticsTarget(
-                        tbl.identifier().schema(),
-                        tbl.getName(),
-                        cols.toArray(EMPTY_STRINGS)
-                    )
-                ),
-                false);
-        }
-    };
-
-    /** Drop table listener to clear its statistics configuration. */
-    private final BiConsumer<String, String> dropTblLsnr = new BiConsumer<String, String>() {
-        /**
-         * Drop statistics after table dropped.
-         *
-         * @param schema Schema name.
-         * @param name Table name.
-         */
-        @Override public void accept(String schema, String name) {
-            assert !F.isEmpty(schema) && !F.isEmpty(name) : schema + ":" + name;
-
-            StatisticsKey key = new StatisticsKey(schema, name);
-
-            try {
-                StatisticsObjectConfiguration cfg = config(key);
-
-                if (cfg != null && !F.isEmpty(cfg.columns()))
-                    dropStatistics(Collections.singletonList(new StatisticsTarget(schema, name)), false);
-            }
-            catch (Throwable e) {
-                if (!X.hasCause(e, NodeStoppingException.class))
-                    throw new IgniteSQLException("Error on drop statistics for dropped table [key=" + key + ']', e);
-            }
-        }
-    };
-
     /**
      * Pass all necessary parameters to schedule statistics key update.
      *
      * @param cfg Statistics object configuration to update statistics by.
      */
     private void updateLocalStatistics(StatisticsObjectConfiguration cfg) {
-        GridH2Table tbl = schemaMgr.dataTable(cfg.key().schema(), cfg.key().obj());
+        GridQueryTypeDescriptor tbl = schemaMgr.typeDescriptorForTable(cfg.key().schema(), cfg.key().obj());
+        GridCacheContextInfo<?, ?> cacheInfo = schemaMgr.cacheInfoForTable(cfg.key().schema(), cfg.key().obj());
+        GridCacheContext<?, ?> cctx = cacheInfo != null ? cacheInfo.cacheContext() : null;
 
         if (tbl == null || cfg.columns().isEmpty()) {
             // Can be drop table event, need to ensure that there is no stored data left for this table.
@@ -268,8 +280,8 @@ public class IgniteStatisticsConfigurationManager {
             }
 
             // Ensure to clean local metastorage.
-            LocalStatisticsGatheringContext ctx = new LocalStatisticsGatheringContext(false, tbl, cfg,
-                Collections.emptySet(), topVer);
+            LocalStatisticsGatheringContext ctx = new LocalStatisticsGatheringContext(false, tbl, cacheInfo,
+                cfg, Collections.emptySet(), topVer);
 
             statProc.updateLocalStatistics(ctx);
 
@@ -283,8 +295,6 @@ public class IgniteStatisticsConfigurationManager {
             return;
         }
 
-        GridCacheContext<?, ?> cctx = tbl.cacheContext();
-
         if (cctx == null || !cctx.gate().enterIfNotStopped()) {
             if (log.isDebugEnabled())
                 log.debug("Unable to lock table by key " + cfg.key() + ". Skipping statistics collection.");
@@ -297,8 +307,8 @@ public class IgniteStatisticsConfigurationManager {
 
             final Set<Integer> primParts = cctx.affinity().primaryPartitions(cctx.localNodeId(), topVer0);
 
-            LocalStatisticsGatheringContext ctx = new LocalStatisticsGatheringContext(false, tbl, cfg,
-                primParts, topVer0);
+            LocalStatisticsGatheringContext ctx = new LocalStatisticsGatheringContext(false, tbl, cacheInfo,
+                cfg, primParts, topVer0);
             statProc.updateLocalStatistics(ctx);
         }
         catch (IgniteCheckedException e) {
@@ -330,13 +340,9 @@ public class IgniteStatisticsConfigurationManager {
         if (log.isTraceEnabled())
             log.trace("Statistics configuration manager starting...");
 
+        active = true;
         mgmtBusyExecutor.activate();
 
-        if (isServerNode) {
-            schemaMgr.registerDropColumnsListener(dropColsLsnr);
-            schemaMgr.registerDropTableListener(dropTblLsnr);
-        }
-
         if (log.isDebugEnabled())
             log.debug("Statistics configuration manager started.");
 
@@ -367,11 +373,7 @@ public class IgniteStatisticsConfigurationManager {
         if (log.isTraceEnabled())
             log.trace("Statistics configuration manager stopping...");
 
-        if (isServerNode) {
-            schemaMgr.unregisterDropColumnsListener(dropColsLsnr);
-            schemaMgr.unregisterDropTableListener(dropTblLsnr);
-        }
-
+        active = false;
         mgmtBusyExecutor.deactivate();
 
         if (log.isDebugEnabled())
@@ -390,16 +392,16 @@ public class IgniteStatisticsConfigurationManager {
 
         for (StatisticsObjectConfiguration target : targets) {
 
-            GridH2Table tbl = schemaMgr.dataTable(target.key().schema(), target.key().obj());
+            GridQueryTypeDescriptor tbl = schemaMgr.typeDescriptorForTable(target.key().schema(), target.key().obj());
 
             validate(target, tbl);
 
             List<StatisticsColumnConfiguration> colCfgs;
 
             if (F.isEmpty(target.columns()))
-                colCfgs = Arrays.stream(tbl.getColumns())
-                    .filter(c -> c.getColumnId() >= QueryUtils.DEFAULT_COLUMNS_COUNT)
-                    .map(c -> new StatisticsColumnConfiguration(c.getName(), null))
+                colCfgs = tbl.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());
             else
                 colCfgs = new ArrayList<>(target.columns().values());
@@ -577,9 +579,9 @@ public class IgniteStatisticsConfigurationManager {
      * Validate specified configuration: check that specified table exist and contains all specified columns.
      *
      * @param cfg Statistics object configuration to check.
-     * @param tbl Corresponding GridH2Table (if exists).
+     * @param tbl Corresponding table (if exists).
      */
-    private void validate(StatisticsObjectConfiguration cfg, GridH2Table tbl) {
+    private void validate(StatisticsObjectConfiguration cfg, GridQueryTypeDescriptor tbl) {
         if (tbl == null) {
             throw new IgniteSQLException(
                 "Table doesn't exist [schema=" + cfg.key().schema() + ", table=" + cfg.key().obj() + ']',
@@ -588,7 +590,7 @@ public class IgniteStatisticsConfigurationManager {
 
         if (!F.isEmpty(cfg.columns())) {
             for (String col : cfg.columns().keySet()) {
-                if (!tbl.doesColumnExist(col)) {
+                if (!tbl.fields().containsKey(col)) {
                     throw new IgniteSQLException(
                         "Column doesn't exist [schema=" + cfg.key().schema() +
                             ", table=" + cfg.key().obj() +
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsDummyStoreImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsDummyStoreImpl.java
similarity index 99%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsDummyStoreImpl.java
rename to modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsDummyStoreImpl.java
index ed2a7b40edf..0d923d43b6d 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsDummyStoreImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsDummyStoreImpl.java
@@ -21,7 +21,6 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.Map;
 import java.util.function.Function;
-
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.internal.util.collection.IntMap;
 
diff --git a/modules/indexing/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
similarity index 78%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsHelper.java
rename to modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsHelper.java
index 44b6cdeb416..3dd953915cd 100644
--- a/modules/indexing/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
@@ -23,27 +23,32 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.Spliterator;
+import java.util.Spliterators;
 import java.util.UUID;
 import java.util.function.Function;
 import java.util.stream.Collectors;
-
+import java.util.stream.Stream;
+import java.util.stream.StreamSupport;
 import org.apache.ignite.IgniteCheckedException;
 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.h2.SchemaManager;
-import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table;
 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;
 import org.apache.ignite.internal.processors.query.stat.messages.StatisticsRequest;
 import org.apache.ignite.internal.util.typedef.F;
-import org.h2.table.Column;
+import org.apache.ignite.internal.util.typedef.T2;
 import org.jetbrains.annotations.Nullable;
 
 /**
@@ -54,7 +59,7 @@ public class IgniteStatisticsHelper {
     private final IgniteLogger log;
 
     /** Schema manager. */
-    private final SchemaManager schemaMgr;
+    private final GridQuerySchemaManager schemaMgr;
 
     /**
      * Constructor.
@@ -65,7 +70,7 @@ public class IgniteStatisticsHelper {
      */
     public IgniteStatisticsHelper(
         UUID locNodeId,
-        SchemaManager schemaMgr,
+        GridQuerySchemaManager schemaMgr,
         Function<Class<?>, IgniteLogger> logSupplier
     ) {
         this.schemaMgr = schemaMgr;
@@ -80,7 +85,7 @@ public class IgniteStatisticsHelper {
      * @throws IgniteCheckedException If unable to find table by specified key.
      */
     public CacheGroupContext groupContext(StatisticsKey key) throws IgniteCheckedException {
-        GridH2Table tbl = schemaMgr.dataTable(key.schema(), key.obj());
+        GridCacheContextInfo<?, ?> tbl = schemaMgr.cacheInfoForTable(key.schema(), key.obj());
 
         if (tbl == null)
             throw new IgniteCheckedException(String.format("Can't find object %s.%s", key.schema(), key.obj()));
@@ -145,7 +150,7 @@ public class IgniteStatisticsHelper {
         );
 
         // For now there can be only tables
-        GridH2Table tbl = schemaMgr.dataTable(keyMsg.schema(), keyMsg.obj());
+        GridQueryTypeDescriptor tbl = schemaMgr.typeDescriptorForTable(keyMsg.schema(), keyMsg.obj());
 
         if (tbl == null) {
             // remove all loaded statistics.
@@ -169,23 +174,24 @@ public class IgniteStatisticsHelper {
      * @return Local level statistics.
      */
     public static ObjectStatisticsImpl aggregateLocalStatistics(
-        GridH2Table tbl,
+        GridQueryTypeDescriptor tbl,
         StatisticsObjectConfiguration cfg,
         Collection<? extends ObjectStatisticsImpl> stats,
         IgniteLogger log
     ) {
         assert !stats.isEmpty();
-        Column[] selectedCols = filterColumns(tbl.getColumns(), cfg.columns().keySet());
+        List<String> selectedCols = filterColumns(tbl, cfg.columns().keySet()).stream().map(T2::getValue)
+            .collect(Collectors.toList());
 
-        Map<Column, List<ColumnStatistics>> colPartStats = new HashMap<>(selectedCols.length);
+        Map<String, List<ColumnStatistics>> colPartStats = new HashMap<>(selectedCols.size());
         long rowCnt = 0;
 
-        for (Column col : selectedCols)
+        for (String col : selectedCols)
             colPartStats.put(col, new ArrayList<>());
 
         for (ObjectStatisticsImpl partStat : stats) {
-            for (Column col : selectedCols) {
-                ColumnStatistics colPartStat = partStat.columnStatistics(col.getName());
+            for (String col : selectedCols) {
+                ColumnStatistics colPartStat = partStat.columnStatistics(col);
 
                 if (colPartStat != null)
                     colPartStats.get(col).add(colPartStat);
@@ -194,17 +200,17 @@ public class IgniteStatisticsHelper {
             rowCnt += partStat.rowCount();
         }
 
-        Map<String, ColumnStatistics> colStats = new HashMap<>(selectedCols.length);
+        Map<String, ColumnStatistics> colStats = new HashMap<>(selectedCols.size());
 
-        for (Column col : selectedCols) {
-            StatisticsColumnConfiguration colCfg = cfg.columns().get(col.getName());
-            ColumnStatistics stat = ColumnStatisticsCollector.aggregate(tbl::compareTypeSafe, colPartStats.get(col),
+        for (String col : selectedCols) {
+            StatisticsColumnConfiguration colCfg = cfg.columns().get(col);
+            ColumnStatistics stat = ColumnStatisticsCollector.aggregate(colPartStats.get(col),
                 colCfg.overrides());
 
             if (log.isDebugEnabled())
-                log.debug("Aggregate column statistic done [col=" + col.getName() + ", stat=" + stat + ']');
+                log.debug("Aggregate column statistic done [col=" + col + ", stat=" + stat + ']');
 
-            colStats.put(col.getName(), stat);
+            colStats.put(col, stat);
         }
 
         rowCnt = calculateRowCount(cfg, rowCnt);
@@ -260,19 +266,44 @@ public class IgniteStatisticsHelper {
     /**
      * Filter columns by specified names.
      *
-     * @param cols Columns to filter.
+     * @param typeDescriptor Table descriptor.
      * @param colNames Column names.
      * @return Column with specified names.
      */
-    public static Column[] filterColumns(Column[] cols, @Nullable Collection<String> colNames) {
+    public static List<T2<Integer, String>> filterColumns(
+        GridQueryTypeDescriptor typeDescriptor,
+        @Nullable Collection<String> colNames
+    ) {
+        Stream<T2<Integer, String>> colStream = enumerate(typeDescriptor.fields().keySet().stream(),
+            QueryUtils.DEFAULT_COLUMNS_COUNT);
+
         if (F.isEmpty(colNames)) {
-            return Arrays.stream(cols)
-                .filter(c -> c.getColumnId() >= QueryUtils.DEFAULT_COLUMNS_COUNT)
-                .toArray(Column[]::new);
+            return colStream.filter(col -> !QueryUtils.KEY_FIELD_NAME.equals(col.getValue()) &&
+                    !QueryUtils.VAL_FIELD_NAME.equals(col.getValue())).collect(Collectors.toList());
         }
 
         Set<String> colNamesSet = new HashSet<>(colNames);
 
-        return Arrays.stream(cols).filter(c -> colNamesSet.contains(c.getName())).toArray(Column[]::new);
+        return colStream.filter(col -> colNamesSet.contains(col.getValue())).collect(Collectors.toList());
+    }
+
+    /** */
+    private static <T> Stream<T2<Integer, T>> enumerate(Stream<? extends T> stream, int startIdx) {
+        Iterator<T2<Integer, T>> iter = new Iterator<T2<Integer, T>>() {
+            private final Iterator<? extends T> streamIter = stream.iterator();
+
+            private int idx = startIdx;
+
+            @Override public boolean hasNext() {
+                return streamIter.hasNext();
+            }
+
+            @Override public T2<Integer, T> next() {
+                return new T2<>(idx++, streamIter.next());
+            }
+        };
+
+        return StreamSupport.stream(Spliterators.spliteratorUnknownSize(iter, Spliterator.ORDERED |
+            Spliterator.IMMUTABLE), false);
     }
 }
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsInMemoryStoreImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsInMemoryStoreImpl.java
similarity index 100%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsInMemoryStoreImpl.java
rename to modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsInMemoryStoreImpl.java
diff --git a/modules/indexing/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
similarity index 96%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsManagerImpl.java
rename to modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsManagerImpl.java
index 9c6b988351a..f63d625bec9 100644
--- a/modules/indexing/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
@@ -23,7 +23,6 @@ import java.util.List;
 import java.util.Set;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.function.Consumer;
-
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
@@ -32,13 +31,14 @@ import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.managers.communication.GridIoPolicy;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.cache.GridCacheContextInfo;
 import org.apache.ignite.internal.processors.cache.GridCacheUtils;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture;
 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.h2.SchemaManager;
-import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table;
+import org.apache.ignite.internal.processors.query.GridQuerySchemaManager;
+import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
 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 SchemaManager schemaMgr;
+    private final GridQuerySchemaManager schemaMgr;
 
     /** Statistics repository. */
     private final IgniteStatisticsRepository statsRepos;
@@ -106,7 +106,7 @@ public class IgniteStatisticsManagerImpl implements IgniteStatisticsManager {
     private volatile StatisticsUsageState lastUsageState = null;
 
     /** Started flag to prevent double start on change statistics usage state and activation and vice versa. */
-    private boolean started = false;
+    private volatile boolean started;
 
     /** Schedule to process obsolescence statistics. */
     private GridTimeoutProcessor.CancelableTask obsolescenceSchedule;
@@ -136,7 +136,10 @@ public class IgniteStatisticsManagerImpl implements IgniteStatisticsManager {
      * @param ctx Kernal context.
      * @param schemaMgr Schema manager.
      */
-    public IgniteStatisticsManagerImpl(GridKernalContext ctx, SchemaManager schemaMgr) {
+    public IgniteStatisticsManagerImpl(
+        GridKernalContext ctx,
+        GridQuerySchemaManager schemaMgr
+    ) {
         this.ctx = ctx;
         this.schemaMgr = schemaMgr;
 
@@ -482,7 +485,8 @@ public class IgniteStatisticsManagerImpl implements IgniteStatisticsManager {
 
             Set<Integer> tasksParts = calculateObsolescencedPartitions(cfg, statsRepos.getObsolescence(key));
 
-            GridH2Table tbl = schemaMgr.dataTable(key.schema(), key.obj());
+            GridQueryTypeDescriptor tbl = schemaMgr.typeDescriptorForTable(key.schema(), key.obj());
+            GridCacheContextInfo<?, ?> cacheInfo = schemaMgr.cacheInfoForTable(key.schema(), key.obj());
 
             if (tbl == null) {
                 // Table can be removed earlier, but not already processed. Or somethink goes wrong. Try to reschedule.
@@ -490,7 +494,7 @@ public class IgniteStatisticsManagerImpl implements IgniteStatisticsManager {
                     log.debug(String.format("Got obsolescence statistics for unknown table %s", key));
             }
 
-            LocalStatisticsGatheringContext ctx = new LocalStatisticsGatheringContext(true, tbl, cfg,
+            LocalStatisticsGatheringContext ctx = new LocalStatisticsGatheringContext(true, tbl, cacheInfo, cfg,
                 tasksParts, null);
             statProc.updateLocalStatistics(ctx);
         }
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsPersistenceStoreImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsPersistenceStoreImpl.java
similarity index 99%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsPersistenceStoreImpl.java
rename to modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsPersistenceStoreImpl.java
index a2fbb282446..0132c11a62c 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsPersistenceStoreImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsPersistenceStoreImpl.java
@@ -29,7 +29,6 @@ import java.util.Set;
 import java.util.function.BiConsumer;
 import java.util.function.Function;
 import java.util.stream.Collectors;
-
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.internal.processors.cache.persistence.IgniteCacheDatabaseSharedManager;
@@ -64,7 +63,7 @@ public class IgniteStatisticsPersistenceStoreImpl implements IgniteStatisticsSto
     private static final String META_VERSION_KEY = META_STAT_PREFIX + META_SEPARATOR + "version";
 
     /** Actual statistics version. */
-    public static final Integer VERSION = 2;
+    public static final Integer VERSION = 3;
 
     /** Logger. */
     private final IgniteLogger log;
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsRepository.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsRepository.java
similarity index 99%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsRepository.java
rename to modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsRepository.java
index ea4360d1c26..be12e89aa5a 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsRepository.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsRepository.java
@@ -26,7 +26,6 @@ import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.function.Consumer;
 import java.util.function.Function;
-
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.internal.managers.systemview.GridSystemViewManager;
 import org.apache.ignite.internal.managers.systemview.walker.StatisticsColumnLocalDataViewWalker;
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsStore.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsStore.java
similarity index 99%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsStore.java
rename to modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsStore.java
index d4f8b7b935b..7ac68526d47 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsStore.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsStore.java
@@ -19,7 +19,6 @@ package org.apache.ignite.internal.processors.query.stat;
 
 import java.util.Collection;
 import java.util.Map;
-
 import org.apache.ignite.internal.util.collection.IntMap;
 
 /**
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/LocalStatisticsGatheringContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/LocalStatisticsGatheringContext.java
similarity index 88%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/LocalStatisticsGatheringContext.java
rename to modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/LocalStatisticsGatheringContext.java
index e0659ed6934..4df521d39d0 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/LocalStatisticsGatheringContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/LocalStatisticsGatheringContext.java
@@ -20,9 +20,9 @@ package org.apache.ignite.internal.processors.query.stat;
 import java.util.HashSet;
 import java.util.Set;
 import java.util.concurrent.CompletableFuture;
-
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
-import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table;
+import org.apache.ignite.internal.processors.cache.GridCacheContextInfo;
+import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
 import org.apache.ignite.internal.processors.query.stat.config.StatisticsObjectConfiguration;
 import org.apache.ignite.internal.util.typedef.internal.S;
 
@@ -34,7 +34,10 @@ public class LocalStatisticsGatheringContext {
     private final boolean forceRecollect;
 
     /** Table to process. */
-    private final GridH2Table tbl;
+    private final GridQueryTypeDescriptor tbl;
+
+    /** Table cache context. */
+    private final GridCacheContextInfo<?, ?> cctxInfo;
 
     /** Statistics configuration to use. */
     private final StatisticsObjectConfiguration cfg;
@@ -59,18 +62,21 @@ public class LocalStatisticsGatheringContext {
      *
      * @param forceRecollect Force recollect flag.
      * @param tbl Table to process.
+     * @param cctxInfo Cache context info;
      * @param cfg Statistics configuration to use.
      * @param remainingParts Set of partition ids to collect.
      */
     public LocalStatisticsGatheringContext(
         boolean forceRecollect,
-        GridH2Table tbl,
+        GridQueryTypeDescriptor tbl,
+        GridCacheContextInfo<?, ?> cctxInfo,
         StatisticsObjectConfiguration cfg,
         Set<Integer> remainingParts,
         AffinityTopologyVersion topVer
     ) {
         this.forceRecollect = forceRecollect;
         this.tbl = tbl;
+        this.cctxInfo = cctxInfo;
         this.cfg = cfg;
         this.remainingParts = new HashSet<>(remainingParts);
         this.allParts = (forceRecollect) ? null : new HashSet<>(remainingParts);
@@ -88,10 +94,17 @@ public class LocalStatisticsGatheringContext {
     /**
      * @return Table to process.
      */
-    public GridH2Table table() {
+    public GridQueryTypeDescriptor table() {
         return tbl;
     }
 
+    /**
+     * @return Cache context of processing table.
+     */
+    public GridCacheContextInfo<?, ?> cacheContextInfo() {
+        return cctxInfo;
+    }
+
     /**
      * @return Statistics configuration to collect with.
      */
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/ObjectPartitionStatisticsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/ObjectPartitionStatisticsImpl.java
similarity index 99%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/ObjectPartitionStatisticsImpl.java
rename to modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/ObjectPartitionStatisticsImpl.java
index 97c7f808b1e..812340ddb19 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/ObjectPartitionStatisticsImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/ObjectPartitionStatisticsImpl.java
@@ -19,7 +19,6 @@ package org.apache.ignite.internal.processors.query.stat;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Objects;
-
 import org.apache.ignite.internal.util.typedef.internal.S;
 
 /**
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/ObjectPartitionStatisticsObsolescence.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/ObjectPartitionStatisticsObsolescence.java
similarity index 99%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/ObjectPartitionStatisticsObsolescence.java
rename to modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/ObjectPartitionStatisticsObsolescence.java
index e73b9d85809..65e68be09d8 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/ObjectPartitionStatisticsObsolescence.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/ObjectPartitionStatisticsObsolescence.java
@@ -20,7 +20,6 @@ import java.io.Externalizable;
 import java.io.IOException;
 import java.io.ObjectInput;
 import java.io.ObjectOutput;
-
 import org.apache.ignite.internal.processors.query.stat.hll.HLL;
 
 /**
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/ObjectStatisticsEvent.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/ObjectStatisticsEvent.java
similarity index 100%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/ObjectStatisticsEvent.java
rename to modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/ObjectStatisticsEvent.java
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/ObjectStatisticsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/ObjectStatisticsImpl.java
similarity index 99%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/ObjectStatisticsImpl.java
rename to modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/ObjectStatisticsImpl.java
index 172f679bbeb..4cd6374d071 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/ObjectStatisticsImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/ObjectStatisticsImpl.java
@@ -21,7 +21,6 @@ import java.util.HashMap;
 import java.util.Map;
 import java.util.Objects;
 import java.util.Set;
-
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.internal.S;
 
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/StatisticsAddressedRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/StatisticsAddressedRequest.java
similarity index 100%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/StatisticsAddressedRequest.java
rename to modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/StatisticsAddressedRequest.java
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/StatisticsProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/StatisticsProcessor.java
similarity index 100%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/StatisticsProcessor.java
rename to modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/StatisticsProcessor.java
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/StatisticsType.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/StatisticsType.java
similarity index 100%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/StatisticsType.java
rename to modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/StatisticsType.java
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/StatisticsUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/StatisticsUtils.java
similarity index 73%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/StatisticsUtils.java
rename to modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/StatisticsUtils.java
index be7c34e4a44..5d7c8ed3229 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/StatisticsUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/StatisticsUtils.java
@@ -17,20 +17,28 @@
 
 package org.apache.ignite.internal.processors.query.stat;
 
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
 import java.util.HashMap;
 import java.util.Map;
-
+import java.util.UUID;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.GridKernalContext;
-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.stat.config.StatisticsColumnConfiguration;
 import org.apache.ignite.internal.processors.query.stat.config.StatisticsObjectConfiguration;
 import org.apache.ignite.internal.processors.query.stat.messages.StatisticsColumnData;
+import org.apache.ignite.internal.processors.query.stat.messages.StatisticsDecimalMessage;
 import org.apache.ignite.internal.processors.query.stat.messages.StatisticsKeyMessage;
 import org.apache.ignite.internal.processors.query.stat.messages.StatisticsObjectData;
 import org.apache.ignite.internal.util.typedef.F;
-import org.h2.value.Value;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+import static org.apache.ignite.internal.cache.query.index.sorted.inline.types.DateValueUtils.convertToSqlDate;
+import static org.apache.ignite.internal.cache.query.index.sorted.inline.types.DateValueUtils.convertToSqlTime;
+import static org.apache.ignite.internal.cache.query.index.sorted.inline.types.DateValueUtils.convertToTimestamp;
 
 /**
  * Utilities to convert statistics from/to messages, validate configurations with statistics and so on.
@@ -44,8 +52,8 @@ public class StatisticsUtils {
      * @throws IgniteCheckedException In case of errors.
      */
     public static StatisticsColumnData toMessage(ColumnStatistics stat) throws IgniteCheckedException {
-        GridH2ValueMessage msgMin = stat.min() == null ? null : GridH2ValueMessageFactory.toMessage(stat.min());
-        GridH2ValueMessage msgMax = stat.max() == null ? null : GridH2ValueMessageFactory.toMessage(stat.max());
+        StatisticsDecimalMessage msgMin = new StatisticsDecimalMessage(stat.min());
+        StatisticsDecimalMessage msgMax = new StatisticsDecimalMessage(stat.max());
 
         return new StatisticsColumnData(msgMin, msgMax, stat.nulls(), stat.distinct(),
             stat.total(), stat.size(), stat.raw(), stat.version(), stat.createdAt());
@@ -57,16 +65,9 @@ public class StatisticsUtils {
      * @param ctx Kernal context.
      * @param data Statistics column data message to convert.
      * @return ColumnStatistics object.
-     * @throws IgniteCheckedException In case of errors.
      */
-    public static ColumnStatistics toColumnStatistics(
-        GridKernalContext ctx,
-        StatisticsColumnData data
-    ) throws IgniteCheckedException {
-        Value min = (data.min() == null) ? null : data.min().value(ctx);
-        Value max = (data.max() == null) ? null : data.max().value(ctx);
-
-        return new ColumnStatistics(min, max, data.nulls(), data.distinct(),
+    public static ColumnStatistics toColumnStatistics(GridKernalContext ctx, StatisticsColumnData data) {
+        return new ColumnStatistics(data.min().value(), data.max().value(), data.nulls(), data.distinct(),
             data.total(), data.size(), data.rawData(), data.version(), data.createdAt());
     }
 
@@ -150,12 +151,8 @@ public class StatisticsUtils {
      * @param ctx Kernal context to use during conversion.
      * @param data Statistics object data message to convert.
      * @return Converted object statistics.
-     * @throws IgniteCheckedException  In case of errors.
      */
-    public static ObjectStatisticsImpl toObjectStatistics(
-        GridKernalContext ctx,
-        StatisticsObjectData data
-    ) throws IgniteCheckedException {
+    public static ObjectStatisticsImpl toObjectStatistics(GridKernalContext ctx, StatisticsObjectData data) {
         Map<String, ColumnStatistics> colNameToStat = new HashMap<>(data.data().size());
 
         for (Map.Entry<String, StatisticsColumnData> cs : data.data().entrySet())
@@ -233,4 +230,43 @@ public class StatisticsUtils {
 
         return 0;
     }
+
+    /** */
+    public static BigDecimal toDecimal(Object obj) {
+        if (obj == null)
+            return null;
+
+        Class<?> cls = U.box(obj.getClass());
+
+        if (Boolean.class.isAssignableFrom(cls))
+            return (Boolean)obj ? BigDecimal.ONE : BigDecimal.ZERO;
+        else if (Byte.class.isAssignableFrom(cls))
+            return BigDecimal.valueOf((Byte)obj);
+        else if (Short.class.isAssignableFrom(cls))
+            return BigDecimal.valueOf((Short)obj);
+        else if (Integer.class.isAssignableFrom(cls))
+            return BigDecimal.valueOf((Integer)obj);
+        else if (Long.class.isAssignableFrom(cls))
+            return new BigDecimal((Long)obj);
+        else if (Float.class.isAssignableFrom(cls))
+            return BigDecimal.valueOf((Float)obj);
+        else if (Double.class.isAssignableFrom(cls))
+            return BigDecimal.valueOf((Double)obj);
+        else if (BigDecimal.class.isAssignableFrom(cls))
+            return (BigDecimal)obj;
+        else if (UUID.class.isAssignableFrom(cls)) {
+            BigInteger bigInt = new BigInteger(1, U.uuidToBytes((UUID)obj));
+            return new BigDecimal(bigInt);
+        }
+        else if (java.util.Date.class.isAssignableFrom(cls))
+            return new BigDecimal(((java.util.Date)obj).getTime());
+        else if (LocalDate.class.isAssignableFrom(cls))
+            return new BigDecimal(convertToSqlDate((LocalDate)obj).getTime());
+        else if (LocalTime.class.isAssignableFrom(cls))
+            return new BigDecimal(convertToSqlTime((LocalTime)obj).getTime());
+        else if (LocalDateTime.class.isAssignableFrom(cls))
+            return new BigDecimal(convertToTimestamp((LocalDateTime)obj).getTime());
+
+        throw new IllegalArgumentException("Value of type " + cls.getName() + " is not expected");
+    }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/config/StatisticsColumnConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/config/StatisticsColumnConfiguration.java
index 27f94d4faae..1290a11b0a9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/config/StatisticsColumnConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/config/StatisticsColumnConfiguration.java
@@ -18,7 +18,6 @@ package org.apache.ignite.internal.processors.query.stat.config;
 
 import java.io.Serializable;
 import java.util.Objects;
-
 import org.apache.ignite.internal.util.typedef.internal.S;
 
 /**
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/config/StatisticsColumnOverrides.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/config/StatisticsColumnOverrides.java
index 2e980a2ecf0..bba2c9125b3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/config/StatisticsColumnOverrides.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/config/StatisticsColumnOverrides.java
@@ -18,7 +18,6 @@ package org.apache.ignite.internal.processors.query.stat.config;
 
 import java.io.Serializable;
 import java.util.Objects;
-
 import org.apache.ignite.internal.util.typedef.internal.S;
 
 /**
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/config/StatisticsObjectConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/config/StatisticsObjectConfiguration.java
index d39706f9a53..067338e08a3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/config/StatisticsObjectConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/config/StatisticsObjectConfiguration.java
@@ -28,7 +28,6 @@ import java.util.Objects;
 import java.util.Set;
 import java.util.function.Function;
 import java.util.stream.Collectors;
-
 import org.apache.ignite.internal.processors.query.stat.StatisticsKey;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.F;
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/hll/HLL.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/hll/HLL.java
similarity index 99%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/hll/HLL.java
rename to modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/hll/HLL.java
index b8cb0986a41..31319b42046 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/hll/HLL.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/hll/HLL.java
@@ -18,7 +18,6 @@ package org.apache.ignite.internal.processors.query.stat.hll;
 import java.util.Arrays;
 import java.util.HashMap;
 import java.util.HashSet;
-
 import org.apache.ignite.internal.processors.query.stat.hll.serialization.HLLMetadata;
 import org.apache.ignite.internal.processors.query.stat.hll.serialization.IHLLMetadata;
 import org.apache.ignite.internal.processors.query.stat.hll.serialization.ISchemaVersion;
@@ -1112,7 +1111,7 @@ public class HLL implements Cloneable {
     /**
      * Create a deep copy of this HLL.
      *
-     * @see java.lang.Object#clone()
+     * @see Object#clone()
      */
     @Override public HLL clone() throws CloneNotSupportedException {
         // NOTE: Since the package-only constructor assumes both explicit and
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/hll/HLLType.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/hll/HLLType.java
similarity index 100%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/hll/HLLType.java
rename to modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/hll/HLLType.java
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/hll/serialization/BigEndianAscendingWordDeserializer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/hll/serialization/BigEndianAscendingWordDeserializer.java
similarity index 100%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/hll/serialization/BigEndianAscendingWordDeserializer.java
rename to modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/hll/serialization/BigEndianAscendingWordDeserializer.java
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/hll/serialization/BigEndianAscendingWordSerializer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/hll/serialization/BigEndianAscendingWordSerializer.java
similarity index 100%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/hll/serialization/BigEndianAscendingWordSerializer.java
rename to modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/hll/serialization/BigEndianAscendingWordSerializer.java
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/hll/serialization/HLLMetadata.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/hll/serialization/HLLMetadata.java
similarity index 100%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/hll/serialization/HLLMetadata.java
rename to modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/hll/serialization/HLLMetadata.java
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/hll/serialization/IHLLMetadata.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/hll/serialization/IHLLMetadata.java
similarity index 100%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/hll/serialization/IHLLMetadata.java
rename to modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/hll/serialization/IHLLMetadata.java
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/hll/serialization/ISchemaVersion.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/hll/serialization/ISchemaVersion.java
similarity index 100%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/hll/serialization/ISchemaVersion.java
rename to modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/hll/serialization/ISchemaVersion.java
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/hll/serialization/IWordDeserializer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/hll/serialization/IWordDeserializer.java
similarity index 100%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/hll/serialization/IWordDeserializer.java
rename to modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/hll/serialization/IWordDeserializer.java
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/hll/serialization/IWordSerializer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/hll/serialization/IWordSerializer.java
similarity index 100%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/hll/serialization/IWordSerializer.java
rename to modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/hll/serialization/IWordSerializer.java
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/hll/serialization/SchemaVersionOne.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/hll/serialization/SchemaVersionOne.java
similarity index 100%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/hll/serialization/SchemaVersionOne.java
rename to modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/hll/serialization/SchemaVersionOne.java
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/hll/serialization/SerializationUtil.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/hll/serialization/SerializationUtil.java
similarity index 100%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/hll/serialization/SerializationUtil.java
rename to modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/hll/serialization/SerializationUtil.java
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/hll/util/BitUtil.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/hll/util/BitUtil.java
similarity index 100%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/hll/util/BitUtil.java
rename to modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/hll/util/BitUtil.java
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/hll/util/BitVector.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/hll/util/BitVector.java
similarity index 98%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/hll/util/BitVector.java
rename to modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/hll/util/BitVector.java
index fc9d713d282..e6cc6385a7e 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/hll/util/BitVector.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/hll/util/BitVector.java
@@ -156,7 +156,7 @@ public class BitVector implements Cloneable {
      * @return a <code>LongIterator</code> for iterating starting at the register
      *         with index zero. This will never be <code>null</code>.
      */
-    public org.apache.ignite.internal.processors.query.stat.hll.util.LongIterator registerIterator() {
+    public LongIterator registerIterator() {
         LongIterator longIterator = new LongIterator() {
             final int registerWidth = BitVector.this.registerWidth;
             final long[] words = BitVector.this.words;
@@ -218,7 +218,7 @@ public class BitVector implements Cloneable {
      *         otherwise.
      * @see #getRegister(long)
      * @see #setRegister(long, long)
-     * @see java.lang.Math#max(long, long)
+     * @see Math#max(long, long)
      */
     // NOTE:  if this changes then setRegister() must change
     public boolean setMaxRegister(final long registerIndex, final long value) {
@@ -284,7 +284,7 @@ public class BitVector implements Cloneable {
     /**
      * Creates a deep copy of this vector.
      *
-     * @see java.lang.Object#clone()
+     * @see Object#clone()
      */
     @Override public BitVector clone() {
         final BitVector copy = new BitVector(registerWidth, count);
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/hll/util/HLLUtil.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/hll/util/HLLUtil.java
similarity index 100%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/hll/util/HLLUtil.java
rename to modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/hll/util/HLLUtil.java
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/hll/util/LongIterator.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/hll/util/LongIterator.java
similarity index 100%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/hll/util/LongIterator.java
rename to modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/hll/util/LongIterator.java
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/hll/util/NumberUtil.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/hll/util/NumberUtil.java
similarity index 100%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/hll/util/NumberUtil.java
rename to modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/hll/util/NumberUtil.java
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/messages/StatisticsColumnData.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/messages/StatisticsColumnData.java
similarity index 95%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/messages/StatisticsColumnData.java
rename to modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/messages/StatisticsColumnData.java
index c41ebbce5a4..463876bcc49 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/messages/StatisticsColumnData.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/messages/StatisticsColumnData.java
@@ -18,8 +18,6 @@
 package org.apache.ignite.internal.processors.query.stat.messages;
 
 import java.nio.ByteBuffer;
-
-import org.apache.ignite.internal.processors.query.h2.twostep.msg.GridH2ValueMessage;
 import org.apache.ignite.plugin.extensions.communication.Message;
 import org.apache.ignite.plugin.extensions.communication.MessageReader;
 import org.apache.ignite.plugin.extensions.communication.MessageWriter;
@@ -32,13 +30,13 @@ public class StatisticsColumnData implements Message {
     private static final long serialVersionUID = 0L;
 
     /** */
-    public static final short TYPE_CODE = 185;
+    public static final short TYPE_CODE = 186;
 
     /** Min value in column. */
-    private GridH2ValueMessage min;
+    private StatisticsDecimalMessage min;
 
     /** Max value in column. */
-    private GridH2ValueMessage max;
+    private StatisticsDecimalMessage max;
 
     /** Number of null values in column. */
     private long nulls;
@@ -65,6 +63,7 @@ public class StatisticsColumnData implements Message {
      * Default constructor.
      */
     public StatisticsColumnData() {
+        // No-op.
     }
 
     /**
@@ -81,8 +80,8 @@ public class StatisticsColumnData implements Message {
      * @param createdAt Created at time, milliseconds.
      */
     public StatisticsColumnData(
-        GridH2ValueMessage min,
-        GridH2ValueMessage max,
+        StatisticsDecimalMessage min,
+        StatisticsDecimalMessage max,
         long nulls,
         long distinct,
         long total,
@@ -105,14 +104,14 @@ public class StatisticsColumnData implements Message {
     /**
      * @return Min value in column.
      */
-    public GridH2ValueMessage min() {
+    public StatisticsDecimalMessage min() {
         return min;
     }
 
     /**
      * @return Max value in column.
      */
-    public GridH2ValueMessage max() {
+    public StatisticsDecimalMessage max() {
         return max;
     }
 
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/messages/StatisticsResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/messages/StatisticsDecimalMessage.java
similarity index 70%
copy from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/messages/StatisticsResponse.java
copy to modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/messages/StatisticsDecimalMessage.java
index fcf7063e335..42dd6e4ad1f 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/messages/StatisticsResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/messages/StatisticsDecimalMessage.java
@@ -17,66 +17,59 @@
 
 package org.apache.ignite.internal.processors.query.stat.messages;
 
+import java.math.BigDecimal;
+import java.math.BigInteger;
 import java.nio.ByteBuffer;
-import java.util.UUID;
-import org.apache.ignite.internal.util.typedef.internal.S;
+import java.util.Objects;
 import org.apache.ignite.plugin.extensions.communication.Message;
 import org.apache.ignite.plugin.extensions.communication.MessageReader;
 import org.apache.ignite.plugin.extensions.communication.MessageWriter;
 
 /**
- * Response for statistics request.
+ * H2 Decimal.
  */
-public class StatisticsResponse implements Message {
+public class StatisticsDecimalMessage implements Message {
     /** */
     private static final long serialVersionUID = 0L;
 
     /** */
-    public static final short TYPE_CODE = 187;
+    public static final short TYPE_CODE = 184;
 
-    /** Request id. */
-    private UUID reqId;
-
-    /** Requested statistics. */
-    private StatisticsObjectData data;
+    /** */
+    private int scale;
 
-    /**
-     * Constructor.
-     */
-    public StatisticsResponse() {
-    }
+    /** */
+    private byte[] b;
 
     /**
-     * Constructor.
      *
-     * @param reqId Request id
-     * @param data Statistics data.
      */
-    public StatisticsResponse(
-        UUID reqId,
-        StatisticsObjectData data
-    ) {
-        this.reqId = reqId;
-        this.data = data;
+    public StatisticsDecimalMessage() {
+        // No-op.
     }
 
     /**
-     * @return Request id.
+     * @param val Value.
      */
-    public UUID reqId() {
-        return reqId;
+    public StatisticsDecimalMessage(BigDecimal val) {
+        if (val == null) {
+            scale = 0;
+            b = null;
+        }
+        else {
+            scale = val.scale();
+            b = val.unscaledValue().toByteArray();
+        }
     }
 
     /**
-     * @return Statitics data.
+     * @return Decimal value.
      */
-    public StatisticsObjectData data() {
-        return data;
-    }
+    public BigDecimal value() {
+        if (b == null && scale == 0)
+            return null;
 
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(StatisticsResponse.class, this);
+        return new BigDecimal(new BigInteger(b), scale);
     }
 
     /** {@inheritDoc} */
@@ -92,17 +85,16 @@ public class StatisticsResponse implements Message {
 
         switch (writer.state()) {
             case 0:
-                if (!writer.writeMessage("data", data))
+                if (!writer.writeByteArray("b", b))
                     return false;
 
                 writer.incrementState();
 
             case 1:
-                if (!writer.writeUuid("reqId", reqId))
+                if (!writer.writeInt("scale", scale))
                     return false;
 
                 writer.incrementState();
-
         }
 
         return true;
@@ -117,7 +109,7 @@ public class StatisticsResponse implements Message {
 
         switch (reader.state()) {
             case 0:
-                data = reader.readMessage("data");
+                b = reader.readByteArray("b");
 
                 if (!reader.isLastRead())
                     return false;
@@ -125,16 +117,15 @@ public class StatisticsResponse implements Message {
                 reader.incrementState();
 
             case 1:
-                reqId = reader.readUuid("reqId");
+                scale = reader.readInt("scale");
 
                 if (!reader.isLastRead())
                     return false;
 
                 reader.incrementState();
-
         }
 
-        return reader.afterMessageRead(StatisticsResponse.class);
+        return reader.afterMessageRead(StatisticsDecimalMessage.class);
     }
 
     /** {@inheritDoc} */
@@ -149,6 +140,11 @@ public class StatisticsResponse implements Message {
 
     /** {@inheritDoc} */
     @Override public void onAckReceived() {
+        // No-op.
+    }
 
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return Objects.toString(value());
     }
 }
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/messages/StatisticsKeyMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/messages/StatisticsKeyMessage.java
similarity index 99%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/messages/StatisticsKeyMessage.java
rename to modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/messages/StatisticsKeyMessage.java
index c20a5164a04..50e6752f0d8 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/messages/StatisticsKeyMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/messages/StatisticsKeyMessage.java
@@ -21,7 +21,6 @@ import java.io.Externalizable;
 import java.nio.ByteBuffer;
 import java.util.List;
 import java.util.Objects;
-
 import org.apache.ignite.internal.GridDirectCollection;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.plugin.extensions.communication.Message;
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/messages/StatisticsObjectData.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/messages/StatisticsObjectData.java
similarity index 99%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/messages/StatisticsObjectData.java
rename to modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/messages/StatisticsObjectData.java
index 95a0c173156..5e3ba7027d1 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/messages/StatisticsObjectData.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/messages/StatisticsObjectData.java
@@ -19,7 +19,6 @@ package org.apache.ignite.internal.processors.query.stat.messages;
 
 import java.nio.ByteBuffer;
 import java.util.Map;
-
 import org.apache.ignite.internal.GridDirectMap;
 import org.apache.ignite.internal.processors.query.stat.StatisticsType;
 import org.apache.ignite.plugin.extensions.communication.Message;
@@ -35,7 +34,7 @@ public class StatisticsObjectData implements Message {
     private static final long serialVersionUID = 0L;
 
     /** */
-    public static final short TYPE_CODE = 184;
+    public static final short TYPE_CODE = 185;
 
     /** Statistics key. */
     private StatisticsKeyMessage key;
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/messages/StatisticsRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/messages/StatisticsRequest.java
similarity index 99%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/messages/StatisticsRequest.java
rename to modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/messages/StatisticsRequest.java
index ea7c38ddcd7..a83c10695d7 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/messages/StatisticsRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/messages/StatisticsRequest.java
@@ -37,7 +37,7 @@ public class StatisticsRequest implements Message {
     private static final long serialVersionUID = 0L;
 
     /** */
-    public static final short TYPE_CODE = 186;
+    public static final short TYPE_CODE = 187;
 
     /** Gathering id. */
     private UUID reqId;
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/messages/StatisticsResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/messages/StatisticsResponse.java
similarity index 98%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/messages/StatisticsResponse.java
rename to modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/messages/StatisticsResponse.java
index fcf7063e335..b542f157d37 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/messages/StatisticsResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/messages/StatisticsResponse.java
@@ -32,7 +32,7 @@ public class StatisticsResponse implements Message {
     private static final long serialVersionUID = 0L;
 
     /** */
-    public static final short TYPE_CODE = 187;
+    public static final short TYPE_CODE = 188;
 
     /** Request id. */
     private UUID reqId;
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/task/GatherPartitionStatistics.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/task/GatherPartitionStatistics.java
similarity index 80%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/task/GatherPartitionStatistics.java
rename to modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/task/GatherPartitionStatistics.java
index b1f9878ddf2..b14d236aa78 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/task/GatherPartitionStatistics.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/task/GatherPartitionStatistics.java
@@ -17,10 +17,11 @@
 
 package org.apache.ignite.internal.processors.query.stat.task;
 
-import java.util.Arrays;
+import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.Callable;
@@ -30,15 +31,15 @@ import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.CacheObject;
 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.dht.topology.GridDhtPartitionTopology;
 import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+import org.apache.ignite.internal.processors.query.GridQueryRowDescriptor;
+import org.apache.ignite.internal.processors.query.GridQueryRowDescriptorImpl;
 import org.apache.ignite.internal.processors.query.GridQueryTypeDescriptor;
-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.opt.H2CacheRow;
-import org.apache.ignite.internal.processors.query.h2.opt.H2Row;
+import org.apache.ignite.internal.processors.query.QueryUtils;
 import org.apache.ignite.internal.processors.query.stat.ColumnStatistics;
 import org.apache.ignite.internal.processors.query.stat.ColumnStatisticsCollector;
 import org.apache.ignite.internal.processors.query.stat.GatherStatisticCancelException;
@@ -48,8 +49,8 @@ import org.apache.ignite.internal.processors.query.stat.LocalStatisticsGathering
 import org.apache.ignite.internal.processors.query.stat.ObjectPartitionStatisticsImpl;
 import org.apache.ignite.internal.processors.query.stat.config.StatisticsColumnConfiguration;
 import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.T2;
 import org.apache.ignite.internal.util.typedef.internal.U;
-import org.h2.table.Column;
 import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState.OWNING;
@@ -129,7 +130,8 @@ public class GatherPartitionStatistics implements Callable<ObjectPartitionStatis
         if (gathCtx.cancelled())
             throw new GatherStatisticCancelException();
 
-        GridCacheContext<?, ?> cctx = gathCtx.table().cacheContext();
+        GridCacheContext<?, ?> cctx = gathCtx.cacheContextInfo() != null ? gathCtx.cacheContextInfo().cacheContext()
+            : null;
 
         if (cctx == null || !(cctx.gate().enterIfNotStopped()))
             throw new GatherStatisticCancelException();
@@ -225,44 +227,46 @@ public class GatherPartitionStatistics implements Callable<ObjectPartitionStatis
 
         boolean reserved = locPart.reserve();
 
-        GridH2Table tbl = gathCtx.table();
+        GridQueryTypeDescriptor tbl = gathCtx.table();
 
-        ObjectPartitionStatisticsImpl res = null;
+        ObjectPartitionStatisticsImpl res;
 
         try {
             if (!reserved || (locPart.state() != OWNING)) {
                 if (log.isDebugEnabled()) {
                     log.debug("Partition not owning. Need to retry [part=" + partId +
-                        ", tbl=" + tbl.identifier() + ']');
+                        ", tbl=" + tbl.tableName() + ']');
                 }
 
                 throw new GatherStatisticCancelException();
             }
 
-            Column cols[] = IgniteStatisticsHelper.filterColumns(tbl.getColumns(), colsToCollect.keySet());
+            List<T2<Integer, String>> cols = IgniteStatisticsHelper.filterColumns(tbl, colsToCollect.keySet());
 
-            ColumnStatisticsCollector[] collectors = new ColumnStatisticsCollector[cols.length];
+            List<ColumnStatisticsCollector> collectors = new ArrayList<>();
 
-            for (int i = 0; i < cols.length; ++i) {
-                long colCfgVer = colsToCollect.get(cols[i].getName()).version();
+            for (T2<Integer, String> col: cols) {
+                Integer colId = col.getKey();
+                String colName = col.getValue();
 
-                collectors[i] = new ColumnStatisticsCollector(cols[i], tbl::compareTypeSafe, colCfgVer);
-            }
+                long colCfgVer = colsToCollect.get(colName).version();
+                Class<?> colCls = tbl.fields().get(colName);
 
-            GridH2RowDescriptor rowDesc = tbl.rowDescriptor();
-            GridQueryTypeDescriptor typeDesc = rowDesc.type();
+                collectors.add(new ColumnStatisticsCollector(colId, colName, colCls, colCfgVer));
+            }
 
             try {
                 int checkInt = CANCELLED_CHECK_INTERVAL;
 
                 if (log.isDebugEnabled()) {
                     log.debug("Start partition scan [part=" + partId +
-                        ", tbl=" + gathCtx.table().identifier() + ']');
+                        ", tbl=" + tbl.tableName() + ']');
                 }
 
-                for (CacheDataRow row : grp.offheap().cachePartitionIterator(
-                    gathCtx.table().cacheId(), partId, null, false)
-                ) {
+                GridQueryRowDescriptor rowDesc = new GridQueryRowDescriptorImpl(gathCtx.cacheContextInfo(), tbl);
+
+                for (CacheDataRow row : grp.offheap().cachePartitionIterator(gathCtx.cacheContextInfo().cacheId(), partId,
+                    null, false)) {
                     if (--checkInt == 0) {
                         if (gathCtx.future().isCancelled())
                             throw new GatherStatisticCancelException();
@@ -270,24 +274,22 @@ public class GatherPartitionStatistics implements Callable<ObjectPartitionStatis
                         checkInt = CANCELLED_CHECK_INTERVAL;
                     }
 
-                    if (!typeDesc.matchType(row.value()) || wasExpired(row))
+                    if (!tbl.matchType(row.value()) || wasExpired(row))
                         continue;
 
-                    H2Row h2row = new H2CacheRow(rowDesc, row);
-
                     for (ColumnStatisticsCollector colStat : collectors)
-                        colStat.add(h2row.getValue(colStat.col().getColumnId()));
+                        colStat.add(getValue(cctx, rowDesc, row, colStat));
                 }
             }
             catch (IgniteCheckedException e) {
                 log.warning(String.format("Unable to collect partition level statistics by %s.%s:%d due to %s",
-                    tbl.identifier().schema(), tbl.identifier().table(), partId, e.getMessage()));
+                    tbl.schemaName(), tbl.tableName(), partId, e.getMessage()));
 
                 throw new IgniteException("Unable to collect partition level statistics", e);
             }
 
-            Map<String, ColumnStatistics> colStats = Arrays.stream(collectors).collect(
-                Collectors.toMap(csc -> csc.col().getName(), ColumnStatisticsCollector::finish));
+            Map<String, ColumnStatistics> colStats = collectors.stream().collect(
+                Collectors.toMap(ColumnStatisticsCollector::columnName, ColumnStatisticsCollector::finish));
 
             // Add existing to full replace existing statistics with new one.
             if (partStat != null) {
@@ -318,6 +320,41 @@ public class GatherPartitionStatistics implements Callable<ObjectPartitionStatis
         return res;
     }
 
+    /**
+     * @param cctx Cache contex.
+     * @param desc Row descriptor.
+     * @param row Cache data row
+     * @param coll Column collector.
+     * @return IndexKey containing value extracted from row.
+     */
+    private Object getValue(
+        GridCacheContext<?, ?> cctx,
+        GridQueryRowDescriptor desc,
+        CacheDataRow row,
+        ColumnStatisticsCollector coll
+    ) {
+        if (desc.isKeyColumn(coll.columnId()))
+            return unwrap(cctx, row.key(), desc.type().keyClass());
+
+        if (desc.isValueColumn(coll.columnId()))
+            return unwrap(cctx, row.value(), desc.type().valueClass());
+
+        Object val = desc.getFieldValue(row.key(), row.value(), coll.columnId() - QueryUtils.DEFAULT_COLUMNS_COUNT);
+
+        return unwrap(cctx, val, coll.columnType());
+    }
+
+    /** */
+    private Object unwrap(GridCacheContext<?, ?> cctx, Object val, Class<?> cls) {
+        if (val == null)
+            return null;
+
+        if (val instanceof CacheObject && QueryUtils.isSqlType(cls))
+            return ((CacheObject)val).value(cctx.cacheObjectContext(), false);
+
+        return val;
+    }
+
     /**
      * Row count should be calculated as max(total) of existing columns.
      *
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/view/ColumnConfigurationViewSupplier.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/view/ColumnConfigurationViewSupplier.java
similarity index 100%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/view/ColumnConfigurationViewSupplier.java
rename to modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/view/ColumnConfigurationViewSupplier.java
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/view/ColumnLocalDataViewSupplier.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/view/ColumnLocalDataViewSupplier.java
similarity index 100%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/view/ColumnLocalDataViewSupplier.java
rename to modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/view/ColumnLocalDataViewSupplier.java
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/view/ColumnPartitionDataViewSupplier.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/view/ColumnPartitionDataViewSupplier.java
similarity index 100%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/view/ColumnPartitionDataViewSupplier.java
rename to modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/view/ColumnPartitionDataViewSupplier.java
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/view/StatisticsColumnConfigurationView.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/view/StatisticsColumnConfigurationView.java
similarity index 100%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/view/StatisticsColumnConfigurationView.java
rename to modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/view/StatisticsColumnConfigurationView.java
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/view/StatisticsColumnGlobalDataView.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/view/StatisticsColumnGlobalDataView.java
similarity index 100%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/view/StatisticsColumnGlobalDataView.java
rename to modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/view/StatisticsColumnGlobalDataView.java
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/view/StatisticsColumnLocalDataView.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/view/StatisticsColumnLocalDataView.java
similarity index 100%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/view/StatisticsColumnLocalDataView.java
rename to modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/view/StatisticsColumnLocalDataView.java
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/view/StatisticsColumnPartitionDataView.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/view/StatisticsColumnPartitionDataView.java
similarity index 100%
rename from modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/view/StatisticsColumnPartitionDataView.java
rename to modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/view/StatisticsColumnPartitionDataView.java
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 e58a5eb3ce2..43541370511 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,17 +35,23 @@ 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.stat.StatisticsKey;
+import org.apache.ignite.internal.processors.query.stat.StatisticsTarget;
+import org.apache.ignite.internal.processors.query.stat.config.StatisticsObjectConfiguration;
 import org.apache.ignite.internal.sql.command.SqlAlterTableCommand;
 import org.apache.ignite.internal.sql.command.SqlAlterUserCommand;
+import org.apache.ignite.internal.sql.command.SqlAnalyzeCommand;
 import org.apache.ignite.internal.sql.command.SqlCommand;
 import org.apache.ignite.internal.sql.command.SqlCreateIndexCommand;
 import org.apache.ignite.internal.sql.command.SqlCreateUserCommand;
 import org.apache.ignite.internal.sql.command.SqlDropIndexCommand;
+import org.apache.ignite.internal.sql.command.SqlDropStatisticsCommand;
 import org.apache.ignite.internal.sql.command.SqlDropUserCommand;
 import org.apache.ignite.internal.sql.command.SqlIndexColumn;
 import org.apache.ignite.internal.sql.command.SqlKillComputeTaskCommand;
@@ -54,7 +60,10 @@ import org.apache.ignite.internal.sql.command.SqlKillQueryCommand;
 import org.apache.ignite.internal.sql.command.SqlKillScanQueryCommand;
 import org.apache.ignite.internal.sql.command.SqlKillServiceCommand;
 import org.apache.ignite.internal.sql.command.SqlKillTransactionCommand;
+import org.apache.ignite.internal.sql.command.SqlRefreshStatitsicsCommand;
+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;
@@ -107,6 +116,12 @@ public class SqlCommandProcessor {
             processKillContinuousQueryCommand((SqlKillContinuousQueryCommand)cmdNative);
         else if (cmdNative instanceof SqlKillQueryCommand)
             processKillQueryCommand((SqlKillQueryCommand)cmdNative);
+        else if (cmdNative instanceof SqlAnalyzeCommand)
+            processAnalyzeCommand((SqlAnalyzeCommand)cmdNative);
+        else if (cmdNative instanceof SqlRefreshStatitsicsCommand)
+            processRefreshStatisticsCommand((SqlRefreshStatitsicsCommand)cmdNative);
+        else if (cmdNative instanceof SqlDropStatisticsCommand)
+            processDropStatisticsCommand((SqlDropStatisticsCommand)cmdNative);
 
         return null;
     }
@@ -126,7 +141,8 @@ public class SqlCommandProcessor {
             || cmd instanceof SqlKillTransactionCommand
             || cmd instanceof SqlKillScanQueryCommand
             || cmd instanceof SqlKillContinuousQueryCommand
-            || cmd instanceof SqlKillQueryCommand;
+            || cmd instanceof SqlKillQueryCommand
+            || cmd instanceof SqlStatisticsCommands;
     }
 
     /**
@@ -197,6 +213,80 @@ public class SqlCommandProcessor {
         new QueryMXBeanImpl(ctx).cancelContinuous(cmd.getOriginNodeId(), cmd.getRoutineId());
     }
 
+    /**
+     * Process analyze command.
+     *
+     * @param cmd Sql analyze command.
+     */
+    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) {
+                    StatisticsKey key = new StatisticsKey(cmd.schemaName(), t.key().obj());
+
+                    return new StatisticsObjectConfiguration(key, t.columns().values(),
+                        t.maxPartitionObsolescencePercent());
+                }
+                else
+                    return t;
+            }).toArray(StatisticsObjectConfiguration[]::new);
+
+        try {
+            indexing.statsManager().collectStatistics(objCfgs);
+        }
+        catch (IgniteCheckedException e) {
+            throw new IgniteSQLException(e.getMessage(), e);
+        }
+    }
+
+    /**
+     * Process refresh statistics command.
+     *
+     * @param cmd Refresh statistics command.
+     */
+    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);
+        }
+        catch (IgniteCheckedException e) {
+            throw new IgniteSQLException(e.getMessage(), e);
+        }
+    }
+
+    /**
+     * Process drop statistics command.
+     *
+     * @param cmd Drop statistics command.
+     */
+    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);
+        }
+        catch (IgniteCheckedException e) {
+            throw new IgniteSQLException(e.getMessage(), e);
+        }
+    }
+
     /**
      * Run DDL statement.
      *
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 01715a75670..3cbf8900e66 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
@@ -34,6 +34,7 @@ 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;
@@ -307,4 +308,9 @@ public class DummyQueryIndexing implements GridQueryIndexing {
     @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 85de47bacf0..3392d2f74e8 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
@@ -66,18 +66,11 @@ import org.apache.ignite.internal.processors.query.h2.sql.GridSqlDropIndex;
 import org.apache.ignite.internal.processors.query.h2.sql.GridSqlDropTable;
 import org.apache.ignite.internal.processors.query.h2.sql.GridSqlStatement;
 import org.apache.ignite.internal.processors.query.schema.SchemaOperationException;
-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;
 import org.apache.ignite.internal.sql.SqlCommandProcessor;
-import org.apache.ignite.internal.sql.command.SqlAnalyzeCommand;
 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.SqlDropStatisticsCommand;
-import org.apache.ignite.internal.sql.command.SqlKillQueryCommand;
-import org.apache.ignite.internal.sql.command.SqlRefreshStatitsicsCommand;
 import org.apache.ignite.internal.sql.command.SqlRollbackTransactionCommand;
 import org.apache.ignite.internal.sql.command.SqlSetStreamingCommand;
 import org.apache.ignite.internal.util.future.GridFinishedFuture;
@@ -194,12 +187,6 @@ public class CommandProcessor extends SqlCommandProcessor {
             return processBulkLoadCommand((SqlBulkLoadCommand)cmdNative, qryId);
         else if (cmdNative instanceof SqlSetStreamingCommand)
             processSetStreamingCommand((SqlSetStreamingCommand)cmdNative, cliCtx);
-        else if (cmdNative instanceof SqlAnalyzeCommand)
-            processAnalyzeCommand((SqlAnalyzeCommand)cmdNative);
-        else if (cmdNative instanceof SqlRefreshStatitsicsCommand)
-            processRefreshStatisticsCommand((SqlRefreshStatitsicsCommand)cmdNative);
-        else if (cmdNative instanceof SqlDropStatisticsCommand)
-            processDropStatisticsCommand((SqlDropStatisticsCommand)cmdNative);
         else
             processTxCommand(cmdNative, params);
 
@@ -216,74 +203,6 @@ public class CommandProcessor extends SqlCommandProcessor {
             || cmd instanceof SqlSetStreamingCommand;
     }
 
-    /**
-     * Process kill query command
-     *
-     * @param cmd Command.
-     */
-    private void processKillQueryCommand(SqlKillQueryCommand cmd) {
-        idx.runningQueryManager().cancelQuery(cmd.nodeQueryId(), cmd.nodeId(), cmd.async());
-    }
-
-    /**
-     * Process analyze command.
-     *
-     * @param cmd Sql analyze command.
-     */
-    private void processAnalyzeCommand(SqlAnalyzeCommand cmd) throws IgniteCheckedException {
-        ctx.security().authorize(SecurityPermission.CHANGE_STATISTICS);
-
-        IgniteH2Indexing indexing = (IgniteH2Indexing)ctx.query().getIndexing();
-
-        StatisticsObjectConfiguration objCfgs[] = cmd.configurations().stream()
-            .map(t -> {
-                if (t.key().schema() == null) {
-                    StatisticsKey key = new StatisticsKey(cmd.schemaName(), t.key().obj());
-
-                    return new StatisticsObjectConfiguration(key, t.columns().values(),
-                        t.maxPartitionObsolescencePercent());
-                }
-                else
-                    return t;
-            }).toArray(StatisticsObjectConfiguration[]::new);
-
-        indexing.statsManager().collectStatistics(objCfgs);
-    }
-
-    /**
-     * Process refresh statistics command.
-     *
-     * @param cmd Refresh statistics command.
-     */
-    private void processRefreshStatisticsCommand(SqlRefreshStatitsicsCommand cmd) throws IgniteCheckedException {
-        ctx.security().authorize(SecurityPermission.REFRESH_STATISTICS);
-
-        IgniteH2Indexing indexing = (IgniteH2Indexing)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);
-
-        indexing.statsManager().refreshStatistics(targets);
-    }
-
-    /**
-     * Process drop statistics command.
-     *
-     * @param cmd Drop statistics command.
-     */
-    private void processDropStatisticsCommand(SqlDropStatisticsCommand cmd) throws IgniteCheckedException {
-        ctx.security().authorize(SecurityPermission.CHANGE_STATISTICS);
-
-        IgniteH2Indexing indexing = (IgniteH2Indexing)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);
-
-        indexing.statsManager().dropStatistics(targets);
-    }
-
     /**
      * Execute DDL statement.
      *
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
index 5afe9e97155..7a5c824e60e 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
@@ -2139,10 +2139,10 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         parser = new QueryParser(this, connMgr, cmd -> cmdProc.isCommandSupported(cmd));
 
         schemaMgr = new SchemaManager(ctx, connMgr);
-        schemaMgr.start(ctx.config().getSqlConfiguration().getSqlSchemas());
-
         statsMgr = new IgniteStatisticsManagerImpl(ctx, schemaMgr);
 
+        schemaMgr.start(ctx.config().getSqlConfiguration().getSqlSchemas());
+
         nodeId = ctx.localNodeId();
         marshaller = ctx.config().getMarshaller();
 
@@ -3115,10 +3115,8 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         return map;
     }
 
-    /**
-     * @return Statistics manager.
-     */
-    public IgniteStatisticsManager statsManager() {
+    /** {@inheritDoc} */
+    @Override public IgniteStatisticsManager statsManager() {
         return statsMgr;
     }
 }
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/SchemaManager.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/SchemaManager.java
index 185daa0e5d2..2faadeb87a7 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/SchemaManager.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/SchemaManager.java
@@ -31,7 +31,6 @@ import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
-import java.util.function.BiConsumer;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
 import org.apache.ignite.IgniteCheckedException;
@@ -66,6 +65,7 @@ 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.SqlSystemView;
+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.util.GridConcurrentHashSet;
@@ -81,9 +81,7 @@ import org.apache.ignite.spi.systemview.view.SqlViewView;
 import org.apache.ignite.spi.systemview.view.SystemView;
 import org.h2.index.Index;
 import org.h2.table.Column;
-import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
-import static java.util.Objects.requireNonNull;
 import static org.apache.ignite.internal.processors.metric.impl.MetricUtils.metricName;
 
 /**
@@ -127,7 +125,7 @@ public class SchemaManager implements GridQuerySchemaManager {
     public static final String SQL_VIEW_COLS_VIEW_DESC = "SQL view columns";
 
     /** */
-    private final SchemaChangeListener lsnr;
+    private volatile SchemaChangeListener lsnr;
 
     /** Connection manager. */
     private final ConnectionManager connMgr;
@@ -153,12 +151,6 @@ public class SchemaManager implements GridQuerySchemaManager {
     /** Logger. */
     private final IgniteLogger log;
 
-    /** Drop column listeners. */
-    private final Set<BiConsumer<GridH2Table, List<String>>> dropColsLsnrs = ConcurrentHashMap.newKeySet();
-
-    /** Drop table listeners. */
-    private final Set<BiConsumer<String, String>> dropTblLsnrs = ConcurrentHashMap.newKeySet();
-
     /**
      * Constructor.
      *
@@ -169,7 +161,6 @@ public class SchemaManager implements GridQuerySchemaManager {
         this.ctx = ctx;
         this.connMgr = connMgr;
 
-        lsnr = schemaChangeListener(ctx);
         log = ctx.log(SchemaManager.class);
 
         ctx.systemView().registerView(SQL_SCHEMA_VIEW, SQL_SCHEMA_VIEW_DESC,
@@ -212,6 +203,9 @@ public class SchemaManager implements GridQuerySchemaManager {
      * @param schemaNames Schema names.
      */
     public void start(String[] schemaNames) throws IgniteCheckedException {
+        // Set schema change listener.
+        lsnr = schemaChangeListener(ctx);
+
         // Register PUBLIC schema which is always present.
         schemas.put(QueryUtils.DFLT_SCHEMA, new H2Schema(QueryUtils.DFLT_SCHEMA, true));
 
@@ -368,7 +362,7 @@ public class SchemaManager implements GridQuerySchemaManager {
                     tbl.table().setRemoveIndexOnDestroy(clearIdx);
 
                     dropTable(tbl, rmvIdx);
-                    lsnr.onSqlTypeDropped(schemaName, tbl.type());
+                    lsnr.onSqlTypeDropped(schemaName, tbl.type(), rmvIdx);
                 }
                 catch (Exception e) {
                     U.error(log, "Failed to drop table on cache stop (will ignore): " + tbl.fullTableName(), e);
@@ -630,9 +624,6 @@ public class SchemaManager implements GridQuerySchemaManager {
                     log.debug("Dropping database index table with SQL: " + sql);
 
                 stmt.executeUpdate(sql);
-
-                if (destroy)
-                    afterDropTable(tbl.schemaName(), tbl.tableName());
             }
             catch (SQLException e) {
                 throw new IgniteSQLException("Failed to drop database index table [type=" + tbl.type().name() +
@@ -824,7 +815,7 @@ public class SchemaManager implements GridQuerySchemaManager {
 
         desc.table().addColumns(cols, ifColNotExists);
 
-        lsnr.onSqlTypeUpdated(schemaName, desc.type(), desc.table().cacheInfo());
+        lsnr.onColumnsAdded(schemaName, desc.type(), desc.cacheInfo(), cols);
     }
 
     /**
@@ -854,9 +845,7 @@ public class SchemaManager implements GridQuerySchemaManager {
 
         desc.table().dropColumns(cols, ifColExists);
 
-        lsnr.onSqlTypeUpdated(schemaName, desc.type(), desc.table().cacheInfo());
-
-        dropColsLsnrs.forEach(l -> l.accept(desc.table(), cols));
+        lsnr.onColumnsDropped(schemaName, desc.type(), desc.cacheInfo(), cols);
     }
 
     /**
@@ -994,48 +983,7 @@ public class SchemaManager implements GridQuerySchemaManager {
     }
 
     /** */
-    private static final class NoOpSchemaChangeListener implements SchemaChangeListener {
-        /** {@inheritDoc} */
-        @Override public void onSchemaCreated(String schemaName) {}
-
-        /** {@inheritDoc} */
-        @Override public void onSchemaDropped(String schemaName) {}
-
-        /** {@inheritDoc} */
-        @Override public void onIndexCreated(String schemaName, String tblName, String idxName,
-            GridQueryIndexDescriptor idxDesc, org.apache.ignite.internal.cache.query.index.Index idx) {}
-
-        /** {@inheritDoc} */
-        @Override public void onIndexDropped(String schemaName, String tblName, String idxName) {}
-
-        /** {@inheritDoc} */
-        @Override public void onIndexRebuildStarted(String schemaName, String tblName) {}
-
-        /** {@inheritDoc} */
-        @Override public void onIndexRebuildFinished(String schemaName, String tblName) {}
-
-        /** {@inheritDoc} */
-        @Override public void onSqlTypeCreated(
-            String schemaName,
-            GridQueryTypeDescriptor typeDesc,
-            GridCacheContextInfo<?, ?> cacheInfo
-        ) {}
-
-        /** {@inheritDoc} */
-        @Override public void onSqlTypeUpdated(
-            String schemaName,
-            GridQueryTypeDescriptor typeDesc,
-            GridCacheContextInfo<?, ?> cacheInfo
-        ) {}
-
-        /** {@inheritDoc} */
-        @Override public void onSqlTypeDropped(String schemaName, GridQueryTypeDescriptor typeDescriptor) {}
-
-        /** {@inheritDoc} */
-        @Override public void onFunctionCreated(String schemaName, String name, Method method) {}
-
-        /** {@inheritDoc} */
-        @Override public void onSystemViewCreated(String schemaName, SystemView<?> sysView) {}
+    private static final class NoOpSchemaChangeListener extends AbstractSchemaChangeListener {
     }
 
     /** */
@@ -1080,19 +1028,36 @@ public class SchemaManager implements GridQuerySchemaManager {
         /**
          * {@inheritDoc}
          */
-        @Override public void onSqlTypeUpdated(
+        @Override public void onColumnsAdded(
             String schemaName,
             GridQueryTypeDescriptor typeDesc,
-            GridCacheContextInfo<?, ?> cacheInfo
+            GridCacheContextInfo<?, ?> cacheInfo,
+            List<QueryField> cols
+        ) {
+            lsnrs.forEach(lsnr -> lsnr.onColumnsAdded(schemaName, typeDesc, cacheInfo, cols));
+        }
+
+        /**
+         * {@inheritDoc}
+         */
+        @Override public void onColumnsDropped(
+            String schemaName,
+            GridQueryTypeDescriptor typeDesc,
+            GridCacheContextInfo<?, ?> cacheInfo,
+            List<String> cols
         ) {
-            lsnrs.forEach(lsnr -> lsnr.onSqlTypeUpdated(schemaName, typeDesc, cacheInfo));
+            lsnrs.forEach(lsnr -> lsnr.onColumnsDropped(schemaName, typeDesc, cacheInfo, cols));
         }
 
         /**
          * {@inheritDoc}
          */
-        @Override public void onSqlTypeDropped(String schemaName, GridQueryTypeDescriptor typeDescriptor) {
-            lsnrs.forEach(lsnr -> lsnr.onSqlTypeDropped(schemaName, typeDescriptor));
+        @Override public void onSqlTypeDropped(
+            String schemaName,
+            GridQueryTypeDescriptor typeDescriptor,
+            boolean destroy
+        ) {
+            lsnrs.forEach(lsnr -> lsnr.onSqlTypeDropped(schemaName, typeDescriptor, destroy));
         }
 
         /**
@@ -1130,58 +1095,4 @@ public class SchemaManager implements GridQuerySchemaManager {
             lsnrs.forEach(lsnr -> lsnr.onSystemViewCreated(schemaName, sysView));
         }
     }
-
-    /**
-     * Register listener for drop columns event.
-     *
-     * @param lsnr Drop columns event listener.
-     */
-    public void registerDropColumnsListener(@NotNull BiConsumer<GridH2Table, List<String>> lsnr) {
-        requireNonNull(lsnr, "Drop columns listener should be not-null.");
-
-        dropColsLsnrs.add(lsnr);
-    }
-
-    /**
-     * Unregister listener for drop columns event.
-     *
-     * @param lsnr Drop columns event listener.
-     */
-    public void unregisterDropColumnsListener(@NotNull BiConsumer<GridH2Table, List<String>> lsnr) {
-        requireNonNull(lsnr, "Drop columns listener should be not-null.");
-
-        dropColsLsnrs.remove(lsnr);
-    }
-
-    /**
-     * Register listener for drop table event.
-     *
-     * @param lsnr Drop table event listener.
-     */
-    public void registerDropTableListener(@NotNull BiConsumer<String, String> lsnr) {
-        requireNonNull(lsnr, "Drop table listener should be not-null.");
-
-        dropTblLsnrs.add(lsnr);
-    }
-
-    /**
-     * Unregister listener for drop table event.
-     *
-     * @param lsnr Drop table event listener.
-     */
-    public void unregisterDropTableListener(@NotNull BiConsumer<String, String> lsnr) {
-        requireNonNull(lsnr, "Drop table listener should be not-null.");
-
-        dropTblLsnrs.remove(lsnr);
-    }
-
-    /**
-     * Fire each listener after table drop.
-     *
-     * @param schema Dropped table schema.
-     * @param tblName Dropped table name.
-     */
-    private void afterDropTable(String schema, String tblName) {
-        dropTblLsnrs.forEach(l -> l.accept(schema, tblName));
-    }
 }
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/H2IndexCostedBase.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/H2IndexCostedBase.java
index d897beab320..13524982acf 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/H2IndexCostedBase.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/H2IndexCostedBase.java
@@ -22,7 +22,6 @@ import java.math.BigInteger;
 import java.math.MathContext;
 import java.util.ArrayList;
 import java.util.HashSet;
-
 import org.apache.ignite.internal.processors.query.stat.ColumnStatistics;
 import org.apache.ignite.internal.processors.query.stat.ObjectStatisticsImpl;
 import org.apache.ignite.internal.util.typedef.F;
@@ -375,8 +374,8 @@ public abstract class H2IndexCostedBase extends BaseIndex {
         if (minValue == null && maxValue == null)
             return estimatePercentFallback(min, max);
 
-        BigDecimal minStat = getComparableValue(colStat.min());
-        BigDecimal maxStat = getComparableValue(colStat.max());
+        BigDecimal minStat = colStat.min();
+        BigDecimal maxStat = colStat.max();
 
         if (minStat == null || maxStat == null)
             return estimatePercentFallback(min, max);
@@ -429,7 +428,7 @@ public abstract class H2IndexCostedBase extends BaseIndex {
                 throw new IllegalArgumentException("Can't compare null values");
 
             case Value.BOOLEAN:
-                return new BigDecimal(value.getBoolean() ? 1 : 0);
+                return value.getBoolean() ? BigDecimal.ONE : BigDecimal.ZERO;
 
             case Value.BYTE:
                 return new BigDecimal(value.getByte());
@@ -447,10 +446,10 @@ public abstract class H2IndexCostedBase extends BaseIndex {
                 return value.getBigDecimal();
 
             case Value.DOUBLE:
-                return new BigDecimal(value.getDouble());
+                return BigDecimal.valueOf(value.getDouble());
 
             case Value.FLOAT:
-                return new BigDecimal(value.getFloat());
+                return BigDecimal.valueOf(value.getFloat());
 
             case Value.DATE:
                 return new BigDecimal(value.getDate().getTime());
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2ValueMessageFactory.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2ValueMessageFactory.java
index 867af779599..ff955ec94fe 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2ValueMessageFactory.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/msg/GridH2ValueMessageFactory.java
@@ -23,11 +23,6 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.processors.query.h2.QueryTable;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2ValueCacheObject;
-import org.apache.ignite.internal.processors.query.stat.messages.StatisticsColumnData;
-import org.apache.ignite.internal.processors.query.stat.messages.StatisticsKeyMessage;
-import org.apache.ignite.internal.processors.query.stat.messages.StatisticsObjectData;
-import org.apache.ignite.internal.processors.query.stat.messages.StatisticsRequest;
-import org.apache.ignite.internal.processors.query.stat.messages.StatisticsResponse;
 import org.apache.ignite.plugin.extensions.communication.IgniteMessageFactory;
 import org.apache.ignite.plugin.extensions.communication.Message;
 import org.apache.ignite.plugin.extensions.communication.MessageFactoryProvider;
@@ -69,12 +64,6 @@ public class GridH2ValueMessageFactory implements MessageFactoryProvider {
         factory.register((short)-55, GridH2DmlRequest::new);
         factory.register((short)-56, GridH2DmlResponse::new);
         factory.register((short)-57, GridH2SelectForUpdateTxDetails::new);
-
-        factory.register(StatisticsKeyMessage.TYPE_CODE, StatisticsKeyMessage::new);
-        factory.register(StatisticsObjectData.TYPE_CODE, StatisticsObjectData::new);
-        factory.register(StatisticsColumnData.TYPE_CODE, StatisticsColumnData::new);
-        factory.register(StatisticsRequest.TYPE_CODE, StatisticsRequest::new);
-        factory.register(StatisticsResponse.TYPE_CODE, StatisticsResponse::new);
     }
 
     /** {@inheritDoc} */
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/ColumnStatisticsCollector.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/ColumnStatisticsCollector.java
deleted file mode 100644
index 582c8d1c78f..00000000000
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/ColumnStatisticsCollector.java
+++ /dev/null
@@ -1,274 +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.stat;
-
-import java.math.BigInteger;
-import java.nio.charset.StandardCharsets;
-import java.util.Comparator;
-import java.util.List;
-
-import org.apache.ignite.internal.processors.query.stat.config.StatisticsColumnOverrides;
-import org.apache.ignite.internal.processors.query.stat.hll.HLL;
-import org.apache.ignite.internal.util.typedef.F;
-import org.apache.ignite.internal.util.typedef.internal.U;
-import org.h2.table.Column;
-import org.h2.value.Value;
-
-/**
- * Collector to compute statistic by single column.
- */
-public class ColumnStatisticsCollector {
-    /** Column. */
-    private final Column col;
-
-    /** Hyper Log Log structure */
-    private final HLL hll = buildHll();
-
-    /** Minimum value. */
-    private Value min = null;
-
-    /** Maximum value. */
-    private Value max = null;
-
-    /** Total values in column. */
-    private long total = 0;
-
-    /** Total size of all non nulls values (in bytes).*/
-    private long size = 0;
-
-    /** Column value comparator. */
-    private final Comparator<Value> comp;
-
-    /** Null values counter. */
-    private long nullsCnt;
-
-    /** Is column has complex type. */
-    private final boolean complexType;
-
-    /** Hasher. */
-    private final Hasher hash = new Hasher();
-
-    /** Version. */
-    private final long ver;
-
-    /**
-     * Constructor.
-     *
-     * @param col Column to collect statistics by.
-     * @param comp Column values comparator.
-     */
-    public ColumnStatisticsCollector(Column col, Comparator<Value> comp) {
-        this(col, comp, 0);
-    }
-
-    /**
-     * Constructor.
-     *
-     * @param col Column to collect statistics by.
-     * @param comp Column values comparator.
-     * @param ver Target statistic version.
-     */
-    public ColumnStatisticsCollector(Column col, Comparator<Value> comp, long ver) {
-        this.col = col;
-        this.comp = comp;
-        this.ver = ver;
-
-        int colType = col.getType();
-        complexType = colType == Value.ARRAY || colType == Value.ENUM || colType == Value.JAVA_OBJECT ||
-            colType == Value.RESULT_SET || colType == Value.UNKNOWN;
-    }
-
-    /**
-     * Try to fix unexpected behaviour of base Value class.
-     *
-     * @param val Value to convert.
-     * @return Byte array.
-     */
-    private byte[] getBytes(Value val) {
-        switch (val.getType()) {
-            case Value.STRING:
-                String strVal = val.getString();
-                return strVal.getBytes(StandardCharsets.UTF_8);
-            case Value.BOOLEAN:
-                return val.getBoolean() ? new byte[]{1} : new byte[]{0};
-            case Value.DECIMAL:
-            case Value.DOUBLE:
-            case Value.FLOAT:
-                return U.join(val.getBigDecimal().unscaledValue().toByteArray(),
-                    BigInteger.valueOf(val.getBigDecimal().scale()).toByteArray());
-            case Value.TIME:
-                return BigInteger.valueOf(val.getTime().getTime()).toByteArray();
-            case Value.DATE:
-                return BigInteger.valueOf(val.getDate().getTime()).toByteArray();
-            case Value.TIMESTAMP:
-                return BigInteger.valueOf(val.getTimestamp().getTime()).toByteArray();
-            default:
-                return val.getBytes();
-        }
-    }
-
-    /**
-     * Add value to statistics.
-     *
-     * @param val Value to add to statistics.
-     */
-    public void add(Value val) {
-        total++;
-
-        if (isNullValue(val)) {
-            nullsCnt++;
-
-            return;
-        }
-
-        byte[] bytes = getBytes(val);
-        size += bytes.length;
-
-        hll.addRaw(hash.fastHash(bytes));
-
-        if (!complexType) {
-            if (null == min || comp.compare(val, min) < 0)
-                min = val;
-
-            if (null == max || comp.compare(val, max) > 0)
-                max = val;
-        }
-    }
-
-    /**
-     * Get total column statistics.
-     *
-     * @return Aggregated column statistics.
-     */
-    public ColumnStatistics finish() {
-
-        int averageSize = averageSize(size, total, nullsCnt);
-
-        return new ColumnStatistics(min, max, nullsCnt, hll.cardinality(), total, averageSize, hll.toBytes(), ver,
-            U.currentTimeMillis());
-    }
-
-    /**
-     * Calculate average record size in bytes.
-     *
-     * @param size Total size of all records.
-     * @param total Total number of all records.
-     * @param nullsCnt Number of nulls record.
-     * @return Average size of not null record in byte.
-     */
-    private static int averageSize(long size, long total, long nullsCnt) {
-        long averageSizeLong = (total - nullsCnt > 0) ? (size / (total - nullsCnt)) : 0;
-        return (averageSizeLong > Integer.MAX_VALUE) ? Integer.MAX_VALUE : (int)averageSizeLong;
-    }
-
-    /**
-     * @return get column.
-     */
-    public Column col() {
-        return col;
-    }
-
-    /**
-     * Aggregate specified (partition or local) column statistics into (local or global) single one.
-     *
-     * @param comp Value comparator.
-     * @param partStats Column statistics by partitions.
-     * @param overrides Overrides or {@code null} to keep calculated values.
-     * @return Column statistics for all partitions.
-     */
-    public static ColumnStatistics aggregate(
-        Comparator<Value> comp,
-        List<ColumnStatistics> partStats,
-        StatisticsColumnOverrides overrides
-    ) {
-        assert !F.isEmpty(partStats);
-
-        Long overrideDistinct = (overrides == null) ? null : overrides.distinct();
-        HLL hll = buildHll();
-
-        Value min = null;
-        Value max = null;
-
-        // Total number of nulls
-        long nullsCnt = 0;
-
-        // Total values (null and not null) counter)
-        long total = 0;
-
-        // Total size in bytes
-        long totalSize = 0;
-
-        ColumnStatistics firstStat = F.first(partStats);
-        long ver = firstStat.version();
-        long createdAt = firstStat.createdAt();
-
-        for (ColumnStatistics partStat : partStats) {
-            assert ver == partStat.version() : "Aggregate statistics with different version [stats=" + partStats + ']';
-
-            if (overrideDistinct == null) {
-                HLL partHll = HLL.fromBytes(partStat.raw());
-                hll.union(partHll);
-            }
-
-            total += partStat.total();
-            nullsCnt += partStat.nulls();
-            totalSize += (long)partStat.size() * (partStat.total() - partStat.nulls());
-
-            if (min == null || (partStat.min() != null && comp.compare(partStat.min(), min) < 0))
-                min = partStat.min();
-
-            if (max == null || (partStat.max() != null && comp.compare(partStat.max(), max) > 0))
-                max = partStat.max();
-
-            if (createdAt < partStat.createdAt())
-                createdAt = partStat.createdAt();
-        }
-
-        Integer overrideSize = (overrides == null) ? null : overrides.size();
-        int averageSize = (overrideSize == null) ? averageSize(totalSize, total, nullsCnt) : overrideSize;
-
-        long distinct = (overrideDistinct == null) ? hll.cardinality() : overrideDistinct;
-
-        Long overrideNulls = (overrides == null) ? null : overrides.nulls();
-        long nulls = (overrideNulls == null) ? nullsCnt : overrideNulls;
-
-        Long overrideTotal = (overrides == null) ? null : overrides.total();
-        total = (overrideTotal == null) ? total : overrideTotal;
-
-        return new ColumnStatistics(min, max, nulls, distinct, total, averageSize, hll.toBytes(), ver, createdAt);
-    }
-
-    /**
-     * Get HLL with default params.
-     *
-     * @return Empty hll structure.
-     */
-    private static HLL buildHll() {
-        return new HLL(13, 5);
-    }
-
-    /**
-     * Test if specified value is null.
-     *
-     * @param v Value to test.
-     * @return {@code true} if value is null, {@code false} - otherwise.
-     */
-    public static boolean isNullValue(Value v) {
-        return v == null || v.getType() == Value.NULL;
-    }
-}
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/stat/ColumnStatisticsCollectorAggregationTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/stat/ColumnStatisticsCollectorAggregationTest.java
index e6a1fa9393f..e743c60c4dd 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/stat/ColumnStatisticsCollectorAggregationTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/stat/ColumnStatisticsCollectorAggregationTest.java
@@ -19,26 +19,18 @@ package org.apache.ignite.internal.processors.query.stat;
 
 import java.math.BigDecimal;
 import java.util.ArrayList;
-import java.util.Comparator;
 import java.util.List;
 import java.util.Random;
 import java.util.concurrent.ThreadLocalRandom;
-
 import org.apache.ignite.internal.processors.query.stat.hll.HLL;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-import org.h2.value.Value;
-import org.h2.value.ValueDecimal;
 import org.junit.Test;
 
 /**
  * Test different scenarious with column statistics aggregation.
  */
 public class ColumnStatisticsCollectorAggregationTest extends GridCommonAbstractTest {
-    /** Decimal comparator. */
-    private static final Comparator<Value> DECIMAL_VALUE_COMPARATOR = (v1, v2) ->
-        v1.getBigDecimal().compareTo(v2.getBigDecimal());
-
     /**
      * Aggregate single column statistics object.
      * Test that aggregated object are the same as original.
@@ -50,7 +42,7 @@ public class ColumnStatisticsCollectorAggregationTest extends GridCommonAbstract
             getHLL(-1).toBytes(), 0, U.currentTimeMillis());
         statistics.add(stat1);
 
-        ColumnStatistics res = ColumnStatisticsCollector.aggregate(DECIMAL_VALUE_COMPARATOR, statistics, null);
+        ColumnStatistics res = ColumnStatisticsCollector.aggregate(statistics, null);
 
         assertEquals(stat1, res);
     }
@@ -70,7 +62,7 @@ public class ColumnStatisticsCollectorAggregationTest extends GridCommonAbstract
         statistics.add(stat1);
         statistics.add(stat2);
 
-        ColumnStatistics res = ColumnStatisticsCollector.aggregate(DECIMAL_VALUE_COMPARATOR, statistics, null);
+        ColumnStatistics res = ColumnStatisticsCollector.aggregate(statistics, null);
 
         assertNull(res.min());
         assertNull(res.max());
@@ -88,18 +80,18 @@ public class ColumnStatisticsCollectorAggregationTest extends GridCommonAbstract
     @Test
     public void aggregateTest() {
         List<ColumnStatistics> statistics = new ArrayList<>();
-        ColumnStatistics stat1 = new ColumnStatistics(ValueDecimal.get(BigDecimal.ONE), ValueDecimal.get(BigDecimal.TEN),
-            50, 10, 1000, 0, getHLL(50).toBytes(), 0, U.currentTimeMillis());
-        ColumnStatistics stat2 = new ColumnStatistics(ValueDecimal.get(BigDecimal.ZERO), ValueDecimal.get(BigDecimal.ONE),
-            10, 100, 10, 0, getHLL(9).toBytes(), 0, U.currentTimeMillis());
+        ColumnStatistics stat1 = new ColumnStatistics(BigDecimal.ONE, BigDecimal.TEN, 50, 10, 1000, 0,
+            getHLL(50).toBytes(), 0, U.currentTimeMillis());
+        ColumnStatistics stat2 = new ColumnStatistics(BigDecimal.ZERO, BigDecimal.ONE, 10, 100, 10, 0,
+            getHLL(9).toBytes(), 0, U.currentTimeMillis());
 
         statistics.add(stat1);
         statistics.add(stat2);
 
-        ColumnStatistics res = ColumnStatisticsCollector.aggregate(DECIMAL_VALUE_COMPARATOR, statistics, null);
+        ColumnStatistics res = ColumnStatisticsCollector.aggregate(statistics, null);
 
-        assertEquals(ValueDecimal.get(BigDecimal.ZERO), res.min());
-        assertEquals(ValueDecimal.get(BigDecimal.TEN), res.max());
+        assertEquals(BigDecimal.ZERO, res.min());
+        assertEquals(BigDecimal.TEN, res.max());
         assertEquals(60, res.nulls());
         assertEquals(59, res.distinct());
         assertEquals(1010, res.total());
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/stat/ColumnStatisticsCollectorTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/stat/ColumnStatisticsCollectorTest.java
index ca002f7c9d8..e845e20e2da 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/stat/ColumnStatisticsCollectorTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/stat/ColumnStatisticsCollectorTest.java
@@ -18,27 +18,14 @@
 package org.apache.ignite.internal.processors.query.stat;
 
 import java.math.BigDecimal;
+import java.time.LocalDate;
+import java.time.Month;
 import java.util.Arrays;
-import java.util.Comparator;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
 import java.util.UUID;
-
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-import org.h2.table.Column;
-import org.h2.value.Value;
-import org.h2.value.ValueBoolean;
-import org.h2.value.ValueByte;
-import org.h2.value.ValueDate;
-import org.h2.value.ValueDecimal;
-import org.h2.value.ValueDouble;
-import org.h2.value.ValueFloat;
-import org.h2.value.ValueInt;
-import org.h2.value.ValueNull;
-import org.h2.value.ValueShort;
-import org.h2.value.ValueString;
-import org.h2.value.ValueUuid;
 import org.junit.Test;
 
 /**
@@ -46,30 +33,26 @@ import org.junit.Test;
  */
 public class ColumnStatisticsCollectorTest extends GridCommonAbstractTest {
     /** Types with its comparators for tests.  */
-    private static final Map<Value[], Comparator<Value>> types = new HashMap<>();
+    private static final Map<Class<?>, Object[]> types = new HashMap<>();
+
+    /** */
+    private static final Object[] ZERO_ARR = new Object[0];
 
     static {
-        types.put(new Value[]{ValueBoolean.get(false), ValueBoolean.get(true)},
-            (v1, v2) -> Boolean.compare(v1.getBoolean(), v2.getBoolean()));
-        types.put(new Value[]{ValueInt.get(1), ValueInt.get(2), ValueInt.get(10)},
-            (v1, v2) -> Integer.compare(v1.getInt(), v2.getInt()));
-        types.put(new Value[]{ValueShort.get((short)1), ValueShort.get((short)3)},
-            (v1, v2) -> Short.compare(v1.getShort(), v2.getShort()));
-        types.put(new Value[]{ValueString.get("1"), ValueString.get("9")},
-            (v1, v2) -> v1.getString().compareTo(v2.getString()));
-        types.put(new Value[]{ValueDecimal.get(BigDecimal.ONE), ValueDecimal.get(BigDecimal.TEN)},
-            (v1, v2) -> v1.getBigDecimal().compareTo(v2.getBigDecimal()));
-        types.put(new Value[]{ValueDate.fromMillis(1), ValueDate.fromMillis(10000), ValueDate.fromMillis(9999999)},
-            (v1, v2) -> v1.getDate().compareTo(v2.getDate()));
-        types.put(new Value[]{ValueUuid.get(1, 2), ValueUuid.get(2, 1), ValueUuid.get(2, 2)},
-            (v1, v2) -> new UUID(((ValueUuid)v1).getHigh(), ((ValueUuid)v1).getLow())
-                .compareTo(new UUID(((ValueUuid)v2).getHigh(), ((ValueUuid)v2).getLow())));
-        types.put(new Value[]{ValueFloat.get(1f), ValueFloat.get(10f)},
-            (v1, v2) -> Float.compare(v1.getFloat(), v2.getFloat()));
-        types.put(new Value[]{ValueDouble.get(1.), ValueDouble.get(10.)},
-            (v1, v2) -> Double.compare(v1.getDouble(), v2.getDouble()));
-        types.put(new Value[]{ValueByte.get((byte)1), ValueByte.get((byte)2)},
-            (v1, v2) -> Byte.compare(v1.getByte(), v2.getByte()));
+        types.put(Boolean.class, new Object[]{false, true});
+        types.put(Integer.class, new Object[]{1, 2, 10});
+        types.put(Short.class, new Object[]{(short)1, (short)3});
+        types.put(String.class, new Object[]{"1", "9"});
+        types.put(BigDecimal.class, new Object[]{BigDecimal.ONE, BigDecimal.TEN});
+        types.put(LocalDate.class, new Object[]{
+            LocalDate.of(1945, Month.MAY, 9),
+            LocalDate.of(1957, Month.OCTOBER, 4),
+            LocalDate.of(1961, Month.APRIL, 12),
+        });
+        types.put(UUID.class, new Object[]{new UUID(1, 2), new UUID(2, 1), new UUID(2, 2)});
+        types.put(Float.class, new Object[]{1f, 10f});
+        types.put(Double.class, new Object[]{1., 10.});
+        types.put(Byte.class, new Object[]{(byte)1, (byte)2});
     }
 
     /**
@@ -77,10 +60,9 @@ public class ColumnStatisticsCollectorTest extends GridCommonAbstractTest {
      * Check that statistics collected properly.
      */
     @Test
-    public void testZeroAggregation() {
-        Value[] zeroArr = new Value[0];
-        for (Map.Entry<Value[], Comparator<Value>> type : types.entrySet())
-            testAggregation(type.getValue(), type.getKey()[0].getType(), 0, zeroArr);
+    public void testZeroAggregation() throws Exception {
+        for (Map.Entry<Class<?>, Object[]> tv: types.entrySet())
+            testAggregation(tv.getKey(), 0, ZERO_ARR);
     }
 
     /**
@@ -88,9 +70,9 @@ public class ColumnStatisticsCollectorTest extends GridCommonAbstractTest {
      * Check that statistics collected properly.
      */
     @Test
-    public void testSingleNullAggregation() {
-        for (Map.Entry<Value[], Comparator<Value>> type : types.entrySet())
-            testAggregation(type.getValue(), type.getKey()[0].getType(), 1);
+    public void testSingleNullAggregation() throws Exception {
+        for (Map.Entry<Class<?>, Object[]> tv: types.entrySet())
+            testAggregation(tv.getKey(), 1);
     }
 
     /**
@@ -98,10 +80,9 @@ public class ColumnStatisticsCollectorTest extends GridCommonAbstractTest {
      * Check that statistics collected properly.
      */
     @Test
-    public void testMultipleNullsAggregation() {
-        Value[] zeroArr = new Value[0];
-        for (Map.Entry<Value[], Comparator<Value>> type : types.entrySet())
-            testAggregation(type.getValue(), type.getKey()[0].getType(), 1000, zeroArr);
+    public void testMultipleNullsAggregation() throws Exception {
+        for (Map.Entry<Class<?>, Object[]> tv: types.entrySet())
+            testAggregation(tv.getKey(), 1000, ZERO_ARR);
     }
 
     /**
@@ -109,10 +90,10 @@ public class ColumnStatisticsCollectorTest extends GridCommonAbstractTest {
      * Check that statistics collected properly.
      */
     @Test
-    public void testSingleAggregation() {
-        for (Map.Entry<Value[], Comparator<Value>> type : types.entrySet()) {
-            for (Value v : type.getKey())
-                testAggregation(type.getValue(), v.getType(), 0, v);
+    public void testSingleAggregation() throws Exception {
+        for (Map.Entry<Class<?>, Object[]> tv: types.entrySet()) {
+            for (Object val : tv.getValue())
+                testAggregation(tv.getKey(), 0, val);
         }
     }
 
@@ -121,11 +102,9 @@ public class ColumnStatisticsCollectorTest extends GridCommonAbstractTest {
      * Check that statistics collected properly.
      */
     @Test
-    public void testMultipleAggregation() {
-        for (Map.Entry<Value[], Comparator<Value>> type : types.entrySet()) {
-            Value[] vals = type.getKey();
-            testAggregation(type.getValue(), vals[0].getType(), 0, vals);
-        }
+    public void testMultipleAggregation() throws Exception {
+        for (Map.Entry<Class<?>, Object[]> tv: types.entrySet())
+            testAggregation(tv.getKey(), 0, tv.getValue());
     }
 
     /**
@@ -133,59 +112,55 @@ public class ColumnStatisticsCollectorTest extends GridCommonAbstractTest {
      * Check that statistics collected properly.
      */
     @Test
-    public void testMultipleWithNullsAggregation() {
-        for (Map.Entry<Value[], Comparator<Value>> type : types.entrySet()) {
-            Value[] vals = type.getKey();
-            testAggregation(type.getValue(), vals[0].getType(), vals.length, vals);
-        }
+    public void testMultipleWithNullsAggregation() throws Exception {
+        for (Map.Entry<Class<?>, Object[]> tv: types.entrySet())
+            testAggregation(tv.getKey(), tv.getValue().length, tv.getValue());
     }
 
     /**
      * Test aggregation with specified values.
      * Check that statistics collected properly.
      *
-     * @param comp Value comparator.
      * @param type Value type.
      * @param nulls Nulls count.
      * @param vals Values to aggregate where the first one is the smallest and the last one is the biggest one.
      */
-    private static void testAggregation(Comparator<Value> comp, int type, int nulls, Value... vals) {
-        Column intCol = new Column("test", type);
-
-        ColumnStatisticsCollector collector = new ColumnStatisticsCollector(intCol, comp);
-        ColumnStatisticsCollector collectorInverted = new ColumnStatisticsCollector(intCol, comp);
+    private static void testAggregation(Class<?> type, int nulls, Object... vals) throws Exception {
+        ColumnStatisticsCollector collector = new ColumnStatisticsCollector(0, "test", type);
+        ColumnStatisticsCollector collectorInverted = new ColumnStatisticsCollector(0, "test", type);
 
         for (int i = 0; i < vals.length; i++) {
             collector.add(vals[i]);
             collectorInverted.add(vals[vals.length - 1 - i]);
         }
         for (int i = 0; i < nulls; i++) {
-            collector.add(ValueNull.INSTANCE);
-            collectorInverted.add(ValueNull.INSTANCE);
+            collector.add(null);
+            collectorInverted.add(null);
         }
 
         ColumnStatistics res = collector.finish();
         ColumnStatistics resInverted = collectorInverted.finish();
 
-        testAggregationResult(res, nulls, vals);
-        testAggregationResult(resInverted, nulls, vals);
+        testAggregationResult(type, res, nulls, vals);
+        testAggregationResult(type, resInverted, nulls, vals);
     }
 
     /**
      * Check column statistics collection results.
      *
+     * @param type Field type.
      * @param res Column statistics to test.
      * @param nulls Count of null values in statistics.
      * @param vals Values included into statistics where first one is the smallest one and the last one is the biggest.
      */
-    private static void testAggregationResult(ColumnStatistics res, int nulls, Value... vals) {
+    private static void testAggregationResult(Class<?> type, ColumnStatistics res, int nulls, Object... vals) {
         if (vals.length == 0) {
             assertNull(res.min());
             assertNull(res.max());
         }
-        else {
-            assertEquals(vals[0], res.min());
-            assertEquals(vals[vals.length - 1], res.max());
+        else if (!type.isAssignableFrom(String.class)) {
+            assertEquals(StatisticsUtils.toDecimal(vals[0]), res.min());
+            assertEquals(StatisticsUtils.toDecimal(vals[vals.length - 1]), res.max());
         }
 
         assertEquals(nulls, res.nulls());
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsRepositoryStaticTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsRepositoryStaticTest.java
index 4d0a2ae79a5..9877d7dc0b8 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsRepositoryStaticTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsRepositoryStaticTest.java
@@ -17,13 +17,12 @@
 
 package org.apache.ignite.internal.processors.query.stat;
 
+import java.math.BigDecimal;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.stream.Collectors;
-
 import org.apache.ignite.internal.util.typedef.internal.U;
-import org.h2.value.ValueInt;
 import org.junit.Test;
 
 /**
@@ -41,7 +40,7 @@ public class IgniteStatisticsRepositoryStaticTest extends StatisticsAbstractTest
         0, new byte[0], 0, U.currentTimeMillis());
 
     /** Column statistics with 100 integers 0-100. */
-    protected ColumnStatistics cs2 = new ColumnStatistics(ValueInt.get(0), ValueInt.get(100), 0, 100, 100,
+    protected ColumnStatistics cs2 = new ColumnStatistics(new BigDecimal(0), new BigDecimal(100), 0, 100, 100,
         4, new byte[0], 0, U.currentTimeMillis());
 
     /** Column statistics with 0 rows. */
@@ -49,7 +48,7 @@ public class IgniteStatisticsRepositoryStaticTest extends StatisticsAbstractTest
         new byte[0], 0, U.currentTimeMillis());
 
     /** Column statistics with 100 integers 0-10. */
-    protected ColumnStatistics cs4 = new ColumnStatistics(ValueInt.get(0), ValueInt.get(10), 0, 10, 100,
+    protected ColumnStatistics cs4 = new ColumnStatistics(new BigDecimal(0), new BigDecimal(10), 0, 10, 100,
         4, new byte[0], 0, U.currentTimeMillis());
 
     /**
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsRepositoryTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsRepositoryTest.java
index c8b64eb027a..2171dec3343 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsRepositoryTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsRepositoryTest.java
@@ -17,10 +17,10 @@
 
 package org.apache.ignite.internal.processors.query.stat;
 
+import java.math.BigDecimal;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
-
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.internal.managers.systemview.GridSystemViewManager;
@@ -33,7 +33,6 @@ import org.apache.ignite.internal.util.collection.IntMap;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.logger.GridTestLog4jLogger;
-import org.h2.value.ValueInt;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
@@ -55,7 +54,7 @@ public class IgniteStatisticsRepositoryTest extends StatisticsAbstractTest {
         0, new byte[0], 0, U.currentTimeMillis());
 
     /** Column statistics with 100 integers 0-100. */
-    protected ColumnStatistics cs2 = new ColumnStatistics(ValueInt.get(0), ValueInt.get(100), 0, 100, 100,
+    protected ColumnStatistics cs2 = new ColumnStatistics(new BigDecimal(0), new BigDecimal(100), 0, 100, 100,
         4, new byte[0], 0, U.currentTimeMillis());
 
     /** Column statistics with 0 rows. */
@@ -63,7 +62,7 @@ public class IgniteStatisticsRepositoryTest extends StatisticsAbstractTest {
         new byte[0], 0, U.currentTimeMillis());
 
     /** Column statistics with 100 integers 0-10. */
-    protected ColumnStatistics cs4 = new ColumnStatistics(ValueInt.get(0), ValueInt.get(10), 0, 10, 100,
+    protected ColumnStatistics cs4 = new ColumnStatistics(new BigDecimal(0), new BigDecimal(10), 0, 10, 100,
         4, new byte[0], 0, U.currentTimeMillis());
 
     /** Persistence enabled flag. */
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/stat/ManagerStatisticsTypesTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/stat/ManagerStatisticsTypesTest.java
index b33f4556c1b..c7b1c3ba772 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/stat/ManagerStatisticsTypesTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/stat/ManagerStatisticsTypesTest.java
@@ -18,11 +18,20 @@
 package org.apache.ignite.internal.processors.query.stat;
 
 import java.math.BigDecimal;
-
+import java.math.BigInteger;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.Month;
+import java.util.UUID;
 import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
-import org.h2.value.ValueUuid;
+import org.apache.ignite.internal.util.GridClientByteUtils;
 import org.junit.Test;
 
+import static org.apache.ignite.internal.cache.query.index.sorted.inline.types.DateValueUtils.convertToSqlDate;
+import static org.apache.ignite.internal.cache.query.index.sorted.inline.types.DateValueUtils.convertToSqlTime;
+import static org.apache.ignite.internal.cache.query.index.sorted.inline.types.DateValueUtils.convertToTimestamp;
+
 /**
  * Gather statistics on test table dtypes and check that statistics manager will return correct statistics for
  * different data types.
@@ -58,13 +67,13 @@ public class ManagerStatisticsTypesTest extends StatisticsTypesAbstractTest {
             assertEquals(dtypesStat.rowCount(), colStat.total());
             assertNotNull(colStat.raw());
 
-            if (colName.equals("COL_GEOMETRY")) {
-                assertNull(colStat.min());
-                assertNull(colStat.max());
+            if ("BINARY".equals(type) || "GEOMETRY".equals(type) || "VARCHAR".equals(type) || "CHAR".equals(type)) {
+                assertNull("Unexpected min for " + type, colStat.min());
+                assertNull("Unexpected max for " + type, colStat.max());
             }
             else {
-                assertNotNull(colStat.min());
-                assertNotNull(colStat.max());
+                assertNotNull("Unexpected min for " + type, colStat.min());
+                assertNotNull("Unexpected max for " + type, colStat.max());
             }
         }
     }
@@ -78,8 +87,8 @@ public class ManagerStatisticsTypesTest extends StatisticsTypesAbstractTest {
         ColumnStatistics booleanStats = getTypesStats().columnStatistics(colName);
 
         assertEquals(2, booleanStats.distinct());
-        assertFalse(booleanStats.min().getBoolean());
-        assertTrue(booleanStats.max().getBoolean());
+        assertTrue(booleanStats.min().compareTo(BigDecimal.ZERO) == 0);
+        assertTrue(booleanStats.max().compareTo(BigDecimal.ONE) == 0);
         assertEquals(1, booleanStats.size());
     }
 
@@ -92,8 +101,8 @@ public class ManagerStatisticsTypesTest extends StatisticsTypesAbstractTest {
         ColumnStatistics intStats = getTypesStats().columnStatistics(colName);
 
         assertEquals(SMALL_SIZE - 1, intStats.distinct());
-        assertEquals(1, intStats.min().getInt());
-        assertEquals(SMALL_SIZE - 1, intStats.max().getInt());
+        assertEquals(1, intStats.min().intValue());
+        assertEquals(SMALL_SIZE - 1, intStats.max().intValue());
         assertEquals(4, intStats.size());
     }
 
@@ -106,8 +115,8 @@ public class ManagerStatisticsTypesTest extends StatisticsTypesAbstractTest {
         ColumnStatistics tinyintStats = getTypesStats().columnStatistics(colName);
 
         assertEquals(SMALL_SIZE - 1, tinyintStats.distinct());
-        assertEquals(1, tinyintStats.min().getShort());
-        assertEquals(SMALL_SIZE - 1, tinyintStats.max().getShort());
+        assertEquals(1, tinyintStats.min().byteValue());
+        assertEquals(SMALL_SIZE - 1, tinyintStats.max().byteValue());
         assertEquals(1, tinyintStats.size());
     }
 
@@ -120,8 +129,8 @@ public class ManagerStatisticsTypesTest extends StatisticsTypesAbstractTest {
         ColumnStatistics smallintStats = getTypesStats().columnStatistics(colName);
 
         assertEquals(SMALL_SIZE - 1, smallintStats.distinct());
-        assertEquals(1, smallintStats.min().getShort());
-        assertEquals(SMALL_SIZE - 1, smallintStats.max().getShort());
+        assertEquals(1, smallintStats.min().shortValue());
+        assertEquals(SMALL_SIZE - 1, smallintStats.max().shortValue());
         assertEquals(2, smallintStats.size());
     }
 
@@ -134,8 +143,8 @@ public class ManagerStatisticsTypesTest extends StatisticsTypesAbstractTest {
         ColumnStatistics bigintStats = getTypesStats().columnStatistics(colName);
 
         assertEquals(SMALL_SIZE - 1, bigintStats.distinct());
-        assertEquals(1, bigintStats.min().getBigDecimal().intValue());
-        assertEquals(SMALL_SIZE - 1, bigintStats.max().getBigDecimal().intValue());
+        assertEquals(1, bigintStats.min().intValue());
+        assertEquals(SMALL_SIZE - 1, bigintStats.max().intValue());
         assertEquals(8, bigintStats.size());
     }
 
@@ -148,9 +157,11 @@ public class ManagerStatisticsTypesTest extends StatisticsTypesAbstractTest {
         ColumnStatistics decimalStats = getTypesStats().columnStatistics(colName);
 
         assertEquals(SMALL_SIZE - 1, decimalStats.distinct());
-        assertEquals(new BigDecimal("0.01"), decimalStats.min().getBigDecimal());
-        assertEquals(new BigDecimal("" + ((double)SMALL_SIZE - 1) / 100), decimalStats.max().getBigDecimal());
-        assertEquals(2, decimalStats.size());
+        assertEquals(new BigDecimal("0.01"), decimalStats.min());
+        assertEquals(new BigDecimal("" + ((double)SMALL_SIZE - 1) / 100), decimalStats.max());
+
+        // Size of unscaled value plus scale.
+        assertEquals(new BigDecimal("0.01").unscaledValue().toByteArray().length + 4, decimalStats.size());
     }
 
     /**
@@ -162,9 +173,9 @@ public class ManagerStatisticsTypesTest extends StatisticsTypesAbstractTest {
         ColumnStatistics doubleStats = getTypesStats().columnStatistics(colName);
 
         assertEquals(SMALL_SIZE - 1, doubleStats.distinct());
-        assertEquals(0.01, doubleStats.min().getDouble());
-        assertEquals(((double)SMALL_SIZE - 1) / 100, doubleStats.max().getDouble());
-        assertEquals(2, doubleStats.size());
+        assertEquals(0.01, doubleStats.min().doubleValue());
+        assertEquals(((double)SMALL_SIZE - 1) / 100, doubleStats.max().doubleValue());
+        assertEquals(8, doubleStats.size());
     }
 
     /**
@@ -176,9 +187,9 @@ public class ManagerStatisticsTypesTest extends StatisticsTypesAbstractTest {
         ColumnStatistics realStats = getTypesStats().columnStatistics(colName);
 
         assertEquals(SMALL_SIZE - 1, realStats.distinct());
-        assertEquals(new BigDecimal("0.01"), realStats.min().getBigDecimal());
-        assertEquals(new BigDecimal("" + ((double)SMALL_SIZE - 1) / 100), realStats.max().getBigDecimal());
-        assertEquals(2, realStats.size());
+        assertEquals(0.01f, realStats.min().floatValue());
+        assertEquals(((float)SMALL_SIZE - 1) / 100, realStats.max().floatValue());
+        assertEquals(4, realStats.size());
     }
 
     /**
@@ -190,9 +201,9 @@ public class ManagerStatisticsTypesTest extends StatisticsTypesAbstractTest {
         ColumnStatistics timeStats = getTypesStats().columnStatistics(colName);
 
         assertEquals(SMALL_SIZE - 1, timeStats.distinct());
-        assertEquals("12:00:01", timeStats.min().getTime().toString());
-        assertEquals("12:01:39", timeStats.max().getTime().toString());
-        assertEquals(4, timeStats.size());
+        assertEquals(convert(LocalTime.of(12, 0, 1)), timeStats.min());
+        assertEquals(convert(LocalTime.of(12, 1, 39)), timeStats.max());
+        assertEquals(8, timeStats.size());
     }
 
     /**
@@ -204,9 +215,9 @@ public class ManagerStatisticsTypesTest extends StatisticsTypesAbstractTest {
         ColumnStatistics dateStats = getTypesStats().columnStatistics(colName);
 
         assertEquals(SMALL_SIZE - 1, dateStats.distinct());
-        assertEquals("1970-01-02", dateStats.min().getDate().toString());
-        assertEquals("1970-04-10", dateStats.max().getDate().toString());
-        assertEquals(4, dateStats.size());
+        assertEquals(convert(LocalDate.of(1970, Month.JANUARY, 2)), dateStats.min());
+        assertEquals(convert(LocalDate.of(1970, Month.APRIL, 10)), dateStats.max());
+        assertEquals(8, dateStats.size());
     }
 
     /**
@@ -218,9 +229,9 @@ public class ManagerStatisticsTypesTest extends StatisticsTypesAbstractTest {
         ColumnStatistics timestampStats = getTypesStats().columnStatistics(colName);
 
         assertEquals(SMALL_SIZE - 1, timestampStats.distinct());
-        assertEquals("1970-01-01 12:00:01.0", timestampStats.min().getTimestamp().toString());
-        assertEquals("1970-01-01 12:01:39.0", timestampStats.max().getTimestamp().toString());
-        assertEquals(4, timestampStats.size());
+        assertEquals(convert(LocalDateTime.of(1970, Month.JANUARY, 1, 12, 0, 1)), timestampStats.min());
+        assertEquals(convert(LocalDateTime.of(1970, Month.JANUARY, 1, 12, 1, 39)), timestampStats.max());
+        assertEquals(12, timestampStats.size());
     }
 
     /**
@@ -232,8 +243,8 @@ public class ManagerStatisticsTypesTest extends StatisticsTypesAbstractTest {
         ColumnStatistics varcharStats = getTypesStats().columnStatistics(colName);
 
         assertEquals(SMALL_SIZE - 1, varcharStats.distinct());
-        assertEquals("varchar" + 1, varcharStats.min().getString());
-        assertEquals("varchar" + (SMALL_SIZE - 1), varcharStats.max().getString());
+        assertNull(varcharStats.min());
+        assertNull(varcharStats.max());
         assertEquals(8, varcharStats.size());
     }
 
@@ -246,8 +257,8 @@ public class ManagerStatisticsTypesTest extends StatisticsTypesAbstractTest {
         ColumnStatistics charStats = getTypesStats().columnStatistics(colName);
 
         assertEquals(26, charStats.distinct());
-        assertEquals('A', charStats.min().getString().charAt(0));
-        assertEquals('Z', charStats.max().getString().charAt(0));
+        assertNull(charStats.min());
+        assertNull(charStats.max());
         assertEquals(1, charStats.size());
     }
 
@@ -260,8 +271,10 @@ public class ManagerStatisticsTypesTest extends StatisticsTypesAbstractTest {
         ColumnStatistics decimalStats = getTypesStats().columnStatistics(colName);
 
         assertEquals(SMALL_SIZE - 1, decimalStats.distinct());
-        assertEquals(1L, ((ValueUuid)decimalStats.min()).getLow());
-        assertEquals(SMALL_SIZE - 1L, ((ValueUuid)decimalStats.max()).getLow());
+        assertEquals(new BigDecimal(new BigInteger(1, GridClientByteUtils.uuidToBytes(new UUID(0L, 1L)))),
+            decimalStats.min());
+        assertEquals(new BigDecimal(new BigInteger(1, GridClientByteUtils.uuidToBytes(new UUID(0L, SMALL_SIZE - 1L)))),
+            decimalStats.max());
         assertEquals(16, decimalStats.size());
     }
 
@@ -274,8 +287,8 @@ public class ManagerStatisticsTypesTest extends StatisticsTypesAbstractTest {
         ColumnStatistics binaryStats = getTypesStats().columnStatistics(colName);
 
         assertEquals(SMALL_SIZE - 1, binaryStats.distinct());
-        assertEquals((byte)1, binaryStats.min().getBytes()[3]);
-        assertEquals((byte)99, binaryStats.max().getBytes()[3]);
+        assertNull(binaryStats.min());
+        assertNull(binaryStats.max());
         assertEquals(4, binaryStats.size());
     }
 
@@ -287,4 +300,19 @@ public class ManagerStatisticsTypesTest extends StatisticsTypesAbstractTest {
     private ObjectStatisticsImpl getTypesStats() {
         return (ObjectStatisticsImpl)statisticsMgr(0).getLocalStatistics(new StatisticsKey(SCHEMA, "DTYPES"));
     }
+
+    /** */
+    private static BigDecimal convert(LocalTime time) {
+        return new BigDecimal(convertToSqlTime(time).getTime());
+    }
+
+    /** */
+    private static BigDecimal convert(LocalDate date) {
+        return new BigDecimal(convertToSqlDate(date).getTime());
+    }
+
+    /** */
+    private static BigDecimal convert(LocalDateTime ts) {
+        return new BigDecimal(convertToTimestamp(ts).getTime());
+    }
 }
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/stat/SqlStatisticsCommandTests.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/stat/SqlStatisticsCommandTests.java
index f395c190142..e464a513930 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/stat/SqlStatisticsCommandTests.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/stat/SqlStatisticsCommandTests.java
@@ -18,7 +18,6 @@
 package org.apache.ignite.internal.processors.query.stat;
 
 import java.util.function.Predicate;
-
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.IgniteEx;
@@ -28,7 +27,6 @@ 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.testframework.GridTestUtils;
-import org.junit.Ignore;
 import org.junit.Test;
 
 import static org.apache.ignite.testframework.GridTestUtils.waitForCondition;
@@ -36,7 +34,6 @@ import static org.apache.ignite.testframework.GridTestUtils.waitForCondition;
 /**
  * Integration tests for statistics collection.
  */
-@Ignore("https://issues.apache.org/jira/browse/IGNITE-15455")
 public class SqlStatisticsCommandTests extends StatisticsAbstractTest {
     /** {@inheritDoc} */
     @Override protected void beforeTestsStarted() throws Exception {
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/stat/StatisticsAbstractTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/stat/StatisticsAbstractTest.java
index 0f308875dc4..6dcda40d36d 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/stat/StatisticsAbstractTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/stat/StatisticsAbstractTest.java
@@ -83,7 +83,7 @@ public abstract class StatisticsAbstractTest extends GridCommonAbstractTest {
     static final StatisticsTarget SMALL_TARGET = new StatisticsTarget(SMALL_KEY, null);
 
     /** Async operation timeout for test */
-    static final int TIMEOUT = 5_000;
+    static final int TIMEOUT = 10_000;
 
     static {
         assertTrue(SMALL_SIZE < MED_SIZE && MED_SIZE < BIG_SIZE);
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/stat/StatisticsConfigurationTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/stat/StatisticsConfigurationTest.java
index b032482fd72..af2893032ed 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/stat/StatisticsConfigurationTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/stat/StatisticsConfigurationTest.java
@@ -22,7 +22,6 @@ import java.util.Arrays;
 import java.util.List;
 import java.util.function.Consumer;
 import java.util.stream.Collectors;
-
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
@@ -105,7 +104,7 @@ public class StatisticsConfigurationTest extends StatisticsAbstractTest {
                 assertNotNull("Column: " + col, colStat);
 
                 assertTrue("Column: " + col, colStat.distinct() > 0);
-                assertTrue("Column: " + col, colStat.max().getInt() > 0);
+                assertTrue("Column: " + col, colStat.max().intValue() > 0);
                 assertTrue("Column: " + col, colStat.total() == stat.rowCount());
             }
         }
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/stat/StatisticsGatheringTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/stat/StatisticsGatheringTest.java
index d0ef1ff5d5f..92f691789ee 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/stat/StatisticsGatheringTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/stat/StatisticsGatheringTest.java
@@ -20,7 +20,6 @@ package org.apache.ignite.internal.processors.query.stat;
 import java.util.Collections;
 import java.util.Map;
 import java.util.Objects;
-
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.cluster.ClusterState;
@@ -128,11 +127,9 @@ public class StatisticsGatheringTest extends StatisticsRestartAbstractTest {
 
     /**
      * Collect statistics for group of object at once and check it collected in each node.
-     *
-     * @throws Exception In case of errors.
      */
     @Test
-    public void testGroupGathering() throws Exception {
+    public void testGroupGathering() {
         StatisticsTarget t100 = createStatisticTarget(100);
         StatisticsTarget t101 = createStatisticTarget(101);
         StatisticsTarget tWrong = new StatisticsTarget(t101.schema(), t101.obj() + "wrong");
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/stat/StatisticsObsolescenceTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/stat/StatisticsObsolescenceTest.java
index c9769bea1ae..1baa806d1a2 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/stat/StatisticsObsolescenceTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/stat/StatisticsObsolescenceTest.java
@@ -65,7 +65,7 @@ public class StatisticsObsolescenceTest extends StatisticsAbstractTest {
         assertTrue(GridTestUtils.waitForCondition(() -> {
             ObjectStatisticsImpl stat2 = (ObjectStatisticsImpl)statisticsMgr(0).getLocalStatistics(SMALL_KEY);
 
-            return stat2.rowCount() > stat1.rowCount();
+            return stat2 != null && stat2.rowCount() > stat1.rowCount();
         }, TIMEOUT));
     }
 
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/stat/StatisticsViewsTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/stat/StatisticsViewsTest.java
index b0ca914f182..2981f06e05d 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/stat/StatisticsViewsTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/stat/StatisticsViewsTest.java
@@ -20,13 +20,9 @@ package org.apache.ignite.internal.processors.query.stat;
 import java.sql.Timestamp;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Collections;
 import java.util.List;
-
+import org.apache.ignite.cache.query.SqlFieldsQuery;
 import org.apache.ignite.cluster.ClusterState;
-import org.apache.ignite.internal.processors.query.stat.config.StatisticsColumnConfiguration;
-import org.apache.ignite.internal.processors.query.stat.config.StatisticsColumnOverrides;
-import org.apache.ignite.internal.processors.query.stat.config.StatisticsObjectConfiguration;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.core.config.Configurator;
@@ -50,7 +46,6 @@ public abstract class StatisticsViewsTest extends StatisticsAbstractTest {
         collectStatistics(StatisticsType.GLOBAL, SMALL_TARGET);
     }
 
-
     /**
      * Check small table configuration in statistics column configuration view.
      */
@@ -92,9 +87,7 @@ public abstract class StatisticsViewsTest extends StatisticsAbstractTest {
         name = name.toUpperCase();
 
         // 2) Create statistics for new table.
-        // TODO: revert after IGNITE-15455
-        //grid(0).cache(DEFAULT_CACHE_NAME).query(new SqlFieldsQuery("ANALYZE " + name)).getAll();
-        collectStatistics(StatisticsType.GLOBAL, name);
+        grid(0).cache(DEFAULT_CACHE_NAME).query(new SqlFieldsQuery("ANALYZE " + name)).getAll();
 
         // 3) Check statistics configuration presence.
         List<List<Object>> config = new ArrayList<>();
@@ -105,9 +98,7 @@ public abstract class StatisticsViewsTest extends StatisticsAbstractTest {
         checkSqlResult("select * from SYS.STATISTICS_CONFIGURATION where NAME = '" + name + "'", null, config::equals);
 
         // 4) Drop statistics for some column of new table.
-        //grid(0).cache(DEFAULT_CACHE_NAME).query(new SqlFieldsQuery("DROP STATISTICS " + name + "(A);")).getAll();
-        statisticsMgr(0).statisticConfiguration().dropStatistics(
-            Collections.singletonList(new StatisticsTarget(SCHEMA, name, "A")), true);
+        grid(0).cache(DEFAULT_CACHE_NAME).query(new SqlFieldsQuery("DROP STATISTICS " + name + "(A);")).getAll();
 
         // 5) Check statistics configuration without dropped column.
         List<Object> removed = config.remove(0);
@@ -115,10 +106,7 @@ public abstract class StatisticsViewsTest extends StatisticsAbstractTest {
             act -> testContains(config, act) == null && testContains(Arrays.asList(removed), act) != null);
 
         // 6) Drop statistics for new table.
-        // TODO: revert after IGNITE-15455
-        //grid(0).cache(DEFAULT_CACHE_NAME).query(new SqlFieldsQuery("DROP STATISTICS " + name)).getAll();
-        statisticsMgr(0).statisticConfiguration().dropStatistics(
-            Collections.singletonList(new StatisticsTarget(SCHEMA, name)), true);
+        grid(0).cache(DEFAULT_CACHE_NAME).query(new SqlFieldsQuery("DROP STATISTICS " + name)).getAll();
 
         // 7) Check statistics configuration without it.
         checkSqlResult("select * from SYS.STATISTICS_CONFIGURATION where NAME = '" + name + "'", null, List::isEmpty);
@@ -177,35 +165,13 @@ public abstract class StatisticsViewsTest extends StatisticsAbstractTest {
         assertNotNull(smallStat);
         assertEquals(size, smallStat.rowCount());
 
-        // TODO: revert after IGNITE-15455
-        // sql("DROP STATISTICS SMALL");
-
-        // sql("ANALYZE SMALL (A) WITH \"DISTINCT=5,NULLS=6,TOTAL=7,SIZE=8\"");
-        // sql("ANALYZE SMALL (B) WITH \"DISTINCT=6,NULLS=7,TOTAL=8\"");
-        // sql("ANALYZE SMALL (C)");
-        IgniteStatisticsConfigurationManager cfgMgr = statisticsMgr(0).statisticConfiguration();
-
-        cfgMgr.dropStatistics(Collections.singletonList(SMALL_TARGET), true);
-
-        StatisticsColumnConfiguration aCfg = new StatisticsColumnConfiguration("A",
-            new StatisticsColumnOverrides(6L, 5L, 7L, 8));
-        StatisticsObjectConfiguration smallACfg = new StatisticsObjectConfiguration(SMALL_KEY,
-            Collections.singleton(aCfg), StatisticsObjectConfiguration.DEFAULT_OBSOLESCENCE_MAX_PERCENT);
-
-        cfgMgr.updateStatistics(smallACfg);
-
-        StatisticsColumnConfiguration bCfg = new StatisticsColumnConfiguration("B",
-            new StatisticsColumnOverrides(7L, 6L, 8L, null));
-        StatisticsObjectConfiguration smallBCfg = new StatisticsObjectConfiguration(SMALL_KEY,
-            Collections.singleton(bCfg), StatisticsObjectConfiguration.DEFAULT_OBSOLESCENCE_MAX_PERCENT);
-
-        cfgMgr.updateStatistics(smallBCfg);
+        sql("DROP STATISTICS SMALL");
 
-        StatisticsColumnConfiguration cCfg = new StatisticsColumnConfiguration("C", null);
-        StatisticsObjectConfiguration smallCCfg = new StatisticsObjectConfiguration(SMALL_KEY,
-            Collections.singleton(cCfg), StatisticsObjectConfiguration.DEFAULT_OBSOLESCENCE_MAX_PERCENT);
+        checkSqlResult("select * from SYS.STATISTICS_LOCAL_DATA where NAME = 'SMALL'", null, List::isEmpty);
 
-        cfgMgr.updateStatistics(smallCCfg);
+        sql("ANALYZE SMALL (A) WITH \"DISTINCT=5,NULLS=6,TOTAL=7,SIZE=8\"");
+        sql("ANALYZE SMALL (B) WITH \"DISTINCT=6,NULLS=7,TOTAL=8\"");
+        sql("ANALYZE SMALL (C)");
 
         checkSqlResult("select * from SYS.STATISTICS_LOCAL_DATA where NAME = 'SMALL' and COLUMN = 'C'", null,
             list -> !list.isEmpty());