You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ni...@apache.org on 2019/06/14 12:38:55 UTC

[ignite] branch master updated: IGNITE-11848: New Monitoring. Phase1 (#6546)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new fdaa310  IGNITE-11848: New Monitoring. Phase1 (#6546)
fdaa310 is described below

commit fdaa310430aefff07994eb35510d3416886b5bbe
Author: Nikolay <ni...@apache.org>
AuthorDate: Fri Jun 14 15:38:44 2019 +0300

    IGNITE-11848: New Monitoring. Phase1 (#6546)
    
    * Metrics and Counters API.
    * MonitoringExporterSPI: JMX, SQL View, Log implementations.
    * GridMonitoringManager.
    * MonitoringRegistry.
    * ThreadPools metrics, DataRegion metrics, DataStorage metrics migration.
---
 .../benchmarks/jol/GridMetricsJolBenchmark.java    | 135 +++++++
 modules/clients/pom.xml                            |   2 +-
 modules/core/pom.xml                               |   2 +-
 .../java/org/apache/ignite/DataRegionMetrics.java  |   4 +
 .../apache/ignite/DataRegionMetricsAdapter.java    |   4 +
 .../java/org/apache/ignite/DataStorageMetrics.java |   4 +
 .../apache/ignite/DataStorageMetricsAdapter.java   |   5 +
 .../ignite/configuration/IgniteConfiguration.java  |  27 ++
 .../apache/ignite/internal/GridKernalContext.java  |  16 +-
 .../ignite/internal/GridKernalContextImpl.java     |  35 +-
 .../org/apache/ignite/internal/IgniteKernal.java   |  18 +-
 .../org/apache/ignite/internal/IgnitionEx.java     |   4 +
 .../internal/managers/IgniteMBeansManager.java     |   6 -
 .../internal/{stat => metric}/IndexPageType.java   |   2 +-
 .../{stat => metric}/IoStatisticsHolder.java       |  19 +-
 .../{stat => metric}/IoStatisticsHolderCache.java  |  56 ++-
 .../{stat => metric}/IoStatisticsHolderIndex.java  |  67 ++--
 .../{stat => metric}/IoStatisticsHolderNoOp.java   |  20 +-
 .../{stat => metric}/IoStatisticsHolderQuery.java  |  29 +-
 .../{stat => metric}/IoStatisticsQueryHelper.java  |   2 +-
 .../{stat => metric}/IoStatisticsType.java         |  27 +-
 .../ignite/internal/pagemem/PageSupport.java       |   2 +-
 .../pagemem/impl/PageMemoryNoStoreImpl.java        |   4 +-
 .../processors/cache/CacheGroupContext.java        |  17 +-
 .../cache/CacheLocalMetricsMXBeanImpl.java         |   4 +
 .../processors/cache/CacheMetricsImpl.java         | 378 ++++++++++++++------
 .../processors/cache/GridCacheAdapter.java         |   9 +-
 .../cache/IgniteCacheOffheapManagerImpl.java       |   2 +-
 .../cache/persistence/CacheDataRowAdapter.java     |   4 +-
 .../cache/persistence/DataRegionMetricsImpl.java   | 196 +++++++---
 .../persistence/DataRegionMetricsMXBeanImpl.java   |   4 +
 .../cache/persistence/DataStorageMetricsImpl.java  | 197 +++++++----
 .../persistence/DataStorageMetricsSnapshot.java    |   4 +-
 .../cache/persistence/DataStructure.java           |   4 +-
 .../GridCacheDatabaseSharedManager.java            |   3 +-
 .../IgniteCacheDatabaseSharedManager.java          |  11 +-
 .../processors/cache/persistence/RowStore.java     |   2 +-
 .../persistence/freelist/AbstractFreeList.java     |   4 +-
 .../cache/persistence/freelist/CacheFreeList.java  |   2 +-
 .../cache/persistence/freelist/FreeList.java       |   2 +-
 .../cache/persistence/freelist/PagesList.java      |   4 +-
 .../metastorage/MetastorageRowStore.java           |   2 +-
 .../cache/persistence/pagemem/PageMemoryEx.java    |   2 +-
 .../cache/persistence/pagemem/PageMemoryImpl.java  |   4 +-
 .../partstorage/PartitionMetaStorage.java          |   2 +-
 .../cache/persistence/tree/BPlusTree.java          |   4 +-
 .../cache/persistence/tree/io/PageIO.java          |   4 +-
 .../persistence/tree/reuse/ReuseListImpl.java      |   2 +-
 .../cache/persistence/tree/util/PageHandler.java   |   2 +-
 .../wal/reader/StandaloneGridKernalContext.java    |  13 +-
 .../cache/ratemetrics/HitRateMetrics.java          | 179 ----------
 .../cache/ratemetrics/HitRateMetricsSandbox.java   |  95 -----
 .../processors/cache/tree/CacheDataTree.java       |   2 +-
 .../metric/AbstractMetric.java}                    |  60 ++--
 .../processors/metric/GridMetricManager.java       | 293 +++++++++++++++
 .../internal/processors/metric/MetricRegistry.java | 196 ++++++++++
 .../processors/metric/MetricRegistryImpl.java      | 212 +++++++++++
 .../metric/MetricRegistryPrefixProxy.java          | 177 ++++++++++
 .../metric/PushMetricsExporterAdapter.java         | 106 ++++++
 .../processors/metric/impl/BooleanGauge.java}      |  29 +-
 .../processors/metric/impl/BooleanMetricImpl.java  |  57 +++
 .../processors/metric/impl/DoubleGauge.java}       |  29 +-
 .../processors/metric/impl/DoubleMetricImpl.java   |  69 ++++
 .../processors/metric/impl/HistogramMetric.java    | 118 +++++++
 .../processors/metric/impl/HitRateMetric.java      | 222 ++++++++++++
 .../internal/processors/metric/impl/IntGauge.java} |  29 +-
 .../processors/metric/impl/IntMetricImpl.java      |  71 ++++
 .../metric/impl/LongAdderMetricImpl.java           |  68 ++++
 .../processors/metric/impl/LongGauge.java}         |  29 +-
 .../processors/metric/impl/LongMetricImpl.java     |  81 +++++
 .../processors/metric/impl/MetricUtils.java        | 157 ++++++++
 .../processors/metric/impl/ObjectGauge.java        |  57 +++
 .../processors/metric/impl/ObjectMetricImpl.java   |  63 ++++
 .../internal/processors/query/QueryUtils.java      |   3 +
 .../ignite/internal/stat/IoStatisticsManager.java  | 226 ------------
 .../stat/IoStatisticsMetricsLocalMXBeanImpl.java   | 146 --------
 .../apache/ignite/internal/util/IgniteUtils.java   |  11 +-
 .../apache/ignite/internal/util/lang/GridFunc.java | 193 ++++++++++
 .../apache/ignite/mxbean/CacheMetricsMXBean.java   |   4 +
 .../ignite/mxbean/DataRegionMetricsMXBean.java     |   4 +
 .../ignite/mxbean/DataStorageMetricsMXBean.java    |   4 +
 .../org/apache/ignite/mxbean/IgniteMXBean.java     |  10 +
 .../ignite/mxbean/IoStatisticsMetricsMXBean.java   | 172 ---------
 .../org/apache/ignite/mxbean/ThreadPoolMXBean.java |   7 +-
 .../org/apache/ignite/spi/IgniteSpiAdapter.java    |   2 +-
 .../apache/ignite/spi/metric/BooleanMetric.java}   |  13 +-
 .../apache/ignite/spi/metric/DoubleMetric.java}    |  13 +-
 .../org/apache/ignite/spi/metric/IntMetric.java}   |  13 +-
 .../org/apache/ignite/spi/metric/LongMetric.java}  |  23 +-
 .../java/org/apache/ignite/spi/metric/Metric.java} |  23 +-
 .../ignite/spi/metric/MetricExporterSpi.java       |  63 ++++
 .../apache/ignite/spi/metric/ObjectMetric.java}    |  23 +-
 .../ignite/spi/metric/ReadOnlyMetricRegistry.java  |  59 ++++
 .../ignite/spi/metric/jmx/JmxExporterSpi.java      | 125 +++++++
 .../ignite/spi/metric/jmx/MetricSetMBean.java      | 172 +++++++++
 .../metric/jmx/package-info.java}                  |  20 +-
 .../ignite/spi/metric/log/LogExporterSpi.java}     |  29 +-
 .../metric/log/package-info.java}                  |  20 +-
 .../spi/metric/noop/NoopMetricExporterSpi.java     |  53 +++
 .../metric/noop/package-info.java}                 |  20 +-
 .../metric/package-info.java}                      |  20 +-
 .../main/resources/META-INF/classnames.properties  |   2 +-
 .../internal/metric/AbstractExporterSpiTest.java   |  79 +++++
 .../internal/metric/CacheMetricsAddRemoveTest.java | 149 ++++++++
 .../IoStatisticsCachePersistenceSelfTest.java      |   2 +-
 .../IoStatisticsCacheSelfTest.java                 | 106 ++++--
 ...IoStatisticsMetricsLocalMXBeanImplSelfTest.java | 179 ++++++++++
 .../IoStatisticsSelfTest.java}                     | 124 +++++--
 .../ignite/internal/metric/JmxExporterSpiTest.java | 126 +++++++
 .../ignite/internal/metric/LogExporterSpiTest.java | 113 ++++++
 .../ignite/internal/metric/MetricsSelfTest.java    | 393 +++++++++++++++++++++
 .../db/wal/IgniteWalIteratorSwitchSegmentTest.java |   7 +
 .../pagemem/BPlusTreePageMemoryImplTest.java       |   4 +
 .../BPlusTreeReuseListPageMemoryImplTest.java      |   4 +
 .../IgnitePageMemReplaceDelayedWriteUnitTest.java  |   9 +
 .../pagemem/IndexStoragePageMemoryImplTest.java    |   4 +
 .../pagemem/PageMemoryImplNoLoadTest.java          |   4 +
 .../persistence/pagemem/PageMemoryImplTest.java    |   4 +
 .../pagemem/PagesWriteThrottleSandboxTest.java     |  12 +-
 .../pagemem/PagesWriteThrottleSmokeTest.java       |  16 +-
 ...ionIntegrityWithPrimaryIndexCorruptionTest.java |   2 +-
 .../processors/database/CacheFreeListSelfTest.java |   2 +-
 .../database/DataRegionMetricsSelfTest.java        |   4 +-
 ...IoStatisticsMetricsLocalMXBeanImplSelfTest.java | 220 ------------
 .../testsuites/IgniteCacheMvccTestSuite9.java      |   4 +-
 .../ignite/testsuites/IgniteCacheTestSuite9.java   |  22 +-
 modules/ignored-tests/pom.xml                      |   2 +-
 modules/indexing/pom.xml                           |   2 +-
 .../processors/query/h2/SchemaManager.java         |  30 +-
 .../processors/query/h2/database/H2Tree.java       |   2 +-
 .../processors/query/h2/database/H2TreeIndex.java  |  13 +-
 .../view/SqlSystemViewCacheGroupsIOStatistics.java |  95 +++--
 .../ignite/spi/metric/sql/SqlViewExporterSpi.java  | 124 +++++++
 .../cache/index/AbstractSchemaSelfTest.java        |   2 +-
 .../index/IoStatisticsBasicIndexSelfTest.java      |  86 ++++-
 .../cache/metric/SqlViewExporterSpiTest.java       | 134 +++++++
 .../persistence/db/wal/IgniteWalRecoveryTest.java  |   2 +-
 .../IgniteBinaryCacheQueryTestSuite.java           |   5 +-
 .../util/GridCommandHandlerIndexingTest.java       |   2 +-
 modules/opencensus/README.txt                      |   5 +
 modules/opencensus/licenses/apache-2.0.txt         | 202 +++++++++++
 modules/{tensorflow => opencensus}/pom.xml         |  97 ++---
 .../opencensus/OpenCensusMetricExporterSpi.java    | 279 +++++++++++++++
 .../OpenCensusMetricExporterSpiTest.java           | 169 +++++++++
 .../Apache.Ignite.Core/Apache.Ignite.Core.csproj   |   4 +-
 .../Apache.Ignite.Core/IgniteConfiguration.cs      |   6 +
 .../IgniteConfigurationSection.xsd                 |   5 +
 .../Metric/IMetricExporterSpi.cs}                  |  22 +-
 .../Apache.Ignite.Core/Metric/Package-Info.cs      |  26 ++
 modules/tensorflow/pom.xml                         |   2 +-
 parent/pom.xml                                     |   3 +
 pom.xml                                            |   1 +
 152 files changed, 6247 insertions(+), 1906 deletions(-)

diff --git a/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jol/GridMetricsJolBenchmark.java b/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jol/GridMetricsJolBenchmark.java
new file mode 100644
index 0000000..c7a1529
--- /dev/null
+++ b/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jol/GridMetricsJolBenchmark.java
@@ -0,0 +1,135 @@
+/*
+ * 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.benchmarks.jol;
+
+import org.apache.ignite.internal.processors.metric.MetricRegistryImpl;
+import org.apache.ignite.internal.processors.metric.impl.BooleanMetricImpl;
+import org.apache.ignite.internal.processors.metric.impl.DoubleMetricImpl;
+import org.apache.ignite.internal.processors.metric.impl.IntMetricImpl;
+import org.apache.ignite.internal.processors.metric.impl.LongAdderMetricImpl;
+import org.apache.ignite.internal.processors.metric.impl.LongMetricImpl;
+import org.openjdk.jol.info.GraphLayout;
+
+/**
+ * Benchmark to measure heap space for metrics.
+ */
+public class GridMetricsJolBenchmark {
+    /** */
+    private static final int BOOLEAN_CNT = 100;
+
+    /** */
+    private static final int DOUBLE_CNT = 100;
+
+    /** */
+    private static final int INT_CNT = 100;
+
+    /** */
+    private static final int LONG_CNT = 100;
+
+    /** */
+    private static final int LONG_ADDER_CNT = 100;
+
+    /** */
+    private static final int TOTAL = BOOLEAN_CNT + DOUBLE_CNT + INT_CNT + LONG_CNT + LONG_ADDER_CNT;
+
+    /** */
+    public static final String BOOLEAN_METRIC = "boolean.metric.";
+
+    /** */
+    public static final String DOUBLE_METRIC = "double.metric.";
+
+    /** */
+    public static final String INT_METRIC = "int.metric.";
+
+    /** */
+    public static final String LONG_METRIC = "long.metric.";
+
+    /** */
+    public static final String LONG_ADDER_METRIC = "long.adder.metric.";
+
+    /** */
+    public static void main(String[] args) {
+        measureMetricRegistry();
+
+        measureArray();
+    }
+
+    /**
+     * Calculates and prints the size of metrics array of {@code TOTAL} size;
+     */
+    private static void measureArray() {
+        Object[] metrics = new Object[TOTAL];
+
+        int start = 0;
+
+        for(int i=0; i<BOOLEAN_CNT; i++)
+            metrics[start + i] = new BooleanMetricImpl(BOOLEAN_METRIC + i, null);
+
+        start += BOOLEAN_CNT;
+
+        for(int i=0; i<DOUBLE_CNT; i++)
+            metrics[start + i] = new DoubleMetricImpl(DOUBLE_METRIC + i, null);
+
+        start += DOUBLE_CNT;
+
+        for(int i=0; i<INT_CNT; i++)
+            metrics[start + i] = new IntMetricImpl(INT_METRIC + i, null);
+
+        start += INT_CNT;
+
+        for(int i=0; i<LONG_CNT; i++)
+            metrics[start + i] = new LongMetricImpl(LONG_METRIC + i, null);
+
+        start += LONG_CNT;
+
+        for(int i=0; i<LONG_ADDER_CNT; i++)
+            metrics[start + i] = new LongAdderMetricImpl(LONG_ADDER_METRIC + i, null);
+
+        start += LONG_ADDER_CNT;
+
+        long sz = GraphLayout.parseInstance(metrics).totalSize();
+
+        System.out.println("Total size of " + TOTAL + " metric array is " + (sz/1024) + "KiB, " + sz + " bytes.");
+    }
+
+    /**
+     * Calculates and prints the size of metric registry of {@code TOTAL} size;
+     */
+    private static void measureMetricRegistry() {
+        MetricRegistryImpl mreg = new MetricRegistryImpl();
+
+        for(int i=0; i<BOOLEAN_CNT; i++)
+            mreg.booleanMetric(BOOLEAN_METRIC + i, null);
+
+        for(int i=0; i<DOUBLE_CNT; i++)
+            mreg.doubleMetric(DOUBLE_METRIC + i, null);
+
+        for(int i=0; i<INT_CNT; i++)
+            mreg.doubleMetric(INT_METRIC + i, null);
+
+        for(int i=0; i<LONG_CNT; i++)
+            mreg.metric(LONG_METRIC + i, null);
+
+        for(int i=0; i<LONG_ADDER_CNT; i++)
+            mreg.metric(LONG_ADDER_METRIC + i, null);
+
+        long sz = GraphLayout.parseInstance(mreg).totalSize();
+
+        System.out.println("Total size of " + TOTAL + " metric registry is " + (sz/1024) + "KiB, " + sz + " bytes.");
+    }
+}
diff --git a/modules/clients/pom.xml b/modules/clients/pom.xml
index d04fc70..c2ecda9 100644
--- a/modules/clients/pom.xml
+++ b/modules/clients/pom.xml
@@ -94,7 +94,7 @@
         <dependency>
             <groupId>commons-io</groupId>
             <artifactId>commons-io</artifactId>
-            <version>2.4</version>
+            <version>${commons.io.version}</version>
             <scope>test</scope>
         </dependency>
 
diff --git a/modules/core/pom.xml b/modules/core/pom.xml
index 667c1cf..3642eb4 100644
--- a/modules/core/pom.xml
+++ b/modules/core/pom.xml
@@ -187,7 +187,7 @@
         <dependency>
             <groupId>commons-io</groupId>
             <artifactId>commons-io</artifactId>
-            <version>2.4</version>
+            <version>${commons.io.version}</version>
             <scope>test</scope>
         </dependency>
 
diff --git a/modules/core/src/main/java/org/apache/ignite/DataRegionMetrics.java b/modules/core/src/main/java/org/apache/ignite/DataRegionMetrics.java
index 87a9e19..40ffbde 100644
--- a/modules/core/src/main/java/org/apache/ignite/DataRegionMetrics.java
+++ b/modules/core/src/main/java/org/apache/ignite/DataRegionMetrics.java
@@ -19,6 +19,7 @@ package org.apache.ignite;
 
 import org.apache.ignite.configuration.DataRegionConfiguration;
 import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.internal.processors.metric.GridMetricManager;
 import org.apache.ignite.mxbean.DataRegionMetricsMXBean;
 
 /**
@@ -45,7 +46,10 @@ import org.apache.ignite.mxbean.DataRegionMetricsMXBean;
  * why the metrics are turned off by default. To enable the collection you can use both
  * {@link DataRegionConfiguration#setMetricsEnabled(boolean)} configuration property or
  * {@link DataRegionMetricsMXBean#enableMetrics()} method of a respective JMX bean.
+ *
+ * @deprecated Use {@link GridMetricManager} instead.
  */
+@Deprecated
 public interface DataRegionMetrics {
     /**
      * A name of a memory region the metrics are collected for.
diff --git a/modules/core/src/main/java/org/apache/ignite/DataRegionMetricsAdapter.java b/modules/core/src/main/java/org/apache/ignite/DataRegionMetricsAdapter.java
index dcf2049..85f32c5 100644
--- a/modules/core/src/main/java/org/apache/ignite/DataRegionMetricsAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/DataRegionMetricsAdapter.java
@@ -18,10 +18,14 @@ package org.apache.ignite;
 
 import java.util.ArrayList;
 import java.util.Collection;
+import org.apache.ignite.internal.processors.metric.GridMetricManager;
 
 /**
  * Converter class from {@link DataRegionMetrics} to legacy {@link MemoryMetrics}.
+ *
+ * @deprecated Use {@link GridMetricManager} instead.
  */
+@Deprecated
 public class DataRegionMetricsAdapter implements MemoryMetrics {
     /** Delegate. */
     private final DataRegionMetrics delegate;
diff --git a/modules/core/src/main/java/org/apache/ignite/DataStorageMetrics.java b/modules/core/src/main/java/org/apache/ignite/DataStorageMetrics.java
index e23d188..c2534cb 100644
--- a/modules/core/src/main/java/org/apache/ignite/DataStorageMetrics.java
+++ b/modules/core/src/main/java/org/apache/ignite/DataStorageMetrics.java
@@ -17,10 +17,14 @@
 package org.apache.ignite;
 
 import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.internal.processors.metric.GridMetricManager;
 
 /**
  * Data storage metrics are used to obtain statistics on persistent store and whole data storage.
+ *
+ * @deprecated Use {@link GridMetricManager} instead.
  */
+@Deprecated
 public interface DataStorageMetrics {
     /**
      * Gets the average number of WAL records per second written during the last time interval.
diff --git a/modules/core/src/main/java/org/apache/ignite/DataStorageMetricsAdapter.java b/modules/core/src/main/java/org/apache/ignite/DataStorageMetricsAdapter.java
index 431e264..09c25f1 100644
--- a/modules/core/src/main/java/org/apache/ignite/DataStorageMetricsAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/DataStorageMetricsAdapter.java
@@ -16,9 +16,14 @@
 */
 package org.apache.ignite;
 
+import org.apache.ignite.internal.processors.metric.GridMetricManager;
+
 /**
  * Converter class from {@link DataStorageMetrics} to legacy {@link PersistenceMetrics}.
+ *
+ * @deprecated Use {@link GridMetricManager} instead.
  */
+@Deprecated
 public class DataStorageMetricsAdapter implements PersistenceMetrics {
     /** Delegate. */
     private final DataStorageMetrics delegate;
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
index f1e9124..6bc6e9d 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
@@ -77,6 +77,7 @@ import org.apache.ignite.spi.failover.always.AlwaysFailoverSpi;
 import org.apache.ignite.spi.indexing.IndexingSpi;
 import org.apache.ignite.spi.loadbalancing.LoadBalancingSpi;
 import org.apache.ignite.spi.loadbalancing.roundrobin.RoundRobinLoadBalancingSpi;
+import org.apache.ignite.spi.metric.MetricExporterSpi;
 import org.apache.ignite.ssl.SslContextFactory;
 import org.jetbrains.annotations.Nullable;
 
@@ -396,6 +397,9 @@ public class IgniteConfiguration {
     /** Encryption SPI. */
     private EncryptionSpi encryptionSpi;
 
+    /** Metric exporter SPI. */
+    private MetricExporterSpi[] metricExporterSpi;
+
     /** Cache configurations. */
     private CacheConfiguration[] cacheCfg;
 
@@ -585,6 +589,7 @@ public class IgniteConfiguration {
         loadBalancingSpi = cfg.getLoadBalancingSpi();
         indexingSpi = cfg.getIndexingSpi();
         encryptionSpi = cfg.getEncryptionSpi();
+        metricExporterSpi = cfg.getMetricExporterSpi();
 
         commFailureRslvr = cfg.getCommunicationFailureResolver();
 
@@ -2329,6 +2334,28 @@ public class IgniteConfiguration {
     }
 
     /**
+     * Sets fully configured instances of {@link MetricExporterSpi}.
+     *
+     * @param metricExporterSpi Fully configured instances of {@link MetricExporterSpi}.
+     * @see IgniteConfiguration#getMetricExporterSpi()
+     * @return {@code this} for chaining.
+     */
+    public IgniteConfiguration setMetricExporterSpi(MetricExporterSpi... metricExporterSpi) {
+        this.metricExporterSpi = metricExporterSpi;
+
+        return this;
+    }
+
+    /**
+     * Gets fully configured monitoring SPI implementations.
+     *
+     * @return Metric exporter SPI implementations.
+     */
+    public MetricExporterSpi[] getMetricExporterSpi() {
+        return metricExporterSpi;
+    }
+
+    /**
      * Gets address resolver for addresses mapping determination.
      *
      * @return Address resolver.
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
index 8c67e45..c8f256e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
@@ -43,6 +43,7 @@ import org.apache.ignite.internal.processors.closure.GridClosureProcessor;
 import org.apache.ignite.internal.processors.cluster.ClusterProcessor;
 import org.apache.ignite.internal.processors.cluster.GridClusterStateProcessor;
 import org.apache.ignite.internal.processors.compress.CompressionProcessor;
+import org.apache.ignite.internal.processors.configuration.distributed.DistributedConfigurationProcessor;
 import org.apache.ignite.internal.processors.continuous.GridContinuousProcessor;
 import org.apache.ignite.internal.processors.datastreamer.DataStreamProcessor;
 import org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor;
@@ -55,8 +56,8 @@ import org.apache.ignite.internal.processors.igfs.IgfsProcessorAdapter;
 import org.apache.ignite.internal.processors.job.GridJobProcessor;
 import org.apache.ignite.internal.processors.jobmetrics.GridJobMetricsProcessor;
 import org.apache.ignite.internal.processors.marshaller.GridMarshallerMappingProcessor;
-import org.apache.ignite.internal.processors.configuration.distributed.DistributedConfigurationProcessor;
 import org.apache.ignite.internal.processors.metastorage.DistributedMetaStorage;
+import org.apache.ignite.internal.processors.metric.GridMetricManager;
 import org.apache.ignite.internal.processors.odbc.ClientListenerProcessor;
 import org.apache.ignite.internal.processors.platform.PlatformProcessor;
 import org.apache.ignite.internal.processors.plugin.IgnitePluginProcessor;
@@ -73,7 +74,6 @@ import org.apache.ignite.internal.processors.session.GridTaskSessionProcessor;
 import org.apache.ignite.internal.processors.subscription.GridInternalSubscriptionProcessor;
 import org.apache.ignite.internal.processors.task.GridTaskProcessor;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor;
-import org.apache.ignite.internal.stat.IoStatisticsManager;
 import org.apache.ignite.internal.suggestions.GridPerformanceSuggestions;
 import org.apache.ignite.internal.util.IgniteExceptionRegistry;
 import org.apache.ignite.internal.util.StripedExecutor;
@@ -195,6 +195,13 @@ public interface GridKernalContext extends Iterable<GridComponent> {
     public GridJobMetricsProcessor jobMetric();
 
     /**
+     * Gets metric manager.
+     *
+     * @return Monitoring manager.
+     */
+    public GridMetricManager metric();
+
+    /**
      * Gets caches processor.
      *
      * @return Cache processor.
@@ -728,11 +735,6 @@ public interface GridKernalContext extends Iterable<GridComponent> {
     public GridInternalSubscriptionProcessor internalSubscriptionProcessor();
 
     /**
-     * @return IO statistic manager.
-     */
-    public IoStatisticsManager ioStats();
-
-    /**
      * @return Default uncaught exception handler used by thread pools.
      */
     public Thread.UncaughtExceptionHandler uncaughtExceptionHandler();
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
index 20cde25e..2261f48 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
@@ -48,8 +48,6 @@ import org.apache.ignite.internal.managers.eventstorage.GridEventStorageManager;
 import org.apache.ignite.internal.managers.failover.GridFailoverManager;
 import org.apache.ignite.internal.managers.indexing.GridIndexingManager;
 import org.apache.ignite.internal.managers.loadbalancer.GridLoadBalancerManager;
-import org.apache.ignite.internal.processors.diagnostic.DiagnosticProcessor;
-import org.apache.ignite.internal.processors.service.ServiceProcessorAdapter;
 import org.apache.ignite.internal.processors.affinity.GridAffinityProcessor;
 import org.apache.ignite.internal.processors.authentication.IgniteAuthenticationProcessor;
 import org.apache.ignite.internal.processors.cache.CacheConflictResolutionManager;
@@ -62,9 +60,11 @@ import org.apache.ignite.internal.processors.closure.GridClosureProcessor;
 import org.apache.ignite.internal.processors.cluster.ClusterProcessor;
 import org.apache.ignite.internal.processors.cluster.GridClusterStateProcessor;
 import org.apache.ignite.internal.processors.compress.CompressionProcessor;
+import org.apache.ignite.internal.processors.configuration.distributed.DistributedConfigurationProcessor;
 import org.apache.ignite.internal.processors.continuous.GridContinuousProcessor;
 import org.apache.ignite.internal.processors.datastreamer.DataStreamProcessor;
 import org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor;
+import org.apache.ignite.internal.processors.diagnostic.DiagnosticProcessor;
 import org.apache.ignite.internal.processors.failure.FailureProcessor;
 import org.apache.ignite.internal.processors.hadoop.HadoopHelper;
 import org.apache.ignite.internal.processors.hadoop.HadoopProcessorAdapter;
@@ -73,8 +73,8 @@ import org.apache.ignite.internal.processors.igfs.IgfsProcessorAdapter;
 import org.apache.ignite.internal.processors.job.GridJobProcessor;
 import org.apache.ignite.internal.processors.jobmetrics.GridJobMetricsProcessor;
 import org.apache.ignite.internal.processors.marshaller.GridMarshallerMappingProcessor;
-import org.apache.ignite.internal.processors.configuration.distributed.DistributedConfigurationProcessor;
 import org.apache.ignite.internal.processors.metastorage.DistributedMetaStorage;
+import org.apache.ignite.internal.processors.metric.GridMetricManager;
 import org.apache.ignite.internal.processors.nodevalidation.DiscoveryNodeValidationProcessor;
 import org.apache.ignite.internal.processors.odbc.ClientListenerProcessor;
 import org.apache.ignite.internal.processors.platform.PlatformProcessor;
@@ -88,11 +88,11 @@ import org.apache.ignite.internal.processors.rest.GridRestProcessor;
 import org.apache.ignite.internal.processors.schedule.IgniteScheduleProcessorAdapter;
 import org.apache.ignite.internal.processors.security.IgniteSecurity;
 import org.apache.ignite.internal.processors.segmentation.GridSegmentationProcessor;
+import org.apache.ignite.internal.processors.service.ServiceProcessorAdapter;
 import org.apache.ignite.internal.processors.session.GridTaskSessionProcessor;
 import org.apache.ignite.internal.processors.subscription.GridInternalSubscriptionProcessor;
 import org.apache.ignite.internal.processors.task.GridTaskProcessor;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor;
-import org.apache.ignite.internal.stat.IoStatisticsManager;
 import org.apache.ignite.internal.suggestions.GridPerformanceSuggestions;
 import org.apache.ignite.internal.util.IgniteExceptionRegistry;
 import org.apache.ignite.internal.util.StripedExecutor;
@@ -203,7 +203,11 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
 
     /** */
     @GridToStringInclude
-    private GridJobMetricsProcessor metricsProc;
+    private GridJobMetricsProcessor jobMetricsProc;
+
+    /** */
+    @GridToStringInclude
+    private GridMetricManager metricMgr;
 
     /** */
     @GridToStringInclude
@@ -443,9 +447,6 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
     /** Recovery mode flag. Flag is set to {@code false} when discovery manager started. */
     private boolean recoveryMode = true;
 
-    /** IO statistics manager. */
-    private IoStatisticsManager ioStatMgr;
-
     /**
      * No-arg constructor is required by externalization.
      */
@@ -545,8 +546,6 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
                 log.debug("Failed to load spring component, will not be able to extract userVersion from " +
                     "META-INF/ignite.xml.");
         }
-
-        ioStatMgr = new IoStatisticsManager();
     }
 
     /** {@inheritDoc} */
@@ -615,7 +614,9 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
         else if (comp instanceof GridResourceProcessor)
             rsrcProc = (GridResourceProcessor)comp;
         else if (comp instanceof GridJobMetricsProcessor)
-            metricsProc = (GridJobMetricsProcessor)comp;
+            jobMetricsProc = (GridJobMetricsProcessor)comp;
+        else if (comp instanceof GridMetricManager)
+            metricMgr = (GridMetricManager)comp;
         else if (comp instanceof GridCacheProcessor)
             cacheProc = (GridCacheProcessor)comp;
         else if (comp instanceof GridClusterStateProcessor)
@@ -764,7 +765,12 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
 
     /** {@inheritDoc} */
     @Override public GridJobMetricsProcessor jobMetric() {
-        return metricsProc;
+        return jobMetricsProc;
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridMetricManager metric() {
+        return metricMgr;
     }
 
     /** {@inheritDoc} */
@@ -1202,11 +1208,6 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
         return internalSubscriptionProc;
     }
 
-    /** {@inheritDoc} */
-    @Override public IoStatisticsManager ioStats() {
-        return ioStatMgr;
-    }
-
     /**
      * @param disconnected Disconnected flag.
      */
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
index bda34b9..e1a6c93 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
@@ -156,6 +156,8 @@ import org.apache.ignite.internal.processors.job.GridJobProcessor;
 import org.apache.ignite.internal.processors.jobmetrics.GridJobMetricsProcessor;
 import org.apache.ignite.internal.processors.marshaller.GridMarshallerMappingProcessor;
 import org.apache.ignite.internal.processors.metastorage.persistence.DistributedMetaStorageImpl;
+import org.apache.ignite.internal.processors.metric.GridMetricManager;
+import org.apache.ignite.internal.processors.metric.MetricRegistry;
 import org.apache.ignite.internal.processors.nodevalidation.DiscoveryNodeValidationProcessor;
 import org.apache.ignite.internal.processors.nodevalidation.OsDiscoveryNodeValidationProcessor;
 import org.apache.ignite.internal.processors.odbc.ClientListenerProcessor;
@@ -1098,6 +1100,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
 
             // Start SPI managers.
             // NOTE: that order matters as there are dependencies between managers.
+            startManager(new GridMetricManager(ctx));
             startManager(new GridIoManager(ctx));
             startManager(new GridCheckpointManager(ctx));
 
@@ -1276,6 +1279,10 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
             if (recon)
                 reconnectState.waitFirstReconnect();
 
+            ctx.metric().registerThreadPools(utilityCachePool, execSvc, svcExecSvc, sysExecSvc, stripedExecSvc,
+                p2pExecSvc, mgmtExecSvc, igfsExecSvc, dataStreamExecSvc, restExecSvc, affExecSvc, idxExecSvc,
+                callbackExecSvc, qryExecSvc, schemaExecSvc, customExecSvcs);
+
             // Register MBeans.
             mBeansMgr.registerAllMBeans(utilityCachePool, execSvc, svcExecSvc, sysExecSvc, stripedExecSvc, p2pExecSvc,
                 mgmtExecSvc, igfsExecSvc, dataStreamExecSvc, restExecSvc, affExecSvc, idxExecSvc, callbackExecSvc,
@@ -2617,8 +2624,6 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
                 // Preserve interrupt status.
                 Thread.currentThread().interrupt();
             }
-
-            ctx.ioStats().stop();
         }
         else {
             // Proper notification.
@@ -4495,6 +4500,15 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
     }
 
     /** {@inheritDoc} */
+    @Override public void resetMetrics(String prefix) {
+        assert prefix != null;
+
+        MetricRegistry mreg = ctx.metric().registry().withPrefix(prefix);
+
+        mreg.reset();
+    }
+
+    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(IgniteKernal.class, this);
     }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
index 5839168..2d2a093 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
@@ -121,6 +121,7 @@ import org.apache.ignite.spi.failover.always.AlwaysFailoverSpi;
 import org.apache.ignite.spi.indexing.noop.NoopIndexingSpi;
 import org.apache.ignite.spi.loadbalancing.LoadBalancingSpi;
 import org.apache.ignite.spi.loadbalancing.roundrobin.RoundRobinLoadBalancingSpi;
+import org.apache.ignite.spi.metric.noop.NoopMetricExporterSpi;
 import org.apache.ignite.thread.IgniteStripedThreadPoolExecutor;
 import org.apache.ignite.thread.IgniteThread;
 import org.apache.ignite.thread.IgniteThreadPoolExecutor;
@@ -2413,6 +2414,9 @@ public class IgnitionEx {
 
             if (cfg.getEncryptionSpi() == null)
                 cfg.setEncryptionSpi(new NoopEncryptionSpi());
+
+            if (F.isEmpty(cfg.getMetricExporterSpi()))
+                cfg.setMetricExporterSpi(new NoopMetricExporterSpi());
         }
 
         /**
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/IgniteMBeansManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/IgniteMBeansManager.java
index 07da645..ad0ff24 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/IgniteMBeansManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/IgniteMBeansManager.java
@@ -35,7 +35,6 @@ import org.apache.ignite.internal.TransactionMetricsMxBeanImpl;
 import org.apache.ignite.internal.TransactionsMXBeanImpl;
 import org.apache.ignite.internal.processors.cache.persistence.DataStorageMXBeanImpl;
 import org.apache.ignite.internal.processors.cluster.BaselineAutoAdjustMXBeanImpl;
-import org.apache.ignite.internal.stat.IoStatisticsMetricsLocalMXBeanImpl;
 import org.apache.ignite.internal.util.StripedExecutor;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.internal.worker.FailureHandlingMxBeanImpl;
@@ -46,7 +45,6 @@ import org.apache.ignite.mxbean.ClusterMetricsMXBean;
 import org.apache.ignite.mxbean.DataStorageMXBean;
 import org.apache.ignite.mxbean.FailureHandlingMxBean;
 import org.apache.ignite.mxbean.IgniteMXBean;
-import org.apache.ignite.mxbean.IoStatisticsMetricsMXBean;
 import org.apache.ignite.mxbean.StripedExecutorMXBean;
 import org.apache.ignite.mxbean.ThreadPoolMXBean;
 import org.apache.ignite.mxbean.TransactionMetricsMxBean;
@@ -133,10 +131,6 @@ public class IgniteMBeansManager {
         ClusterMetricsMXBean metricsBean = new ClusterMetricsMXBeanImpl(kernal.cluster());
         registerMBean("Kernal", metricsBean.getClass().getSimpleName(), metricsBean, ClusterMetricsMXBean.class);
 
-        //IO metrics
-        IoStatisticsMetricsMXBean ioStatMetricsBean = new IoStatisticsMetricsLocalMXBeanImpl(ctx.ioStats());
-        registerMBean("IOMetrics", ioStatMetricsBean.getClass().getSimpleName(), ioStatMetricsBean, IoStatisticsMetricsMXBean.class);
-
         // Transaction metrics
         TransactionMetricsMxBean txMetricsMXBean = new TransactionMetricsMxBeanImpl(ctx.cache().transactions().metrics());
         registerMBean("TransactionMetrics", txMetricsMXBean.getClass().getSimpleName(), txMetricsMXBean, TransactionMetricsMxBean.class);
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/stat/IndexPageType.java b/modules/core/src/main/java/org/apache/ignite/internal/metric/IndexPageType.java
similarity index 95%
rename from modules/core/src/main/java/org/apache/ignite/internal/stat/IndexPageType.java
rename to modules/core/src/main/java/org/apache/ignite/internal/metric/IndexPageType.java
index a64c5f7..7f1a686 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/stat/IndexPageType.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/metric/IndexPageType.java
@@ -16,7 +16,7 @@
  *
  */
 
-package org.apache.ignite.internal.stat;
+package org.apache.ignite.internal.metric;
 
 /**
  * Enumeration of types of b-tree index pages.
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/stat/IoStatisticsHolder.java b/modules/core/src/main/java/org/apache/ignite/internal/metric/IoStatisticsHolder.java
similarity index 77%
rename from modules/core/src/main/java/org/apache/ignite/internal/stat/IoStatisticsHolder.java
rename to modules/core/src/main/java/org/apache/ignite/internal/metric/IoStatisticsHolder.java
index 5b7a57f..e6f9f7b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/stat/IoStatisticsHolder.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/metric/IoStatisticsHolder.java
@@ -16,9 +16,7 @@
  *
  */
 
-package org.apache.ignite.internal.stat;
-
-import java.util.Map;
+package org.apache.ignite.internal.metric;
 
 /**
  * Holder of IO statistics.
@@ -47,19 +45,4 @@ public interface IoStatisticsHolder {
      * @return Number of physical reads.
      */
     public long physicalReads();
-
-    /**
-     * @return Logical reads statistics per page types.
-     */
-    public Map<String, Long> logicalReadsMap();
-
-    /**
-     * @return Physical reads statistics per page types.
-     */
-    public Map<String, Long> physicalReadsMap();
-
-    /**
-     * Reset statistics. All counters will be reset to 0.
-     */
-    public void resetStatistics();
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/stat/IoStatisticsHolderCache.java b/modules/core/src/main/java/org/apache/ignite/internal/metric/IoStatisticsHolderCache.java
similarity index 70%
rename from modules/core/src/main/java/org/apache/ignite/internal/stat/IoStatisticsHolderCache.java
rename to modules/core/src/main/java/org/apache/ignite/internal/metric/IoStatisticsHolderCache.java
index c0dd85a..c89f8e2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/stat/IoStatisticsHolderCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/metric/IoStatisticsHolderCache.java
@@ -16,29 +16,31 @@
  *
  */
 
-package org.apache.ignite.internal.stat;
+package org.apache.ignite.internal.metric;
 
-import java.util.HashMap;
-import java.util.Map;
-import java.util.concurrent.atomic.LongAdder;
 import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
 import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.processors.metric.MetricRegistry;
+import org.apache.ignite.internal.processors.metric.impl.LongAdderMetricImpl;
+
+import static org.apache.ignite.internal.metric.IoStatisticsType.CACHE_GROUP;
 
 /**
  * Cache statistics holder to gather statistics related to concrete cache.
  */
 public class IoStatisticsHolderCache implements IoStatisticsHolder {
     /** */
-    private static final String PHYSICAL_READS = "PHYSICAL_READS";
+    public static final String PHYSICAL_READS = "PHYSICAL_READS";
 
     /** */
-    private static final String LOGICAL_READS = "LOGICAL_READS";
+    public static final String LOGICAL_READS = "LOGICAL_READS";
 
     /** */
-    private LongAdder logicalReadCtr = new LongAdder();
+    private final LongAdderMetricImpl logicalReadCtr;
 
     /** */
-    private LongAdder physicalReadCtr = new LongAdder();
+    private final LongAdderMetricImpl physicalReadCtr;
 
     /** */
     private final String cacheName;
@@ -48,12 +50,23 @@ public class IoStatisticsHolderCache implements IoStatisticsHolder {
 
     /**
      * @param cacheName Name of cache.
+     * @param grpId Group id.
+     * @param mreg Metric registry.
      */
-    public IoStatisticsHolderCache(String cacheName, int grpId) {
+    public IoStatisticsHolderCache(String cacheName, int grpId, MetricRegistry mreg) {
         assert cacheName != null;
 
         this.cacheName = cacheName;
         this.grpId = grpId;
+
+        MetricRegistry mset = mreg.withPrefix(CACHE_GROUP.metricGroupName(), cacheName);
+
+        mset.metric("startTime", null).value(U.currentTimeMillis());
+        mset.objectMetric("name", String.class, null).value(cacheName);
+        mset.intMetric("grpId", null).value(grpId);
+
+        this.logicalReadCtr = mset.longAdderMetric(LOGICAL_READS, null);
+        this.physicalReadCtr = mset.longAdderMetric(PHYSICAL_READS, null);
     }
 
     /** {@inheritDoc} */
@@ -65,7 +78,6 @@ public class IoStatisticsHolderCache implements IoStatisticsHolder {
 
             IoStatisticsQueryHelper.trackLogicalReadQuery(pageAddr);
         }
-
     }
 
     /** {@inheritDoc} */
@@ -91,30 +103,6 @@ public class IoStatisticsHolderCache implements IoStatisticsHolder {
         return physicalReadCtr.longValue();
     }
 
-    /** {@inheritDoc} */
-    @Override public Map<String, Long> logicalReadsMap() {
-        Map<String, Long> res = new HashMap<>(2);
-
-        res.put(LOGICAL_READS, logicalReads());
-
-        return res;
-    }
-
-    /** {@inheritDoc} */
-    @Override public Map<String, Long> physicalReadsMap() {
-        Map<String, Long> res = new HashMap<>(2);
-
-        res.put(PHYSICAL_READS, physicalReads());
-
-        return res;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void resetStatistics() {
-        logicalReadCtr.reset();
-        physicalReadCtr.reset();
-    }
-
     /**
      * @return Cache group id.
      */
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/stat/IoStatisticsHolderIndex.java b/modules/core/src/main/java/org/apache/ignite/internal/metric/IoStatisticsHolderIndex.java
similarity index 73%
rename from modules/core/src/main/java/org/apache/ignite/internal/stat/IoStatisticsHolderIndex.java
rename to modules/core/src/main/java/org/apache/ignite/internal/metric/IoStatisticsHolderIndex.java
index 8c09d67..c1d10d0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/stat/IoStatisticsHolderIndex.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/metric/IoStatisticsHolderIndex.java
@@ -16,13 +16,13 @@
  *
  */
 
-package org.apache.ignite.internal.stat;
+package org.apache.ignite.internal.metric;
 
-import java.util.HashMap;
-import java.util.Map;
-import java.util.concurrent.atomic.LongAdder;
 import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
 import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.processors.metric.MetricRegistry;
+import org.apache.ignite.internal.processors.metric.impl.LongAdderMetricImpl;
 
 /**
  * Index statistics holder to gather statistics related to concrete index.
@@ -44,16 +44,16 @@ public class IoStatisticsHolderIndex implements IoStatisticsHolder {
     public static final String PHYSICAL_READS_INNER = "PHYSICAL_READS_INNER";
 
     /** */
-    private LongAdder logicalReadLeafCtr = new LongAdder();
+    private final LongAdderMetricImpl logicalReadLeafCtr;
 
     /** */
-    private LongAdder logicalReadInnerCtr = new LongAdder();
+    private final LongAdderMetricImpl logicalReadInnerCtr;
 
     /** */
-    private LongAdder physicalReadLeafCtr = new LongAdder();
+    private final LongAdderMetricImpl physicalReadLeafCtr;
 
     /** */
-    private LongAdder physicalReadInnerCtr = new LongAdder();
+    private final LongAdderMetricImpl physicalReadInnerCtr;
 
     /** */
     private final String cacheName;
@@ -62,14 +62,31 @@ public class IoStatisticsHolderIndex implements IoStatisticsHolder {
     private final String idxName;
 
     /**
-     * @param cacheName Name of cache.
-     * @param idxName Name of index.
+     * @param type Type of statistics.
+     * @param cacheName Cache name.
+     * @param idxName Index name.
+     * @param mreg Metric registry.
      */
-    public IoStatisticsHolderIndex(String cacheName, String idxName) {
+    public IoStatisticsHolderIndex(
+        IoStatisticsType type,
+        String cacheName,
+        String idxName,
+        MetricRegistry mreg) {
         assert cacheName != null && idxName != null;
 
         this.cacheName = cacheName;
         this.idxName = idxName;
+
+        MetricRegistry mset = mreg.withPrefix(type.metricGroupName(), cacheName, idxName);
+
+        mset.metric("startTime", null).value(U.currentTimeMillis());
+        mset.objectMetric("name", String.class, null).value(cacheName);
+        mset.objectMetric("indexName", String.class, null).value(idxName);
+
+        logicalReadLeafCtr = mset.longAdderMetric(LOGICAL_READS_LEAF, null);
+        logicalReadInnerCtr = mset.longAdderMetric(LOGICAL_READS_INNER, null);
+        physicalReadLeafCtr = mset.longAdderMetric(PHYSICAL_READS_LEAF, null);
+        physicalReadInnerCtr = mset.longAdderMetric(PHYSICAL_READS_INNER, null);
     }
 
     /** {@inheritDoc} */
@@ -128,34 +145,6 @@ public class IoStatisticsHolderIndex implements IoStatisticsHolder {
     }
 
     /** {@inheritDoc} */
-    @Override public Map<String, Long> logicalReadsMap() {
-        Map<String, Long> res = new HashMap<>(3);
-
-        res.put(LOGICAL_READS_LEAF, logicalReadLeafCtr.longValue());
-        res.put(LOGICAL_READS_INNER, logicalReadInnerCtr.longValue());
-
-        return res;
-    }
-
-    /** {@inheritDoc} */
-    @Override public Map<String, Long> physicalReadsMap() {
-        Map<String, Long> res = new HashMap<>(3);
-
-        res.put(PHYSICAL_READS_LEAF, physicalReadLeafCtr.longValue());
-        res.put(PHYSICAL_READS_INNER, physicalReadInnerCtr.longValue());
-
-        return res;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void resetStatistics() {
-        logicalReadLeafCtr.reset();
-        logicalReadInnerCtr.reset();
-        physicalReadLeafCtr.reset();
-        physicalReadInnerCtr.reset();
-    }
-
-    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(IoStatisticsHolderIndex.class, this,
             "logicalReadLeafCtr", logicalReadLeafCtr,
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/stat/IoStatisticsHolderNoOp.java b/modules/core/src/main/java/org/apache/ignite/internal/metric/IoStatisticsHolderNoOp.java
similarity index 77%
rename from modules/core/src/main/java/org/apache/ignite/internal/stat/IoStatisticsHolderNoOp.java
rename to modules/core/src/main/java/org/apache/ignite/internal/metric/IoStatisticsHolderNoOp.java
index 65f855b..43c75c1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/stat/IoStatisticsHolderNoOp.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/metric/IoStatisticsHolderNoOp.java
@@ -16,10 +16,7 @@
  *
  */
 
-package org.apache.ignite.internal.stat;
-
-import java.util.Collections;
-import java.util.Map;
+package org.apache.ignite.internal.metric;
 
 /**
  * No Operation IO statistics holder. Use in case statistics shouldn't be gathered.
@@ -52,19 +49,4 @@ public class IoStatisticsHolderNoOp implements IoStatisticsHolder {
     @Override public long physicalReads() {
         return 0;
     }
-
-    /** {@inheritDoc} */
-    @Override public Map<String, Long> logicalReadsMap() {
-        return Collections.emptyMap();
-    }
-
-    /** {@inheritDoc} */
-    @Override public Map<String, Long> physicalReadsMap() {
-        return Collections.emptyMap();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void resetStatistics() {
-        // No-op.
-    }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/stat/IoStatisticsHolderQuery.java b/modules/core/src/main/java/org/apache/ignite/internal/metric/IoStatisticsHolderQuery.java
similarity index 82%
rename from modules/core/src/main/java/org/apache/ignite/internal/stat/IoStatisticsHolderQuery.java
rename to modules/core/src/main/java/org/apache/ignite/internal/metric/IoStatisticsHolderQuery.java
index 71684de..bdb5b67 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/stat/IoStatisticsHolderQuery.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/metric/IoStatisticsHolderQuery.java
@@ -16,10 +16,8 @@
  *
  */
 
-package org.apache.ignite.internal.stat;
+package org.apache.ignite.internal.metric;
 
-import java.util.HashMap;
-import java.util.Map;
 import java.util.concurrent.atomic.LongAdder;
 import org.apache.ignite.internal.util.typedef.internal.S;
 
@@ -74,31 +72,6 @@ public class IoStatisticsHolderQuery implements IoStatisticsHolder {
         return physicalReadCtr.longValue();
     }
 
-    /** {@inheritDoc} */
-    @Override public Map<String, Long> logicalReadsMap() {
-        Map<String, Long> res = new HashMap<>(2);
-
-        res.put(LOGICAL_READS, logicalReads());
-
-        return res;
-    }
-
-    /** {@inheritDoc} */
-    @Override public Map<String, Long> physicalReadsMap() {
-        Map<String, Long> res = new HashMap<>(2);
-
-        res.put(PHYSICAL_READS, physicalReads());
-
-        return res;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void resetStatistics() {
-        logicalReadCtr.reset();
-
-        physicalReadCtr.reset();
-    }
-
     /**
      * @return Query id.
      */
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/stat/IoStatisticsQueryHelper.java b/modules/core/src/main/java/org/apache/ignite/internal/metric/IoStatisticsQueryHelper.java
similarity index 98%
rename from modules/core/src/main/java/org/apache/ignite/internal/stat/IoStatisticsQueryHelper.java
rename to modules/core/src/main/java/org/apache/ignite/internal/metric/IoStatisticsQueryHelper.java
index 399aca2..60f08f5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/stat/IoStatisticsQueryHelper.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/metric/IoStatisticsQueryHelper.java
@@ -16,7 +16,7 @@
  *
  */
 
-package org.apache.ignite.internal.stat;
+package org.apache.ignite.internal.metric;
 
 /**
  * Helper for gathering IO statistics.
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/stat/IoStatisticsType.java b/modules/core/src/main/java/org/apache/ignite/internal/metric/IoStatisticsType.java
similarity index 62%
copy from modules/core/src/main/java/org/apache/ignite/internal/stat/IoStatisticsType.java
copy to modules/core/src/main/java/org/apache/ignite/internal/metric/IoStatisticsType.java
index 67577a4..34f0b28 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/stat/IoStatisticsType.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/metric/IoStatisticsType.java
@@ -16,21 +16,38 @@
  *
  */
 
-package org.apache.ignite.internal.stat;
+package org.apache.ignite.internal.metric;
 
 /**
  * Type of statistics.
  */
 public enum IoStatisticsType {
     /** Cache group. */
-    CACHE_GROUP,
+    CACHE_GROUP("io.statistics.cacheGroups"),
 
     /** Hash index. */
-    HASH_INDEX,
+    HASH_INDEX("io.statistics.hashIndexes"),
 
     /** Sorted index. */
-    SORTED_INDEX,
+    SORTED_INDEX("io.statistics.sortedIndexes"),
 
     /** SQL. */
-    SQL
+    SQL("io.statistics.sql");
+
+    /** Metric group. */
+    private String metricGroupName;
+
+    /**
+     * @param monitoringGroup Monitoring group.
+     */
+    IoStatisticsType(String metricGroupName) {
+        this.metricGroupName = metricGroupName;
+    }
+
+    /**
+     * @return Metric group name.
+     */
+    public String metricGroupName() {
+        return metricGroupName;
+    }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageSupport.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageSupport.java
index 877234d..50bf4be 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageSupport.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageSupport.java
@@ -18,7 +18,7 @@
 package org.apache.ignite.internal.pagemem;
 
 import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.internal.stat.IoStatisticsHolder;
+import org.apache.ignite.internal.metric.IoStatisticsHolder;
 
 /**
  * Supports operations on pages.
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/impl/PageMemoryNoStoreImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/impl/PageMemoryNoStoreImpl.java
index e0ddd2b..77985d4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/impl/PageMemoryNoStoreImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/impl/PageMemoryNoStoreImpl.java
@@ -38,8 +38,8 @@ import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
 import org.apache.ignite.internal.processors.cache.persistence.DataRegionMetricsImpl;
 import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
-import org.apache.ignite.internal.stat.IoStatisticsHolder;
-import org.apache.ignite.internal.stat.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.metric.IoStatisticsHolder;
+import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp;
 import org.apache.ignite.internal.util.GridUnsafe;
 import org.apache.ignite.internal.util.IgniteUtils;
 import org.apache.ignite.internal.util.OffheapReadWriteLock;
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupContext.java
index 0be1dff..07b42bb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheGroupContext.java
@@ -50,9 +50,10 @@ import org.apache.ignite.internal.processors.cache.persistence.freelist.FreeList
 import org.apache.ignite.internal.processors.cache.persistence.tree.reuse.ReuseList;
 import org.apache.ignite.internal.processors.cache.query.continuous.CounterSkipContext;
 import org.apache.ignite.internal.processors.query.QueryUtils;
-import org.apache.ignite.internal.stat.IoStatisticsHolder;
-import org.apache.ignite.internal.stat.IoStatisticsHolderNoOp;
-import org.apache.ignite.internal.stat.IoStatisticsType;
+import org.apache.ignite.internal.metric.IoStatisticsHolder;
+import org.apache.ignite.internal.metric.IoStatisticsHolderCache;
+import org.apache.ignite.internal.metric.IoStatisticsHolderIndex;
+import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp;
 import org.apache.ignite.internal.util.StripedCompositeReadWriteLock;
 import org.apache.ignite.internal.util.typedef.CI1;
 import org.apache.ignite.internal.util.typedef.F;
@@ -63,6 +64,7 @@ import org.apache.ignite.lang.IgniteBiInClosure;
 import org.apache.ignite.lang.IgniteFuture;
 import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.mxbean.CacheGroupMetricsMXBean;
+import org.apache.ignite.internal.processors.metric.MetricRegistry;
 import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
@@ -74,7 +76,8 @@ import static org.apache.ignite.events.EventType.EVT_CACHE_REBALANCE_PART_MISSED
 import static org.apache.ignite.events.EventType.EVT_CACHE_REBALANCE_PART_SUPPLIED;
 import static org.apache.ignite.events.EventType.EVT_CACHE_REBALANCE_PART_UNLOADED;
 import static org.apache.ignite.internal.managers.communication.GridIoPolicy.AFFINITY_POOL;
-import static org.apache.ignite.internal.stat.IoStatisticsHolderIndex.HASH_PK_IDX_NAME;
+import static org.apache.ignite.internal.metric.IoStatisticsHolderIndex.HASH_PK_IDX_NAME;
+import static org.apache.ignite.internal.metric.IoStatisticsType.HASH_INDEX;
 
 /**
  *
@@ -259,10 +262,10 @@ public class CacheGroupContext {
             statHolderData = IoStatisticsHolderNoOp.INSTANCE;
         }
         else {
-            statHolderIdx = ctx.kernalContext().ioStats().registerIndex(IoStatisticsType.HASH_INDEX,
-                cacheOrGroupName(), HASH_PK_IDX_NAME);
+            MetricRegistry mreg = ctx.kernalContext().metric().registry();
 
-            statHolderData = ctx.kernalContext().ioStats().registerCacheGroup(cacheOrGroupName(), grpId);
+            statHolderIdx = new IoStatisticsHolderIndex(HASH_INDEX, cacheOrGroupName(), HASH_PK_IDX_NAME, mreg);
+            statHolderData = new IoStatisticsHolderCache(cacheOrGroupName(), grpId, mreg);
         }
     }
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLocalMetricsMXBeanImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLocalMetricsMXBeanImpl.java
index 790fe00..fe67968 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLocalMetricsMXBeanImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLocalMetricsMXBeanImpl.java
@@ -20,11 +20,15 @@ package org.apache.ignite.internal.processors.cache;
 import java.util.Collections;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.metric.GridMetricManager;
 import org.apache.ignite.mxbean.CacheMetricsMXBean;
 
 /**
  * Management bean that provides access to {@link IgniteCache IgniteCache}.
+ *
+ * @deprecated Use {@link GridMetricManager} instead.
  */
+@Deprecated
 class CacheLocalMetricsMXBeanImpl implements CacheMetricsMXBean {
     /** Cache. */
     private GridCacheAdapter<?, ?> cache;
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java
index 8ce21c5..b9af7d4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java
@@ -18,7 +18,6 @@
 package org.apache.ignite.internal.processors.cache;
 
 import java.util.Set;
-import java.util.concurrent.atomic.AtomicLong;
 import org.apache.ignite.IgniteSystemProperties;
 import org.apache.ignite.cache.CacheMetrics;
 import org.apache.ignite.cache.CachePeekMode;
@@ -27,11 +26,17 @@ import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTopologyFuture;
 import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition;
 import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState;
-import org.apache.ignite.internal.processors.cache.ratemetrics.HitRateMetrics;
 import org.apache.ignite.internal.processors.cache.store.GridCacheWriteBehindStore;
+import org.apache.ignite.internal.processors.metric.impl.MetricUtils;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.processors.metric.MetricRegistry;
+import org.apache.ignite.internal.processors.metric.impl.HitRateMetric;
+import org.apache.ignite.internal.processors.metric.impl.LongMetricImpl;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.internal.processors.metric.impl.MetricUtils.metricName;
 
 /**
  * Adapter for cache metrics.
@@ -48,107 +53,114 @@ public class CacheMetricsImpl implements CacheMetrics {
     /** */
     private static final long NANOS_IN_MICROSECOND = 1000L;
 
+    /**
+     * Cache metrics prefix.
+     * Full name will contain {@link CacheConfiguration#getName()} also.
+     * {@code "cache.sys-cache"}, for example.
+     */
+    public static final String CACHE_METRICS_PREFIX = "cache";
+
     /** Number of reads. */
-    private AtomicLong reads = new AtomicLong();
+    private final LongMetricImpl reads;
 
     /** Number of invocations caused update. */
-    private AtomicLong entryProcessorPuts = new AtomicLong();
+    private final LongMetricImpl entryProcessorPuts;
 
     /** Number of invocations caused removal. */
-    private AtomicLong entryProcessorRemovals = new AtomicLong();
+    private final LongMetricImpl entryProcessorRemovals;
 
     /** Number of invocations caused update. */
-    private AtomicLong entryProcessorReadOnlyInvocations = new AtomicLong();
+    private final LongMetricImpl entryProcessorReadOnlyInvocations;
 
     /** Entry processor invoke time taken nanos. */
-    private AtomicLong entryProcessorInvokeTimeNanos = new AtomicLong();
+    private final LongMetricImpl entryProcessorInvokeTimeNanos;
 
     /** So far, the minimum time to execute cache invokes. */
-    private AtomicLong entryProcessorMinInvocationTime = new AtomicLong();
+    private final LongMetricImpl entryProcessorMinInvocationTime;
 
     /** So far, the maximum time to execute cache invokes. */
-    private AtomicLong entryProcessorMaxInvocationTime = new AtomicLong();
+    private final LongMetricImpl entryProcessorMaxInvocationTime;
 
     /** Number of entry processor invokes on keys, which exist in cache. */
-    private AtomicLong entryProcessorHits = new AtomicLong();
+    private final LongMetricImpl entryProcessorHits;
 
     /** Number of entry processor invokes on keys, which don't exist in cache. */
-    private AtomicLong entryProcessorMisses = new AtomicLong();
+    private final LongMetricImpl entryProcessorMisses;
 
     /** Number of writes. */
-    private AtomicLong writes = new AtomicLong();
+    private final LongMetricImpl writes;
 
     /** Number of hits. */
-    private AtomicLong hits = new AtomicLong();
+    private final LongMetricImpl hits;
 
     /** Number of misses. */
-    private AtomicLong misses = new AtomicLong();
+    private final LongMetricImpl misses;
 
     /** Number of transaction commits. */
-    private AtomicLong txCommits = new AtomicLong();
+    private final LongMetricImpl txCommits;
 
     /** Number of transaction rollbacks. */
-    private AtomicLong txRollbacks = new AtomicLong();
+    private final LongMetricImpl txRollbacks;
 
     /** Number of evictions. */
-    private AtomicLong evictCnt = new AtomicLong();
+    private final LongMetricImpl evictCnt;
 
     /** Number of removed entries. */
-    private AtomicLong rmCnt = new AtomicLong();
+    private final LongMetricImpl rmCnt;
 
     /** Put time taken nanos. */
-    private AtomicLong putTimeNanos = new AtomicLong();
+    private final LongMetricImpl putTimeNanos;
 
     /** Get time taken nanos. */
-    private AtomicLong getTimeNanos = new AtomicLong();
+    private final LongMetricImpl getTimeNanos;
 
     /** Remove time taken nanos. */
-    private AtomicLong rmvTimeNanos = new AtomicLong();
+    private final LongMetricImpl rmvTimeNanos;
 
     /** Commit transaction time taken nanos. */
-    private AtomicLong commitTimeNanos = new AtomicLong();
+    private final LongMetricImpl commitTimeNanos;
 
     /** Commit transaction time taken nanos. */
-    private AtomicLong rollbackTimeNanos = new AtomicLong();
+    private final LongMetricImpl rollbackTimeNanos;
 
     /** Number of reads from off-heap memory. */
-    private AtomicLong offHeapGets = new AtomicLong();
+    private final LongMetricImpl offHeapGets;
 
     /** Number of writes to off-heap memory. */
-    private AtomicLong offHeapPuts = new AtomicLong();
+    private final LongMetricImpl offHeapPuts;
 
     /** Number of removed entries from off-heap memory. */
-    private AtomicLong offHeapRemoves = new AtomicLong();
+    private final LongMetricImpl offHeapRemoves;
 
     /** Number of evictions from off-heap memory. */
-    private AtomicLong offHeapEvicts = new AtomicLong();
+    private final LongMetricImpl offHeapEvicts;
 
     /** Number of off-heap hits. */
-    private AtomicLong offHeapHits = new AtomicLong();
+    private final LongMetricImpl offHeapHits;
 
     /** Number of off-heap misses. */
-    private AtomicLong offHeapMisses = new AtomicLong();
+    private final LongMetricImpl offHeapMisses;
 
     /** Rebalanced keys count. */
-    private AtomicLong rebalancedKeys = new AtomicLong();
+    private final LongMetricImpl rebalancedKeys;
 
     /** Total rebalanced bytes count. */
-    private AtomicLong totalRebalancedBytes = new AtomicLong();
+    private final LongMetricImpl totalRebalancedBytes;
 
     /** Rebalanced start time. */
-    private AtomicLong rebalanceStartTime = new AtomicLong(-1L);
+    private final LongMetricImpl rebalanceStartTime;
 
     /** Estimated rebalancing keys count. */
-    private AtomicLong estimatedRebalancingKeys = new AtomicLong();
+    private final LongMetricImpl estimatedRebalancingKeys;
 
     /** Rebalancing rate in keys. */
-    private HitRateMetrics rebalancingKeysRate = new HitRateMetrics(REBALANCE_RATE_INTERVAL, 20);
+    private final HitRateMetric rebalancingKeysRate;
 
     /** Rebalancing rate in bytes. */
-    private HitRateMetrics rebalancingBytesRate = new HitRateMetrics(REBALANCE_RATE_INTERVAL, 20);
+    private final HitRateMetric rebalancingBytesRate;
 
     /** Number of currently clearing partitions for rebalancing. */
-    private AtomicLong rebalanceClearingPartitions = new AtomicLong();
+    private final LongMetricImpl rebalanceClearingPartitions;
 
     /** Cache metrics. */
     @GridToStringExclude
@@ -163,12 +175,25 @@ public class CacheMetricsImpl implements CacheMetrics {
     /** Write-behind store, if configured. */
     private GridCacheWriteBehindStore store;
 
+    /** Prefix for the cache metrics. */
+    private String prefix;
+
     /**
-     * Creates cache metrics;
+     * Creates cache metrics.
      *
      * @param cctx Cache context.
      */
     public CacheMetricsImpl(GridCacheContext<?, ?> cctx) {
+        this(cctx, null);
+    }
+
+    /**
+     * Creates cache metrics.
+     *
+     * @param cctx Cache context.
+     * @param suffix Suffix for the metric set name.
+     */
+    public CacheMetricsImpl(GridCacheContext<?, ?> cctx, @Nullable String suffix) {
         assert cctx != null;
 
         this.cctx = cctx;
@@ -180,6 +205,120 @@ public class CacheMetricsImpl implements CacheMetrics {
             store = (GridCacheWriteBehindStore)cctx.store().store();
 
         delegate = null;
+
+        if (suffix == null)
+            prefix = metricName(CACHE_METRICS_PREFIX, cctx.name());
+        else
+            prefix = metricName(CACHE_METRICS_PREFIX, cctx.name(), suffix);
+
+        MetricRegistry mreg = cctx.kernalContext().metric().registry().withPrefix(prefix);
+
+        reads = mreg.metric("CacheGets",
+            "The total number of gets to the cache.");
+
+        entryProcessorPuts = mreg.metric("EntryProcessorPuts",
+            "The total number of cache invocations, caused update.");
+
+        entryProcessorRemovals = mreg.metric("EntryProcessorRemovals",
+            "The total number of cache invocations, caused removals.");
+
+        entryProcessorReadOnlyInvocations = mreg.metric("EntryProcessorReadOnlyInvocations",
+            "The total number of cache invocations, caused no updates.");
+
+        entryProcessorInvokeTimeNanos = mreg.metric("EntryProcessorInvokeTimeNanos",
+            "The total time of cache invocations, in nanoseconds.");
+
+        entryProcessorMinInvocationTime = mreg.metric("EntryProcessorMinInvocationTime",
+            "So far, the minimum time to execute cache invokes.");
+
+        entryProcessorMaxInvocationTime = mreg.metric("EntryProcessorMaxInvocationTime",
+            "So far, the maximum time to execute cache invokes.");
+
+        entryProcessorHits = mreg.metric("EntryProcessorHits",
+            "The total number of invocations on keys, which exist in cache.");
+
+        entryProcessorMisses = mreg.metric("EntryProcessorMisses",
+            "The total number of invocations on keys, which don't exist in cache.");
+
+        writes = mreg.metric("CachePuts",
+            "The total number of puts to the cache.");
+
+        hits = mreg.metric("CacheHits",
+            "The number of get requests that were satisfied by the cache.");
+
+        misses = mreg.metric("CacheMisses",
+            "A miss is a get request that is not satisfied.");
+
+        txCommits = mreg.metric("CacheTxCommits",
+            "Total number of transaction commits.");
+
+        txRollbacks = mreg.metric("CacheTxRollbacks",
+            "Total number of transaction rollbacks.");
+
+        evictCnt = mreg.metric("CacheEvictions",
+            "The total number of evictions from the cache.");
+
+        rmCnt = mreg.metric("CacheRemovals", "The total number of removals from the cache.");
+
+        putTimeNanos = mreg.metric("PutTime",
+            "The total time of cache puts, in nanoseconds.");
+
+        getTimeNanos = mreg.metric("GetTime",
+            "The total time of cache gets, in nanoseconds.");
+
+        rmvTimeNanos = mreg.metric("RemovalTime",
+            "The total time of cache removal, in nanoseconds.");
+
+        commitTimeNanos = mreg.metric("CommitTime",
+            "The total time of commit, in nanoseconds.");
+
+        rollbackTimeNanos = mreg.metric("RollbackTime",
+            "The total time of rollback, in nanoseconds.");
+
+        offHeapGets = mreg.metric("OffHeapGets",
+            "The total number of get requests to the off-heap memory.");
+
+        offHeapPuts = mreg.metric("OffHeapPuts",
+            "The total number of put requests to the off-heap memory.");
+
+        offHeapRemoves = mreg.metric("OffHeapRemovals",
+            "The total number of removals from the off-heap memory.");
+
+        offHeapEvicts = mreg.metric("OffHeapEvictions",
+            "The total number of evictions from the off-heap memory.");
+
+        offHeapHits = mreg.metric("OffHeapHits",
+            "The number of get requests that were satisfied by the off-heap memory.");
+
+        offHeapMisses = mreg.metric("OffHeapMisses",
+            "A miss is a get request that is not satisfied by off-heap memory.");
+
+        rebalancedKeys = mreg.metric("RebalancedKeys",
+            "Number of already rebalanced keys.");
+
+        totalRebalancedBytes = mreg.metric("TotalRebalancedBytes",
+            "Number of already rebalanced bytes.");
+
+        rebalanceStartTime = mreg.metric("RebalanceStartTime",
+            "Rebalance start time");
+
+        rebalanceStartTime.value(-1);
+
+        estimatedRebalancingKeys = mreg.metric("EstimatedRebalancingKeys",
+            "Number estimated to rebalance keys.");
+
+        rebalancingKeysRate = mreg.hitRateMetric("RebalancingKeysRate",
+            "Estimated rebalancing speed in keys",
+            REBALANCE_RATE_INTERVAL,
+            20);
+
+        rebalancingBytesRate = mreg.hitRateMetric("RebalancingBytesRate",
+            "Estimated rebalancing speed in bytes",
+            REBALANCE_RATE_INTERVAL,
+            20);
+
+        rebalanceClearingPartitions = mreg.metric("RebalanceClearingPartitionsLeft",
+            "Number of partitions need to be cleared before actual rebalance start.");
     }
 
     /**
@@ -449,35 +588,35 @@ public class CacheMetricsImpl implements CacheMetrics {
      * Clear metrics.
      */
     public void clear() {
-        reads.set(0);
-        writes.set(0);
-        rmCnt.set(0);
-        hits.set(0);
-        misses.set(0);
-        evictCnt.set(0);
-        txCommits.set(0);
-        txRollbacks.set(0);
-        putTimeNanos.set(0);
-        rmvTimeNanos.set(0);
-        getTimeNanos.set(0);
-        commitTimeNanos.set(0);
-        rollbackTimeNanos.set(0);
-
-        entryProcessorPuts.set(0);
-        entryProcessorRemovals.set(0);
-        entryProcessorReadOnlyInvocations.set(0);
-        entryProcessorMisses.set(0);
-        entryProcessorHits.set(0);
-        entryProcessorInvokeTimeNanos.set(0);
-        entryProcessorMaxInvocationTime.set(0);
-        entryProcessorMinInvocationTime.set(0);
-
-        offHeapGets.set(0);
-        offHeapPuts.set(0);
-        offHeapRemoves.set(0);
-        offHeapHits.set(0);
-        offHeapMisses.set(0);
-        offHeapEvicts.set(0);
+        reads.reset();
+        writes.reset();
+        rmCnt.reset();
+        hits.reset();
+        misses.reset();
+        evictCnt.reset();
+        txCommits.reset();
+        txRollbacks.reset();
+        putTimeNanos.reset();
+        rmvTimeNanos.reset();
+        getTimeNanos.reset();
+        commitTimeNanos.reset();
+        rollbackTimeNanos.reset();
+
+        entryProcessorPuts.reset();
+        entryProcessorRemovals.reset();
+        entryProcessorReadOnlyInvocations.reset();
+        entryProcessorMisses.reset();
+        entryProcessorHits.reset();
+        entryProcessorInvokeTimeNanos.reset();
+        entryProcessorMaxInvocationTime.reset();
+        entryProcessorMinInvocationTime.reset();
+
+        offHeapGets.reset();
+        offHeapPuts.reset();
+        offHeapRemoves.reset();
+        offHeapHits.reset();
+        offHeapMisses.reset();
+        offHeapEvicts.reset();
 
         clearRebalanceCounters();
 
@@ -648,12 +787,12 @@ public class CacheMetricsImpl implements CacheMetrics {
      * @param isHit Hit or miss flag.
      */
     public void onRead(boolean isHit) {
-        reads.incrementAndGet();
+        reads.increment();
 
         if (isHit)
-            hits.incrementAndGet();
+            hits.increment();
         else
-            misses.incrementAndGet();
+            misses.increment();
 
         if (delegate != null)
             delegate.onRead(isHit);
@@ -665,12 +804,12 @@ public class CacheMetricsImpl implements CacheMetrics {
      * @param isHit Hit or miss flag.
      */
     public void onInvokeUpdate(boolean isHit) {
-        entryProcessorPuts.incrementAndGet();
+        entryProcessorPuts.increment();
 
         if (isHit)
-            entryProcessorHits.incrementAndGet();
+            entryProcessorHits.increment();
         else
-            entryProcessorMisses.incrementAndGet();
+            entryProcessorMisses.increment();
 
         if (delegate != null)
             delegate.onInvokeUpdate(isHit);
@@ -682,12 +821,12 @@ public class CacheMetricsImpl implements CacheMetrics {
      * @param isHit Hit or miss flag.
      */
     public void onInvokeRemove(boolean isHit) {
-        entryProcessorRemovals.incrementAndGet();
+        entryProcessorRemovals.increment();
 
         if (isHit)
-            entryProcessorHits.incrementAndGet();
+            entryProcessorHits.increment();
         else
-            entryProcessorMisses.incrementAndGet();
+            entryProcessorMisses.increment();
 
         if (delegate != null)
             delegate.onInvokeRemove(isHit);
@@ -699,12 +838,12 @@ public class CacheMetricsImpl implements CacheMetrics {
      * @param isHit Hit or miss flag.
      */
     public void onReadOnlyInvoke(boolean isHit) {
-        entryProcessorReadOnlyInvocations.incrementAndGet();
+        entryProcessorReadOnlyInvocations.increment();
 
         if (isHit)
-            entryProcessorHits.incrementAndGet();
+            entryProcessorHits.increment();
         else
-            entryProcessorMisses.incrementAndGet();
+            entryProcessorMisses.increment();
 
         if (delegate != null)
             delegate.onReadOnlyInvoke(isHit);
@@ -716,7 +855,7 @@ public class CacheMetricsImpl implements CacheMetrics {
      * @param duration Duration.
      */
     public void addInvokeTimeNanos(long duration) {
-        entryProcessorInvokeTimeNanos.addAndGet(duration);
+        entryProcessorInvokeTimeNanos.add(duration);
 
         recalculateInvokeMinTimeNanos(duration);
 
@@ -736,7 +875,7 @@ public class CacheMetricsImpl implements CacheMetrics {
         long minTime = entryProcessorMinInvocationTime.longValue();
 
         while (minTime > duration || minTime == 0) {
-            if (entryProcessorMinInvocationTime.compareAndSet(minTime, duration))
+            if (MetricUtils.compareAndSet(entryProcessorMinInvocationTime, minTime, duration))
                 break;
             else
                 minTime = entryProcessorMinInvocationTime.longValue();
@@ -752,7 +891,7 @@ public class CacheMetricsImpl implements CacheMetrics {
         long maxTime = entryProcessorMaxInvocationTime.longValue();
 
         while (maxTime < duration) {
-            if (entryProcessorMaxInvocationTime.compareAndSet(maxTime, duration))
+            if (MetricUtils.compareAndSet(entryProcessorMaxInvocationTime, maxTime, duration))
                 break;
             else
                 maxTime = entryProcessorMaxInvocationTime.longValue();
@@ -763,7 +902,7 @@ public class CacheMetricsImpl implements CacheMetrics {
      * Cache write callback.
      */
     public void onWrite() {
-        writes.incrementAndGet();
+        writes.increment();
 
         if (delegate != null)
             delegate.onWrite();
@@ -773,7 +912,7 @@ public class CacheMetricsImpl implements CacheMetrics {
      * Cache remove callback.
      */
     public void onRemove(){
-        rmCnt.incrementAndGet();
+        rmCnt.increment();
 
         if (delegate != null)
             delegate.onRemove();
@@ -783,7 +922,7 @@ public class CacheMetricsImpl implements CacheMetrics {
      * Cache remove callback.
      */
     public void onEvict() {
-        evictCnt.incrementAndGet();
+        evictCnt.increment();
 
         if (delegate != null)
             delegate.onEvict();
@@ -795,8 +934,8 @@ public class CacheMetricsImpl implements CacheMetrics {
      * @param duration the time taken in nanoseconds.
      */
     public void onTxCommit(long duration) {
-        txCommits.incrementAndGet();
-        commitTimeNanos.addAndGet(duration);
+        txCommits.increment();
+        commitTimeNanos.add(duration);
 
         if (delegate != null)
             delegate.onTxCommit(duration);
@@ -808,8 +947,8 @@ public class CacheMetricsImpl implements CacheMetrics {
      * @param duration the time taken in nanoseconds.
      */
     public void onTxRollback(long duration) {
-        txRollbacks.incrementAndGet();
-        rollbackTimeNanos.addAndGet(duration);
+        txRollbacks.increment();
+        rollbackTimeNanos.add(duration);
 
         if (delegate != null)
             delegate.onTxRollback(duration);
@@ -821,7 +960,7 @@ public class CacheMetricsImpl implements CacheMetrics {
      * @param duration the time taken in nanoseconds.
      */
     public void addGetTimeNanos(long duration) {
-        getTimeNanos.addAndGet(duration);
+        getTimeNanos.add(duration);
 
         if (delegate != null)
             delegate.addGetTimeNanos(duration);
@@ -833,7 +972,7 @@ public class CacheMetricsImpl implements CacheMetrics {
      * @param duration the time taken in nanoseconds.
      */
     public void addPutTimeNanos(long duration) {
-        putTimeNanos.addAndGet(duration);
+        putTimeNanos.add(duration);
 
         if (delegate != null)
             delegate.addPutTimeNanos(duration);
@@ -845,7 +984,7 @@ public class CacheMetricsImpl implements CacheMetrics {
      * @param duration the time taken in nanoseconds.
      */
     public void addRemoveTimeNanos(long duration) {
-        rmvTimeNanos.addAndGet(duration);
+        rmvTimeNanos.add(duration);
 
         if (delegate != null)
             delegate.addRemoveTimeNanos(duration);
@@ -857,8 +996,8 @@ public class CacheMetricsImpl implements CacheMetrics {
      * @param duration the time taken in nanoseconds.
      */
     public void addRemoveAndGetTimeNanos(long duration) {
-        rmvTimeNanos.addAndGet(duration);
-        getTimeNanos.addAndGet(duration);
+        rmvTimeNanos.add(duration);
+        getTimeNanos.add(duration);
 
         if (delegate != null)
             delegate.addRemoveAndGetTimeNanos(duration);
@@ -870,8 +1009,8 @@ public class CacheMetricsImpl implements CacheMetrics {
      * @param duration the time taken in nanoseconds.
      */
     public void addPutAndGetTimeNanos(long duration) {
-        putTimeNanos.addAndGet(duration);
-        getTimeNanos.addAndGet(duration);
+        putTimeNanos.add(duration);
+        getTimeNanos.add(duration);
 
         if (delegate != null)
             delegate.addPutAndGetTimeNanos(duration);
@@ -1075,36 +1214,36 @@ public class CacheMetricsImpl implements CacheMetrics {
 
     /** {@inheritDoc} */
     @Override public long getRebalancingKeysRate() {
-        return rebalancingKeysRate.getRate();
+        return rebalancingKeysRate.value();
     }
 
     /** {@inheritDoc} */
     @Override public long getRebalancingBytesRate() {
-        return rebalancingBytesRate.getRate();
+        return rebalancingBytesRate.value();
     }
 
     /**
      * Clear rebalance counters.
      */
     public void clearRebalanceCounters() {
-        estimatedRebalancingKeys.set(0);
+        estimatedRebalancingKeys.reset();
 
-        rebalancedKeys.set(0);
+        rebalancedKeys.reset();
 
-        totalRebalancedBytes.set(0);
+        totalRebalancedBytes.reset();
 
-        rebalancingBytesRate.clear();
+        rebalancingBytesRate.reset();
 
-        rebalancingKeysRate.clear();
+        rebalancingKeysRate.reset();
 
-        rebalanceStartTime.set(-1L);
+        rebalanceStartTime.value(-1L);
     }
 
     /**
      *
      */
     public void startRebalance(long delay){
-        rebalanceStartTime.set(delay + U.currentTimeMillis());
+        rebalanceStartTime.value(delay + U.currentTimeMillis());
     }
 
     /** {@inheritDoc} */
@@ -1119,7 +1258,7 @@ public class CacheMetricsImpl implements CacheMetrics {
 
     /** {@inheritDoc} */
     @Override public long getEstimatedRebalancingFinishTime() {
-        long rate = rebalancingKeysRate.getRate();
+        long rate = rebalancingKeysRate.value();
 
         return rate <= 0 ? -1L :
             ((getKeysToRebalanceLeft() / rate) * REBALANCE_RATE_INTERVAL) + U.currentTimeMillis();
@@ -1140,7 +1279,7 @@ public class CacheMetricsImpl implements CacheMetrics {
      * @param partitions Partitions number.
      */
     public void rebalanceClearingPartitions(int partitions) {
-        rebalanceClearingPartitions.set(partitions);
+        rebalanceClearingPartitions.value(partitions);
     }
 
     /**
@@ -1151,16 +1290,16 @@ public class CacheMetricsImpl implements CacheMetrics {
         if (keysCnt == null)
             return;
 
-        estimatedRebalancingKeys.addAndGet(keysCnt);
+        estimatedRebalancingKeys.add(keysCnt);
     }
 
     /**
      * Rebalance entry store callback.
      */
     public void onRebalanceKeyReceived() {
-        rebalancedKeys.incrementAndGet();
+        rebalancedKeys.increment();
 
-        rebalancingKeysRate.onHit();
+        rebalancingKeysRate.increment();
     }
 
     /**
@@ -1169,9 +1308,9 @@ public class CacheMetricsImpl implements CacheMetrics {
      * @param batchSize Batch size in bytes.
      */
     public void onRebalanceBatchReceived(long batchSize) {
-        totalRebalancedBytes.addAndGet(batchSize);
+        totalRebalancedBytes.add(batchSize);
 
-        rebalancingBytesRate.onHits(batchSize);
+        rebalancingBytesRate.add(batchSize);
     }
 
     /**
@@ -1194,12 +1333,12 @@ public class CacheMetricsImpl implements CacheMetrics {
      * @param hit Hit or miss flag.
      */
     public void onOffHeapRead(boolean hit) {
-        offHeapGets.incrementAndGet();
+        offHeapGets.increment();
 
         if (hit)
-            offHeapHits.incrementAndGet();
+            offHeapHits.increment();
         else
-            offHeapMisses.incrementAndGet();
+            offHeapMisses.increment();
 
         if (delegate != null)
             delegate.onOffHeapRead(hit);
@@ -1209,7 +1348,7 @@ public class CacheMetricsImpl implements CacheMetrics {
      * Off-heap write callback.
      */
     public void onOffHeapWrite() {
-        offHeapPuts.incrementAndGet();
+        offHeapPuts.increment();
 
         if (delegate != null)
             delegate.onOffHeapWrite();
@@ -1219,7 +1358,7 @@ public class CacheMetricsImpl implements CacheMetrics {
      * Off-heap remove callback.
      */
     public void onOffHeapRemove() {
-        offHeapRemoves.incrementAndGet();
+        offHeapRemoves.increment();
 
         if (delegate != null)
             delegate.onOffHeapRemove();
@@ -1229,12 +1368,17 @@ public class CacheMetricsImpl implements CacheMetrics {
      * Off-heap evict callback.
      */
     public void onOffHeapEvict() {
-        offHeapEvicts.incrementAndGet();
+        offHeapEvicts.increment();
 
         if (delegate != null)
             delegate.onOffHeapEvict();
     }
 
+    /** @return Prefix for the cache metrics. */
+    public String metricsPrefix() {
+        return prefix;
+    }
+
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(CacheMetricsImpl.class, this);
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index b456172..04d73b5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
@@ -105,6 +105,7 @@ import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.processors.datastreamer.DataStreamerEntry;
 import org.apache.ignite.internal.processors.datastreamer.DataStreamerImpl;
 import org.apache.ignite.internal.processors.dr.IgniteDrDataStreamerCacheUpdater;
+import org.apache.ignite.internal.processors.metric.MetricRegistry;
 import org.apache.ignite.internal.processors.platform.cache.PlatformCacheEntryFilter;
 import org.apache.ignite.internal.processors.task.GridInternal;
 import org.apache.ignite.internal.transactions.IgniteTxHeuristicCheckedException;
@@ -339,7 +340,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         log = ctx.logger(getClass());
         txLockMsgLog = ctx.shared().txLockMessageLogger();
 
-        metrics = new CacheMetricsImpl(ctx);
+        metrics = new CacheMetricsImpl(ctx, isNear() ? "near" : null);
 
         locMxBean = new CacheLocalMetricsMXBeanImpl(this);
         clusterMxBean = new CacheClusterMetricsMXBeanImpl(this);
@@ -651,6 +652,12 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         // Nulling thread local reference to ensure values will be eventually GCed
         // no matter what references these futures are holding.
         lastFut = null;
+
+        MetricRegistry mreg = ctx.kernalContext().metric().registry();
+
+        mreg.withPrefix(metrics.metricsPrefix())
+            .getMetrics()
+            .forEach(m -> mreg.remove(m.name()));
     }
 
     /**
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
index ff87d70..acd9ca0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
@@ -93,7 +93,7 @@ import org.apache.ignite.internal.processors.cache.tree.mvcc.search.MvccSnapshot
 import org.apache.ignite.internal.processors.cache.tree.mvcc.search.MvccTreeClosure;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.processors.query.GridQueryRowCacheCleaner;
-import org.apache.ignite.internal.stat.IoStatisticsHolder;
+import org.apache.ignite.internal.metric.IoStatisticsHolder;
 import org.apache.ignite.internal.transactions.IgniteTxUnexpectedStateCheckedException;
 import org.apache.ignite.internal.util.GridAtomicLong;
 import org.apache.ignite.internal.util.GridCloseableIteratorAdapter;
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/CacheDataRowAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/CacheDataRowAdapter.java
index 8e30e9f..06e7214 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/CacheDataRowAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/CacheDataRowAdapter.java
@@ -19,6 +19,8 @@ package org.apache.ignite.internal.processors.cache.persistence;
 
 import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.metric.IoStatisticsHolder;
+import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp;
 import org.apache.ignite.internal.pagemem.PageIdUtils;
 import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.pagemem.PageUtils;
@@ -36,8 +38,6 @@ import org.apache.ignite.internal.processors.cache.persistence.tree.io.CacheVers
 import org.apache.ignite.internal.processors.cache.persistence.tree.io.DataPageIO;
 import org.apache.ignite.internal.processors.cache.persistence.tree.io.DataPagePayload;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
-import org.apache.ignite.internal.stat.IoStatisticsHolder;
-import org.apache.ignite.internal.stat.IoStatisticsHolderNoOp;
 import org.apache.ignite.internal.util.GridLongList;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DataRegionMetricsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DataRegionMetricsImpl.java
index 15064c8..606c2c7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DataRegionMetricsImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DataRegionMetricsImpl.java
@@ -18,26 +18,34 @@ package org.apache.ignite.internal.processors.cache.persistence;
 
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.atomic.LongAdder;
 import org.apache.ignite.DataRegionMetrics;
 import org.apache.ignite.DataRegionMetricsProvider;
 import org.apache.ignite.configuration.DataRegionConfiguration;
 import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.processors.cache.CacheGroupMetricsMXBeanImpl.GroupAllocationTracker;
-import org.apache.ignite.internal.processors.cache.ratemetrics.HitRateMetrics;
+import org.apache.ignite.internal.processors.metric.MetricRegistryImpl;
 import org.apache.ignite.internal.util.typedef.internal.U;
-import org.jetbrains.annotations.Nullable;
+import org.apache.ignite.internal.processors.metric.MetricRegistry;
+import org.apache.ignite.internal.processors.metric.impl.HitRateMetric;
+import org.apache.ignite.internal.processors.metric.impl.LongAdderMetricImpl;
+import org.apache.ignite.internal.processors.metric.impl.LongMetricImpl;
 
 /**
  *
  */
 public class DataRegionMetricsImpl implements DataRegionMetrics, AllocatedPageTracker {
+    /**
+     * Data region metrics prefix.
+     * Full name will contain {@link DataRegionConfiguration#getName()} also.
+     * {@code "io.dataregion.default"}, for example.
+     */
+    public static final String DATAREGION_METRICS_PREFIX = "io.dataregion";
+
     /** */
     private final DataRegionMetricsProvider dataRegionMetricsProvider;
 
     /** */
-    private final LongAdder totalAllocatedPages = new LongAdder();
+    private final LongAdderMetricImpl totalAllocatedPages;
 
     /** */
     private final ConcurrentMap<Integer, GroupAllocationTracker> grpAllocationTrackers = new ConcurrentHashMap<>();
@@ -45,25 +53,25 @@ public class DataRegionMetricsImpl implements DataRegionMetrics, AllocatedPageTr
     /**
      * Counter for number of pages occupied by large entries (one entry is larger than one page).
      */
-    private final LongAdder largeEntriesPages = new LongAdder();
+    private final LongAdderMetricImpl largeEntriesPages;
 
     /** Counter for number of dirty pages. */
-    private final LongAdder dirtyPages = new LongAdder();
+    private final LongAdderMetricImpl dirtyPages;
 
     /** */
-    private final LongAdder readPages = new LongAdder();
+    private final LongAdderMetricImpl readPages;
 
     /** */
-    private final LongAdder writtenPages = new LongAdder();
+    private final LongAdderMetricImpl writtenPages;
 
     /** */
-    private final LongAdder replacedPages = new LongAdder();
+    private final LongAdderMetricImpl replacedPages;
 
     /** */
-    private final AtomicLong offHeapSize = new AtomicLong();
+    private final LongMetricImpl offHeapSize;
 
     /** */
-    private final AtomicLong checkpointBufferSize = new AtomicLong();
+    private final LongMetricImpl checkpointBufferSize;
 
     /** */
     private volatile boolean metricsEnabled;
@@ -75,16 +83,16 @@ public class DataRegionMetricsImpl implements DataRegionMetrics, AllocatedPageTr
     private volatile int subInts;
 
     /** Allocation rate calculator. */
-    private volatile HitRateMetrics allocRate = new HitRateMetrics(60_000, 5);
+    private final HitRateMetric allocRate;
 
     /** Eviction rate calculator. */
-    private volatile HitRateMetrics evictRate = new HitRateMetrics(60_000, 5);
+    private final HitRateMetric evictRate;
 
     /** */
-    private volatile HitRateMetrics pageReplaceRate = new HitRateMetrics(60_000, 5);
+    private final HitRateMetric pageReplaceRate;
 
     /** */
-    private volatile HitRateMetrics pageReplaceAge = new HitRateMetrics(60_000, 5);
+    private final HitRateMetric pageReplaceAge;
 
     /** */
     private final DataRegionConfiguration memPlcCfg;
@@ -95,26 +103,119 @@ public class DataRegionMetricsImpl implements DataRegionMetrics, AllocatedPageTr
     /** Time interval (in milliseconds) when allocations/evictions are counted to calculate rate. */
     private volatile long rateTimeInterval;
 
+    /** For test purposes only. */
+    private static final DataRegionMetricsProvider NO_OP_METRICS = new DataRegionMetricsProvider() {
+        /** {@inheritDoc} */
+        @Override public long partiallyFilledPagesFreeSpace() {
+            return 0;
+        }
+
+        /** {@inheritDoc} */
+        @Override public long emptyDataPages() {
+            return 0;
+        }
+    };
+
     /**
      * @param memPlcCfg DataRegionConfiguration.
      */
     public DataRegionMetricsImpl(DataRegionConfiguration memPlcCfg) {
-        this(memPlcCfg, null);
+        this(memPlcCfg, new MetricRegistryImpl(), NO_OP_METRICS);
     }
 
     /**
      * @param memPlcCfg DataRegionConfiguration.
+     * @param mreg Metrics registry.
+     * @param dataRegionMetricsProvider Data region metrics provider.
      */
     public DataRegionMetricsImpl(DataRegionConfiguration memPlcCfg,
-                                 @Nullable DataRegionMetricsProvider dataRegionMetricsProvider) {
+        MetricRegistry mreg,
+        DataRegionMetricsProvider dataRegionMetricsProvider) {
         this.memPlcCfg = memPlcCfg;
         this.dataRegionMetricsProvider = dataRegionMetricsProvider;
 
         metricsEnabled = memPlcCfg.isMetricsEnabled();
 
+        persistenceEnabled = memPlcCfg.isPersistenceEnabled();
+
         rateTimeInterval = memPlcCfg.getMetricsRateTimeInterval();
 
         subInts = memPlcCfg.getMetricsSubIntervalCount();
+
+        MetricRegistry mset = mreg.withPrefix(DATAREGION_METRICS_PREFIX, memPlcCfg.getName());
+
+        totalAllocatedPages = mset.longAdderMetric("TotalAllocatedPages",
+            "Total number of allocated pages.");
+
+        allocRate = mset.hitRateMetric("AllocationRate",
+            "Allocation rate (pages per second) averaged across rateTimeInternal.",
+            60_000,
+            5);
+
+        evictRate = mset.hitRateMetric("EvictionRate",
+            "Eviction rate (pages per second).",
+            60_000,
+            5);
+
+        pageReplaceRate = mset.hitRateMetric("PagesReplaceRate",
+            "Rate at which pages in memory are replaced with pages from persistent storage (pages per second).",
+            60_000,
+            5);
+
+        pageReplaceAge = mset.hitRateMetric("PagesReplaceAge",
+            "Average age at which pages in memory are replaced with pages from persistent storage (milliseconds).",
+            60_000,
+            5);
+
+        largeEntriesPages = mset.longAdderMetric("LargeEntriesPagesCount",
+            "Count of pages that fully ocupied by large entries that go beyond page size");
+
+        dirtyPages = mset.longAdderMetric("DirtyPages",
+            "Number of pages in memory not yet synchronized with persistent storage.");
+
+        readPages = mset.longAdderMetric("PagesRead",
+            "Number of pages read from last restart.");
+
+        writtenPages = mset.longAdderMetric("PagesWritten",
+            "Number of pages written from last restart.");
+
+        replacedPages = mset.longAdderMetric("PagesReplaced",
+            "Number of pages replaced from last restart.");
+
+        offHeapSize = mset.metric("OffHeapSize",
+            "Offheap size in bytes.");
+
+        checkpointBufferSize = mset.metric("CheckpointBufferSize",
+            "Checkpoint buffer size in bytes.");
+
+        mset.register("EmptyDataPages",
+            dataRegionMetricsProvider::emptyDataPages,
+            "Calculates empty data pages count for region. It counts only totally free pages that can be reused " +
+                "(e. g. pages that are contained in reuse bucket of free list).");
+
+        mset.register("PagesFillFactor",
+            this::getPagesFillFactor,
+            "The percentage of the used space.");
+
+        mset.register("PhysicalMemoryPages",
+            this::getPhysicalMemoryPages,
+            "Number of pages residing in physical RAM.");
+
+        mset.register("OffheapUsedSize",
+            this::getOffheapUsedSize,
+            "Offheap used size in bytes.");
+
+        mset.register("TotalAllocatedSize",
+            this::getTotalAllocatedSize,
+            "Gets a total size of memory allocated in the data region, in bytes");
+
+        mset.register("PhysicalMemorySize",
+            this::getPhysicalMemorySize,
+            "Gets total size of pages loaded to the RAM, in bytes");
+
+        mset.register("UsedCheckpointBufferSize",
+            this::getUsedCheckpointBufferSize,
+            "Gets used checkpoint buffer size in bytes");
     }
 
     /** {@inheritDoc} */
@@ -144,7 +245,7 @@ public class DataRegionMetricsImpl implements DataRegionMetrics, AllocatedPageTr
         if (!metricsEnabled)
             return 0;
 
-        return ((float)allocRate.getRate() * 1000) / rateTimeInterval;
+        return ((float)allocRate.value() * 1000) / rateTimeInterval;
     }
 
     /** {@inheritDoc} */
@@ -152,7 +253,7 @@ public class DataRegionMetricsImpl implements DataRegionMetrics, AllocatedPageTr
         if (!metricsEnabled)
             return 0;
 
-        return ((float)evictRate.getRate() * 1000) / rateTimeInterval;
+        return ((float)evictRate.value() * 1000) / rateTimeInterval;
     }
 
     /** {@inheritDoc} */
@@ -161,7 +262,7 @@ public class DataRegionMetricsImpl implements DataRegionMetrics, AllocatedPageTr
             return 0;
 
         return totalAllocatedPages.longValue() != 0 ?
-                (float) largeEntriesPages.doubleValue() / totalAllocatedPages.longValue()
+                (float) largeEntriesPages.longValue() / totalAllocatedPages.longValue()
                 : 0;
     }
 
@@ -192,7 +293,7 @@ public class DataRegionMetricsImpl implements DataRegionMetrics, AllocatedPageTr
         if (!metricsEnabled || !persistenceEnabled)
             return 0;
 
-        return ((float)pageReplaceRate.getRate() * 1000) / rateTimeInterval;
+        return ((float)pageReplaceRate.value() * 1000) / rateTimeInterval;
     }
 
     /** {@inheritDoc} */
@@ -200,9 +301,9 @@ public class DataRegionMetricsImpl implements DataRegionMetrics, AllocatedPageTr
         if (!metricsEnabled || !persistenceEnabled)
             return 0;
 
-        long rep = pageReplaceRate.getRate();
+        long rep = pageReplaceRate.value();
 
-        return rep == 0 ? 0 : ((float)pageReplaceAge.getRate() / rep);
+        return rep == 0 ? 0 : ((float)pageReplaceAge.value() / rep);
     }
 
     /** {@inheritDoc} */
@@ -297,14 +398,14 @@ public class DataRegionMetricsImpl implements DataRegionMetrics, AllocatedPageTr
      * @param size Region size.
      */
     public void updateOffHeapSize(long size) {
-        this.offHeapSize.addAndGet(size);
+        this.offHeapSize.add(size);
     }
 
     /**
      * @param size Checkpoint buffer size.
      */
     public void updateCheckpointBufferSize(long size) {
-        this.checkpointBufferSize.addAndGet(size);
+        this.checkpointBufferSize.add(size);
     }
 
     /**
@@ -312,9 +413,9 @@ public class DataRegionMetricsImpl implements DataRegionMetrics, AllocatedPageTr
      */
     public void updatePageReplaceRate(long pageAge) {
         if (metricsEnabled) {
-            pageReplaceRate.onHit();
+            pageReplaceRate.increment();
 
-            pageReplaceAge.onHits(pageAge);
+            pageReplaceAge.add(pageAge);
 
             replacedPages.increment();
         }
@@ -393,7 +494,7 @@ public class DataRegionMetricsImpl implements DataRegionMetrics, AllocatedPageTr
      *
      */
     private void updateAllocationRateMetrics(long hits) {
-        allocRate.onHits(hits);
+        allocRate.add(hits);
     }
 
     /**
@@ -401,14 +502,7 @@ public class DataRegionMetricsImpl implements DataRegionMetrics, AllocatedPageTr
      */
     public void updateEvictionRate() {
         if (metricsEnabled)
-            evictRate.onHit();
-    }
-
-    /**
-     * @param intervalNum Interval number.
-     */
-    private long subInt(int intervalNum) {
-        return (rateTimeInterval * intervalNum) / subInts;
+            evictRate.increment();
     }
 
     /**
@@ -461,10 +555,10 @@ public class DataRegionMetricsImpl implements DataRegionMetrics, AllocatedPageTr
     public void rateTimeInterval(long rateTimeInterval) {
         this.rateTimeInterval = rateTimeInterval;
 
-        allocRate = new HitRateMetrics((int) rateTimeInterval, subInts);
-        evictRate = new HitRateMetrics((int) rateTimeInterval, subInts);
-        pageReplaceRate = new HitRateMetrics((int)rateTimeInterval, subInts);
-        pageReplaceAge = new HitRateMetrics((int)rateTimeInterval, subInts);
+        allocRate.reset(rateTimeInterval, subInts);
+        evictRate.reset(rateTimeInterval, subInts);
+        pageReplaceRate.reset(rateTimeInterval, subInts);
+        pageReplaceAge.reset(rateTimeInterval, subInts);
     }
 
     /**
@@ -481,10 +575,10 @@ public class DataRegionMetricsImpl implements DataRegionMetrics, AllocatedPageTr
         if (rateTimeInterval / subInts < 10)
             subInts = (int) rateTimeInterval / 10;
 
-        allocRate = new HitRateMetrics((int) rateTimeInterval, subInts);
-        evictRate = new HitRateMetrics((int) rateTimeInterval, subInts);
-        pageReplaceRate = new HitRateMetrics((int)rateTimeInterval, subInts);
-        pageReplaceAge = new HitRateMetrics((int)rateTimeInterval, subInts);
+        allocRate.reset(rateTimeInterval, subInts);
+        evictRate.reset(rateTimeInterval, subInts);
+        pageReplaceRate.reset(rateTimeInterval, subInts);
+        pageReplaceAge.reset(rateTimeInterval, subInts);
     }
 
     /**
@@ -498,11 +592,11 @@ public class DataRegionMetricsImpl implements DataRegionMetrics, AllocatedPageTr
         readPages.reset();
         writtenPages.reset();
         replacedPages.reset();
-        offHeapSize.set(0);
-        checkpointBufferSize.set(0);
-        allocRate.clear();
-        evictRate.clear();
-        pageReplaceRate.clear();
-        pageReplaceAge.clear();
+        offHeapSize.reset();
+        checkpointBufferSize.reset();
+        allocRate.reset();
+        evictRate.reset();
+        pageReplaceRate.reset();
+        pageReplaceAge.reset();
     }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DataRegionMetricsMXBeanImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DataRegionMetricsMXBeanImpl.java
index 046593d..2559a7b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DataRegionMetricsMXBeanImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DataRegionMetricsMXBeanImpl.java
@@ -18,11 +18,15 @@ package org.apache.ignite.internal.processors.cache.persistence;
 
 import org.apache.ignite.DataRegionMetrics;
 import org.apache.ignite.configuration.DataRegionConfiguration;
+import org.apache.ignite.internal.processors.metric.GridMetricManager;
 import org.apache.ignite.mxbean.DataRegionMetricsMXBean;
 
 /**
  * MBean to expose {@link DataRegionMetrics} through JMX interface.
+ *
+ * @deprecated Use {@link GridMetricManager} instead.
  */
+@Deprecated
 class DataRegionMetricsMXBeanImpl implements DataRegionMetricsMXBean {
     /** */
     private final DataRegionMetricsImpl memMetrics;
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DataStorageMetricsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DataStorageMetricsImpl.java
index 4565b58..675ae16 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DataStorageMetricsImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DataStorageMetricsImpl.java
@@ -17,57 +17,61 @@
 package org.apache.ignite.internal.processors.cache.persistence;
 
 import java.util.Collection;
-import java.util.concurrent.atomic.AtomicLong;
 import org.apache.ignite.DataRegionMetrics;
 import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager;
-import org.apache.ignite.internal.processors.cache.ratemetrics.HitRateMetrics;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteOutClosure;
 import org.apache.ignite.mxbean.DataStorageMetricsMXBean;
+import org.apache.ignite.internal.processors.metric.MetricRegistry;
+import org.apache.ignite.internal.processors.metric.impl.HitRateMetric;
+import org.apache.ignite.internal.processors.metric.impl.LongMetricImpl;
 
 /**
  *
  */
 public class DataStorageMetricsImpl implements DataStorageMetricsMXBean {
+    /** Prefix for all data storage metrics. */
+    public static final String DATASTORAGE_METRIC_PREFIX = "io.datastorage";
+
     /** */
-    private volatile HitRateMetrics walLoggingRate;
+    private final HitRateMetric walLoggingRate;
 
     /** */
-    private volatile HitRateMetrics walWritingRate;
+    private final HitRateMetric walWritingRate;
 
     /** */
-    private volatile HitRateMetrics walFsyncTimeDuration;
+    private final HitRateMetric walFsyncTimeDuration;
 
     /** */
-    private volatile HitRateMetrics walFsyncTimeNum;
+    private final HitRateMetric walFsyncTimeNum;
 
     /** */
-    private volatile HitRateMetrics walBuffPollSpinsNum;
+    private final HitRateMetric walBuffPollSpinsNum;
 
     /** */
-    private volatile long lastCpLockWaitDuration;
+    private final LongMetricImpl lastCpLockWaitDuration;
 
     /** */
-    private volatile long lastCpMarkDuration;
+    private final LongMetricImpl lastCpMarkDuration;
 
     /** */
-    private volatile long lastCpPagesWriteDuration;
+    private final LongMetricImpl lastCpPagesWriteDuration;
 
     /** */
-    private volatile long lastCpDuration;
+    private final LongMetricImpl lastCpDuration;
 
     /** */
-    private volatile long lastCpFsyncDuration;
+    private final LongMetricImpl lastCpFsyncDuration;
 
     /** */
-    private volatile long lastCpTotalPages;
+    private final LongMetricImpl lastCpTotalPages;
 
     /** */
-    private volatile long lastCpDataPages;
+    private final LongMetricImpl lastCpDataPages;
 
     /** */
-    private volatile long lastCpCowPages;
+    private final LongMetricImpl lastCpCowPages;
 
     /** */
     private volatile long rateTimeInterval;
@@ -85,26 +89,28 @@ public class DataStorageMetricsImpl implements DataStorageMetricsMXBean {
     private volatile IgniteOutClosure<Long> walSizeProvider;
 
     /** */
-    private volatile long lastWalSegmentRollOverTime;
+    private final LongMetricImpl lastWalSegmentRollOverTime;
 
     /** */
-    private final AtomicLong totalCheckpointTime = new AtomicLong();
+    private final LongMetricImpl totalCheckpointTime;
 
     /** */
     private volatile Collection<DataRegionMetrics> regionMetrics;
 
     /** */
-    private volatile long storageSize;
+    private final LongMetricImpl storageSize;
 
     /** */
-    private volatile long sparseStorageSize;
+    private final LongMetricImpl sparseStorageSize;
 
     /**
+     * @param mreg Metrics registry.
      * @param metricsEnabled Metrics enabled flag.
      * @param rateTimeInterval Rate time interval.
      * @param subInts Number of sub-intervals.
      */
     public DataStorageMetricsImpl(
+        MetricRegistry mreg,
         boolean metricsEnabled,
         long rateTimeInterval,
         int subInts
@@ -113,7 +119,80 @@ public class DataStorageMetricsImpl implements DataStorageMetricsMXBean {
         this.rateTimeInterval = rateTimeInterval;
         this.subInts = subInts;
 
-        resetRates();
+        MetricRegistry mset = mreg.withPrefix(DATASTORAGE_METRIC_PREFIX);
+
+        walLoggingRate = mset.hitRateMetric("WalLoggingRate",
+            "Average number of WAL records per second written during the last time interval.",
+            rateTimeInterval,
+            subInts);
+
+        walWritingRate = mset.hitRateMetric(
+            "WalWritingRate",
+            "Average number of bytes per second written during the last time interval.",
+            rateTimeInterval,
+            subInts);
+
+        walFsyncTimeDuration = mset.hitRateMetric(
+            "WalFsyncTimeDuration",
+            "Total duration of fsync",
+            rateTimeInterval,
+            subInts);
+
+        walFsyncTimeNum = mset.hitRateMetric(
+            "WalFsyncTimeNum",
+            "Total count of fsync",
+            rateTimeInterval,
+            subInts);
+
+        walBuffPollSpinsNum = mset.hitRateMetric(
+            "WalBuffPollSpinsRate",
+            "WAL buffer poll spins number over the last time interval.",
+            rateTimeInterval,
+            subInts);
+
+        lastCpLockWaitDuration = mset.metric("LastCheckpointLockWaitDuration",
+            "Duration of the checkpoint lock wait in milliseconds.");
+
+        lastCpMarkDuration = mset.metric("LastCheckpointMarkDuration",
+            "Duration of the checkpoint lock wait in milliseconds.");
+
+        lastCpPagesWriteDuration = mset.metric("LastCheckpointPagesWriteDuration",
+            "Duration of the checkpoint pages write in milliseconds.");
+
+        lastCpDuration = mset.metric("LastCheckpointDuration",
+            "Duration of the last checkpoint in milliseconds.");
+
+        lastCpFsyncDuration = mset.metric("LastCheckpointFsyncDuration",
+            "Duration of the sync phase of the last checkpoint in milliseconds.");
+
+        lastCpTotalPages = mset.metric("LastCheckpointTotalPagesNumber",
+            "Total number of pages written during the last checkpoint.");
+
+        lastCpDataPages = mset.metric("LastCheckpointDataPagesNumber",
+            "Total number of data pages written during the last checkpoint.");
+
+        lastCpCowPages = mset.metric("LastCheckpointCopiedOnWritePagesNumber",
+            "Number of pages copied to a temporary checkpoint buffer during the last checkpoint.");
+
+        lastWalSegmentRollOverTime = mset.metric("WalLastRollOverTime",
+            "Time of the last WAL segment rollover.");
+
+        totalCheckpointTime = mset.metric("CheckpointTotalTime",
+            "Total duration of checkpoint");
+
+        storageSize = mset.metric("StorageSize",
+            "Storage space allocated, in bytes.");
+
+        sparseStorageSize = mset.metric("SparseStorageSize",
+            "Storage space allocated adjusted for possible sparsity, in bytes.");
+
+        mset.register("WalArchiveSegments",
+            this::getWalArchiveSegments,
+            "Current number of WAL segments in the WAL archive.");
+
+        mset.register("WalTotalSize",
+            this::getWalTotalSize,
+            "Total size in bytes for storage wal files.");
     }
 
     /** {@inheritDoc} */
@@ -121,7 +200,7 @@ public class DataStorageMetricsImpl implements DataStorageMetricsMXBean {
         if (!metricsEnabled)
             return 0;
 
-        return ((float)walLoggingRate.getRate() * 1000) / rateTimeInterval;
+        return ((float)walLoggingRate.value() * 1000) / rateTimeInterval;
     }
 
     /** {@inheritDoc} */
@@ -129,7 +208,7 @@ public class DataStorageMetricsImpl implements DataStorageMetricsMXBean {
         if (!metricsEnabled)
             return 0;
 
-        return ((float)walWritingRate.getRate() * 1000) / rateTimeInterval;
+        return ((float)walWritingRate.value() * 1000) / rateTimeInterval;
     }
 
     /** {@inheritDoc} */
@@ -145,12 +224,12 @@ public class DataStorageMetricsImpl implements DataStorageMetricsMXBean {
         if (!metricsEnabled)
             return 0;
 
-        long numRate = walFsyncTimeNum.getRate();
+        long numRate = walFsyncTimeNum.value();
 
         if (numRate == 0)
             return 0;
 
-        return (float)walFsyncTimeDuration.getRate() / numRate;
+        return (float)walFsyncTimeDuration.value() / numRate;
     }
 
     /** {@inheritDoc} */
@@ -158,7 +237,7 @@ public class DataStorageMetricsImpl implements DataStorageMetricsMXBean {
         if (!metricsEnabled)
             return 0;
 
-        return walBuffPollSpinsNum.getRate();
+        return walBuffPollSpinsNum.value();
     }
 
 
@@ -167,7 +246,7 @@ public class DataStorageMetricsImpl implements DataStorageMetricsMXBean {
         if (!metricsEnabled)
             return 0;
 
-        return lastCpDuration;
+        return lastCpDuration.value();
     }
 
     /** {@inheritDoc} */
@@ -175,7 +254,7 @@ public class DataStorageMetricsImpl implements DataStorageMetricsMXBean {
         if (!metricsEnabled)
             return 0;
 
-        return lastCpLockWaitDuration;
+        return lastCpLockWaitDuration.value();
     }
 
     /** {@inheritDoc} */
@@ -183,7 +262,7 @@ public class DataStorageMetricsImpl implements DataStorageMetricsMXBean {
         if (!metricsEnabled)
             return 0;
 
-        return lastCpMarkDuration;
+        return lastCpMarkDuration.value();
     }
 
     /** {@inheritDoc} */
@@ -191,7 +270,7 @@ public class DataStorageMetricsImpl implements DataStorageMetricsMXBean {
         if (!metricsEnabled)
             return 0;
 
-        return lastCpPagesWriteDuration;
+        return lastCpPagesWriteDuration.value();
     }
 
     /** {@inheritDoc} */
@@ -199,7 +278,7 @@ public class DataStorageMetricsImpl implements DataStorageMetricsMXBean {
         if (!metricsEnabled)
             return 0;
 
-        return lastCpFsyncDuration;
+        return lastCpFsyncDuration.value();
     }
 
     /** {@inheritDoc} */
@@ -207,7 +286,7 @@ public class DataStorageMetricsImpl implements DataStorageMetricsMXBean {
         if (!metricsEnabled)
             return 0;
 
-        return lastCpTotalPages;
+        return lastCpTotalPages.value();
     }
 
     /** {@inheritDoc} */
@@ -215,7 +294,7 @@ public class DataStorageMetricsImpl implements DataStorageMetricsMXBean {
         if (!metricsEnabled)
             return 0;
 
-        return lastCpDataPages;
+        return lastCpDataPages.value();
     }
 
     /** {@inheritDoc} */
@@ -223,7 +302,7 @@ public class DataStorageMetricsImpl implements DataStorageMetricsMXBean {
         if (!metricsEnabled)
             return 0;
 
-        return lastCpCowPages;
+        return lastCpCowPages.value();
     }
 
     /** {@inheritDoc} */
@@ -265,7 +344,7 @@ public class DataStorageMetricsImpl implements DataStorageMetricsMXBean {
         if (!metricsEnabled)
             return 0;
 
-        return lastWalSegmentRollOverTime;
+        return lastWalSegmentRollOverTime.value();
     }
 
     /** {@inheritDoc} */
@@ -474,7 +553,7 @@ public class DataStorageMetricsImpl implements DataStorageMetricsMXBean {
      *
      */
     public void onWallRollOver() {
-        this.lastWalSegmentRollOverTime = U.currentTimeMillis();
+        this.lastWalSegmentRollOverTime.value(U.currentTimeMillis());
     }
 
     /**
@@ -493,12 +572,12 @@ public class DataStorageMetricsImpl implements DataStorageMetricsMXBean {
 
     /** {@inheritDoc} */
     @Override public long getStorageSize() {
-        return storageSize;
+        return storageSize.value();
     }
 
     /** {@inheritDoc} */
     @Override public long getSparseStorageSize() {
-        return sparseStorageSize;
+        return sparseStorageSize.value();
     }
 
     /**
@@ -524,18 +603,18 @@ public class DataStorageMetricsImpl implements DataStorageMetricsMXBean {
         long sparseStorageSize
     ) {
         if (metricsEnabled) {
-            lastCpLockWaitDuration = lockWaitDuration;
-            lastCpMarkDuration = markDuration;
-            lastCpPagesWriteDuration = pagesWriteDuration;
-            lastCpFsyncDuration = fsyncDuration;
-            lastCpDuration = duration;
-            lastCpTotalPages = totalPages;
-            lastCpDataPages = dataPages;
-            lastCpCowPages = cowPages;
-            this.storageSize = storageSize;
-            this.sparseStorageSize = sparseStorageSize;
-
-            totalCheckpointTime.addAndGet(duration);
+            lastCpLockWaitDuration.value(lockWaitDuration);
+            lastCpMarkDuration.value(markDuration);
+            lastCpPagesWriteDuration.value(pagesWriteDuration);
+            lastCpFsyncDuration.value(fsyncDuration);
+            lastCpDuration.value(duration);
+            lastCpTotalPages.value(totalPages);
+            lastCpDataPages.value(dataPages);
+            lastCpCowPages.value(cowPages);
+            this.storageSize.value(storageSize);
+            this.sparseStorageSize.value(sparseStorageSize);
+
+            totalCheckpointTime.add(duration);
         }
     }
 
@@ -543,14 +622,14 @@ public class DataStorageMetricsImpl implements DataStorageMetricsMXBean {
      *
      */
     public void onWalRecordLogged() {
-        walLoggingRate.onHit();
+        walLoggingRate.increment();
     }
 
     /**
      * @param size Size written.
      */
     public void onWalBytesWritten(int size) {
-        walWritingRate.onHits(size);
+        walWritingRate.add(size);
     }
 
     /**
@@ -559,26 +638,26 @@ public class DataStorageMetricsImpl implements DataStorageMetricsMXBean {
     public void onFsync(long nanoTime) {
         long microseconds = nanoTime / 1_000;
 
-        walFsyncTimeDuration.onHits(microseconds);
-        walFsyncTimeNum.onHit();
+        walFsyncTimeDuration.add(microseconds);
+        walFsyncTimeNum.increment();
     }
 
     /**
      * @param num Number.
      */
     public void onBuffPollSpin(int num) {
-        walBuffPollSpinsNum.onHits(num);
+        walBuffPollSpinsNum.add(num);
     }
 
     /**
      *
      */
     private void resetRates() {
-        walLoggingRate = new HitRateMetrics((int)rateTimeInterval, subInts);
-        walWritingRate = new HitRateMetrics((int)rateTimeInterval, subInts);
-        walBuffPollSpinsNum = new HitRateMetrics((int)rateTimeInterval, subInts);
+        walLoggingRate.reset(rateTimeInterval, subInts);
+        walWritingRate.reset(rateTimeInterval, subInts);
+        walBuffPollSpinsNum.reset(rateTimeInterval, subInts);
 
-        walFsyncTimeDuration = new HitRateMetrics((int)rateTimeInterval, subInts);
-        walFsyncTimeNum = new HitRateMetrics((int)rateTimeInterval, subInts);
+        walFsyncTimeDuration.reset(rateTimeInterval, subInts);
+        walFsyncTimeNum.reset(rateTimeInterval, subInts);
     }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DataStorageMetricsSnapshot.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DataStorageMetricsSnapshot.java
index 78b08bc..d99848c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DataStorageMetricsSnapshot.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DataStorageMetricsSnapshot.java
@@ -17,11 +17,13 @@
 package org.apache.ignite.internal.processors.cache.persistence;
 
 import org.apache.ignite.DataStorageMetrics;
+import org.apache.ignite.internal.processors.metric.GridMetricManager;
 import org.apache.ignite.internal.util.typedef.internal.S;
 
 /**
- *
+ * @deprecated Use {@link GridMetricManager} instead.
  */
+@Deprecated
 public class DataStorageMetricsSnapshot implements DataStorageMetrics {
     /** */
     private float walLoggingRate;
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DataStructure.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DataStructure.java
index e3c7eff..252e48f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DataStructure.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/DataStructure.java
@@ -31,8 +31,8 @@ import org.apache.ignite.internal.processors.cache.persistence.tree.reuse.ReuseB
 import org.apache.ignite.internal.processors.cache.persistence.tree.reuse.ReuseList;
 import org.apache.ignite.internal.processors.cache.persistence.tree.util.PageHandler;
 import org.apache.ignite.internal.processors.cache.persistence.tree.util.PageLockListener;
-import org.apache.ignite.internal.stat.IoStatisticsHolder;
-import org.apache.ignite.internal.stat.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.metric.IoStatisticsHolder;
+import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp;
 import org.apache.ignite.internal.util.typedef.internal.U;
 
 import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_DATA;
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
index 559309c..935f403 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
@@ -152,7 +152,7 @@ import org.apache.ignite.internal.processors.cache.persistence.wal.crc.IgniteDat
 import org.apache.ignite.internal.processors.compress.CompressionProcessor;
 import org.apache.ignite.internal.processors.port.GridPortRecord;
 import org.apache.ignite.internal.processors.query.GridQueryProcessor;
-import org.apache.ignite.internal.stat.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp;
 import org.apache.ignite.internal.util.GridConcurrentHashSet;
 import org.apache.ignite.internal.util.GridCountDownCallback;
 import org.apache.ignite.internal.util.GridMultiCollectionWrapper;
@@ -415,6 +415,7 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
         lockWaitTime = persistenceCfg.getLockWaitTime();
 
         persStoreMetrics = new DataStorageMetricsImpl(
+            ctx.metric().registry(),
             persistenceCfg.isMetricsEnabled(),
             persistenceCfg.getMetricsRateTimeInterval(),
             persistenceCfg.getMetricsSubIntervalCount()
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java
index 9803a50..f72ca10 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheDatabaseSharedManager.java
@@ -27,13 +27,12 @@ import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import javax.management.InstanceNotFoundException;
-
 import org.apache.ignite.DataRegionMetrics;
+import org.apache.ignite.DataRegionMetricsProvider;
 import org.apache.ignite.DataStorageMetrics;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.IgniteSystemProperties;
-import org.apache.ignite.DataRegionMetricsProvider;
 import org.apache.ignite.configuration.DataPageEvictionMode;
 import org.apache.ignite.configuration.DataRegionConfiguration;
 import org.apache.ignite.configuration.DataStorageConfiguration;
@@ -363,8 +362,10 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
         if (dfltMemPlcName == null)
             dfltMemPlcName = DFLT_DATA_REG_DEFAULT_NAME;
 
-        DataRegionMetricsImpl memMetrics =
-                new DataRegionMetricsImpl(dataRegionCfg, dataRegionMetricsProvider(dataRegionCfg));
+        DataRegionMetricsImpl memMetrics = new DataRegionMetricsImpl(
+            dataRegionCfg,
+            cctx.kernalContext().metric().registry(),
+            dataRegionMetricsProvider(dataRegionCfg));
 
         DataRegion region = initMemory(dataStorageCfg, dataRegionCfg, memMetrics, trackable);
 
@@ -1277,7 +1278,7 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
                 MBEAN_GROUP_NAME,
                 region.memoryMetrics().getName()
             );
-            }
+        }
 
         dataRegionMap.clear();
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/RowStore.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/RowStore.java
index 9a3d9d2..bdd1c2d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/RowStore.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/RowStore.java
@@ -26,7 +26,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
 import org.apache.ignite.internal.processors.cache.persistence.freelist.FreeList;
 import org.apache.ignite.internal.processors.cache.persistence.tree.util.PageHandler;
 import org.apache.ignite.internal.processors.query.GridQueryRowCacheCleaner;
-import org.apache.ignite.internal.stat.IoStatisticsHolder;
+import org.apache.ignite.internal.metric.IoStatisticsHolder;
 import org.apache.ignite.internal.util.typedef.internal.U;
 
 /**
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/AbstractFreeList.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/AbstractFreeList.java
index 22a79cc..b677116 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/AbstractFreeList.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/AbstractFreeList.java
@@ -40,8 +40,8 @@ import org.apache.ignite.internal.processors.cache.persistence.tree.reuse.ReuseB
 import org.apache.ignite.internal.processors.cache.persistence.tree.reuse.ReuseList;
 import org.apache.ignite.internal.processors.cache.persistence.tree.util.PageHandler;
 import org.apache.ignite.internal.processors.cache.persistence.tree.util.PageLockListener;
-import org.apache.ignite.internal.stat.IoStatisticsHolder;
-import org.apache.ignite.internal.stat.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.metric.IoStatisticsHolder;
+import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp;
 import org.apache.ignite.internal.util.typedef.internal.U;
 
 /**
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/CacheFreeList.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/CacheFreeList.java
index 755610c..ec63f21 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/CacheFreeList.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/CacheFreeList.java
@@ -25,7 +25,7 @@ import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
 import org.apache.ignite.internal.processors.cache.persistence.DataRegionMetricsImpl;
 import org.apache.ignite.internal.processors.cache.persistence.tree.reuse.ReuseList;
 import org.apache.ignite.internal.processors.cache.persistence.tree.util.PageLockListener;
-import org.apache.ignite.internal.stat.IoStatisticsHolder;
+import org.apache.ignite.internal.metric.IoStatisticsHolder;
 import org.apache.ignite.internal.util.typedef.internal.U;
 
 /**
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/FreeList.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/FreeList.java
index e28d421..28f5a50 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/FreeList.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/FreeList.java
@@ -21,7 +21,7 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.internal.processors.cache.persistence.Storable;
 import org.apache.ignite.internal.processors.cache.persistence.tree.util.PageHandler;
-import org.apache.ignite.internal.stat.IoStatisticsHolder;
+import org.apache.ignite.internal.metric.IoStatisticsHolder;
 
 /**
  */
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/PagesList.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/PagesList.java
index 135552b..7734ba4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/PagesList.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/freelist/PagesList.java
@@ -45,9 +45,9 @@ import org.apache.ignite.internal.processors.cache.persistence.tree.io.IOVersion
 import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
 import org.apache.ignite.internal.processors.cache.persistence.tree.reuse.ReuseBag;
 import org.apache.ignite.internal.processors.cache.persistence.tree.util.PageHandler;
+import org.apache.ignite.internal.metric.IoStatisticsHolder;
+import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp;
 import org.apache.ignite.internal.processors.cache.persistence.tree.util.PageLockListener;
-import org.apache.ignite.internal.stat.IoStatisticsHolder;
-import org.apache.ignite.internal.stat.IoStatisticsHolderNoOp;
 import org.apache.ignite.internal.util.GridArrays;
 import org.apache.ignite.internal.util.GridLongList;
 import org.apache.ignite.internal.util.typedef.F;
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/metastorage/MetastorageRowStore.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/metastorage/MetastorageRowStore.java
index ccc55dc..425c6dc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/metastorage/MetastorageRowStore.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/metastorage/MetastorageRowStore.java
@@ -20,7 +20,7 @@ package org.apache.ignite.internal.processors.cache.persistence.metastorage;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.processors.cache.persistence.IgniteCacheDatabaseSharedManager;
 import org.apache.ignite.internal.processors.cache.persistence.partstorage.PartitionMetaStorage;
-import org.apache.ignite.internal.stat.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp;
 
 /**
  *
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryEx.java
index 6932d0e..f839bed 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryEx.java
@@ -24,7 +24,7 @@ import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.pagemem.FullPageId;
 import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.processors.cache.persistence.StorageException;
-import org.apache.ignite.internal.stat.IoStatisticsHolder;
+import org.apache.ignite.internal.metric.IoStatisticsHolder;
 import org.apache.ignite.internal.util.GridMultiCollectionWrapper;
 import org.jetbrains.annotations.Nullable;
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java
index 9949dc7..cbb8b76 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImpl.java
@@ -75,8 +75,8 @@ import org.apache.ignite.internal.processors.cache.persistence.tree.io.TrackingP
 import org.apache.ignite.internal.processors.cache.persistence.wal.crc.IgniteDataIntegrityViolationException;
 import org.apache.ignite.internal.processors.compress.CompressionProcessor;
 import org.apache.ignite.internal.processors.query.GridQueryRowCacheCleaner;
-import org.apache.ignite.internal.stat.IoStatisticsHolder;
-import org.apache.ignite.internal.stat.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.metric.IoStatisticsHolder;
+import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp;
 import org.apache.ignite.internal.util.GridConcurrentHashSet;
 import org.apache.ignite.internal.util.GridLongList;
 import org.apache.ignite.internal.util.GridMultiCollectionWrapper;
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/partstorage/PartitionMetaStorage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/partstorage/PartitionMetaStorage.java
index 0dbd5f4..40aafae 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/partstorage/PartitionMetaStorage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/partstorage/PartitionMetaStorage.java
@@ -19,7 +19,7 @@ package org.apache.ignite.internal.processors.cache.persistence.partstorage;
 
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.processors.cache.persistence.Storable;
-import org.apache.ignite.internal.stat.IoStatisticsHolder;
+import org.apache.ignite.internal.metric.IoStatisticsHolder;
 
 /**
  * Provides a way to associate any {@link Storable} implementation as partition metadata.
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/BPlusTree.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/BPlusTree.java
index 624ee8d..b4b150c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/BPlusTree.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/BPlusTree.java
@@ -61,8 +61,8 @@ import org.apache.ignite.internal.processors.cache.persistence.tree.util.PageHan
 import org.apache.ignite.internal.processors.cache.persistence.tree.util.PageHandlerWrapper;
 import org.apache.ignite.internal.processors.cache.persistence.tree.util.PageLockListener;
 import org.apache.ignite.internal.processors.failure.FailureProcessor;
-import org.apache.ignite.internal.stat.IoStatisticsHolder;
-import org.apache.ignite.internal.stat.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.metric.IoStatisticsHolder;
+import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp;
 import org.apache.ignite.internal.util.GridArrays;
 import org.apache.ignite.internal.util.GridLongList;
 import org.apache.ignite.internal.util.IgniteTree;
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/PageIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/PageIO.java
index fafea22..93ee3b3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/PageIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/io/PageIO.java
@@ -46,8 +46,8 @@ import org.apache.ignite.internal.processors.cache.tree.mvcc.data.MvccCacheIdAwa
 import org.apache.ignite.internal.processors.cache.tree.mvcc.data.MvccCacheIdAwareDataLeafIO;
 import org.apache.ignite.internal.processors.cache.tree.mvcc.data.MvccDataInnerIO;
 import org.apache.ignite.internal.processors.cache.tree.mvcc.data.MvccDataLeafIO;
-import org.apache.ignite.internal.stat.IndexPageType;
-import org.apache.ignite.internal.stat.IoStatisticsHolder;
+import org.apache.ignite.internal.metric.IndexPageType;
+import org.apache.ignite.internal.metric.IoStatisticsHolder;
 import org.apache.ignite.internal.util.GridStringBuilder;
 import org.apache.ignite.spi.encryption.EncryptionSpi;
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/reuse/ReuseListImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/reuse/ReuseListImpl.java
index fa76c26..34b5779 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/reuse/ReuseListImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/reuse/ReuseListImpl.java
@@ -22,8 +22,8 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager;
 import org.apache.ignite.internal.processors.cache.persistence.freelist.PagesList;
+import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp;
 import org.apache.ignite.internal.processors.cache.persistence.tree.util.PageLockListener;
-import org.apache.ignite.internal.stat.IoStatisticsHolderNoOp;
 
 /**
  * Reuse list.
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/util/PageHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/util/PageHandler.java
index ff980af..283b3ba 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/util/PageHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/util/PageHandler.java
@@ -24,7 +24,7 @@ import org.apache.ignite.internal.pagemem.PageSupport;
 import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager;
 import org.apache.ignite.internal.pagemem.wal.record.delta.InitNewPageRecord;
 import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
-import org.apache.ignite.internal.stat.IoStatisticsHolder;
+import org.apache.ignite.internal.metric.IoStatisticsHolder;
 import org.apache.ignite.internal.util.GridUnsafe;
 import org.jetbrains.annotations.Nullable;
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java
index 30bb1ed..0a25f1c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java
@@ -59,6 +59,7 @@ import org.apache.ignite.internal.processors.closure.GridClosureProcessor;
 import org.apache.ignite.internal.processors.cluster.ClusterProcessor;
 import org.apache.ignite.internal.processors.cluster.GridClusterStateProcessor;
 import org.apache.ignite.internal.processors.compress.CompressionProcessor;
+import org.apache.ignite.internal.processors.configuration.distributed.DistributedConfigurationProcessor;
 import org.apache.ignite.internal.processors.continuous.GridContinuousProcessor;
 import org.apache.ignite.internal.processors.datastreamer.DataStreamProcessor;
 import org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor;
@@ -71,8 +72,8 @@ import org.apache.ignite.internal.processors.igfs.IgfsProcessorAdapter;
 import org.apache.ignite.internal.processors.job.GridJobProcessor;
 import org.apache.ignite.internal.processors.jobmetrics.GridJobMetricsProcessor;
 import org.apache.ignite.internal.processors.marshaller.GridMarshallerMappingProcessor;
-import org.apache.ignite.internal.processors.configuration.distributed.DistributedConfigurationProcessor;
 import org.apache.ignite.internal.processors.metastorage.DistributedMetaStorage;
+import org.apache.ignite.internal.processors.metric.GridMetricManager;
 import org.apache.ignite.internal.processors.odbc.ClientListenerProcessor;
 import org.apache.ignite.internal.processors.platform.PlatformProcessor;
 import org.apache.ignite.internal.processors.plugin.IgnitePluginProcessor;
@@ -89,7 +90,6 @@ import org.apache.ignite.internal.processors.session.GridTaskSessionProcessor;
 import org.apache.ignite.internal.processors.subscription.GridInternalSubscriptionProcessor;
 import org.apache.ignite.internal.processors.task.GridTaskProcessor;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor;
-import org.apache.ignite.internal.stat.IoStatisticsManager;
 import org.apache.ignite.internal.suggestions.GridPerformanceSuggestions;
 import org.apache.ignite.internal.util.IgniteExceptionRegistry;
 import org.apache.ignite.internal.util.StripedExecutor;
@@ -295,6 +295,11 @@ public class StandaloneGridKernalContext implements GridKernalContext {
     }
 
     /** {@inheritDoc} */
+    @Override public GridMetricManager metric() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
     @Override public GridCacheProcessor cache() {
         return null;
     }
@@ -663,10 +668,6 @@ public class StandaloneGridKernalContext implements GridKernalContext {
         return null;
     }
 
-    @Override public IoStatisticsManager ioStats() {
-        return null;
-    }
-
     /** {@inheritDoc} */
     @Override public Thread.UncaughtExceptionHandler uncaughtExceptionHandler() {
         return null;
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ratemetrics/HitRateMetrics.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ratemetrics/HitRateMetrics.java
deleted file mode 100644
index 9c096a6..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ratemetrics/HitRateMetrics.java
+++ /dev/null
@@ -1,179 +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.cache.ratemetrics;
-
-import java.util.concurrent.atomic.AtomicLongArray;
-import org.apache.ignite.internal.util.typedef.internal.A;
-import org.apache.ignite.internal.util.typedef.internal.U;
-
-/**
- * Accumulates approximate hit rate statistics.
- * Calculates number of hits in last {@link #rateTimeInterval} milliseconds.
- * Algorithm is based on circular array of {@link #size} hit counters, each is responsible for last corresponding time
- * interval of {@link #rateTimeInterval}/{@link #size} milliseconds. Resulting number of hits is sum of all counters.
- *
- * <p>Implementation is nonblocking and protected from hits loss.
- * Maximum relative error is 1/{@link #size}.
- * 2^55 - 1 hits per interval can be accumulated without numeric overflow.
- */
-public class HitRateMetrics {
-    /** Bits that store actual hit count. */
-    private static final int TAG_OFFSET = 56;
-
-    /** Useful part mask. */
-    private static final long NO_TAG_MASK = ~(-1L << TAG_OFFSET);
-
-    /** Time interval when hits are counted to calculate rate, in milliseconds. */
-    private final int rateTimeInterval;
-
-    /** Counters array size. */
-    private final int size;
-
-    /** Tagged counters. */
-    private volatile AtomicLongArray taggedCounters;
-
-    /** Last hit times. */
-    private volatile AtomicLongArray lastHitTimes;
-
-    /**
-     * @param rateTimeInterval Rate time interval.
-     * @param size Number of counters.
-     */
-    public HitRateMetrics(int rateTimeInterval, int size) {
-        A.ensure(rateTimeInterval > 0, "should be positive");
-
-        A.ensure(size > 1, "minimum is 2");
-
-        this.rateTimeInterval = rateTimeInterval;
-
-        this.size = size;
-
-        taggedCounters = new AtomicLongArray(size);
-
-        lastHitTimes = new AtomicLongArray(size);
-    }
-
-    /**
-     * Mark one hit.
-     */
-    public void onHit() {
-        onHits(1);
-    }
-
-    /**
-     * Mark multiple hits.
-     *
-     * @param hits Number of hits.
-     */
-    public void onHits(long hits) {
-        long curTs = U.currentTimeMillis();
-
-        int curPos = position(curTs);
-
-        clearIfObsolete(curTs, curPos);
-
-        lastHitTimes.set(curPos, curTs);
-
-        // Order is important. Hit won't be cleared by concurrent #clearIfObsolete.
-        taggedCounters.addAndGet(curPos, hits);
-    }
-
-    /**
-     * @return Total number of hits in last {@link #rateTimeInterval} milliseconds.
-     */
-    public long getRate() {
-        long curTs = U.currentTimeMillis();
-
-        long sum = 0;
-
-        for (int i = 0; i < size; i++) {
-            clearIfObsolete(curTs, i);
-
-            sum += untag(taggedCounters.get(i));
-        }
-
-        return sum;
-    }
-
-    /**
-     * Clear metrics.
-     */
-    public void clear() {
-        taggedCounters = new AtomicLongArray(size);
-
-        lastHitTimes = new AtomicLongArray(size);
-    }
-
-    /**
-     * @param curTs Current timestamp.
-     * @param i Index.
-     */
-    private void clearIfObsolete(long curTs, int i) {
-        long cur = taggedCounters.get(i);
-
-        byte curTag = getTag(cur);
-
-        long lastTs = lastHitTimes.get(i);
-
-        if (isObsolete(curTs, lastTs)) {
-            if (taggedCounters.compareAndSet(i, cur, taggedLongZero(++curTag))) // ABA problem prevention.
-                lastHitTimes.set(i, curTs);
-            // If CAS failed, counter is reset by another thread.
-        }
-    }
-
-    /**
-     * @param curTs Current timestamp.
-     * @param lastHitTime Last hit timestamp.
-     * @return True, is last hit time was too long ago.
-     */
-    private boolean isObsolete(long curTs, long lastHitTime) {
-        return curTs - lastHitTime > rateTimeInterval * (size - 1) / size;
-    }
-
-    /**
-     * @param time Timestamp.
-     * @return Index of counter for given timestamp.
-     */
-    private int position(long time) {
-        return (int)(time % rateTimeInterval * size) / rateTimeInterval;
-    }
-
-    /**
-     * @param tag Tag byte.
-     * @return 0L with given tag byte.
-     */
-    private static long taggedLongZero(byte tag) {
-        return ((long)tag << TAG_OFFSET);
-    }
-
-    /**
-     * @param l Tagged long.
-     * @return Long without tag byte.
-     */
-    private static long untag(long l) {
-        return l & NO_TAG_MASK;
-    }
-
-    /**
-     * @param taggedLong Tagged long.
-     * @return Tag byte.
-     */
-    private static byte getTag(long taggedLong) {
-        return (byte)(taggedLong >> TAG_OFFSET);
-    }
-}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ratemetrics/HitRateMetricsSandbox.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ratemetrics/HitRateMetricsSandbox.java
deleted file mode 100644
index 92ef411..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ratemetrics/HitRateMetricsSandbox.java
+++ /dev/null
@@ -1,95 +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.cache.ratemetrics;
-
-import java.awt.Dimension;
-import java.awt.EventQueue;
-import java.awt.FlowLayout;
-import java.awt.event.ActionEvent;
-import java.awt.event.ActionListener;
-import javax.swing.JButton;
-import javax.swing.JComponent;
-import javax.swing.JFrame;
-import javax.swing.JLabel;
-import javax.swing.JPanel;
-import javax.swing.Timer;
-import org.apache.ignite.internal.util.IgniteUtils;
-
-/**
- * Visualization of {@link HitRateMetrics}.
- */
-public class HitRateMetricsSandbox extends JFrame {
-    /** Serial version uid. */
-    private static final long serialVersionUID = 0L;
-
-    /** */
-    private final JLabel rateLb = new JLabel("0.0");
-
-    /** */
-    private final HitRateMetrics metrics = new HitRateMetrics(5_000, 20);
-
-    /**
-     * Default constructor.
-     */
-    private HitRateMetricsSandbox() {
-        IgniteUtils.onGridStart();
-
-        JButton hitBtn = new JButton("Hit");
-        hitBtn.addActionListener(new ActionListener() {
-            @Override public void actionPerformed(ActionEvent e) {
-                metrics.onHit();
-            }
-        });
-
-        new Timer(100, new ActionListener() {
-            @Override public void actionPerformed(ActionEvent evt) {
-                rateLb.setText(Double.toString(metrics.getRate()));
-            }
-        }).start();
-
-        setContentPane(createPanel(new JLabel("Hits in 5 seconds:"), rateLb, hitBtn));
-
-        setMinimumSize(new Dimension(300, 120));
-    }
-
-    /**
-     * @param components Components.
-     * @return Panel.
-     */
-    private JPanel createPanel(JComponent... components) {
-        JPanel panel = new JPanel();
-
-        panel.setLayout(new FlowLayout());
-
-        for (JComponent component : components)
-            panel.add(component);
-
-        return panel;
-    }
-
-    /**
-     * @param args Args.
-     */
-    public static void main(String[] args) {
-        EventQueue.invokeLater(new Runnable() {
-            @Override public void run() {
-                HitRateMetricsSandbox s = new HitRateMetricsSandbox();
-                s.setVisible(true);
-            }
-        });
-    }
-}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/CacheDataTree.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/CacheDataTree.java
index b85bc9d..e9232a1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/CacheDataTree.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/tree/CacheDataTree.java
@@ -43,7 +43,7 @@ import org.apache.ignite.internal.processors.cache.tree.mvcc.data.MvccDataInnerI
 import org.apache.ignite.internal.processors.cache.tree.mvcc.data.MvccDataLeafIO;
 import org.apache.ignite.internal.processors.cache.tree.mvcc.data.MvccDataRow;
 import org.apache.ignite.internal.processors.cache.tree.mvcc.search.MvccDataPageClosure;
-import org.apache.ignite.internal.stat.IoStatisticsHolder;
+import org.apache.ignite.internal.metric.IoStatisticsHolder;
 import org.apache.ignite.internal.util.GridUnsafe;
 import org.apache.ignite.internal.util.lang.GridCursor;
 import org.apache.ignite.internal.util.typedef.internal.CU;
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/stat/IoStatisticsHolderKey.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/metric/AbstractMetric.java
similarity index 56%
rename from modules/core/src/main/java/org/apache/ignite/internal/stat/IoStatisticsHolderKey.java
rename to modules/core/src/main/java/org/apache/ignite/internal/processors/metric/AbstractMetric.java
index 86fff4f..db411db 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/stat/IoStatisticsHolderKey.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/metric/AbstractMetric.java
@@ -13,57 +13,43 @@
  * 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.stat;
+package org.apache.ignite.internal.processors.metric;
 
-import java.util.Objects;
+import org.apache.ignite.spi.metric.Metric;
+import org.jetbrains.annotations.Nullable;
 
 /**
- * Statistics holder key
+ * Base class for {@code Metric} implementations.
  */
-public class IoStatisticsHolderKey {
+public abstract class AbstractMetric implements Metric {
     /** Name. */
     private final String name;
 
-    /** Second name. */
-    private final String subName;
+    /** Description. */
+    @Nullable private final String descr;
 
     /**
-     * Constructor.
-     *
      * @param name Name.
+     * @param descr Description.
      */
-    public IoStatisticsHolderKey(String name) {
-        this(name, null);
-    }
-
-    /**
-     * Constructor.
-     *
-     * @param name Name.
-     * @param subName Second name.
-     */
-    public IoStatisticsHolderKey(String name, String subName) {
+    public AbstractMetric(String name, String descr) {
         assert name != null;
+        assert !name.isEmpty();
 
         this.name = name;
-        this.subName = subName;
+        this.descr = descr;
     }
 
-    /**
-     * @return Name.
-     */
-    public String name() {
+    /** {@inheritDoc} */
+    @Override public String name() {
         return name;
     }
 
-    /**
-     * @return Second name.
-     */
-    public String subName() {
-        return subName;
+    /** {@inheritDoc} */
+    @Override @Nullable public String description() {
+        return descr;
     }
 
     /** {@inheritDoc} */
@@ -74,18 +60,16 @@ public class IoStatisticsHolderKey {
         if (o == null || getClass() != o.getClass())
             return false;
 
-        IoStatisticsHolderKey other = (IoStatisticsHolderKey)o;
+        AbstractMetric metric = (AbstractMetric)o;
 
-        return Objects.equals(name, other.name) && Objects.equals(subName, other.subName);
-    }
+        if (!name.equals(metric.name))
+            return false;
 
-    /** {@inheritDoc} */
-    @Override public int hashCode() {
-        return 31 * (name != null ? name.hashCode() : 0) + (subName != null ? subName.hashCode() : 0);
+        return true;
     }
 
     /** {@inheritDoc} */
-    @Override public String toString() {
-        return subName == null ? name : name + "." + subName;
+    @Override public int hashCode() {
+        return name.hashCode();
     }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/metric/GridMetricManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/metric/GridMetricManager.java
new file mode 100644
index 0000000..e7e2640
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/metric/GridMetricManager.java
@@ -0,0 +1,293 @@
+/*
+ * 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.metric;
+
+import java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.RejectedExecutionHandler;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.ThreadPoolExecutor;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.managers.GridManagerAdapter;
+import org.apache.ignite.internal.util.StripedExecutor;
+import org.apache.ignite.spi.metric.MetricExporterSpi;
+import org.apache.ignite.thread.IgniteStripedThreadPoolExecutor;
+import org.jetbrains.annotations.Nullable;
+
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
+
+/**
+ * This manager should provide {@link MetricRegistry} for each configured {@link MetricExporterSpi}.
+ *
+ * @see MetricExporterSpi
+ * @see MetricRegistry
+ */
+public class GridMetricManager extends GridManagerAdapter<MetricExporterSpi> {
+    /** */
+    public static final String ACTIVE_COUNT_DESC = "Approximate number of threads that are actively executing tasks.";
+
+    /** */
+    public static final String COMPLETED_TASK_DESC = "Approximate total number of tasks that have completed execution.";
+
+    /** */
+    public static final String CORE_SIZE_DESC = "The core number of threads.";
+
+    /** */
+    public static final String LARGEST_SIZE_DESC = "Largest number of threads that have ever simultaneously been in the pool.";
+
+    /** */
+    public static final String MAX_SIZE_DESC = "The maximum allowed number of threads.";
+
+    /** */
+    public static final String POOL_SIZE_DESC = "Current number of threads in the pool.";
+
+    /** */
+    public static final String TASK_COUNT_DESC = "Approximate total number of tasks that have been scheduled for execution.";
+
+    /** */
+    public static final String QUEUE_SIZE_DESC = "Current size of the execution queue.";
+
+    /** */
+    public static final String KEEP_ALIVE_TIME_DESC = "Thread keep-alive time, which is the amount of time which threads in excess of " +
+        "the core pool size may remain idle before being terminated.";
+
+    /** */
+    public static final String IS_SHUTDOWN_DESC = "True if this executor has been shut down.";
+
+    /** */
+    public static final String IS_TERMINATED_DESC = "True if all tasks have completed following shut down.";
+
+    /** */
+    public static final String IS_TERMINATING_DESC = "True if terminating but not yet terminated.";
+
+    /** */
+    public static final String REJ_HND_DESC = "Class name of current rejection handler.";
+
+    /** */
+    public static final String THRD_FACTORY_DESC = "Class name of thread factory used to create new threads.";
+
+    /** Group for a thread pools. */
+    public static final String THREAD_POOLS = "threadPools";
+
+    /** Monitoring registry. */
+    private MetricRegistry mreg;
+
+    /**
+     * @param ctx Kernal context.
+     */
+    public GridMetricManager(GridKernalContext ctx) {
+        super(ctx, ctx.config().getMetricExporterSpi());
+
+        mreg = new MetricRegistryImpl(ctx.log(MetricRegistryImpl.class));
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws IgniteCheckedException {
+        for (MetricExporterSpi spi : getSpis())
+            spi.setMetricRegistry(mreg);
+
+        startSpi();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop(boolean cancel) throws IgniteCheckedException {
+        stopSpi();
+    }
+
+    /**
+     * @return Metric resitry.
+     */
+    public MetricRegistry registry() {
+        return mreg;
+    }
+
+    /**
+     * Registers all metrics for thread pools.
+     *
+     * @param utilityCachePool Utility cache pool.
+     * @param execSvc Executor service.
+     * @param svcExecSvc Services' executor service.
+     * @param sysExecSvc System executor service.
+     * @param stripedExecSvc Striped executor.
+     * @param p2pExecSvc P2P executor service.
+     * @param mgmtExecSvc Management executor service.
+     * @param igfsExecSvc IGFS executor service.
+     * @param dataStreamExecSvc Data stream executor service.
+     * @param restExecSvc Reset executor service.
+     * @param affExecSvc Affinity executor service.
+     * @param idxExecSvc Indexing executor service.
+     * @param callbackExecSvc Callback executor service.
+     * @param qryExecSvc Query executor service.
+     * @param schemaExecSvc Schema executor service.
+     * @param customExecSvcs Custom named executors.
+     */
+    public void registerThreadPools(
+        ExecutorService utilityCachePool,
+        ExecutorService execSvc,
+        ExecutorService svcExecSvc,
+        ExecutorService sysExecSvc,
+        StripedExecutor stripedExecSvc,
+        ExecutorService p2pExecSvc,
+        ExecutorService mgmtExecSvc,
+        ExecutorService igfsExecSvc,
+        StripedExecutor dataStreamExecSvc,
+        ExecutorService restExecSvc,
+        ExecutorService affExecSvc,
+        @Nullable ExecutorService idxExecSvc,
+        IgniteStripedThreadPoolExecutor callbackExecSvc,
+        ExecutorService qryExecSvc,
+        ExecutorService schemaExecSvc,
+        @Nullable final Map<String, ? extends ExecutorService> customExecSvcs
+    ) {
+        // Executors
+        monitorExecutor("GridUtilityCacheExecutor", utilityCachePool);
+        monitorExecutor("GridExecutionExecutor", execSvc);
+        monitorExecutor("GridServicesExecutor", svcExecSvc);
+        monitorExecutor("GridSystemExecutor", sysExecSvc);
+        monitorExecutor("GridClassLoadingExecutor", p2pExecSvc);
+        monitorExecutor("GridManagementExecutor", mgmtExecSvc);
+        monitorExecutor("GridIgfsExecutor", igfsExecSvc);
+        monitorExecutor("GridDataStreamExecutor", dataStreamExecSvc);
+        monitorExecutor("GridAffinityExecutor", affExecSvc);
+        monitorExecutor("GridCallbackExecutor", callbackExecSvc);
+        monitorExecutor("GridQueryExecutor", qryExecSvc);
+        monitorExecutor("GridSchemaExecutor", schemaExecSvc);
+
+        if (idxExecSvc != null)
+            monitorExecutor("GridIndexingExecutor", idxExecSvc);
+
+        if (ctx.config().getConnectorConfiguration() != null)
+            monitorExecutor("GridRestExecutor", restExecSvc);
+
+        if (stripedExecSvc != null) {
+            // Striped executor uses a custom adapter.
+            monitorStripedPool(stripedExecSvc);
+        }
+
+        if (customExecSvcs != null) {
+            for (Map.Entry<String, ? extends ExecutorService> entry : customExecSvcs.entrySet())
+                monitorExecutor(entry.getKey(), entry.getValue());
+        }
+    }
+
+    /**
+     * Creates a MetricSet for an executor.
+     *
+     * @param name Name of the bean to register.
+     * @param execSvc Executor to register a bean for.
+     */
+    private void monitorExecutor(String name, ExecutorService execSvc) {
+        MetricRegistry mset = mreg.withPrefix(THREAD_POOLS, name);
+
+        if (execSvc instanceof ThreadPoolExecutor) {
+            ThreadPoolExecutor exec = (ThreadPoolExecutor)execSvc;
+
+            mset.register("ActiveCount", exec::getActiveCount, ACTIVE_COUNT_DESC);
+            mset.register("CompletedTaskCount", exec::getCompletedTaskCount, COMPLETED_TASK_DESC);
+            mset.register("CorePoolSize", exec::getCorePoolSize, CORE_SIZE_DESC);
+            mset.register("LargestPoolSize", exec::getLargestPoolSize, LARGEST_SIZE_DESC);
+            mset.register("MaximumPoolSize", exec::getMaximumPoolSize, MAX_SIZE_DESC);
+            mset.register("PoolSize", exec::getPoolSize, POOL_SIZE_DESC);
+            mset.register("TaskCount", exec::getTaskCount, TASK_COUNT_DESC);
+            mset.register("QueueSize", () -> exec.getQueue().size(), QUEUE_SIZE_DESC);
+            mset.register("KeepAliveTime", () -> exec.getKeepAliveTime(MILLISECONDS), KEEP_ALIVE_TIME_DESC);
+            mset.register("Shutdown", exec::isShutdown, IS_SHUTDOWN_DESC);
+            mset.register("Terminated", exec::isTerminated, IS_TERMINATED_DESC);
+            mset.register("Terminating", exec::isTerminating, IS_TERMINATING_DESC);
+            mset.register("RejectedExecutionHandlerClass", () -> {
+                RejectedExecutionHandler hnd = exec.getRejectedExecutionHandler();
+
+                return hnd == null ? "" : hnd.getClass().getName();
+            }, String.class, REJ_HND_DESC);
+            mset.register("ThreadFactoryClass", () -> {
+                ThreadFactory factory = exec.getThreadFactory();
+
+                return factory == null ? "" : factory.getClass().getName();
+            }, String.class, THRD_FACTORY_DESC);
+        }
+        else {
+            mset.metric("ActiveCount", ACTIVE_COUNT_DESC).value(0);
+            mset.metric("CompletedTaskCount", COMPLETED_TASK_DESC).value(0);
+            mset.metric("CorePoolSize", CORE_SIZE_DESC).value(0);
+            mset.metric("LargestPoolSize", LARGEST_SIZE_DESC).value(0);
+            mset.metric("MaximumPoolSize", MAX_SIZE_DESC).value(0);
+            mset.metric("PoolSize", POOL_SIZE_DESC).value(0);
+            mset.metric("TaskCount", TASK_COUNT_DESC);
+            mset.metric("QueueSize", QUEUE_SIZE_DESC).value(0);
+            mset.metric("KeepAliveTime", KEEP_ALIVE_TIME_DESC).value(0);
+            mset.register("Shutdown", execSvc::isShutdown, IS_SHUTDOWN_DESC);
+            mset.register("Terminated", execSvc::isTerminated, IS_TERMINATED_DESC);
+            mset.metric("Terminating", IS_TERMINATING_DESC);
+            mset.objectMetric("RejectedExecutionHandlerClass", String.class, REJ_HND_DESC).value("");
+            mset.objectMetric("ThreadFactoryClass", String.class, THRD_FACTORY_DESC).value("");
+        }
+    }
+
+    /**
+     * Creates a MetricSet for an stripped executor.
+     *
+     * @param svc Executor.
+     */
+    private void monitorStripedPool(StripedExecutor svc) {
+        MetricRegistry mset = mreg.withPrefix(THREAD_POOLS, "StripedExecutor");
+
+        mset.register("DetectStarvation",
+            svc::detectStarvation,
+            "True if possible starvation in striped pool is detected.");
+
+        mset.register("StripesCount",
+            svc::stripes,
+            "Stripes count.");
+
+        mset.register("Shutdown",
+            svc::isShutdown,
+            "True if this executor has been shut down.");
+
+        mset.register("Terminated",
+            svc::isTerminated,
+            "True if all tasks have completed following shut down.");
+
+        mset.register("TotalQueueSize",
+            svc::queueSize,
+            "Total queue size of all stripes.");
+
+        mset.register("TotalCompletedTasksCount",
+            svc::completedTasks,
+            "Completed tasks count of all stripes.");
+
+        mset.register("StripesCompletedTasksCounts",
+            svc::stripesCompletedTasks,
+            long[].class,
+            "Number of completed tasks per stripe.");
+
+        mset.register("ActiveCount",
+            svc::activeStripesCount,
+            "Number of active tasks of all stripes.");
+
+        mset.register("StripesActiveStatuses",
+            svc::stripesActiveStatuses,
+            boolean[].class,
+            "Number of active tasks per stripe.");
+
+        mset.register("StripesQueueSizes",
+            svc::stripesQueueSizes,
+            int[].class,
+            "Size of queue per stripe.");
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/metric/MetricRegistry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/metric/MetricRegistry.java
new file mode 100644
index 0000000..697c8d7
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/metric/MetricRegistry.java
@@ -0,0 +1,196 @@
+/*
+ * 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.metric;
+
+import java.util.function.BooleanSupplier;
+import java.util.function.DoubleSupplier;
+import java.util.function.IntSupplier;
+import java.util.function.LongSupplier;
+import java.util.function.Supplier;
+import org.apache.ignite.internal.processors.metric.impl.DoubleMetricImpl;
+import org.apache.ignite.internal.processors.metric.impl.HitRateMetric;
+import org.apache.ignite.internal.processors.metric.impl.IntMetricImpl;
+import org.apache.ignite.internal.processors.metric.impl.LongAdderMetricImpl;
+import org.apache.ignite.internal.processors.metric.impl.LongMetricImpl;
+import org.apache.ignite.internal.processors.metric.impl.BooleanMetricImpl;
+import org.apache.ignite.internal.processors.metric.impl.HistogramMetric;
+import org.apache.ignite.internal.processors.metric.impl.ObjectMetricImpl;
+import org.apache.ignite.spi.metric.BooleanMetric;
+import org.apache.ignite.spi.metric.IntMetric;
+import org.apache.ignite.spi.metric.LongMetric;
+import org.apache.ignite.spi.metric.Metric;
+import org.apache.ignite.spi.metric.ReadOnlyMetricRegistry;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Metrics registry.
+ * Provide methods to register required metrics, gauges for Ignite internals.
+ * Provide the way to obtain all registered metrics for exporters.
+ */
+public interface MetricRegistry extends ReadOnlyMetricRegistry {
+    /**
+     * Register existing metrics in this group with the specified name.
+     *
+     * @param metric Metric.
+     */
+    public void register(Metric metric);
+
+    /**
+     * Removes metrics with the {@code name} from registry.
+     *
+     * @param name Metric name.
+     */
+    public void remove(String name);
+
+    /**
+     * Resets state of this metric set.
+     */
+    public default void reset() {
+        getMetrics().forEach(Metric::reset);
+    }
+
+    /**
+     * Registers {@link BooleanMetric} which value will be queried from the specified supplier.
+     *
+     * @param name Name.
+     * @param supplier Supplier.
+     * @param description Description.
+     */
+    public void register(String name, BooleanSupplier supplier, @Nullable String description);
+
+    /**
+     * Registers {@link BooleanMetric} which value will be queried from the specified supplier.
+     *
+     * @param name Name.
+     * @param supplier Supplier.
+     * @param description Description.
+     */
+    public void register(String name, DoubleSupplier supplier, @Nullable String description);
+
+    /**
+     * Registers {@link IntMetric} which value will be queried from the specified supplier.
+     *
+     * @param name Name.
+     * @param supplier Supplier.
+     * @param description Description.
+     */
+    public void register(String name, IntSupplier supplier, @Nullable String description);
+
+    /**
+     * Registers {@link LongMetric} which value will be queried from the specified supplier.
+     *
+     * @param name Name.
+     * @param supplier Supplier.
+     * @param description Description.
+     */
+    public void register(String name, LongSupplier supplier, @Nullable String description);
+
+    /**
+     * Registers {@code ObjectMetric} which value will be queried from the specified supplier.
+     *
+     * @param name Name.
+     * @param supplier Supplier.
+     * @param type Type.
+     * @param description Description.
+     */
+    public <T> void register(String name, Supplier<T> supplier, Class<T> type, @Nullable String description);
+
+    /**
+     * Creates and register named metric.
+     * Returned instance are thread safe.
+     *
+     * @param name Name.
+     * @param description Description.
+     * @return Metric
+     */
+    public DoubleMetricImpl doubleMetric(String name, @Nullable String description);
+
+    /**
+     * Creates and register named metric.
+     * Returned instance are thread safe.
+     *
+     * @param name Name.
+     * @param description Description.
+     * @return Metric.
+     */
+    public IntMetricImpl intMetric(String name, @Nullable String description);
+
+    /**
+     * Creates and register named metric.
+     * Returned instance are thread safe.
+     *
+     * @param name Name.
+     * @param description Description.
+     * @return Metric.
+     */
+    public LongMetricImpl metric(String name, @Nullable String description);
+
+    /**
+     * Creates and register named metric.
+     * Returned instance are thread safe.
+     *
+     * @param name Name.
+     * @param description Description.
+     * @return Metric
+     */
+    public LongAdderMetricImpl longAdderMetric(String name, @Nullable String description);
+
+    /**
+     * Creates and register hit rate metric.
+     *
+     * It will accumulates approximate hit rate statistics.
+     * Calculates number of hits in last rateTimeInterval milliseconds.
+     *
+     * @param rateTimeInterval Rate time interval.
+     * @param size Array size for underlying calculations.
+     * @return Metric.
+     * @see HitRateMetric
+     */
+    public HitRateMetric hitRateMetric(String name, @Nullable String description, long rateTimeInterval, int size);
+
+    /**
+     * Creates and register named gauge.
+     * Returned instance are thread safe.
+     *
+     * @param name Name.
+     * @param description Description.
+     * @return Gauge.
+     */
+    public BooleanMetricImpl booleanMetric(String name, @Nullable String description);
+
+    /**
+     * Creates and register named gauge.
+     * Returned instance are thread safe.
+     *
+     * @param name Name.
+     * @param type Type.
+     * @param description Description.
+     * @return Gauge.
+     */
+    public <T> ObjectMetricImpl<T> objectMetric(String name, Class<T> type, @Nullable String description);
+
+    /**
+     * Creates and registre named histogram gauge.
+     *
+     * @param name Name
+     * @param bounds Bounds of measurements.
+     * @param description Description.
+     * @return HistogramGauge.
+     */
+    public HistogramMetric histogram(String name, long[] bounds, @Nullable String description);
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/metric/MetricRegistryImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/metric/MetricRegistryImpl.java
new file mode 100644
index 0000000..a19c5a3
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/metric/MetricRegistryImpl.java
@@ -0,0 +1,212 @@
+/*
+ * 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.metric;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.function.BooleanSupplier;
+import java.util.function.Consumer;
+import java.util.function.DoubleSupplier;
+import java.util.function.IntSupplier;
+import java.util.function.LongSupplier;
+import java.util.function.Supplier;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.processors.metric.impl.BooleanGauge;
+import org.apache.ignite.internal.processors.metric.impl.DoubleGauge;
+import org.apache.ignite.internal.processors.metric.impl.IntGauge;
+import org.apache.ignite.internal.processors.metric.impl.LongGauge;
+import org.apache.ignite.internal.processors.metric.impl.ObjectGauge;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.spi.metric.Metric;
+import org.apache.ignite.internal.processors.metric.impl.DoubleMetricImpl;
+import org.apache.ignite.internal.processors.metric.impl.HitRateMetric;
+import org.apache.ignite.internal.processors.metric.impl.IntMetricImpl;
+import org.apache.ignite.internal.processors.metric.impl.LongAdderMetricImpl;
+import org.apache.ignite.internal.processors.metric.impl.LongMetricImpl;
+import org.apache.ignite.internal.processors.metric.impl.BooleanMetricImpl;
+import org.apache.ignite.internal.processors.metric.impl.HistogramMetric;
+import org.apache.ignite.internal.processors.metric.impl.ObjectMetricImpl;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.internal.processors.metric.impl.MetricUtils.metricName;
+import static org.apache.ignite.internal.util.lang.GridFunc.nonThrowableSupplier;
+
+/**
+ * Simple implementation.
+ */
+public class MetricRegistryImpl implements MetricRegistry {
+    /** Registered metrics. */
+    private final ConcurrentHashMap<String, Metric> metrics = new ConcurrentHashMap<>();
+
+    /** Logger. */
+    @Nullable private final IgniteLogger log;
+
+    /** Metric set creation listeners. */
+    private final List<Consumer<Metric>> metricCreationLsnrs = new CopyOnWriteArrayList<>();
+
+    /** For test usage only. */
+    public MetricRegistryImpl() {
+        this.log = null;
+    }
+
+    /**
+     * @param log Logger.
+     */
+    public MetricRegistryImpl(IgniteLogger log) {
+        this.log = log;
+    }
+
+    /** {@inheritDoc} */
+    @Override public MetricRegistry withPrefix(String prefix) {
+        return new MetricRegistryPrefixProxy(prefix, this);
+    }
+
+    /** {@inheritDoc} */
+    @Override public MetricRegistry withPrefix(String... prefixes) {
+        return withPrefix(metricName(prefixes));
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<Metric> getMetrics() {
+        return metrics.values();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void addMetricCreationListener(Consumer<Metric> lsnr) {
+        metricCreationLsnrs.add(lsnr);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Metric findMetric(String name) {
+        return metrics.get(name);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void register(Metric metric) {
+        addMetric(metric.name(), metric);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void remove(String name) {
+        metrics.remove(name);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void register(String name, BooleanSupplier supplier, @Nullable String description) {
+        addMetric(name, new BooleanGauge(name, description, nonThrowableSupplier(supplier, log)));
+    }
+
+    /** {@inheritDoc} */
+    @Override public void register(String name, DoubleSupplier supplier, @Nullable String description) {
+        addMetric(name, new DoubleGauge(name, description, nonThrowableSupplier(supplier, log)));
+    }
+
+    /** {@inheritDoc} */
+    @Override public void register(String name, IntSupplier supplier, @Nullable String description) {
+        addMetric(name, new IntGauge(name, description, nonThrowableSupplier(supplier, log)));
+    }
+
+    /** {@inheritDoc} */
+    @Override public void register(String name, LongSupplier supplier, @Nullable String description) {
+        addMetric(name, new LongGauge(name, description, nonThrowableSupplier(supplier, log)));
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T> void register(String name, Supplier<T> supplier, Class<T> type, @Nullable String description) {
+        addMetric(name, new ObjectGauge<>(name, description, nonThrowableSupplier(supplier, log), type));
+    }
+
+    /** {@inheritDoc} */
+    @Override public DoubleMetricImpl doubleMetric(String name, @Nullable String description) {
+        return addMetric(name, new DoubleMetricImpl(name, description));
+    }
+
+    /** {@inheritDoc} */
+    @Override public IntMetricImpl intMetric(String name, @Nullable String description) {
+        return addMetric(name, new IntMetricImpl(name, description));
+    }
+
+    /** {@inheritDoc} */
+    @Override public LongMetricImpl metric(String name, @Nullable String description) {
+        return addMetric(name, new LongMetricImpl(name, description));
+    }
+
+    /** {@inheritDoc} */
+    @Override public LongAdderMetricImpl longAdderMetric(String name, @Nullable String description) {
+        return addMetric(name, new LongAdderMetricImpl(name, description));
+    }
+
+    /** {@inheritDoc} */
+    @Override public HitRateMetric hitRateMetric(String name,
+        @Nullable String description, long rateTimeInterval, int size) {
+        return addMetric(name, new HitRateMetric(name, description, rateTimeInterval, size));
+    }
+
+    /** {@inheritDoc} */
+    @Override public BooleanMetricImpl booleanMetric(String name, @Nullable String description) {
+        return addMetric(name, new BooleanMetricImpl(name, description));
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T> ObjectMetricImpl<T> objectMetric(String name, Class<T> type, @Nullable String description) {
+        return addMetric(name, new ObjectMetricImpl<>(name, description, type));
+    }
+
+    /** {@inheritDoc} */
+    @Override public HistogramMetric histogram(String name, long[] bounds, @Nullable String description) {
+        return addMetric(name, new HistogramMetric(name, description, bounds));
+    }
+
+    /**
+     * Adds metrics if not exists already.
+     *
+     * @param name Name.
+     * @param metric Metric
+     * @param <T> Type of metric.
+     * @return Registered metric.
+     */
+    private <T extends Metric> T addMetric(String name, T metric) {
+        T old = (T)metrics.putIfAbsent(name, metric);
+
+        if (old == null) {
+            notifyListeners(metric, metricCreationLsnrs);
+
+            return metric;
+        }
+
+        return old;
+    }
+
+    /**
+     * @param t Consumed object.
+     * @param lsnrs Listeners.
+     * @param <T> Type of consumed object.
+     */
+    private <T> void notifyListeners(T t, List<Consumer<T>> lsnrs) {
+        for (Consumer<T> lsnr : lsnrs) {
+            try {
+                lsnr.accept(t);
+            }
+            catch (Exception e) {
+                U.warn(log, "Metric listener error", e);
+            }
+        }
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/metric/MetricRegistryPrefixProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/metric/MetricRegistryPrefixProxy.java
new file mode 100644
index 0000000..d063b1e
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/metric/MetricRegistryPrefixProxy.java
@@ -0,0 +1,177 @@
+/*
+ * 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.metric;
+
+import java.util.Collection;
+import java.util.function.BooleanSupplier;
+import java.util.function.Consumer;
+import java.util.function.DoubleSupplier;
+import java.util.function.IntSupplier;
+import java.util.function.LongSupplier;
+import java.util.function.Supplier;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.spi.metric.Metric;
+import org.apache.ignite.internal.processors.metric.impl.DoubleMetricImpl;
+import org.apache.ignite.internal.processors.metric.impl.HitRateMetric;
+import org.apache.ignite.internal.processors.metric.impl.IntMetricImpl;
+import org.apache.ignite.internal.processors.metric.impl.LongAdderMetricImpl;
+import org.apache.ignite.internal.processors.metric.impl.LongMetricImpl;
+import org.apache.ignite.internal.processors.metric.impl.BooleanMetricImpl;
+import org.apache.ignite.internal.processors.metric.impl.HistogramMetric;
+import org.apache.ignite.internal.processors.metric.impl.ObjectMetricImpl;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.internal.processors.metric.impl.MetricUtils.SEPARATOR;
+import static org.apache.ignite.internal.processors.metric.impl.MetricUtils.metricName;
+
+/**
+ * Proxy registry that adds {@code prefix} to all metric names on each method call.
+ */
+public class MetricRegistryPrefixProxy implements MetricRegistry {
+    /** Prefix for underlying registry. */
+    private final String prefix;
+
+    /** Underlying implementation. */
+    private final MetricRegistry reg;
+
+    /**
+     * @param prefix Metrics prefix.
+     * @param reg Underlying imlementaion.
+     */
+    public MetricRegistryPrefixProxy(String prefix, MetricRegistry reg) {
+        assert prefix != null && !prefix.isEmpty();
+
+        this.prefix = prefix;
+        this.reg = reg;
+    }
+
+    /** {@inheritDoc} */
+    @Override public MetricRegistry withPrefix(String prefix) {
+        return new MetricRegistryPrefixProxy(prefix, this);
+    }
+
+    /** {@inheritDoc} */
+    @Override public MetricRegistry withPrefix(String... prefixes) {
+        return withPrefix(metricName(prefixes));
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<Metric> getMetrics() {
+        String p = this.prefix + SEPARATOR;
+
+        return F.view(reg.getMetrics(), m -> m.name().startsWith(p));
+    }
+
+    /** {@inheritDoc} */
+    @Override public void addMetricCreationListener(Consumer<Metric> lsnr) {
+        reg.addMetricCreationListener(m -> {
+            if (m.name().startsWith(prefix))
+                lsnr.accept(m);
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public @Nullable Metric findMetric(String name) {
+        return reg.findMetric(fullName(name));
+    }
+
+    /** {@inheritDoc} */
+    @Override public void register(Metric metric) {
+        reg.register(metric);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void remove(String name) {
+        reg.remove(fullName(name));
+    }
+
+    /** {@inheritDoc} */
+    @Override public void register(String name, BooleanSupplier supplier, @Nullable String description) {
+        reg.register(fullName(name), supplier, description);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void register(String name, DoubleSupplier supplier, @Nullable String description) {
+        reg.register(fullName(name), supplier, description);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void register(String name, IntSupplier supplier, @Nullable String description) {
+        reg.register(fullName(name), supplier, description);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void register(String name, LongSupplier supplier, @Nullable String description) {
+        reg.register(fullName(name), supplier, description);
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T> void register(String name, Supplier<T> supplier, Class<T> type, @Nullable String description) {
+        reg.register(fullName(name), supplier, type, description);
+    }
+
+    /** {@inheritDoc} */
+    @Override public DoubleMetricImpl doubleMetric(String name, @Nullable String description) {
+        return reg.doubleMetric(fullName(name), description);
+    }
+
+    /** {@inheritDoc} */
+    @Override public IntMetricImpl intMetric(String name, @Nullable String description) {
+        return reg.intMetric(fullName(name), description);
+    }
+
+    /** {@inheritDoc} */
+    @Override public LongMetricImpl metric(String name, @Nullable String description) {
+        return reg.metric(fullName(name), description);
+    }
+
+    /** {@inheritDoc} */
+    @Override public LongAdderMetricImpl longAdderMetric(String name, @Nullable String description) {
+        return reg.longAdderMetric(fullName(name), description);
+    }
+
+    /** {@inheritDoc} */
+    @Override public HitRateMetric hitRateMetric(String name, @Nullable String description,
+        long rateTimeInterval, int size) {
+        return reg.hitRateMetric(fullName(name), description, rateTimeInterval, size);
+    }
+
+    /** {@inheritDoc} */
+    @Override public BooleanMetricImpl booleanMetric(String name, @Nullable String description) {
+        return reg.booleanMetric(fullName(name), description);
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T> ObjectMetricImpl<T> objectMetric(String name, Class<T> type, @Nullable String description) {
+        return reg.objectMetric(fullName(name), type, description);
+    }
+
+    /** {@inheritDoc} */
+    @Override public HistogramMetric histogram(String name, long[] bounds, @Nullable String description) {
+        return reg.histogram(fullName(name), bounds, description);
+    }
+
+    /**
+     * @param name Metric name.
+     * @return Full name with prefix.
+     */
+    @NotNull private String fullName(String name) {
+        return metricName(prefix, name);
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/metric/PushMetricsExporterAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/metric/PushMetricsExporterAdapter.java
new file mode 100644
index 0000000..445f900
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/metric/PushMetricsExporterAdapter.java
@@ -0,0 +1,106 @@
+/*
+ * 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.metric;
+
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.function.Predicate;
+import org.apache.ignite.spi.IgniteSpiAdapter;
+import org.apache.ignite.spi.IgniteSpiException;
+import org.apache.ignite.spi.metric.Metric;
+import org.apache.ignite.spi.metric.MetricExporterSpi;
+import org.apache.ignite.spi.metric.ReadOnlyMetricRegistry;
+import org.jetbrains.annotations.Nullable;
+
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
+
+/**
+ *
+ */
+public abstract class PushMetricsExporterAdapter extends IgniteSpiAdapter implements MetricExporterSpi {
+    /** Metric registry. */
+    protected ReadOnlyMetricRegistry mreg;
+
+    /** Metric filter. */
+    protected  @Nullable Predicate<Metric> filter;
+
+    /** Export period. */
+    private long period;
+
+    /** Push spi executor. */
+    private ScheduledExecutorService execSvc;
+
+    /** Export task future. */
+    private ScheduledFuture<?> fut;
+
+    /** {@inheritDoc} */
+    @Override public void spiStart(@Nullable String igniteInstanceName) throws IgniteSpiException {
+        execSvc = Executors.newScheduledThreadPool(1);
+
+        fut = execSvc.scheduleWithFixedDelay(() -> {
+            try {
+                export();
+            }
+            catch (Exception e) {
+                log.error("Metrics export error. " +
+                    "This exporter will be stopped [spiClass=" + getClass() + ",name=" + getName() + ']', e);
+
+                throw e;
+            }
+        }, period, period, MILLISECONDS);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void spiStop() throws IgniteSpiException {
+        fut.cancel(false);
+
+        execSvc.shutdown();
+    }
+
+    /**
+     * Callback to do the export of metrics info.
+     * Method will be called into some Ignite managed thread each {@link #getPeriod()} millisecond.
+     */
+    public abstract void export();
+
+    /**
+     * Sets period in milliseconds after {@link #export()} method should be called.
+     *
+     * @param period Period in milliseconds.
+     */
+    public void setPeriod(long period) {
+        this.period = period;
+    }
+
+    /** @return Period in milliseconds after {@link #export()} method should be called. */
+    public long getPeriod() {
+        return period;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void setMetricRegistry(ReadOnlyMetricRegistry mreg) {
+        this.mreg = mreg;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void setExportFilter(Predicate<Metric> filter) {
+        this.filter = filter;
+    }
+
+}
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/stat/IoStatisticsCachePersistenceSelfTest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/metric/impl/BooleanGauge.java
similarity index 52%
copy from modules/core/src/test/java/org/apache/ignite/internal/stat/IoStatisticsCachePersistenceSelfTest.java
copy to modules/core/src/main/java/org/apache/ignite/internal/processors/metric/impl/BooleanGauge.java
index c203295..ac03caf 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/stat/IoStatisticsCachePersistenceSelfTest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/metric/impl/BooleanGauge.java
@@ -13,18 +13,35 @@
  * 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.stat;
+package org.apache.ignite.internal.processors.metric.impl;
+
+import java.util.function.BooleanSupplier;
+import org.apache.ignite.internal.processors.metric.AbstractMetric;
+import org.apache.ignite.spi.metric.BooleanMetric;
+import org.jetbrains.annotations.Nullable;
 
 /**
- * Tests for cache IO statistics for persistence mode.
+ * Implementation based on primitive supplier.
  */
-public class IoStatisticsCachePersistenceSelfTest extends IoStatisticsCacheSelfTest {
+public class BooleanGauge extends AbstractMetric implements BooleanMetric {
+    /** Value supplier. */
+    private final BooleanSupplier val;
+
+    /**
+     * @param name Name.
+     * @param descr Description.
+     * @param val Supplier.
+     */
+    public BooleanGauge(String name, @Nullable String descr, BooleanSupplier val) {
+        super(name, descr);
+
+        this.val = val;
+    }
 
     /** {@inheritDoc} */
-    @Override protected boolean persist() {
-        return true;
+    @Override public boolean value() {
+        return val.getAsBoolean();
     }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/metric/impl/BooleanMetricImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/metric/impl/BooleanMetricImpl.java
new file mode 100644
index 0000000..61e88da
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/metric/impl/BooleanMetricImpl.java
@@ -0,0 +1,57 @@
+/*
+ * 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.metric.impl;
+
+import org.apache.ignite.internal.processors.metric.AbstractMetric;
+import org.apache.ignite.spi.metric.BooleanMetric;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Metric that holds boolean primitive.
+ */
+public class BooleanMetricImpl extends AbstractMetric implements BooleanMetric {
+    /** Value. */
+    private volatile boolean val;
+
+    /**
+     * @param name Name.
+     * @param descr Description.
+     */
+    public BooleanMetricImpl(String name, @Nullable String descr) {
+        super(name, descr);
+    }
+
+    /**
+     * Sets value.
+     *
+     * @param val Value.
+     */
+    public void value(boolean val) {
+        this.val = val;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void reset() {
+        val = false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean value() {
+        return val;
+    }
+}
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/stat/IoStatisticsCachePersistenceSelfTest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/metric/impl/DoubleGauge.java
similarity index 52%
copy from modules/core/src/test/java/org/apache/ignite/internal/stat/IoStatisticsCachePersistenceSelfTest.java
copy to modules/core/src/main/java/org/apache/ignite/internal/processors/metric/impl/DoubleGauge.java
index c203295..60ec4a7 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/stat/IoStatisticsCachePersistenceSelfTest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/metric/impl/DoubleGauge.java
@@ -13,18 +13,35 @@
  * 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.stat;
+package org.apache.ignite.internal.processors.metric.impl;
+
+import java.util.function.DoubleSupplier;
+import org.apache.ignite.internal.processors.metric.AbstractMetric;
+import org.apache.ignite.spi.metric.DoubleMetric;
+import org.jetbrains.annotations.Nullable;
 
 /**
- * Tests for cache IO statistics for persistence mode.
+ * Implementation based on primitive supplier.
  */
-public class IoStatisticsCachePersistenceSelfTest extends IoStatisticsCacheSelfTest {
+public class DoubleGauge extends AbstractMetric implements DoubleMetric {
+    /** Value supplier. */
+    private final DoubleSupplier val;
+
+    /**
+     * @param name Name.
+     * @param descr Description.
+     * @param val Supplier.
+     */
+    public DoubleGauge(String name, @Nullable String descr, DoubleSupplier val) {
+        super(name, descr);
+
+        this.val = val;
+    }
 
     /** {@inheritDoc} */
-    @Override protected boolean persist() {
-        return true;
+    @Override public double value() {
+        return val.getAsDouble();
     }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/metric/impl/DoubleMetricImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/metric/impl/DoubleMetricImpl.java
new file mode 100644
index 0000000..042fcab
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/metric/impl/DoubleMetricImpl.java
@@ -0,0 +1,69 @@
+/*
+ * 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.metric.impl;
+
+import java.util.concurrent.atomic.DoubleAccumulator;
+import org.apache.ignite.internal.processors.metric.AbstractMetric;
+import org.apache.ignite.spi.metric.DoubleMetric;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Double metric.
+ */
+public class DoubleMetricImpl extends AbstractMetric implements DoubleMetric {
+    /** Value. */
+    private volatile DoubleAccumulator val;
+
+    /**
+     * @param name Name.
+     * @param descr Description.
+     */
+    public DoubleMetricImpl(String name, @Nullable String descr) {
+        super(name, descr);
+
+        this.val = new DoubleAccumulator(Double::sum, 0d);
+    }
+
+    /**
+     * Adds x to the metric.
+     *
+     * @param x Value to be added.
+     */
+    public void add(double x) {
+        val.accumulate(x);
+    }
+
+    /**
+     * Sets value.
+     *
+     * @param val Value.
+     */
+    public void value(double val) {
+        this.val = new DoubleAccumulator(Double::sum, val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void reset() {
+        val.reset();
+    }
+
+    /** {@inheritDoc} */
+    @Override public double value() {
+        return val.doubleValue();
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/metric/impl/HistogramMetric.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/metric/impl/HistogramMetric.java
new file mode 100644
index 0000000..6146425
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/metric/impl/HistogramMetric.java
@@ -0,0 +1,118 @@
+/*
+ * 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.metric.impl;
+
+import java.util.concurrent.atomic.AtomicLongArray;
+import org.apache.ignite.internal.processors.metric.AbstractMetric;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.spi.metric.ObjectMetric;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Histogram metric that will calculate counts of measurements that gets into each bounds interval.
+ * Note, that {@link #value()} will return array length of {@code bounds.length + 1}.
+ * Last element will contains count of measurements bigger then most right value of bounds.
+ */
+public class HistogramMetric extends AbstractMetric implements ObjectMetric<long[]> {
+    /** Holder of measurements. */
+    private volatile HistogramHolder holder;
+
+    /**
+     * @param name Name.
+     * @param description Description.
+     * @param bounds Bounds.
+     */
+    public HistogramMetric(String name, @Nullable String description, long[] bounds) {
+        super(name, description);
+
+        holder = new HistogramHolder(bounds);
+    }
+
+    /**
+     * Sets value.
+     *
+     * @param x Value.
+     */
+    public void value(long x) {
+        assert x >= 0;
+
+        HistogramHolder h = holder;
+
+        //Expect arrays of few elements.
+        for (int i = 0; i < h.bounds.length; i++) {
+            if (x <= h.bounds[i]) {
+                h.measurements.incrementAndGet(i);
+
+                return;
+            }
+        }
+
+        h.measurements.incrementAndGet(h.bounds.length);
+    }
+
+    /**
+     * Resets histogram state with the specified bounds.
+     *
+     * @param bounds Bounds.
+     */
+    public void reset(long[] bounds) {
+        holder = new HistogramHolder(bounds);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void reset() {
+        reset(holder.bounds);
+    }
+
+    /** {@inheritDoc} */
+    @Override public long[] value() {
+        HistogramHolder h = holder;
+
+        long[] res = new long[h.measurements.length()];
+
+        for (int i = 0; i < h.measurements.length(); i++)
+            res[i] = h.measurements.get(i);
+
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Class<long[]> type() {
+        return long[].class;
+    }
+
+    /** Histogram holder. */
+    private static class HistogramHolder {
+        /** Count of measurement for each bound. */
+        public final AtomicLongArray measurements;
+
+        /** Bounds of measurements. */
+        public final long[] bounds;
+
+        /**
+         * @param bounds Bounds of measurements.
+         */
+        public HistogramHolder(long[] bounds) {
+            assert !F.isEmpty(bounds) && F.isSorted(bounds);
+
+            this.bounds = bounds;
+
+            this.measurements = new AtomicLongArray(bounds.length + 1);
+        }
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/metric/impl/HitRateMetric.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/metric/impl/HitRateMetric.java
new file mode 100644
index 0000000..81c2c83
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/metric/impl/HitRateMetric.java
@@ -0,0 +1,222 @@
+/*
+* 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.metric.impl;
+
+import java.util.concurrent.atomic.AtomicLongArray;
+import org.apache.ignite.internal.processors.metric.AbstractMetric;
+import org.apache.ignite.internal.util.typedef.internal.A;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.spi.metric.LongMetric;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Accumulates approximate hit rate statistics.
+ * Calculates number of hits in last {@code rateTimeInterval} milliseconds.
+ * Algorithm is based on circular array of {@code size} hit counters, each is responsible for last corresponding time
+ * interval of {@code rateTimeInterval}/{@code size} milliseconds. Resulting number of hits is sum of all counters.
+ *
+ * <p>Implementation is nonblocking and protected from hits loss.
+ * Maximum relative error is 1/{@code size}.
+ * 2^55 - 1 hits per interval can be accumulated without numeric overflow.
+ */
+public class HitRateMetric extends AbstractMetric implements LongMetric {
+    /** Metric instance. */
+    private volatile HitRateMetricImpl cntr;
+
+    /**
+     * @param name Name.
+     * @param description Description.
+     * @param rateTimeInterval Rate time interval.
+     * @param size Counters array size.
+     */
+    public HitRateMetric(String name, @Nullable String description, long rateTimeInterval, int size) {
+        super(name, description);
+
+        cntr = new HitRateMetricImpl(rateTimeInterval, size);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void reset() {
+        cntr = new HitRateMetricImpl(cntr.rateTimeInterval, cntr.size);
+    }
+
+    /**
+     * Resets metric with the new paramters.
+     *
+     * @param rateTimeInterval New rate time interval.
+     * @param size New counters array size.
+     */
+    public void reset(long rateTimeInterval, int size) {
+        cntr = new HitRateMetricImpl(rateTimeInterval, size);
+    }
+
+    /**
+     * Adds x to the metric.
+     *
+     * @param x Value to be added.
+     */
+    public void add(long x) {
+        cntr.add(x);
+    }
+
+    /** Adds 1 to the metric. */
+    public void increment() {
+        add(1);
+    }
+
+    /** {@inheritDoc} */
+    @Override public long value() {
+        return cntr.value();
+    }
+
+    /**
+     * Actual metric.
+     *
+     * Separated class required to
+     */
+    private static class HitRateMetricImpl {
+        /** Bits that store actual hit count. */
+        private static final int TAG_OFFSET = 56;
+
+        /** Useful part mask. */
+        private static final long NO_TAG_MASK = ~(-1L << TAG_OFFSET);
+
+        /** Time interval when hits are counted to calculate rate, in milliseconds. */
+        private final long rateTimeInterval;
+
+        /** Counters array size. */
+        private final int size;
+
+        /** Tagged counters. */
+        private final AtomicLongArray taggedCounters;
+
+        /** Last hit times. */
+        private final AtomicLongArray lastHitTimes;
+
+        /**
+         * @param rateTimeInterval Rate time interval.
+         * @param size Number of counters.
+         */
+        public HitRateMetricImpl(long rateTimeInterval, int size) {
+            A.ensure(rateTimeInterval > 0, "rateTimeInterval should be positive");
+
+            A.ensure(size > 1, "Minimum value for size is 2");
+
+            this.rateTimeInterval = rateTimeInterval;
+
+            this.size = size;
+
+            taggedCounters = new AtomicLongArray(size);
+
+            lastHitTimes = new AtomicLongArray(size);
+        }
+
+        /**
+         * Adds hits to the metric.
+         *
+         * @param hits Number of hits.
+         */
+        public void add(long hits) {
+            long curTs = U.currentTimeMillis();
+
+            int curPos = position(curTs);
+
+            clearIfObsolete(curTs, curPos);
+
+            lastHitTimes.set(curPos, curTs);
+
+            // Order is important. Hit won't be cleared by concurrent #clearIfObsolete.
+            taggedCounters.addAndGet(curPos, hits);
+        }
+
+        /**
+         * @return Total number of hits in last {@link #rateTimeInterval} milliseconds.
+         */
+        public long value() {
+            long curTs = U.currentTimeMillis();
+
+            long sum = 0;
+
+            for (int i = 0; i < size; i++) {
+                clearIfObsolete(curTs, i);
+
+                sum += untag(taggedCounters.get(i));
+            }
+
+            return sum;
+        }
+
+        /**
+         * @param curTs Current timestamp.
+         * @param i Index.
+         */
+        private void clearIfObsolete(long curTs, int i) {
+            long cur = taggedCounters.get(i);
+
+            byte curTag = getTag(cur);
+
+            long lastTs = lastHitTimes.get(i);
+
+            if (isObsolete(curTs, lastTs)) {
+                if (taggedCounters.compareAndSet(i, cur, taggedLongZero(++curTag))) // ABA problem prevention.
+                    lastHitTimes.set(i, curTs);
+                // If CAS failed, counter is reset by another thread.
+            }
+        }
+
+        /**
+         * @param curTs Current timestamp.
+         * @param lastHitTime Last hit timestamp.
+         * @return True, is last hit time was too long ago.
+         */
+        private boolean isObsolete(long curTs, long lastHitTime) {
+            return curTs - lastHitTime > rateTimeInterval * (size - 1) / size;
+        }
+
+        /**
+         * @param time Timestamp.
+         * @return Index of counter for given timestamp.
+         */
+        private int position(long time) {
+            return (int)((time % rateTimeInterval * size) / rateTimeInterval);
+        }
+
+        /**
+         * @param tag Tag byte.
+         * @return 0L with given tag byte.
+         */
+        private static long taggedLongZero(byte tag) {
+            return ((long)tag << TAG_OFFSET);
+        }
+
+        /**
+         * @param l Tagged long.
+         * @return Long without tag byte.
+         */
+        private static long untag(long l) {
+            return l & NO_TAG_MASK;
+        }
+
+        /**
+         * @param taggedLong Tagged long.
+         * @return Tag byte.
+         */
+        private static byte getTag(long taggedLong) {
+            return (byte)(taggedLong >> TAG_OFFSET);
+        }
+    }
+}
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/stat/IoStatisticsCachePersistenceSelfTest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/metric/impl/IntGauge.java
similarity index 53%
copy from modules/core/src/test/java/org/apache/ignite/internal/stat/IoStatisticsCachePersistenceSelfTest.java
copy to modules/core/src/main/java/org/apache/ignite/internal/processors/metric/impl/IntGauge.java
index c203295..07aeb0a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/stat/IoStatisticsCachePersistenceSelfTest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/metric/impl/IntGauge.java
@@ -13,18 +13,35 @@
  * 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.stat;
+package org.apache.ignite.internal.processors.metric.impl;
+
+import java.util.function.IntSupplier;
+import org.apache.ignite.internal.processors.metric.AbstractMetric;
+import org.apache.ignite.spi.metric.IntMetric;
+import org.jetbrains.annotations.Nullable;
 
 /**
- * Tests for cache IO statistics for persistence mode.
+ * Implementation based on primitive supplier.
  */
-public class IoStatisticsCachePersistenceSelfTest extends IoStatisticsCacheSelfTest {
+public class IntGauge extends AbstractMetric implements IntMetric {
+    /** Value supplier. */
+    private final IntSupplier val;
+
+    /**
+     * @param name Name.
+     * @param descr Description.
+     * @param val Supplier.
+     */
+    public IntGauge(String name, @Nullable String descr, IntSupplier val) {
+        super(name, descr);
+
+        this.val = val;
+    }
 
     /** {@inheritDoc} */
-    @Override protected boolean persist() {
-        return true;
+    @Override public int value() {
+        return val.getAsInt();
     }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/metric/impl/IntMetricImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/metric/impl/IntMetricImpl.java
new file mode 100644
index 0000000..830f31b
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/metric/impl/IntMetricImpl.java
@@ -0,0 +1,71 @@
+/*
+ * 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.metric.impl;
+
+import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
+import org.apache.ignite.internal.processors.metric.AbstractMetric;
+import org.apache.ignite.spi.metric.IntMetric;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Int metric implementation.
+ */
+public class IntMetricImpl extends AbstractMetric implements IntMetric {
+    /** Field updater. */
+    private static final AtomicIntegerFieldUpdater<IntMetricImpl> updater =
+        AtomicIntegerFieldUpdater.newUpdater(IntMetricImpl.class, "val");
+
+    /** Value. */
+    private volatile int val;
+
+    /**
+     * @param name Name.
+     * @param descr Description.
+     */
+    public IntMetricImpl(String name, @Nullable String descr) {
+        super(name, descr);
+    }
+
+    /**
+     * Adds x to the metric.
+     *
+     * @param x Value to be added.
+     */
+    public void add(int x) {
+        updater.addAndGet(this, x);
+    }
+
+    /**
+     * Sets value.
+     *
+     * @param val Value.
+     */
+    public void value(int val) {
+        this.val = val;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void reset() {
+        updater.set(this, 0);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int value() {
+        return val;
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/metric/impl/LongAdderMetricImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/metric/impl/LongAdderMetricImpl.java
new file mode 100644
index 0000000..3948ba3
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/metric/impl/LongAdderMetricImpl.java
@@ -0,0 +1,68 @@
+/*
+ * 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.metric.impl;
+
+import java.util.concurrent.atomic.LongAdder;
+import org.apache.ignite.internal.processors.metric.AbstractMetric;
+import org.apache.ignite.spi.metric.LongMetric;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Long metric implementation based on {@link LongAdder}.
+ */
+public class LongAdderMetricImpl extends AbstractMetric implements LongMetric {
+    /** Field value. */
+    private volatile LongAdder val = new LongAdder();
+
+    /**
+     * @param name Name.
+     * @param descr Description.
+     */
+    public LongAdderMetricImpl(String name, @Nullable String descr) {
+        super(name, descr);
+    }
+
+    /**
+     * Adds x to the metric.
+     *
+     * @param x Value to be added.
+     */
+    public void add(long x) {
+        val.add(x);
+    }
+
+    /** Adds 1 to the metric. */
+    public void increment() {
+        add(1);
+    }
+
+    /** Adds -1 to the metric. */
+    public void decrement() {
+        add(-1);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void reset() {
+        val = new LongAdder();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long value() {
+        return val.longValue();
+    }
+}
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/stat/IoStatisticsCachePersistenceSelfTest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/metric/impl/LongGauge.java
similarity index 53%
copy from modules/core/src/test/java/org/apache/ignite/internal/stat/IoStatisticsCachePersistenceSelfTest.java
copy to modules/core/src/main/java/org/apache/ignite/internal/processors/metric/impl/LongGauge.java
index c203295..946f38e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/stat/IoStatisticsCachePersistenceSelfTest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/metric/impl/LongGauge.java
@@ -13,18 +13,35 @@
  * 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.stat;
+package org.apache.ignite.internal.processors.metric.impl;
+
+import java.util.function.LongSupplier;
+import org.apache.ignite.internal.processors.metric.AbstractMetric;
+import org.apache.ignite.spi.metric.LongMetric;
+import org.jetbrains.annotations.Nullable;
 
 /**
- * Tests for cache IO statistics for persistence mode.
+ * Implementation based on primitive supplier.
  */
-public class IoStatisticsCachePersistenceSelfTest extends IoStatisticsCacheSelfTest {
+public class LongGauge extends AbstractMetric implements LongMetric {
+    /** Value supplier. */
+    private final LongSupplier val;
+
+    /**
+     * @param name Name.
+     * @param descr Description.
+     * @param val Supplier.
+     */
+    public LongGauge(String name, @Nullable String descr, LongSupplier val) {
+        super(name, descr);
+
+        this.val = val;
+    }
 
     /** {@inheritDoc} */
-    @Override protected boolean persist() {
-        return true;
+    @Override public long value() {
+        return val.getAsLong();
     }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/metric/impl/LongMetricImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/metric/impl/LongMetricImpl.java
new file mode 100644
index 0000000..8c16e93
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/metric/impl/LongMetricImpl.java
@@ -0,0 +1,81 @@
+/*
+ * 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.metric.impl;
+
+import java.util.concurrent.atomic.AtomicLongFieldUpdater;
+import org.apache.ignite.internal.processors.metric.AbstractMetric;
+import org.apache.ignite.spi.metric.LongMetric;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Long metric implementation.
+ */
+public class LongMetricImpl extends AbstractMetric implements LongMetric {
+    /** Field updater. */
+    static final AtomicLongFieldUpdater<LongMetricImpl> updater =
+        AtomicLongFieldUpdater.newUpdater(LongMetricImpl.class, "val");
+
+    /** Field value. */
+    private volatile long val;
+
+    /**
+     * @param name Name.
+     * @param descr Description.
+     */
+    public LongMetricImpl(String name, @Nullable String descr) {
+        super(name, descr);
+    }
+
+    /**
+     * Adds x to the metric.
+     *
+     * @param x Value to be added.
+     */
+    public void add(long x) {
+        updater.getAndAdd(this, x);
+    }
+
+    /** Adds 1 to the metric. */
+    public void increment() {
+        add(1);
+    }
+
+    /** Adds -1 to the metric. */
+    public void decrement() {
+        add(-1);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void reset() {
+        updater.set(this, 0);
+    }
+
+    /** {@inheritDoc} */
+    @Override public long value() {
+        return val;
+    }
+
+    /**
+     * Sets value.
+     *
+     * @param val Value.
+     */
+    public void value(long val) {
+        this.val = val;
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/metric/impl/MetricUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/metric/impl/MetricUtils.java
new file mode 100644
index 0000000..bc6577b
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/metric/impl/MetricUtils.java
@@ -0,0 +1,157 @@
+/*
+ * 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.metric.impl;
+
+import org.apache.ignite.internal.processors.metric.GridMetricManager;
+import org.apache.ignite.internal.processors.metric.MetricRegistry;
+
+/**
+ * Utility class to build or parse metric name in dot notation.
+ *
+ * @see GridMetricManager
+ * @see MetricRegistry
+ */
+public class MetricUtils {
+    /** Metric name part separator. */
+    public static final String SEPARATOR = ".";
+
+    /**
+     * Example - metric name - "io.statistics.PRIMARY_KEY_IDX.pagesCount".
+     * root = io - JMX tree root.
+     * subName = statistics.PRIMARY_KEY_IDX - bean name.
+     * msetName = io.statistics.PRIMARY_KEY_IDX - prefix to search metrics for a bean.
+     * mname = pagesCount - metric name.
+     *
+     * @param name Metric name.
+     * @return Parsed names parts.
+     */
+    public static MetricName parse(String name) {
+        int firstDot = name.indexOf('.');
+        int lastDot = name.lastIndexOf('.');
+
+        String grp = name.substring(0, firstDot);
+        String beanName = name.substring(firstDot + 1, lastDot);
+        String msetName = name.substring(0, lastDot);
+        String mname = name.substring(lastDot + 1);
+
+        return new MetricName(grp, beanName, msetName, mname);
+    }
+
+    /**
+     * Builds metric name. Each parameter will separated by '.' char.
+     *
+     * @param names Metric name parts.
+     * @return Metric name.
+     */
+    public static String metricName(String... names) {
+        assert names != null;
+        assert ensureAllNamesNotEmpty(names);
+
+        if (names.length == 1)
+            return names[0];
+
+        return String.join(SEPARATOR, names);
+
+    }
+
+    /**
+     * Atomically sets the value to the given updated value
+     * if the current value {@code ==} the expected value.
+     *
+     * @param m Metric.
+     * @param expect The expected value.
+     * @param update The new value.
+     * @return {@code true} if successful. False return indicates that
+     * the actual value was not equal to the expected value.
+     */
+    public static boolean compareAndSet(LongMetricImpl m, long expect, long update) {
+        return LongMetricImpl.updater.compareAndSet(m, expect, update);
+    }
+
+    /**
+     * Asserts all arguments are not empty.
+     *
+     * @param names Names.
+     * @return True.
+     */
+    private static boolean ensureAllNamesNotEmpty(String... names) {
+        for (int i=0; i<names.length; i++)
+            assert names[i] != null && !names[i].isEmpty() : i + " element is empty [" + String.join(".", names) + "]";
+
+        return true;
+    }
+
+    /**
+     * Parsed metric name parts.
+     *
+     * Example - metric name - "io.statistics.PRIMARY_KEY_IDX.pagesCount".
+     * root = io - JMX tree root.
+     * subName = statistics.PRIMARY_KEY_IDX - bean name.
+     * msetName = io.statistics.PRIMARY_KEY_IDX - prefix to search metrics for a bean.
+     * mname = pagesCount - metric name.
+     */
+    public static class MetricName {
+        /** JMX group name. */
+        private String root;
+
+        /** JMX bean name. */
+        private String subName;
+
+        /** Prefix to search metrics that belongs to metric set. */
+        private String msetName;
+
+        /** Metric name. */
+        private String mname;
+
+        /** */
+        MetricName(String root, String subName, String msetName, String mname) {
+            this.root = root;
+            this.subName = subName;
+            this.msetName = msetName;
+            this.mname = mname;
+        }
+
+        /**
+         * @return JMX group name.
+         */
+        public String root() {
+            return root;
+        }
+
+        /**
+         * @return JMX bean name.
+         */
+        public String subName() {
+            return subName;
+        }
+
+        /**
+         * @return Prefix to search other metrics for metric set represented by this prefix.
+         */
+        public String msetName() {
+            return msetName;
+        }
+
+        /**
+         * @return Metric name.
+         */
+        public String mname() {
+            return mname;
+        }
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/metric/impl/ObjectGauge.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/metric/impl/ObjectGauge.java
new file mode 100644
index 0000000..1d9559f
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/metric/impl/ObjectGauge.java
@@ -0,0 +1,57 @@
+/*
+ * 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.metric.impl;
+
+import java.util.function.Supplier;
+import org.apache.ignite.internal.processors.metric.AbstractMetric;
+import org.apache.ignite.spi.metric.ObjectMetric;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Implementation based on primitive supplier.
+ */
+public class ObjectGauge<T> extends AbstractMetric implements ObjectMetric<T> {
+    /** Value supplier. */
+    private final Supplier<T> val;
+
+    /** Type. */
+    private final Class<T> type;
+
+    /**
+     * @param name Name.
+     * @param descr Description.
+     * @param val Supplier.
+     * @param type Type.
+     */
+    public ObjectGauge(String name, @Nullable String descr, Supplier<T> val, Class<T> type) {
+        super(name, descr);
+
+        this.val = val;
+        this.type = type;
+    }
+
+    /** {@inheritDoc} */
+    @Override public T value() {
+        return val.get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public Class<T> type() {
+        return type;
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/metric/impl/ObjectMetricImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/metric/impl/ObjectMetricImpl.java
new file mode 100644
index 0000000..7dc8ddb
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/metric/impl/ObjectMetricImpl.java
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.metric.impl;
+
+import org.apache.ignite.internal.processors.metric.AbstractMetric;
+import org.apache.ignite.spi.metric.ObjectMetric;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Implementation of {@link ObjectMetric}.
+ */
+public class ObjectMetricImpl<T> extends AbstractMetric implements ObjectMetric<T> {
+    /** Value. */
+    private volatile T val;
+
+    /** Type. */
+    private final Class<T> type;
+
+    /**
+     * @param name Name.
+     * @param description Description.
+     * @param type Type.
+     */
+    public ObjectMetricImpl(String name, @Nullable String description, Class<T> type) {
+        super(name, description);
+
+        this.type = type;
+    }
+
+    /** {@inheritDoc} */
+    @Override public T value() {
+        return val;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Class<T> type() {
+        return type;
+    }
+
+    /**
+     * Sets value.
+     *
+     * @param val Value.
+     */
+    public void value(T val) {
+        this.val = val;
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java
index cc7bc67..49421d3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java
@@ -99,6 +99,9 @@ public class QueryUtils {
     /** Schema for system view. */
     public static final String SCHEMA_SYS = getBoolean(IGNITE_SQL_SYSTEM_SCHEMA_NAME_IGNITE, false) ? "IGNITE" : "SYS";
 
+    /** Schema for monitoring views. */
+    public static final String SCHEMA_MONITORING = "MONITORING";
+
     /** Schema for system view. */
     public static final String SCHEMA_INFORMATION = "INFORMATION_SCHEMA";
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/stat/IoStatisticsManager.java b/modules/core/src/main/java/org/apache/ignite/internal/stat/IoStatisticsManager.java
deleted file mode 100644
index b6e4907..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/stat/IoStatisticsManager.java
+++ /dev/null
@@ -1,226 +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.stat;
-
-import java.time.OffsetDateTime;
-import java.util.Collections;
-import java.util.EnumMap;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.stream.Collectors;
-import org.apache.ignite.IgniteException;
-import org.apache.ignite.IgniteIllegalStateException;
-
-/**
- * IO statistics manager to manage of gathering IO statistics.
- */
-public class IoStatisticsManager {
-    /** All statistic holders */
-    private final Map<IoStatisticsType, Map<IoStatisticsHolderKey, IoStatisticsHolder>> statByType;
-
-    /** Time of since statistics start gathering. */
-    private volatile OffsetDateTime startTime;
-
-    /**
-     * Constructor.
-     */
-    public IoStatisticsManager() {
-        statByType = new EnumMap<>(IoStatisticsType.class);
-
-        for (IoStatisticsType types : IoStatisticsType.values())
-            statByType.put(types, new ConcurrentHashMap<>());
-
-        reset();
-    }
-
-    /**
-     * Create and register statistics holder for cache group.
-     *
-     * @param name Name of cache or cache group.
-     * @param grpId Cache group id.
-     * @return created statistics holder.
-     */
-    public IoStatisticsHolder registerCacheGroup(String name, int grpId) {
-        return register(IoStatisticsType.CACHE_GROUP, name, grpId);
-    }
-
-    /**
-     * Create and register statistics holder for index.
-     *
-     * @param type Type of index statistics.
-     * @param name Name of cache or cache group.
-     * @param idxName Name of index.
-     * @return created statistics holder.
-     */
-    public IoStatisticsHolder registerIndex(IoStatisticsType type, String name, String idxName) {
-        assert type == IoStatisticsType.HASH_INDEX || type == IoStatisticsType.SORTED_INDEX : type;
-
-        return register(type, name, idxName);
-    }
-
-    /**
-     * Create and register statistics holder.
-     *
-     * @param type Type of statistics.
-     * @param name Name of element of statistics.
-     * @param param second parameter of statistic's element.
-     * @return created statistics holder.
-     */
-    private IoStatisticsHolder register(IoStatisticsType type, String name, Object param) {
-        if (statByType.isEmpty())
-            throw new IgniteIllegalStateException("IO Statistics manager has been stopped and can'be used");
-
-        IoStatisticsHolder stat;
-        IoStatisticsHolderKey statKey;
-
-        switch (type) {
-            case CACHE_GROUP:
-                stat = new IoStatisticsHolderCache(name, (Integer)param);
-                statKey = new IoStatisticsHolderKey(name);
-
-                break;
-
-            case HASH_INDEX:
-            case SORTED_INDEX:
-                stat = new IoStatisticsHolderIndex(name, (String)param);
-                statKey = new IoStatisticsHolderKey(name, (String)param);
-
-                break;
-
-            default:
-                throw new IgniteException("Gathering IO statistics for " + type + "doesn't support");
-        }
-
-        IoStatisticsHolder existedStatisitcHolder = statByType.get(type).putIfAbsent(statKey, stat);
-
-        return (existedStatisitcHolder != null) ? existedStatisitcHolder : stat;
-    }
-
-    /**
-     * Remove all holders.
-     */
-    public void stop() {
-        statByType.clear();
-    }
-
-    /**
-     * Reset statistics
-     */
-    public void reset() {
-        statByType.forEach((t, s) ->
-            s.forEach((k, sh) -> sh.resetStatistics())
-        );
-
-        startTime = OffsetDateTime.now();
-    }
-
-    /**
-     * @return When statistics gathering start.
-     */
-    public OffsetDateTime startTime() {
-        return startTime;
-    }
-
-    /**
-     * Extract all tracked names for given statistics type.
-     *
-     * @param statType Type of statistics which tracked names need to extract.
-     * @return Set of present names for given statType
-     */
-    public Set<String> deriveStatisticNames(IoStatisticsType statType) {
-        assert statType != null;
-
-        return statByType.get(statType).keySet().stream()
-            .map(IoStatisticsHolderKey::name)
-            .collect(Collectors.toSet());
-    }
-
-    /**
-     * Extract all tracked subNames for given statistics type .
-     *
-     * @param name Name of element of statistics.
-     * @param statType Type of statistics which tracked names need to extract.
-     * @return Set of present names for given statType
-     */
-    public Set<String> deriveStatisticSubNames(IoStatisticsType statType, String name) {
-        assert statType != null;
-
-        return statByType.get(statType).keySet().stream()
-            .filter(k -> k.name().equalsIgnoreCase(name) && k.subName() != null)
-            .map(IoStatisticsHolderKey::subName)
-            .collect(Collectors.toSet());
-    }
-
-    /**
-     * @param statType Type of statistics which need to take.
-     * @param name name of statistics which need to take, e.g. cache name
-     * @param subName subName of statistics which need to take, e.g. index name.
-     * @return Tracked physical reads by types since last reset statistics.
-     */
-    public Map<String, Long> physicalReadsMap(IoStatisticsType statType, String name, String subName) {
-        IoStatisticsHolder statHolder = statByType.get(statType).get(new IoStatisticsHolderKey(name, subName));
-
-        return (statHolder != null) ? statHolder.physicalReadsMap() : Collections.emptyMap();
-    }
-
-    /**
-     * @param statType Type of statistics which need to take.
-     * @param name name of statistics which need to take, e.g. cache name
-     * @param subName subName of statistics which need to take, e.g. index name.
-     * @return Number of physical reads since last reset statistics.
-     */
-    public Long physicalReads(IoStatisticsType statType, String name, String subName) {
-        IoStatisticsHolder statHolder = statByType.get(statType).get(new IoStatisticsHolderKey(name, subName));
-
-        return (statHolder != null) ? statHolder.physicalReads() : null;
-    }
-
-    /**
-     * @param statType Type of statistics which need to take.
-     * @param name name of statistics which need to take, e.g. cache name
-     * @param subName subName of statistics which need to take, e.g. index name.
-     * @return Tracked logical reads by types since last reset statistics.
-     */
-    public Map<String, Long> logicalReadsMap(IoStatisticsType statType, String name, String subName) {
-        IoStatisticsHolder statHolder = statByType.get(statType).get(new IoStatisticsHolderKey(name, subName));
-
-        return (statHolder != null) ? statHolder.logicalReadsMap() : Collections.emptyMap();
-    }
-
-    /**
-     * @param statType Type of statistics which need to take.
-     * @param name name of statistics which need to take, e.g. cache name
-     * @param subName subName of statistics which need to take, e.g. index name.
-     * @return Number of logical reads since last reset statistics.
-     */
-    public Long logicalReads(IoStatisticsType statType, String name, String subName) {
-        IoStatisticsHolder stat = statByType.get(statType).get(new IoStatisticsHolderKey(name, subName));
-
-        return (stat != null) ? stat.logicalReads() : null;
-    }
-
-    /**
-     * @param statType Type of statistics which need to take.
-     * @return All tracked statistics for given type.
-     */
-    public Map<IoStatisticsHolderKey, IoStatisticsHolder> statistics(IoStatisticsType statType){
-        return Collections.unmodifiableMap(statByType.get(statType));
-    }
-}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/stat/IoStatisticsMetricsLocalMXBeanImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/stat/IoStatisticsMetricsLocalMXBeanImpl.java
deleted file mode 100644
index 8248d34..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/stat/IoStatisticsMetricsLocalMXBeanImpl.java
+++ /dev/null
@@ -1,146 +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.stat;
-
-import java.time.format.DateTimeFormatter;
-import java.util.Map;
-import java.util.stream.Collectors;
-import java.util.stream.Stream;
-import org.apache.ignite.mxbean.IoStatisticsMetricsMXBean;
-
-import static org.apache.ignite.internal.stat.IoStatisticsHolderIndex.HASH_PK_IDX_NAME;
-
-/**
- * JMX bean to expose local node IO statistics.
- */
-public class IoStatisticsMetricsLocalMXBeanImpl implements IoStatisticsMetricsMXBean {
-    /** IO statistic manager. */
-    private IoStatisticsManager statMgr;
-
-    /**
-     * @param statMgr IO statistic manager.
-     */
-    public IoStatisticsMetricsLocalMXBeanImpl(IoStatisticsManager statMgr) {
-        this.statMgr = statMgr;
-    }
-
-    /** {@inheritDoc} */
-    @Override public long getStartTime() {
-        return statMgr.startTime().toEpochSecond();
-    }
-
-    /** {@inheritDoc} */
-    @Override public String getStartTimeLocal() {
-        return statMgr.startTime().format(DateTimeFormatter.ISO_DATE_TIME);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void reset() {
-        statMgr.reset();
-    }
-
-    /** {@inheritDoc} */
-    @Override public String getCacheGroupStatistics(String cacheGrpName) {
-        return formattedStats(IoStatisticsType.CACHE_GROUP, cacheGrpName, null);
-    }
-
-    /** {@inheritDoc} */
-    @Override public Long getCacheGroupPhysicalReads(String cacheGrpName) {
-        return statMgr.physicalReads(IoStatisticsType.CACHE_GROUP, cacheGrpName, null);
-    }
-
-    /** {@inheritDoc} */
-    @Override public Long getCacheGroupLogicalReads(String cacheGrpName) {
-        return statMgr.logicalReads(IoStatisticsType.CACHE_GROUP, cacheGrpName, null);
-    }
-
-    /** {@inheritDoc} */
-    @Override public String getIndexStatistics(String cacheGrpName, String idxName) {
-        return formattedStats(getIndexStatType(idxName), cacheGrpName, idxName);
-    }
-
-    /**
-     * @param idxName Name of index
-     * @return Type of index statistics.
-     */
-    private IoStatisticsType getIndexStatType(String idxName) {
-        return idxName.equals(HASH_PK_IDX_NAME) ? IoStatisticsType.HASH_INDEX : IoStatisticsType.SORTED_INDEX;
-    }
-
-    /**
-     * Gets string presentation of IO statistics for given parameters.
-     *
-     * @param statType Type of statistics.
-     * @param name Name of statistics
-     * @param subName SubName of statistics.
-     * @return String presentation of IO statistics for given parameters.
-     */
-    private String formattedStats(IoStatisticsType statType, String name, String subName) {
-        Map<String, Long> logicalReads = statMgr.logicalReadsMap(statType, name, subName);
-
-        Map<String, Long> physicalReads = statMgr.physicalReadsMap(statType, name, subName);
-
-        String stats = Stream.concat(logicalReads.entrySet().stream(), physicalReads.entrySet().stream())
-            .map(e -> e.getKey() + "=" + e.getValue())
-            .collect(Collectors.joining(", ", "[", "]"));
-
-        String statInfo = statType.name() + " " + (subName != null ? name + "." + subName : name);
-
-        return statInfo + " " + stats;
-    }
-
-    /** {@inheritDoc} */
-    @Override public Long getIndexPhysicalReads(String cacheGrpName, String idxName) {
-        return statMgr.physicalReads(getIndexStatType(idxName), cacheGrpName, idxName);
-    }
-
-    /** {@inheritDoc} */
-    @Override public Long getIndexLogicalReads(String cacheGrpName, String idxName) {
-        return statMgr.logicalReads(getIndexStatType(idxName), cacheGrpName, idxName);
-    }
-
-    /** {@inheritDoc} */
-    @Override public Long getIndexLeafLogicalReads(String cacheGrpName, String idxName) {
-        Map<String, Long> logicalReads = statMgr.logicalReadsMap(getIndexStatType(idxName), cacheGrpName, idxName);
-
-        return logicalReads.get(IoStatisticsHolderIndex.LOGICAL_READS_LEAF);
-    }
-
-    /** {@inheritDoc} */
-    @Override public Long getIndexLeafPhysicalReads(String cacheGrpName, String idxName) {
-        Map<String, Long> logicalReads = statMgr.physicalReadsMap(getIndexStatType(idxName), cacheGrpName, idxName);
-
-        return logicalReads.get(IoStatisticsHolderIndex.PHYSICAL_READS_LEAF);
-    }
-
-    /** {@inheritDoc} */
-    @Override public Long getIndexInnerLogicalReads(String cacheGrpName, String idxName) {
-        Map<String, Long> logicalReads = statMgr.logicalReadsMap(getIndexStatType(idxName), cacheGrpName, idxName);
-
-        return logicalReads.get(IoStatisticsHolderIndex.LOGICAL_READS_INNER);
-    }
-
-    /** {@inheritDoc} */
-    @Override public Long getIndexInnerPhysicalReads(String cacheGrpName, String idxName) {
-        Map<String, Long> logicalReads = statMgr.physicalReadsMap(getIndexStatType(idxName), cacheGrpName, idxName);
-
-        return logicalReads.get(IoStatisticsHolderIndex.PHYSICAL_READS_INNER);
-    }
-
-}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
index bb18c27..ab77b9d 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
@@ -4724,9 +4724,16 @@ public abstract class IgniteUtils {
         assert name != null;
         assert itf != null;
 
-        DynamicMBean mbean = new IgniteStandardMXBean(impl, itf);
+        DynamicMBean mbean;
 
-        mbean.getMBeanInfo();
+        if (impl instanceof DynamicMBean) {
+            mbean = (DynamicMBean)impl;
+        }
+        else {
+            mbean = new IgniteStandardMXBean(impl, itf);
+
+            mbean.getMBeanInfo();
+        }
 
         return mbeanSrv.registerMBean(mbean, name).getObjectName();
     }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java b/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java
index 2d4d1cf..64213c7 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java
@@ -40,9 +40,15 @@ import java.util.concurrent.Callable;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.BooleanSupplier;
+import java.util.function.DoubleSupplier;
+import java.util.function.IntSupplier;
+import java.util.function.LongSupplier;
+import java.util.function.Supplier;
 import javax.cache.Cache;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.cluster.BaselineNode;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.internal.IgniteFutureTimeoutCheckedException;
@@ -96,6 +102,7 @@ import org.apache.ignite.internal.util.lang.gridfunc.TransformMapView2;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.T2;
 import org.apache.ignite.internal.util.typedef.internal.A;
+import org.apache.ignite.internal.util.typedef.internal.LT;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiClosure;
 import org.apache.ignite.lang.IgniteBiTuple;
@@ -3119,4 +3126,190 @@ public class GridFunc {
 
         return rdc == null ? null : rdc.reduce();
     }
+
+    /**
+     * @param arr Array to check.
+     * @return {@code True} if array sorted, {@code false} otherwise.
+     */
+    public static boolean isSorted(long[] arr) {
+        if (isEmpty(arr) || arr.length == 1)
+            return true;
+
+        for (int i = 1; i < arr.length; i++) {
+            if (arr[i - 1] > arr[i])
+                return false;
+        }
+
+        return true;
+    }
+
+    /**
+     * Return supplier that suppress any exception throwed by {@code s}.
+     * Returned supplier will produce {@code 0} on any exception in {@code s}.
+     *
+     * @param s Root supplier.
+     * @param log Logger.
+     * @return Supplier that suppress any exception throwed by {@code s}.
+     */
+    public static BooleanSupplier nonThrowableSupplier(BooleanSupplier s, IgniteLogger log) {
+        return nonThrowableSupplier(s, false, log);
+    }
+
+    /**
+     * Return supplier that suppress any exception throwed by {@code s}.
+     * Returned supplier will produce {@code .0d} on any exception in {@code s}.
+     *
+     * @param s Root supplier.
+     * @param log Logger.
+     * @return Supplier that suppress any exception throwed by {@code s}.
+     */
+    public static DoubleSupplier nonThrowableSupplier(DoubleSupplier s, IgniteLogger log) {
+        return nonThrowableSupplier(s, .0d, log);
+    }
+
+    /**
+     * Return supplier that suppress any exception throwed by {@code s}.
+     * Returned supplier will produce {@code 0} on any exception in {@code s}.
+     *
+     * @param s Root supplier.
+     * @param log Logger.
+     * @return Supplier that suppress any exception throwed by {@code s}.
+     */
+    public static IntSupplier nonThrowableSupplier(IntSupplier s, IgniteLogger log) {
+        return nonThrowableSupplier(s, 0, log);
+    }
+
+    /**
+     * Return supplier that suppress any exception throwed by {@code s}.
+     * Returned supplier will produce {@code 0} on any exception in {@code s}.
+     *
+     * @param s Root supplier.
+     * @param log Logger.
+     * @return Supplier that suppress any exception throwed by {@code s}.
+     */
+    public static LongSupplier nonThrowableSupplier(LongSupplier s, IgniteLogger log) {
+        return nonThrowableSupplier(s, 0, log);
+    }
+
+    /**
+     * Return supplier that suppress any exception throwed by {@code s}.
+     * Returned supplier will produce {@code null} on any exception in {@code s}.
+     *
+     * @param s Root supplier.
+     * @param log Logger.
+     * @return Supplier that suppress any exception throwed by {@code s}.
+     */
+    public static <T> Supplier<T> nonThrowableSupplier(Supplier<T> s, IgniteLogger log) {
+        return nonThrowableSupplier(s, null, log);
+    }
+
+    /**
+     * Return supplier that suppress any exception throwed by {@code s}.
+     * Returned supplier will produce {@code dfltVal} on any exception in {@code s}.
+     *
+     * @param s Root supplier.
+     * @param dfltVal Value returned on exception in {@code s}.
+     * @param log Logger.
+     * @return Supplier that suppress any exception throwed by {@code s}.
+     */
+    public static BooleanSupplier nonThrowableSupplier(BooleanSupplier s, boolean dfltVal, IgniteLogger log) {
+        return () -> {
+            try {
+                return s.getAsBoolean();
+            }
+            catch (Exception e) {
+                LT.warn(log, e, "Exception in supplier", false, true);
+
+                return dfltVal;
+            }
+        };
+    }
+
+    /**
+     * Return supplier that suppress any exception throwed by {@code s}.
+     * Returned supplier will produce {@code dfltVal} on any exception in {@code s}.
+     *
+     * @param s Root supplier.
+     * @param dfltVal Value returned on exception in {@code s}.
+     * @param log Logger.
+     * @return Supplier that suppress any exception throwed by {@code s}.
+     */
+    public static DoubleSupplier nonThrowableSupplier(DoubleSupplier s, double dfltVal, IgniteLogger log) {
+        return () -> {
+            try {
+                return s.getAsDouble();
+            }
+            catch (Exception e) {
+                LT.warn(log, e, "Exception in supplier", false, true);
+
+                return dfltVal;
+            }
+        };
+    }
+
+    /**
+     * Return supplier that suppress any exception throwed by {@code s}.
+     * Returned supplier will produce {@code dfltVal} on any exception in {@code s}.
+     *
+     * @param s Root supplier.
+     * @param dfltVal Value returned on exception in {@code s}.
+     * @param log Logger.
+     * @return Supplier that suppress any exception throwed by {@code s}.
+     */
+    public static IntSupplier nonThrowableSupplier(IntSupplier s, int dfltVal, IgniteLogger log) {
+        return () -> {
+            try {
+                return s.getAsInt();
+            }
+            catch (Exception e) {
+                LT.warn(log, e, "Exception in supplier", false, true);
+
+                return dfltVal;
+            }
+        };
+    }
+
+    /**
+     * Return supplier that suppress any exception throwed by {@code s}.
+     * Returned supplier will produce {@code dfltVal} on any exception in {@code s}.
+     *
+     * @param s Root supplier.
+     * @param dfltVal Value returned on exception in {@code s}.
+     * @param log Logger.
+     * @return Supplier that suppress any exception throwed by {@code s}.
+     */
+    public static LongSupplier nonThrowableSupplier(LongSupplier s, long dfltVal, IgniteLogger log) {
+        return () -> {
+            try {
+                return s.getAsLong();
+            }
+            catch (Exception e) {
+                LT.warn(log, e, "Exception in supplier", false, true);
+
+                return dfltVal;
+            }
+        };
+    }
+
+    /**
+     * Return supplier that suppress any exception throwed by {@code s}.
+     * Returned supplier will produce {@code dfltVal} on any exception in {@code s}.
+     *
+     * @param s Root supplier.
+     * @param dfltVal Value returned on exception in {@code s}.
+     * @param log Logger.
+     * @return Supplier that suppress any exception throwed by {@code s}.
+     */
+    public static <T> Supplier<T> nonThrowableSupplier(Supplier<T> s, T dfltVal, IgniteLogger log) {
+        return () -> {
+            try {
+                return s.get();
+            }
+            catch (Exception e) {
+                LT.warn(log, e, "Exception in supplier", false, true);
+
+                return dfltVal;
+            }
+        };
+    }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/mxbean/CacheMetricsMXBean.java b/modules/core/src/main/java/org/apache/ignite/mxbean/CacheMetricsMXBean.java
index a580d54..f55303a 100644
--- a/modules/core/src/main/java/org/apache/ignite/mxbean/CacheMetricsMXBean.java
+++ b/modules/core/src/main/java/org/apache/ignite/mxbean/CacheMetricsMXBean.java
@@ -21,10 +21,14 @@ import javax.cache.management.CacheMXBean;
 import javax.cache.management.CacheStatisticsMXBean;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.cache.CacheMetrics;
+import org.apache.ignite.internal.processors.metric.GridMetricManager;
 
 /**
  * This interface defines JMX view on {@link IgniteCache}.
+ *
+ * @deprecated Use {@link GridMetricManager} instead.
  */
+@Deprecated
 @MXBeanDescription("MBean that provides access to cache descriptor.")
 public interface CacheMetricsMXBean extends CacheStatisticsMXBean, CacheMXBean, CacheMetrics {
     /** {@inheritDoc} */
diff --git a/modules/core/src/main/java/org/apache/ignite/mxbean/DataRegionMetricsMXBean.java b/modules/core/src/main/java/org/apache/ignite/mxbean/DataRegionMetricsMXBean.java
index 949daea..78bc961 100644
--- a/modules/core/src/main/java/org/apache/ignite/mxbean/DataRegionMetricsMXBean.java
+++ b/modules/core/src/main/java/org/apache/ignite/mxbean/DataRegionMetricsMXBean.java
@@ -18,10 +18,14 @@ package org.apache.ignite.mxbean;
 
 import org.apache.ignite.DataRegionMetrics;
 import org.apache.ignite.configuration.DataRegionConfiguration;
+import org.apache.ignite.internal.processors.metric.GridMetricManager;
 
 /**
  * This interface defines a JMX view on {@link DataRegionMetrics}.
+ *
+ * @deprecated Use {@link GridMetricManager} instead.
  */
+@Deprecated
 @MXBeanDescription("MBean that provides access to DataRegionMetrics of a local Apache Ignite node.")
 public interface DataRegionMetricsMXBean extends DataRegionMetrics {
     /** {@inheritDoc} */
diff --git a/modules/core/src/main/java/org/apache/ignite/mxbean/DataStorageMetricsMXBean.java b/modules/core/src/main/java/org/apache/ignite/mxbean/DataStorageMetricsMXBean.java
index 4689f15..3eec790 100644
--- a/modules/core/src/main/java/org/apache/ignite/mxbean/DataStorageMetricsMXBean.java
+++ b/modules/core/src/main/java/org/apache/ignite/mxbean/DataStorageMetricsMXBean.java
@@ -19,10 +19,14 @@ package org.apache.ignite.mxbean;
 
 import org.apache.ignite.DataStorageMetrics;
 import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.internal.processors.metric.GridMetricManager;
 
 /**
  * An MX bean allowing to monitor and tune persistence metrics.
+ *
+ * @deprecated Use {@link GridMetricManager} instead.
  */
+@Deprecated
 public interface DataStorageMetricsMXBean extends DataStorageMetrics {
     /** {@inheritDoc} */
     @MXBeanDescription("Average number of WAL records per second written during the last time interval.")
diff --git a/modules/core/src/main/java/org/apache/ignite/mxbean/IgniteMXBean.java b/modules/core/src/main/java/org/apache/ignite/mxbean/IgniteMXBean.java
index e0a5fb5..331123b 100644
--- a/modules/core/src/main/java/org/apache/ignite/mxbean/IgniteMXBean.java
+++ b/modules/core/src/main/java/org/apache/ignite/mxbean/IgniteMXBean.java
@@ -495,4 +495,14 @@ public interface IgniteMXBean {
      */
     @MXBeanDescription("Clears local node map.")
     void clearNodeLocalMap();
+
+    /**
+     * Resets metrics for a given prefix.
+     *
+     * @param prefix Prefix for a metrics.
+     */
+    @MXBeanDescription("Resets metrics for a given prefix.")
+    @MXBeanParametersNames("prefix")
+    @MXBeanParametersDescriptions("Metric prefix.")
+    public void resetMetrics(String prefix);
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/mxbean/IoStatisticsMetricsMXBean.java b/modules/core/src/main/java/org/apache/ignite/mxbean/IoStatisticsMetricsMXBean.java
deleted file mode 100644
index bad848c..0000000
--- a/modules/core/src/main/java/org/apache/ignite/mxbean/IoStatisticsMetricsMXBean.java
+++ /dev/null
@@ -1,172 +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.mxbean;
-
-/**
- * This interface defines JMX view for IO statistics.
- */
-@MXBeanDescription("MBean that provides access IO statistics metrics.")
-public interface IoStatisticsMetricsMXBean {
-
-    /**
-     * @return Start time of gathering statistics as UTC milliseconds.
-     */
-    @MXBeanDescription("Start time of gathering staistics.")
-    long getStartTime();
-
-    /**
-     * @return Start time of gathering statistics in ISO-8601 format.
-     */
-    @MXBeanDescription("Start time of gathering staistics.")
-    String getStartTimeLocal();
-
-    /**
-     * Reset all IO statistics.
-     */
-    @MXBeanDescription("Reset gathered statistics.")
-    public void reset();
-
-    /**
-     * Gets string presentation of cache group IO statistics for given cache group.
-     *
-     * @param cacheGrpName Name of cache group.
-     * @return Formatted representation of cache group IO statistics.
-     */
-    @MXBeanDescription("String presentation of cache group IO statistics.")
-    @MXBeanParametersNames("cacheGrpName")
-    @MXBeanParametersDescriptions("Cache group name.")
-    public String getCacheGroupStatistics(String cacheGrpName);
-
-    /**
-     * Gets number of physical page reads for given cache group.
-     *
-     * @param cacheGrpName Name of cache group.
-     * @return Number of physical page reads for given cache group. {@code null} in case such statistics doesn't exists.
-     */
-    @MXBeanDescription("Number of physical page reads for given cache group." +
-        "Can return null in case such statistics doesn't exists.")
-    @MXBeanParametersNames("cacheGrpName")
-    @MXBeanParametersDescriptions("Cache group name.")
-    public Long getCacheGroupPhysicalReads(String cacheGrpName);
-
-    /**
-     * Gets number of logical page reads for given cache group.
-     *
-     * @param cacheGrpName Name of cache group.
-     * @return Number of logical page reads for given cache group. {@code null} in case such statistics doesn't exists.
-     */
-    @MXBeanDescription("Number of logical page reads for given cache group. " +
-        "Can return null in case such statistics doesn't exists.")
-    @MXBeanParametersNames("cacheGrpName")
-    @MXBeanParametersDescriptions("Cache group name.")
-    public Long getCacheGroupLogicalReads(String cacheGrpName);
-
-    /**
-     * Gets string presentation of index IO statistics for given cache group and index.
-     *
-     * @param cacheGrpName Name of cache group.
-     * @param idxName Name of index.
-     * @return Formatted representation of index IO statistics for given cache group and index.
-     */
-    @MXBeanDescription("String presentation of index IO statistics.")
-    @MXBeanParametersNames({"cacheGrpName", "idxName"})
-    @MXBeanParametersDescriptions({"Cache group name.", "Index name."})
-    public String getIndexStatistics(String cacheGrpName, String idxName);
-
-    /**
-     * Gets number of physical index page reads for given cache group and index.
-     *
-     * @param cacheGrpName Name of cache group.
-     * @param idxName Name of index.
-     * @return Number of physical page reads for given cache group and index. {@code null} in case such statistics doesn't exists.
-     */
-    @MXBeanDescription("Number of physical page reads for given cache group." +
-        "Can return null in case such statistics doesn't exists.")
-    @MXBeanParametersNames({"cacheGrpName", "idxName"})
-    @MXBeanParametersDescriptions({"Cache group name.", "Index name."})
-    public Long getIndexPhysicalReads(String cacheGrpName, String idxName);
-
-    /**
-     * Gets number of logical index page reads for given cache group and index.
-     *
-     * @param cacheGrpName Name of cache group.
-     * @param idxName Name of index.
-     * @return Number of logical page reads for given cache group and index. {@code null} in case such statistics doesn't exists.
-     */
-    @MXBeanDescription("Number of logical page reads for given cache group." +
-        "Can return null in case such statistics doesn't exists.")
-    @MXBeanParametersNames({"cacheGrpName", "idxName"})
-    @MXBeanParametersDescriptions({"Cache group name.", "Index name."})
-    public Long getIndexLogicalReads(String cacheGrpName, String idxName);
-
-
-    /**
-     * Gets number of logical leaf index's page reads for given cache group and index.
-     *
-     * @param cacheGrpName Name of cache group.
-     * @param idxName Name of index.
-     * @return Number of logical leaf index's page reads for given cache group and index. {@code null} in case such statistics doesn't exists.
-     */
-    @MXBeanDescription("Number of logical leaf index's page reads for given cache group and index." +
-        "Can return null in case such statistics doesn't exists.")
-    @MXBeanParametersNames({"cacheGrpName", "idxName"})
-    @MXBeanParametersDescriptions({"Cache group name.", "Index name."})
-    public Long getIndexLeafLogicalReads(String cacheGrpName, String idxName);
-
-    /**
-     * Gets number of physical leaf index's page reads for given cache group and index.
-     *
-     * @param cacheGrpName Name of cache group.
-     * @param idxName Name of index.
-     * @return Number of physical leaf index's page reads for given cache group and index. {@code null} in case such statistics doesn't exists.
-     */
-    @MXBeanDescription("Number of physical leaf index's page reads for given cache group and index." +
-        "Can return null in case such statistics doesn't exists.")
-    @MXBeanParametersNames({"cacheGrpName", "idxName"})
-    @MXBeanParametersDescriptions({"Cache group name.", "Index name."})
-    public Long getIndexLeafPhysicalReads(String cacheGrpName, String idxName);
-
-    /**
-     * Gets number of logical inner index's page reads for given cache group and index.
-     *
-     * @param cacheGrpName Name of cache group.
-     * @param idxName Name of index.
-     * @return Number of logical inner index's page reads for given cache group and index. {@code null} in case such statistics doesn't exists.
-     */
-    @MXBeanDescription("Number of logical inner index's page reads for given cache group and index." +
-        "Can return null in case such statistics doesn't exists.")
-    @MXBeanParametersNames({"cacheGrpName", "idxName"})
-    @MXBeanParametersDescriptions({"Cache group name.", "Index name."})
-    public Long getIndexInnerLogicalReads(String cacheGrpName, String idxName);
-
-    /**
-     * Gets number of physical inner index's page reads for given cache group and index.
-     *
-     * @param cacheGrpName Name of cache group.
-     * @param idxName Name of index.
-     * @return Number of physical inner index's page reads for given cache group and index. {@code null} in case such
-     * statistics doesn't exists.
-     */
-    @MXBeanDescription("Number of physical inner index's page reads for given cache group and index." +
-        "Can return null in case such statistics doesn't exists.")
-    @MXBeanParametersNames({"cacheGrpName", "idxName"})
-    @MXBeanParametersDescriptions({"Cache group name.", "Index name."})
-    public Long getIndexInnerPhysicalReads(String cacheGrpName, String idxName);
-
-}
diff --git a/modules/core/src/main/java/org/apache/ignite/mxbean/ThreadPoolMXBean.java b/modules/core/src/main/java/org/apache/ignite/mxbean/ThreadPoolMXBean.java
index f312a1e..daeaa4f 100644
--- a/modules/core/src/main/java/org/apache/ignite/mxbean/ThreadPoolMXBean.java
+++ b/modules/core/src/main/java/org/apache/ignite/mxbean/ThreadPoolMXBean.java
@@ -17,10 +17,15 @@
 
 package org.apache.ignite.mxbean;
 
+import org.apache.ignite.internal.processors.metric.GridMetricManager;
+
 /**
  * MBean that provides access to information about executor service.
+ *
+ * @deprecated Use {@link GridMetricManager} instead.
  */
 @MXBeanDescription("MBean that provides access to information about executor service.")
+@Deprecated
 public interface ThreadPoolMXBean {
     /**
      * Returns the approximate number of threads that are actively executing tasks.
@@ -149,4 +154,4 @@ public interface ThreadPoolMXBean {
      */
     @MXBeanDescription("Class name of thread factory used to create new threads.")
     public String getThreadFactoryClass();
-}
\ No newline at end of file
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
index f68ecd6..f378012 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
@@ -71,7 +71,7 @@ public abstract class IgniteSpiAdapter implements IgniteSpi {
 
     /** */
     @LoggerResource
-    private IgniteLogger log;
+    protected IgniteLogger log;
 
     /** Ignite instance. */
     protected Ignite ignite;
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/stat/IoStatisticsCachePersistenceSelfTest.java b/modules/core/src/main/java/org/apache/ignite/spi/metric/BooleanMetric.java
similarity index 73%
copy from modules/core/src/test/java/org/apache/ignite/internal/stat/IoStatisticsCachePersistenceSelfTest.java
copy to modules/core/src/main/java/org/apache/ignite/spi/metric/BooleanMetric.java
index c203295..193517a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/stat/IoStatisticsCachePersistenceSelfTest.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/metric/BooleanMetric.java
@@ -13,18 +13,19 @@
  * 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.stat;
+package org.apache.ignite.spi.metric;
 
 /**
- * Tests for cache IO statistics for persistence mode.
+ * Interface for the metrics that holds boolean primitive.
  */
-public class IoStatisticsCachePersistenceSelfTest extends IoStatisticsCacheSelfTest {
+public interface BooleanMetric extends Metric {
+    /** @return Value of the metric. */
+    public boolean value();
 
     /** {@inheritDoc} */
-    @Override protected boolean persist() {
-        return true;
+    @Override public default String getAsString() {
+        return Boolean.toString(value());
     }
 }
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/stat/IoStatisticsCachePersistenceSelfTest.java b/modules/core/src/main/java/org/apache/ignite/spi/metric/DoubleMetric.java
similarity index 73%
copy from modules/core/src/test/java/org/apache/ignite/internal/stat/IoStatisticsCachePersistenceSelfTest.java
copy to modules/core/src/main/java/org/apache/ignite/spi/metric/DoubleMetric.java
index c203295..0354244 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/stat/IoStatisticsCachePersistenceSelfTest.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/metric/DoubleMetric.java
@@ -13,18 +13,19 @@
  * 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.stat;
+package org.apache.ignite.spi.metric;
 
 /**
- * Tests for cache IO statistics for persistence mode.
+ * Interface for the metrics that holds double primitive.
  */
-public class IoStatisticsCachePersistenceSelfTest extends IoStatisticsCacheSelfTest {
+public interface DoubleMetric extends Metric {
+    /** @return Value of the metric. */
+    public double value();
 
     /** {@inheritDoc} */
-    @Override protected boolean persist() {
-        return true;
+    @Override public default String getAsString() {
+        return Double.toString(value());
     }
 }
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/stat/IoStatisticsCachePersistenceSelfTest.java b/modules/core/src/main/java/org/apache/ignite/spi/metric/IntMetric.java
similarity index 74%
copy from modules/core/src/test/java/org/apache/ignite/internal/stat/IoStatisticsCachePersistenceSelfTest.java
copy to modules/core/src/main/java/org/apache/ignite/spi/metric/IntMetric.java
index c203295..7ad8ba4 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/stat/IoStatisticsCachePersistenceSelfTest.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/metric/IntMetric.java
@@ -13,18 +13,19 @@
  * 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.stat;
+package org.apache.ignite.spi.metric;
 
 /**
- * Tests for cache IO statistics for persistence mode.
+ * Interface for the metrics that holds int primitive.
  */
-public class IoStatisticsCachePersistenceSelfTest extends IoStatisticsCacheSelfTest {
+public interface IntMetric extends Metric {
+    /** @return Value of the metric. */
+    public int value();
 
     /** {@inheritDoc} */
-    @Override protected boolean persist() {
-        return true;
+    @Override public default String getAsString() {
+        return Integer.toString(value());
     }
 }
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/stat/IoStatisticsCachePersistenceSelfTest.java b/modules/core/src/main/java/org/apache/ignite/spi/metric/LongMetric.java
similarity index 62%
copy from modules/core/src/test/java/org/apache/ignite/internal/stat/IoStatisticsCachePersistenceSelfTest.java
copy to modules/core/src/main/java/org/apache/ignite/spi/metric/LongMetric.java
index c203295..2cdf207 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/stat/IoStatisticsCachePersistenceSelfTest.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/metric/LongMetric.java
@@ -13,18 +13,29 @@
  * 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.stat;
+package org.apache.ignite.spi.metric;
 
 /**
- * Tests for cache IO statistics for persistence mode.
+ * Interface for the metrics that holds long primitive.
  */
-public class IoStatisticsCachePersistenceSelfTest extends IoStatisticsCacheSelfTest {
+public interface LongMetric extends Metric {
+    /** @return Value of the metric. */
+    public long value();
 
     /** {@inheritDoc} */
-    @Override protected boolean persist() {
-        return true;
+    @Override public default String getAsString() {
+        return Long.toString(value());
+    }
+
+    /** @return Value of the metric. */
+    public default long longValue() {
+        return value();
+    }
+
+    /** @return Value of the metric. */
+    public default long get() {
+        return value();
     }
 }
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/stat/IoStatisticsCachePersistenceSelfTest.java b/modules/core/src/main/java/org/apache/ignite/spi/metric/Metric.java
similarity index 63%
copy from modules/core/src/test/java/org/apache/ignite/internal/stat/IoStatisticsCachePersistenceSelfTest.java
copy to modules/core/src/main/java/org/apache/ignite/spi/metric/Metric.java
index c203295..83d244e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/stat/IoStatisticsCachePersistenceSelfTest.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/metric/Metric.java
@@ -13,18 +13,27 @@
  * 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.stat;
+package org.apache.ignite.spi.metric;
+
+import org.jetbrains.annotations.Nullable;
 
 /**
- * Tests for cache IO statistics for persistence mode.
+ * Base interface for all metrics.
  */
-public class IoStatisticsCachePersistenceSelfTest extends IoStatisticsCacheSelfTest {
+public interface Metric {
+    /** @return Name of the metric. */
+    public String name();
+
+    /** @return Description of the metric. */
+    public String description();
+
+    /** @return String representation of metric value. */
+    @Nullable public String getAsString();
 
-    /** {@inheritDoc} */
-    @Override protected boolean persist() {
-        return true;
+    /** Resets metric state. */
+    public default void reset() {
+        // No-op.
     }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/metric/MetricExporterSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/metric/MetricExporterSpi.java
new file mode 100644
index 0000000..70ca699
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/metric/MetricExporterSpi.java
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.spi.metric;
+
+import java.util.function.Consumer;
+import java.util.function.Predicate;
+import org.apache.ignite.internal.processors.metric.MetricRegistry;
+import org.apache.ignite.spi.IgniteSpi;
+import org.apache.ignite.spi.metric.jmx.JmxExporterSpi;
+
+/**
+ * Exporter of monitoring information to the external recepient.
+ * Expected, that each implementation would support some specific protocol.
+ *
+ * Implementation of this Spi should work by pull paradigm.
+ * So after start SPI should respond to some incoming request.
+ * HTTP servlet or JMX bean are good examples of expected implementations.
+ *
+ * @see ReadOnlyMetricRegistry
+ * @see Metric
+ * @see BooleanMetric
+ * @see DoubleMetric
+ * @see IntMetric
+ * @see LongMetric
+ * @see ObjectMetric
+ * @see JmxExporterSpi
+ */
+public interface MetricExporterSpi extends IgniteSpi {
+    /**
+     * Sets metrics registry that SPI should export.
+     * This method called before {@link #spiStart(String)}.
+     *
+     * Registry should be empty in the time this method called.
+     * So all {@link MetricRegistry} that will be created by Ignite internal components can be obtained by
+     * listeners passed to {@link MetricRegistry#addMetricCreationListener(Consumer)}.
+     *
+     * @param registry Metric registry.
+     */
+    public void setMetricRegistry(ReadOnlyMetricRegistry registry);
+
+    /**
+     * Sets export filter.
+     * Metrics that not satisfy {@code filter} shouldn't be exported by this filter.
+     *
+     * @param filter Filter.
+     */
+    public void setExportFilter(Predicate<Metric> filter);
+}
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/stat/IoStatisticsCachePersistenceSelfTest.java b/modules/core/src/main/java/org/apache/ignite/spi/metric/ObjectMetric.java
similarity index 63%
copy from modules/core/src/test/java/org/apache/ignite/internal/stat/IoStatisticsCachePersistenceSelfTest.java
copy to modules/core/src/main/java/org/apache/ignite/spi/metric/ObjectMetric.java
index c203295..f931fef 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/stat/IoStatisticsCachePersistenceSelfTest.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/metric/ObjectMetric.java
@@ -13,18 +13,29 @@
  * 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.stat;
+package org.apache.ignite.spi.metric;
+
+import org.jetbrains.annotations.Nullable;
 
 /**
- * Tests for cache IO statistics for persistence mode.
+ * Interface for the metrics that holds typed value.
  */
-public class IoStatisticsCachePersistenceSelfTest extends IoStatisticsCacheSelfTest {
+public interface ObjectMetric<T> extends Metric {
+    /** @return Value ot the metric. */
+    public T value();
+
+    /** @return Type of metric value. */
+    public Class<T> type();
 
     /** {@inheritDoc} */
-    @Override protected boolean persist() {
-        return true;
+    @Override @Nullable default String getAsString() {
+        T val = value();
+
+        if (val == null)
+            return null;
+
+        return val.toString();
     }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/metric/ReadOnlyMetricRegistry.java b/modules/core/src/main/java/org/apache/ignite/spi/metric/ReadOnlyMetricRegistry.java
new file mode 100644
index 0000000..bd3ec0e
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/metric/ReadOnlyMetricRegistry.java
@@ -0,0 +1,59 @@
+/*
+ * 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.spi.metric;
+
+import java.util.Collection;
+import java.util.function.Consumer;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Read only metric registry.
+ */
+public interface ReadOnlyMetricRegistry {
+    /**
+     * @param prefix prefix for all metrics.
+     * @return Proxy implementation that will search and create only metrics with specified prefix.
+     */
+    public <T extends ReadOnlyMetricRegistry> T withPrefix(String prefix);
+
+    /**
+     * Prefixes combined using dot notation {@code ["io", "stat"] -> "io.stat"}
+     *
+     * @param prefixes prefixes for all metrics.
+     * @return Proxy implementation that will search and create only metrics with specified prefixes.
+     */
+    public <T extends ReadOnlyMetricRegistry> T withPrefix(String... prefixes);
+
+    /**
+     * @return Metrics stored in this group.
+     */
+    public Collection<Metric> getMetrics();
+
+    /**
+     * Adds listener of metrics sets creation events.
+     *
+     * @param lsnr Listener.
+     */
+    public void addMetricCreationListener(Consumer<Metric> lsnr);
+
+    /**
+     * @param name Name of the metric
+     * @return Metric with specified name if exists. Null otherwise.
+     */
+    @Nullable public Metric findMetric(String name);
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/metric/jmx/JmxExporterSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/metric/jmx/JmxExporterSpi.java
new file mode 100644
index 0000000..2e075c5
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/metric/jmx/JmxExporterSpi.java
@@ -0,0 +1,125 @@
+/*
+ * 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.spi.metric.jmx;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.function.Predicate;
+import javax.management.JMException;
+import javax.management.MBeanServer;
+import javax.management.ObjectName;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.internal.processors.metric.impl.MetricUtils.MetricName;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.spi.IgniteSpiAdapter;
+import org.apache.ignite.spi.IgniteSpiException;
+import org.apache.ignite.spi.metric.Metric;
+import org.apache.ignite.spi.metric.MetricExporterSpi;
+import org.apache.ignite.spi.metric.ReadOnlyMetricRegistry;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.internal.processors.metric.impl.MetricUtils.parse;
+
+/**
+ * This SPI implementation exports metrics as JMX beans.
+ */
+public class JmxExporterSpi extends IgniteSpiAdapter implements MetricExporterSpi {
+    /** Monitoring registry. */
+    private ReadOnlyMetricRegistry mreg;
+
+    /** Set of already registered as MBean prefixes. */
+    private Set<String> metricSets = new HashSet<>();
+
+    /** Metric filter. */
+    private @Nullable Predicate<Metric> filter;
+
+    /** Registered beans. */
+    private final List<ObjectName> mBeans = new ArrayList<>();
+
+    /** {@inheritDoc} */
+    @Override public void spiStart(@Nullable String igniteInstanceName) throws IgniteSpiException {
+        mreg.addMetricCreationListener(m -> {
+            if (filter != null && !filter.test(m))
+                return;
+
+            MetricName n = parse(m.name());
+
+            if (metricSets.contains(n.msetName()))
+                return;
+
+            if (log.isDebugEnabled())
+                log.debug("Found new metric set [name=" + n.msetName() + ']');
+
+            metricSets.add(n.msetName());
+
+            try {
+                MetricSetMBean msetBean = new MetricSetMBean(n.msetName(), mreg, m);
+
+                ObjectName mbean = U.registerMBean(
+                    ignite().configuration().getMBeanServer(),
+                    igniteInstanceName,
+                    n.root(),
+                    n.subName(),
+                    msetBean,
+                    MetricSetMBean.class);
+
+                mBeans.add(mbean);
+
+                if (log.isDebugEnabled())
+                    log.debug("MetricSet JMX bean created. " + mbean);
+            }
+            catch (JMException e) {
+                log.error("MBean for " + n.msetName() + " can't be created.", e);
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public void spiStop() throws IgniteSpiException {
+        Ignite ignite = ignite();
+
+        if (ignite == null)
+            return;
+
+        MBeanServer jmx = ignite.configuration().getMBeanServer();
+
+        for (ObjectName bean : mBeans) {
+            try {
+                jmx.unregisterMBean(bean);
+
+                if (log.isDebugEnabled())
+                    log.debug("Unregistered SPI MBean: " + bean);
+            }
+            catch (JMException e) {
+                log.error("Failed to unregister SPI MBean: " + bean, e);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void setMetricRegistry(ReadOnlyMetricRegistry reg) {
+        this.mreg = reg;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void setExportFilter(Predicate<Metric> filter) {
+        this.filter = filter;
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/metric/jmx/MetricSetMBean.java b/modules/core/src/main/java/org/apache/ignite/spi/metric/jmx/MetricSetMBean.java
new file mode 100644
index 0000000..c16fe0f
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/metric/jmx/MetricSetMBean.java
@@ -0,0 +1,172 @@
+/*
+ * 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.spi.metric.jmx;
+
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import javax.management.Attribute;
+import javax.management.AttributeList;
+import javax.management.DynamicMBean;
+import javax.management.MBeanAttributeInfo;
+import javax.management.MBeanInfo;
+import org.apache.ignite.internal.processors.metric.impl.MetricUtils.MetricName;
+import org.apache.ignite.spi.metric.BooleanMetric;
+import org.apache.ignite.spi.metric.DoubleMetric;
+import org.apache.ignite.spi.metric.IntMetric;
+import org.apache.ignite.spi.metric.LongMetric;
+import org.apache.ignite.spi.metric.Metric;
+import org.apache.ignite.spi.metric.ObjectMetric;
+import org.apache.ignite.spi.metric.ReadOnlyMetricRegistry;
+
+import static org.apache.ignite.internal.processors.metric.impl.MetricUtils.parse;
+
+/**
+ * MBean for exporting values of metric set.
+ */
+public class MetricSetMBean implements DynamicMBean {
+    /** Metric set name. */
+    private final String msetName;
+
+    /** Metric set. */
+    private final Map<String, Metric> mset = new HashMap<>();
+
+    /**
+     * @param msetName Metrics set name.
+     * @param mreg Metrics registry.
+     * @param first First set entry.
+     */
+    public MetricSetMBean(String msetName, ReadOnlyMetricRegistry mreg, Metric first) {
+        this.msetName = msetName;
+
+        mreg.addMetricCreationListener(m -> {
+            if (m.name().startsWith(msetName)) {
+                MetricName parsed = parse(m.name());
+
+                if (!parsed.msetName().equals(msetName))
+                    return;
+
+                mset.put(parsed.mname(), m);
+            }
+        });
+
+        mset.put(parse(first.name()).mname(), first);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object getAttribute(String attribute) {
+        if (attribute.equals("MBeanInfo"))
+            return getMBeanInfo();
+
+        Metric metric = mset.get(attribute);
+
+        if (metric == null)
+            return null;
+
+        if (metric instanceof BooleanMetric)
+            return ((BooleanMetric)metric).value();
+        else if (metric instanceof DoubleMetric)
+            return ((DoubleMetric)metric).value();
+        else if (metric instanceof IntMetric)
+            return ((IntMetric)metric).value();
+        else if (metric instanceof LongMetric)
+            return ((LongMetric)metric).value();
+        else if (metric instanceof ObjectMetric)
+            return ((ObjectMetric)metric).value();
+
+        throw new IllegalArgumentException("Unknown metric class. " + metric.getClass());
+    }
+
+    /** {@inheritDoc} */
+    @Override public MBeanInfo getMBeanInfo() {
+        Iterator<Metric> iter = mset.values().iterator();
+
+        MBeanAttributeInfo[] attributes = new MBeanAttributeInfo[mset.size()];
+
+        int sz = attributes.length;
+        for (int i = 0; i < sz; i++) {
+            Metric metric = iter.next();
+
+            attributes[i] = new MBeanAttributeInfo(
+                metric.name().substring(msetName.length() + 1),
+                metricClass(metric),
+                metric.name(),
+                true,
+                false,
+                false);
+        }
+
+        return new MBeanInfo(
+            ReadOnlyMetricRegistry.class.getName(),
+            msetName,
+            attributes,
+            null,
+            null,
+            null);
+    }
+
+    /**
+     * @param metric Metric.
+     * @return Class of metric value.
+     */
+    private String metricClass(Metric metric) {
+        if (metric instanceof BooleanMetric)
+            return Boolean.class.getName();
+        else if (metric instanceof DoubleMetric)
+            return Double.class.getName();
+        else if (metric instanceof IntMetric)
+            return Integer.class.getName();
+        else if (metric instanceof LongMetric)
+            return Long.class.getName();
+        else if (metric instanceof ObjectMetric)
+            return ((ObjectMetric)metric).type().getName();
+
+        throw new IllegalArgumentException("Unknown metric class. " + metric.getClass());
+    }
+
+    /** {@inheritDoc} */
+    @Override public AttributeList getAttributes(String[] attributes) {
+        AttributeList list = new AttributeList();
+
+        for (String attribute : attributes) {
+            Object val = getAttribute(attribute);
+
+            list.add(val);
+        }
+
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void setAttribute(Attribute attribute) {
+        throw new UnsupportedOperationException("setAttribute not supported.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public AttributeList setAttributes(AttributeList attributes) {
+        throw new UnsupportedOperationException("setAttributes not supported.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object invoke(String actionName, Object[] params, String[] signature) {
+        if (actionName.equals("getAttribute"))
+            return getAttribute((String)params[0]);
+
+        throw new UnsupportedOperationException("invoke not supported.");
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/stat/IoStatisticsType.java b/modules/core/src/main/java/org/apache/ignite/spi/metric/jmx/package-info.java
similarity index 76%
copy from modules/core/src/main/java/org/apache/ignite/internal/stat/IoStatisticsType.java
copy to modules/core/src/main/java/org/apache/ignite/spi/metric/jmx/package-info.java
index 67577a4..8672c2f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/stat/IoStatisticsType.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/metric/jmx/package-info.java
@@ -13,24 +13,10 @@
  * 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.stat;
-
 /**
- * Type of statistics.
+ * <!-- Package description. -->
+ * Contains JMX based implementation of MetricExporterSpi.
  */
-public enum IoStatisticsType {
-    /** Cache group. */
-    CACHE_GROUP,
-
-    /** Hash index. */
-    HASH_INDEX,
-
-    /** Sorted index. */
-    SORTED_INDEX,
-
-    /** SQL. */
-    SQL
-}
+package org.apache.ignite.spi.metric.jmx;
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/stat/IoStatisticsCachePersistenceSelfTest.java b/modules/core/src/main/java/org/apache/ignite/spi/metric/log/LogExporterSpi.java
similarity index 51%
copy from modules/core/src/test/java/org/apache/ignite/internal/stat/IoStatisticsCachePersistenceSelfTest.java
copy to modules/core/src/main/java/org/apache/ignite/spi/metric/log/LogExporterSpi.java
index c203295..2fe11b9 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/stat/IoStatisticsCachePersistenceSelfTest.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/metric/log/LogExporterSpi.java
@@ -13,18 +13,33 @@
  * 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.stat;
+package org.apache.ignite.spi.metric.log;
+
+import org.apache.ignite.internal.processors.metric.PushMetricsExporterAdapter;
+import org.apache.ignite.internal.util.typedef.internal.LT;
 
 /**
- * Tests for cache IO statistics for persistence mode.
+ * This SPI implementation exports metrics to Ignite log.
  */
-public class IoStatisticsCachePersistenceSelfTest extends IoStatisticsCacheSelfTest {
-
+public class LogExporterSpi extends PushMetricsExporterAdapter {
     /** {@inheritDoc} */
-    @Override protected boolean persist() {
-        return true;
+    @Override public void export() {
+        if (!log.isInfoEnabled()) {
+            LT.warn(log, "LogExporterSpi configured but INFO level is disabled. " +
+                "Metrics will not be export to the log! Please, check logger settings.");
+
+            return;
+        }
+
+        log.info("Metrics:");
+
+        mreg.getMetrics().forEach(m -> {
+            if (filter != null && !filter.test(m))
+                return;
+
+            log.info(m.name() + " = " + m.getAsString());
+        });
     }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/stat/IoStatisticsType.java b/modules/core/src/main/java/org/apache/ignite/spi/metric/log/package-info.java
similarity index 76%
copy from modules/core/src/main/java/org/apache/ignite/internal/stat/IoStatisticsType.java
copy to modules/core/src/main/java/org/apache/ignite/spi/metric/log/package-info.java
index 67577a4..c5c1d28 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/stat/IoStatisticsType.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/metric/log/package-info.java
@@ -13,24 +13,10 @@
  * 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.stat;
-
 /**
- * Type of statistics.
+ * <!-- Package description. -->
+ * Contains push to log implementation of MetricExporterSpi.
  */
-public enum IoStatisticsType {
-    /** Cache group. */
-    CACHE_GROUP,
-
-    /** Hash index. */
-    HASH_INDEX,
-
-    /** Sorted index. */
-    SORTED_INDEX,
-
-    /** SQL. */
-    SQL
-}
+package org.apache.ignite.spi.metric.log;
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/metric/noop/NoopMetricExporterSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/metric/noop/NoopMetricExporterSpi.java
new file mode 100644
index 0000000..4d1c6eb
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/metric/noop/NoopMetricExporterSpi.java
@@ -0,0 +1,53 @@
+/*
+ * 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.spi.metric.noop;
+
+import java.util.function.Predicate;
+import org.apache.ignite.spi.IgniteSpiAdapter;
+import org.apache.ignite.spi.IgniteSpiException;
+import org.apache.ignite.spi.IgniteSpiNoop;
+import org.apache.ignite.spi.metric.Metric;
+import org.apache.ignite.spi.metric.MetricExporterSpi;
+import org.apache.ignite.spi.metric.ReadOnlyMetricRegistry;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * No-op implementation of metric exporter SPI.
+ */
+@IgniteSpiNoop
+public class NoopMetricExporterSpi extends IgniteSpiAdapter implements MetricExporterSpi {
+    /** {@inheritDoc} */
+    @Override public void spiStart(@Nullable String igniteInstanceName) throws IgniteSpiException {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void spiStop() throws IgniteSpiException {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void setMetricRegistry(ReadOnlyMetricRegistry registry) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void setExportFilter(Predicate<Metric> filter) {
+        // No-op.
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/stat/IoStatisticsType.java b/modules/core/src/main/java/org/apache/ignite/spi/metric/noop/package-info.java
similarity index 76%
copy from modules/core/src/main/java/org/apache/ignite/internal/stat/IoStatisticsType.java
copy to modules/core/src/main/java/org/apache/ignite/spi/metric/noop/package-info.java
index 67577a4..740610f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/stat/IoStatisticsType.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/metric/noop/package-info.java
@@ -13,24 +13,10 @@
  * 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.stat;
-
 /**
- * Type of statistics.
+ * <!-- Package description. -->
+ * Contains no-op implementation of MetricExporterSpi.
  */
-public enum IoStatisticsType {
-    /** Cache group. */
-    CACHE_GROUP,
-
-    /** Hash index. */
-    HASH_INDEX,
-
-    /** Sorted index. */
-    SORTED_INDEX,
-
-    /** SQL. */
-    SQL
-}
+package org.apache.ignite.spi.metric.noop;
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/stat/IoStatisticsType.java b/modules/core/src/main/java/org/apache/ignite/spi/metric/package-info.java
similarity index 76%
rename from modules/core/src/main/java/org/apache/ignite/internal/stat/IoStatisticsType.java
rename to modules/core/src/main/java/org/apache/ignite/spi/metric/package-info.java
index 67577a4..d3794c4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/stat/IoStatisticsType.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/metric/package-info.java
@@ -13,24 +13,10 @@
  * 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.stat;
-
 /**
- * Type of statistics.
+ * <!-- Package description. -->
+ * Contains common classes and interfaces for metric SPI implementations.
  */
-public enum IoStatisticsType {
-    /** Cache group. */
-    CACHE_GROUP,
-
-    /** Hash index. */
-    HASH_INDEX,
-
-    /** Sorted index. */
-    SORTED_INDEX,
-
-    /** SQL. */
-    SQL
-}
+package org.apache.ignite.spi.metric;
diff --git a/modules/core/src/main/resources/META-INF/classnames.properties b/modules/core/src/main/resources/META-INF/classnames.properties
index f806bd0..0f14356 100644
--- a/modules/core/src/main/resources/META-INF/classnames.properties
+++ b/modules/core/src/main/resources/META-INF/classnames.properties
@@ -1175,7 +1175,7 @@ org.apache.ignite.internal.processors.cache.query.jdbc.GridCacheQueryJdbcTask$Jd
 org.apache.ignite.internal.processors.cache.query.jdbc.GridCacheQueryJdbcTask$JdbcDriverJob$1
 org.apache.ignite.internal.processors.cache.query.jdbc.GridCacheQueryJdbcValidationTask
 org.apache.ignite.internal.processors.cache.query.jdbc.GridCacheQueryJdbcValidationTask$1
-org.apache.ignite.internal.processors.cache.ratemetrics.HitRateMetricsSandbox
+org.apache.ignite.spi.monitoring.metric.counter.ratemetrics.HitRateMetricsSandbox
 org.apache.ignite.internal.processors.cache.store.GridCacheStoreManagerAdapter$1
 org.apache.ignite.internal.processors.cache.store.GridCacheStoreManagerAdapter$2
 org.apache.ignite.internal.processors.cache.store.GridCacheStoreManagerAdapter$3
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/metric/AbstractExporterSpiTest.java b/modules/core/src/test/java/org/apache/ignite/internal/metric/AbstractExporterSpiTest.java
new file mode 100644
index 0000000..e31a49b
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/metric/AbstractExporterSpiTest.java
@@ -0,0 +1,79 @@
+/*
+ * 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.metric;
+
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Set;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.processors.metric.MetricRegistry;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+import static org.apache.ignite.internal.processors.metric.impl.MetricUtils.metricName;
+
+/** */
+public abstract class AbstractExporterSpiTest extends GridCommonAbstractTest {
+    /** */
+    protected static final String FILTERED_PREFIX = "filtered.metric";
+
+    /** */
+    public static final int EXPORT_TIMEOUT = 1_000;
+
+    /** */
+    protected Set<String> EXPECTED_ATTRIBUTES = new HashSet<>(Arrays.asList(
+        "TotalAllocatedSize",
+        "LargeEntriesPagesCount",
+        "PagesReplaced",
+        "PhysicalMemorySize",
+        "CheckpointBufferSize",
+        "PagesReplaceRate",
+        "AllocationRate",
+        "PagesRead",
+        "OffHeapSize",
+        "UsedCheckpointBufferSize",
+        "OffheapUsedSize",
+        "EmptyDataPages",
+        "PagesFillFactor",
+        "DirtyPages",
+        "EvictionRate",
+        "PagesWritten",
+        "TotalAllocatedPages",
+        "PagesReplaceAge",
+        "PhysicalMemoryPages"));
+
+    /**
+     * Creates some additional metrics.
+     *
+     * @param ignite Ignite.
+     */
+    protected void createAdditionalMetrics(IgniteEx ignite) {
+        MetricRegistry mreg = ignite.context().metric().registry();
+
+        mreg.metric(metricName(FILTERED_PREFIX, "test"), "")
+            .add(2);
+
+        mreg.metric(metricName("other.prefix", "test"), "")
+            .add(42);
+
+        mreg.metric(metricName("other.prefix", "test2"), "")
+            .add(43);
+
+        mreg.metric(metricName("other.prefix2", "test3"), "")
+            .add(44);
+    }
+}
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/metric/CacheMetricsAddRemoveTest.java b/modules/core/src/test/java/org/apache/ignite/internal/metric/CacheMetricsAddRemoveTest.java
new file mode 100644
index 0000000..38b0d44
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/metric/CacheMetricsAddRemoveTest.java
@@ -0,0 +1,149 @@
+/*
+ * 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.metric;
+
+import java.util.Arrays;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.NearCacheConfiguration;
+import org.apache.ignite.internal.processors.metric.MetricRegistry;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static org.apache.ignite.internal.processors.metric.impl.MetricUtils.metricName;
+
+/** */
+@RunWith(Parameterized.class)
+public class CacheMetricsAddRemoveTest extends GridCommonAbstractTest {
+    /** */
+    public static final String CACHE_GETS = "CacheGets";
+
+    /** */
+    public static final String CACHE_PUTS = "CachePuts";
+
+    /** Cache modes. */
+    @Parameterized.Parameters(name = "cacheMode={0},nearEnabled={1}")
+    public static Iterable<Object[]> params() {
+        return Arrays.asList(
+            new Object[] {CacheMode.PARTITIONED, false},
+            new Object[] {CacheMode.PARTITIONED, true},
+            new Object[] {CacheMode.REPLICATED, false},
+            new Object[] {CacheMode.REPLICATED, true}
+        );
+    }
+
+    /** . */
+    @Parameterized.Parameter(0)
+    public CacheMode mode;
+
+    /** Use index. */
+    @Parameterized.Parameter(1)
+    public boolean nearEnabled;
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        startGridsMultiThreaded(2);
+
+        IgniteConfiguration clientCfg = getConfiguration("client")
+            .setClientMode(true);
+
+        startGrid(clientCfg);
+    }
+
+    /** */
+    @Test
+    public void testCacheMetricsAddRemove() throws Exception {
+        String cachePrefix = metricName("cache", DEFAULT_CACHE_NAME);
+
+        checkMetricsEmpty(cachePrefix);
+
+        createCache();
+
+        checkMetricsNotEmpty(cachePrefix);
+
+        destroyCache();
+
+        checkMetricsEmpty(cachePrefix);
+    }
+
+    /** */
+    private void destroyCache() throws InterruptedException {
+        grid("client").destroyCache(DEFAULT_CACHE_NAME);
+
+        awaitPartitionMapExchange();
+    }
+
+    /** */
+    private void createCache() throws InterruptedException {
+        CacheConfiguration ccfg = new CacheConfiguration(DEFAULT_CACHE_NAME);
+
+        if (nearEnabled)
+            ccfg.setNearConfiguration(new NearCacheConfiguration());
+
+        grid("client").createCache(ccfg);
+
+        awaitPartitionMapExchange();
+    }
+
+    /** */
+    private void checkMetricsNotEmpty(String cachePrefix) {
+        for (int i=0; i<2; i++) {
+            MetricRegistry mreg = metricRegistry(i);
+
+            assertNotNull(mreg.findMetric(metricName(cachePrefix, CACHE_GETS)));
+            assertNotNull(mreg.findMetric(metricName(cachePrefix, CACHE_PUTS)));
+
+            if (nearEnabled) {
+                String nearCachePreifx = metricName("cache", DEFAULT_CACHE_NAME, "near");
+
+                assertNotNull(mreg.findMetric(metricName(nearCachePreifx, CACHE_GETS)));
+                assertNotNull(mreg.findMetric(metricName(nearCachePreifx, CACHE_PUTS)));
+            }
+        }
+    }
+
+    /** */
+    private void checkMetricsEmpty(String cachePrefix) {
+        for (int i=0; i<3; i++) {
+            MetricRegistry mreg = metricRegistry(i);
+
+            assertNull(mreg.findMetric(metricName(cachePrefix, CACHE_GETS)));
+            assertNull(mreg.findMetric(metricName(cachePrefix, CACHE_PUTS)));
+
+            if (nearEnabled) {
+                String nearCachePreifx = metricName("cache", DEFAULT_CACHE_NAME, "near");
+
+                assertNull(mreg.findMetric(metricName(nearCachePreifx, CACHE_GETS)));
+                assertNull(mreg.findMetric(metricName(nearCachePreifx, CACHE_PUTS)));
+            }
+        }
+    }
+
+    /** */
+    private MetricRegistry metricRegistry(int gridIdx) {
+        MetricRegistry mreg;
+        if (gridIdx < 2)
+            mreg = grid(0).context().metric().registry();
+        else
+            mreg = grid("client").context().metric().registry();
+        return mreg;
+    }
+}
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/stat/IoStatisticsCachePersistenceSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/metric/IoStatisticsCachePersistenceSelfTest.java
similarity index 96%
copy from modules/core/src/test/java/org/apache/ignite/internal/stat/IoStatisticsCachePersistenceSelfTest.java
copy to modules/core/src/test/java/org/apache/ignite/internal/metric/IoStatisticsCachePersistenceSelfTest.java
index c203295..66c9bed 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/stat/IoStatisticsCachePersistenceSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/metric/IoStatisticsCachePersistenceSelfTest.java
@@ -16,7 +16,7 @@
  *
  */
 
-package org.apache.ignite.internal.stat;
+package org.apache.ignite.internal.metric;
 
 /**
  * Tests for cache IO statistics for persistence mode.
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/stat/IoStatisticsCacheSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/metric/IoStatisticsCacheSelfTest.java
similarity index 63%
rename from modules/core/src/test/java/org/apache/ignite/internal/stat/IoStatisticsCacheSelfTest.java
rename to modules/core/src/test/java/org/apache/ignite/internal/metric/IoStatisticsCacheSelfTest.java
index fad1524..c098cfd 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/stat/IoStatisticsCacheSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/metric/IoStatisticsCacheSelfTest.java
@@ -16,10 +16,11 @@
  *
  */
 
-package org.apache.ignite.internal.stat;
+package org.apache.ignite.internal.metric;
 
 import com.google.common.collect.Sets;
 import java.util.Set;
+import java.util.stream.Collectors;
 import java.util.stream.Stream;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.cache.CacheAtomicityMode;
@@ -29,11 +30,20 @@ import org.apache.ignite.configuration.DataStorageConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.configuration.WALMode;
 import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.spi.metric.LongMetric;
+import org.apache.ignite.spi.metric.Metric;
+import org.apache.ignite.internal.processors.metric.MetricRegistry;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-import org.junit.Assert;
 import org.junit.Test;
 
-import static org.apache.ignite.internal.stat.IoStatisticsHolderIndex.HASH_PK_IDX_NAME;
+import static org.apache.ignite.internal.metric.IoStatisticsHolderCache.LOGICAL_READS;
+import static org.apache.ignite.internal.metric.IoStatisticsHolderIndex.HASH_PK_IDX_NAME;
+import static org.apache.ignite.internal.metric.IoStatisticsHolderIndex.LOGICAL_READS_INNER;
+import static org.apache.ignite.internal.metric.IoStatisticsHolderIndex.LOGICAL_READS_LEAF;
+import static org.apache.ignite.internal.metric.IoStatisticsMetricsLocalMXBeanImplSelfTest.resetAllIoMetrics;
+import static org.apache.ignite.internal.metric.IoStatisticsType.CACHE_GROUP;
+import static org.apache.ignite.internal.metric.IoStatisticsType.HASH_INDEX;
+import static org.apache.ignite.internal.processors.metric.impl.MetricUtils.metricName;
 
 /**
  * Tests for cache IO statistics for inmemory mode.
@@ -143,14 +153,14 @@ public class IoStatisticsCacheSelfTest extends GridCommonAbstractTest {
     @Override protected void beforeTest() throws Exception {
         super.beforeTest();
 
-        ignite.context().ioStats().reset();
+        resetAllIoMetrics(ignite);
     }
 
     /**
      * Test statistics for TRANSACTIONAL cache.
      */
     @Test
-    public void testTransactonalCache() {
+    public void testTransactonalCache() throws Exception {
         cacheTest(TRANSACTIONAL_CACHE_NAME, RECORD_COUNT, RECORD_COUNT * 3, RECORD_COUNT * 2);
     }
 
@@ -158,7 +168,7 @@ public class IoStatisticsCacheSelfTest extends GridCommonAbstractTest {
      * Test statistics for MVCC cache.
      */
     @Test
-    public void testMvccCache() {
+    public void testMvccCache() throws Exception {
         cacheTest(MVCC_CACHE_NAME, RECORD_COUNT, RECORD_COUNT * 6, RECORD_COUNT * 3);
     }
 
@@ -166,7 +176,7 @@ public class IoStatisticsCacheSelfTest extends GridCommonAbstractTest {
      * Test statistics for ATOMIC cache.
      */
     @Test
-    public void testAtomicCache() {
+    public void testAtomicCache() throws Exception {
         cacheTest(ATOMIC_CACHE_NAME, RECORD_COUNT, RECORD_COUNT * 2, RECORD_COUNT);
     }
 
@@ -174,19 +184,19 @@ public class IoStatisticsCacheSelfTest extends GridCommonAbstractTest {
      * Test statistics for three caches in the same time.
      */
     @Test
-    public void testForThreeCaches() {
+    public void testForThreeCaches() throws Exception {
         prepareData(RECORD_COUNT, ATOMIC_CACHE_NAME, TRANSACTIONAL_CACHE_NAME, MVCC_CACHE_NAME);
 
-        IoStatisticsManager ioStatMgr = ignite.context().ioStats();
+        MetricRegistry mreg = ignite.context().metric().registry();
 
-        Set<String> statisticCacheNames = ioStatMgr.deriveStatisticNames(IoStatisticsType.CACHE_GROUP);
+        Set<String> statisticCacheNames = deriveStatisticNames(CACHE_GROUP);
 
-        Assert.assertEquals(ALL_CACHE_GROUP_NAMES, statisticCacheNames);
+        assertEquals(ALL_CACHE_GROUP_NAMES, statisticCacheNames);
 
         Stream.of(ATOMIC_CACHE_NAME, TRANSACTIONAL_CACHE_NAME, MVCC_CACHE_NAME).forEach((cacheName) -> {
-            long logicalReads = ioStatMgr.logicalReads(IoStatisticsType.CACHE_GROUP, cacheName, null);
+            long logicalReads = logicalReads(mreg, CACHE_GROUP, cacheName);
 
-            Assert.assertTrue(logicalReads > RECORD_COUNT);
+            assertTrue(logicalReads > RECORD_COUNT);
         });
     }
 
@@ -194,18 +204,18 @@ public class IoStatisticsCacheSelfTest extends GridCommonAbstractTest {
      * Test statistics for two caches in the same cache group.
      */
     @Test
-    public void testCacheGroupCaches() {
+    public void testCacheGroupCaches() throws Exception {
         prepareData(RECORD_COUNT, CACHE1_IN_GROUP_NAME, CACHE2_IN_GROUP_NAME);
 
-        IoStatisticsManager ioStatMgr = ignite.context().ioStats();
+        MetricRegistry mreg = ignite.context().metric().registry();
 
-        Set<String> statisticCacheNames = ioStatMgr.deriveStatisticNames(IoStatisticsType.CACHE_GROUP);
+        Set<String> statisticCacheNames = deriveStatisticNames(CACHE_GROUP);
 
-        Assert.assertEquals(ALL_CACHE_GROUP_NAMES, statisticCacheNames);
+        assertEquals(ALL_CACHE_GROUP_NAMES, statisticCacheNames);
 
-        long logicalReads = ioStatMgr.logicalReads(IoStatisticsType.CACHE_GROUP, CACHE_GROUP_NAME, null);
+        long logicalReads = logicalReads(mreg, CACHE_GROUP, CACHE_GROUP_NAME);
 
-        Assert.assertEquals(RECORD_COUNT * 6, logicalReads);
+        assertEquals(RECORD_COUNT * 6, logicalReads);
     }
 
     /**
@@ -214,34 +224,52 @@ public class IoStatisticsCacheSelfTest extends GridCommonAbstractTest {
      * @param dataPageReads How many data page reads operation expected.
      * @param idxPageReadsCnt How many index page reads scan expected.
      */
-    protected void cacheTest(String cacheName, int rowCnt, int dataPageReads, int idxPageReadsCnt) {
+    protected void cacheTest(String cacheName, int rowCnt, int dataPageReads, int idxPageReadsCnt) throws Exception {
         prepareData(rowCnt, cacheName);
 
-        IoStatisticsManager ioStatMgr = ignite.context().ioStats();
+        MetricRegistry mreg = ignite.context().metric().registry();
 
-        Set<String> statisticCacheNames = ioStatMgr.deriveStatisticNames(IoStatisticsType.CACHE_GROUP);
+        Set<String> statisticCacheNames = deriveStatisticNames(CACHE_GROUP);
 
-        Assert.assertEquals(ALL_CACHE_GROUP_NAMES, statisticCacheNames);
+        assertEquals(ALL_CACHE_GROUP_NAMES, statisticCacheNames);
 
-        Assert.assertTrue(statisticCacheNames.contains(cacheName));
+        assertTrue(statisticCacheNames.contains(cacheName));
 
-        long logicalReadsCache = ioStatMgr.logicalReads(IoStatisticsType.CACHE_GROUP, cacheName, null);
+        long logicalReadsCache = logicalReads(mreg, CACHE_GROUP, cacheName);
 
-        Assert.assertEquals(dataPageReads, logicalReadsCache);
+        assertEquals(dataPageReads, logicalReadsCache);
 
-        long logicalReadsIdx = ioStatMgr.logicalReads(IoStatisticsType.HASH_INDEX, cacheName, HASH_PK_IDX_NAME);
+        long logicalReadsIdx = logicalReads(mreg, HASH_INDEX, metricName(cacheName, HASH_PK_IDX_NAME));
 
-        Assert.assertEquals(idxPageReadsCnt, logicalReadsIdx);
+        assertEquals(idxPageReadsCnt, logicalReadsIdx);
 
     }
 
     /**
+     * Extract all tracked names for given statistics type.
+     *
+     * @param statType Type of statistics which tracked names need to extract.
+     * @return Set of present names for given statType
+     */
+    public Set<String> deriveStatisticNames(IoStatisticsType statType) {
+        assert statType != null;
+
+        MetricRegistry msets =
+            ignite.context().metric().registry().withPrefix(statType.metricGroupName());
+
+        return msets.getMetrics().stream()
+            .filter(m -> m.name().endsWith("name"))
+            .map(Metric::getAsString)
+            .collect(Collectors.toSet());
+    }
+
+    /**
      * Warm up and fill cache.
      *
      * @param cacheNames Names of caches to populate.
      * @param cnt Number of entries to put.
      */
-    private void prepareData(int cnt, String... cacheNames) {
+    private void prepareData(int cnt, String... cacheNames) throws Exception {
         //Need to initialize partition and data memory pages
         for (String cacheName : cacheNames) {
 
@@ -253,7 +281,7 @@ public class IoStatisticsCacheSelfTest extends GridCommonAbstractTest {
             }
         }
 
-        ignite.context().ioStats().reset();
+        resetAllIoMetrics(ignite);
 
         for (String cacheName : cacheNames) {
 
@@ -264,4 +292,22 @@ public class IoStatisticsCacheSelfTest extends GridCommonAbstractTest {
         }
     }
 
+    /**
+     * @param mreg Monitoring registry.
+     * @param type Staticstics type.
+     * @param id Metric set id.
+     * @return Logical reads count.
+     */
+    public static long logicalReads(MetricRegistry mreg, IoStatisticsType type, String id) {
+        MetricRegistry mset = mreg.withPrefix(type.metricGroupName(), id);
+
+        if (type == CACHE_GROUP)
+            return ((LongMetric)mset.findMetric(LOGICAL_READS)).value();
+        else {
+            long leaf = ((LongMetric)mset.findMetric(LOGICAL_READS_LEAF)).value();
+            long inner = ((LongMetric)mset.findMetric(LOGICAL_READS_INNER)).value();
+
+            return leaf + inner;
+        }
+    }
 }
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/metric/IoStatisticsMetricsLocalMXBeanImplSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/metric/IoStatisticsMetricsLocalMXBeanImplSelfTest.java
new file mode 100644
index 0000000..76974c1
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/metric/IoStatisticsMetricsLocalMXBeanImplSelfTest.java
@@ -0,0 +1,179 @@
+/*
+ * 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.metric;
+
+import java.lang.management.ManagementFactory;
+import javax.management.MBeanServer;
+import javax.management.MBeanServerInvocationHandler;
+import javax.management.MalformedObjectNameException;
+import javax.management.ObjectName;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.IgniteKernal;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.mxbean.IgniteMXBean;
+import org.apache.ignite.spi.metric.LongMetric;
+import org.apache.ignite.internal.processors.metric.MetricRegistry;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import static org.apache.ignite.internal.metric.IoStatisticsHolderIndex.HASH_PK_IDX_NAME;
+import static org.apache.ignite.internal.metric.IoStatisticsHolderIndex.LOGICAL_READS_INNER;
+import static org.apache.ignite.internal.metric.IoStatisticsHolderIndex.LOGICAL_READS_LEAF;
+import static org.apache.ignite.internal.metric.IoStatisticsHolderIndex.PHYSICAL_READS_INNER;
+import static org.apache.ignite.internal.metric.IoStatisticsHolderIndex.PHYSICAL_READS_LEAF;
+import static org.apache.ignite.internal.metric.IoStatisticsHolderQuery.LOGICAL_READS;
+import static org.apache.ignite.internal.metric.IoStatisticsHolderQuery.PHYSICAL_READS;
+import static org.apache.ignite.internal.metric.IoStatisticsType.CACHE_GROUP;
+import static org.apache.ignite.internal.metric.IoStatisticsType.HASH_INDEX;
+
+/**
+ * Test of local node IO statistics MX bean.
+ */
+public class IoStatisticsMetricsLocalMXBeanImplSelfTest extends GridCommonAbstractTest {
+    /** */
+    private static IgniteEx ignite;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String name) throws Exception {
+        final IgniteConfiguration cfg = super.getConfiguration(name);
+
+        final CacheConfiguration cCfg = new CacheConfiguration()
+            .setName(DEFAULT_CACHE_NAME);
+
+        cfg.setCacheConfiguration(cCfg);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        ignite = startGrid(0);
+    }
+
+    /**
+     * Simple test JMX bean for indexes IO stats.
+     *
+     * @throws Exception In case of failure.
+     */
+    @Test
+    public void testIndexBasic() throws Exception {
+        resetMetric(ignite, HASH_INDEX.metricGroupName());
+
+        int cnt = 100;
+
+        populateCache(cnt);
+
+        MetricRegistry mreg = ignite.context().metric().registry()
+            .withPrefix(HASH_INDEX.metricGroupName(), DEFAULT_CACHE_NAME, HASH_PK_IDX_NAME);
+
+        long idxLeafLogicalCnt = ((LongMetric)mreg.findMetric(LOGICAL_READS_LEAF)).value();
+
+        assertEquals(cnt, idxLeafLogicalCnt);
+
+        long idxLeafPhysicalCnt = ((LongMetric)mreg.findMetric(PHYSICAL_READS_LEAF)).value();
+
+        assertEquals(0, idxLeafPhysicalCnt);
+
+        long idxInnerLogicalCnt = ((LongMetric)mreg.findMetric(LOGICAL_READS_INNER)).value();
+
+        assertEquals(0, idxInnerLogicalCnt);
+
+        long idxInnerPhysicalCnt = ((LongMetric)mreg.findMetric(PHYSICAL_READS_INNER)).value();
+
+        assertEquals(0, idxInnerPhysicalCnt);
+    }
+
+    /**
+     * Simple test JMX bean for caches IO stats.
+     *
+     * @throws Exception In case of failure.
+     */
+    @Test
+    public void testCacheBasic() throws Exception {
+        int cnt = 100;
+
+        populateCache(cnt);
+
+        clearCache(cnt);
+
+        resetMetric(ignite, CACHE_GROUP.metricGroupName());
+
+        populateCache(cnt);
+
+        MetricRegistry mreg = ignite.context().metric().registry()
+            .withPrefix(CACHE_GROUP.metricGroupName(), DEFAULT_CACHE_NAME);
+
+        long cacheLogicalReadsCnt = ((LongMetric)mreg.findMetric(LOGICAL_READS)).value();
+
+        assertEquals(cnt, cacheLogicalReadsCnt);
+
+        long cachePhysicalReadsCnt = ((LongMetric)mreg.findMetric(PHYSICAL_READS)).value();
+
+        assertEquals(0, cachePhysicalReadsCnt);
+    }
+
+    /**
+     * @param cnt Number of inserting elements.
+     */
+    private void populateCache(int cnt) {
+        for (int i = 0; i < cnt; i++)
+            ignite.cache(DEFAULT_CACHE_NAME).put(i, i);
+    }
+
+    /**
+     * @param cnt Number of removing elements.
+     */
+    private void clearCache(int cnt) {
+        for (int i = 0; i < cnt; i++)
+            ignite.cache(DEFAULT_CACHE_NAME).remove(i);
+    }
+
+    /**
+     * Resets all io statistics.
+     *
+     * @param ignite Ignite.
+     */
+    public static void resetAllIoMetrics(IgniteEx ignite) throws MalformedObjectNameException {
+        for (IoStatisticsType type : IoStatisticsType.values())
+            resetMetric(ignite, type.metricGroupName());
+    }
+
+    /**
+     * Resets all metrics for a given prefix.
+     *
+     * @param prefix Prefix to reset metrics.
+     */
+    public static void resetMetric(IgniteEx ignite, String prefix) throws MalformedObjectNameException {
+        ObjectName mbeanName = U.makeMBeanName(ignite.name(), "Kernal",
+            IgniteKernal.class.getSimpleName());
+
+        MBeanServer mbeanSrv = ManagementFactory.getPlatformMBeanServer();
+
+        if (!mbeanSrv.isRegistered(mbeanName))
+            fail("MBean is not registered: " + mbeanName.getCanonicalName());
+
+        IgniteMXBean bean = MBeanServerInvocationHandler.newProxyInstance(mbeanSrv, mbeanName, IgniteMXBean.class, false);
+
+        bean.resetMetrics(prefix);
+    }
+}
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/stat/IoStatisticsManagerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/metric/IoStatisticsSelfTest.java
similarity index 50%
rename from modules/core/src/test/java/org/apache/ignite/internal/stat/IoStatisticsManagerSelfTest.java
rename to modules/core/src/test/java/org/apache/ignite/internal/metric/IoStatisticsSelfTest.java
index 7ba8dc5..e6c1fb4 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/stat/IoStatisticsManagerSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/metric/IoStatisticsSelfTest.java
@@ -16,7 +16,7 @@
  *
  */
 
-package org.apache.ignite.internal.stat;
+package org.apache.ignite.internal.metric;
 
 import java.util.Map;
 import org.apache.ignite.IgniteCache;
@@ -26,22 +26,40 @@ import org.apache.ignite.configuration.DataStorageConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.configuration.WALMode;
 import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.spi.metric.LongMetric;
+import org.apache.ignite.internal.processors.metric.MetricRegistry;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.jetbrains.annotations.NotNull;
 import org.junit.Assert;
 import org.junit.Test;
 
-import static org.apache.ignite.internal.stat.IoStatisticsHolderIndex.HASH_PK_IDX_NAME;
+import static org.apache.ignite.internal.metric.IoStatisticsCacheSelfTest.logicalReads;
+import static org.apache.ignite.internal.metric.IoStatisticsHolderCache.LOGICAL_READS;
+import static org.apache.ignite.internal.metric.IoStatisticsHolderCache.PHYSICAL_READS;
+import static org.apache.ignite.internal.metric.IoStatisticsHolderIndex.HASH_PK_IDX_NAME;
+import static org.apache.ignite.internal.metric.IoStatisticsHolderIndex.LOGICAL_READS_INNER;
+import static org.apache.ignite.internal.metric.IoStatisticsHolderIndex.LOGICAL_READS_LEAF;
+import static org.apache.ignite.internal.metric.IoStatisticsHolderIndex.PHYSICAL_READS_INNER;
+import static org.apache.ignite.internal.metric.IoStatisticsHolderIndex.PHYSICAL_READS_LEAF;
+import static org.apache.ignite.internal.metric.IoStatisticsMetricsLocalMXBeanImplSelfTest.resetAllIoMetrics;
+import static org.apache.ignite.internal.metric.IoStatisticsType.CACHE_GROUP;
+import static org.apache.ignite.internal.metric.IoStatisticsType.HASH_INDEX;
+import static org.apache.ignite.internal.processors.metric.impl.MetricUtils.metricName;
 
 /**
  * Tests for IO statistic manager.
  */
-public class IoStatisticsManagerSelfTest extends GridCommonAbstractTest {
+public class IoStatisticsSelfTest extends GridCommonAbstractTest {
 
     /** */
     private static final int RECORD_COUNT = 5000;
 
     /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        cleanPersistenceDir();
+    }
+
+    /** {@inheritDoc} */
     @Override protected void afterTest() throws Exception {
         super.afterTest();
 
@@ -59,15 +77,17 @@ public class IoStatisticsManagerSelfTest extends GridCommonAbstractTest {
     public void testEmptyIOStat() throws Exception {
         IgniteEx ign = prepareIgnite(true);
 
-        IoStatisticsManager ioStatMgr = ign.context().ioStats();
+        MetricRegistry mreg = ign.context().metric().registry();
 
-        Map<IoStatisticsHolderKey, IoStatisticsHolder> stat = ioStatMgr.statistics(IoStatisticsType.CACHE_GROUP);
+        checkEmptyStat(mreg.withPrefix(CACHE_GROUP.metricGroupName()),
+            DEFAULT_CACHE_NAME,
+            null,
+            CACHE_GROUP);
 
-        checkEmptyStat(stat, DEFAULT_CACHE_NAME, null);
-
-        stat = ioStatMgr.statistics(IoStatisticsType.HASH_INDEX);
-
-        checkEmptyStat(stat, DEFAULT_CACHE_NAME, HASH_PK_IDX_NAME);
+        checkEmptyStat(mreg.withPrefix(HASH_INDEX.metricGroupName()),
+            DEFAULT_CACHE_NAME,
+            HASH_PK_IDX_NAME,
+            HASH_INDEX);
     }
 
     /**
@@ -75,16 +95,34 @@ public class IoStatisticsManagerSelfTest extends GridCommonAbstractTest {
      * @param name Name of statistics.
      * @param subName Subname of statistics.
      */
-    private void checkEmptyStat(Map<IoStatisticsHolderKey, IoStatisticsHolder> stat, String name, String subName) {
-        assertEquals(1, stat.size());
+    private void checkEmptyStat(MetricRegistry mset, String name, String subName, IoStatisticsType type) {
+        MetricRegistry cacheIoStatHolder;
 
-        IoStatisticsHolder cacheIoStatHolder = stat.get(new IoStatisticsHolderKey(name, subName));
+        if (subName == null)
+            cacheIoStatHolder = mset.withPrefix(name);
+        else
+            cacheIoStatHolder = mset.withPrefix(name, subName);
 
         assertNotNull(cacheIoStatHolder);
 
-        assertEquals(0, cacheIoStatHolder.logicalReads());
+        if (type == CACHE_GROUP) {
+            assertEquals(5, mset.getMetrics().size());
 
-        assertEquals(0, cacheIoStatHolder.physicalReads());
+            assertEquals(0, ((LongMetric)cacheIoStatHolder.findMetric(LOGICAL_READS)).longValue());
+
+            assertEquals(0, ((LongMetric)cacheIoStatHolder.findMetric(PHYSICAL_READS)).longValue());
+        }
+        else {
+            assertEquals(7, mset.getMetrics().size());
+
+            assertEquals(0, ((LongMetric)cacheIoStatHolder.findMetric(LOGICAL_READS_LEAF)).longValue());
+
+            assertEquals(0, ((LongMetric)cacheIoStatHolder.findMetric(LOGICAL_READS_INNER)).longValue());
+
+            assertEquals(0, ((LongMetric)cacheIoStatHolder.findMetric(PHYSICAL_READS_LEAF)).longValue());
+
+            assertEquals(0, ((LongMetric)cacheIoStatHolder.findMetric(PHYSICAL_READS_INNER)).longValue());
+        }
     }
 
     /**
@@ -114,16 +152,18 @@ public class IoStatisticsManagerSelfTest extends GridCommonAbstractTest {
      * @throws Exception In case of failure.
      */
     private void ioStatGlobalPageTrackTest(boolean isPersistent) throws Exception {
-        IoStatisticsManager ioStatMgr = prepareData(isPersistent);
+        IgniteEx grid = prepareData(isPersistent);
 
-        long physicalReadsCnt = ioStatMgr.physicalReads(IoStatisticsType.CACHE_GROUP, DEFAULT_CACHE_NAME, null);
+        MetricRegistry mreg = grid.context().metric().registry();
+
+        long physicalReadsCnt = physicalReads(mreg, CACHE_GROUP, DEFAULT_CACHE_NAME, null);
 
         if (isPersistent)
-            Assert.assertTrue(physicalReadsCnt>0);
+            Assert.assertTrue(physicalReadsCnt > 0);
         else
             Assert.assertEquals(0, physicalReadsCnt);
 
-        Long logicalReads = ioStatMgr.logicalReads(IoStatisticsType.HASH_INDEX, DEFAULT_CACHE_NAME, HASH_PK_IDX_NAME);
+        Long logicalReads = logicalReads(mreg, HASH_INDEX, metricName(DEFAULT_CACHE_NAME, HASH_PK_IDX_NAME));
 
         Assert.assertNotNull(logicalReads);
 
@@ -134,22 +174,20 @@ public class IoStatisticsManagerSelfTest extends GridCommonAbstractTest {
      * Prepare Ignite instance and fill cache.
      *
      * @param isPersistent {@code true} in case persistence should be enable.
-     * @return IO statistic manager.
+     * @return Ignite instance.
      * @throws Exception In case of failure.
      */
-    @NotNull private IoStatisticsManager prepareData(boolean isPersistent) throws Exception {
-        IgniteEx ign = prepareIgnite(isPersistent);
-
-        IoStatisticsManager ioStatMgr = ign.context().ioStats();
+    @NotNull private IgniteEx prepareData(boolean isPersistent) throws Exception {
+        IgniteEx grid = prepareIgnite(isPersistent);
 
-        IgniteCache cache = ign.getOrCreateCache(DEFAULT_CACHE_NAME);
+        IgniteCache cache = grid.getOrCreateCache(DEFAULT_CACHE_NAME);
 
-        ioStatMgr.reset();
+        resetAllIoMetrics(grid);
 
         for (int i = 0; i < RECORD_COUNT; i++)
             cache.put("KEY-" + i, "VAL-" + i);
 
-        return ioStatMgr;
+        return grid;
     }
 
     /**
@@ -192,4 +230,36 @@ public class IoStatisticsManagerSelfTest extends GridCommonAbstractTest {
 
         return ignite;
     }
+
+    /**
+     * @param statType Type of statistics which need to take.
+     * @param name name of statistics which need to take, e.g. cache name
+     * @param subName subName of statistics which need to take, e.g. index name.
+     * @return Number of physical reads since last reset statistics.
+     */
+    public Long physicalReads(MetricRegistry mreg, IoStatisticsType statType, String name, String subName) {
+        String fullName = subName == null ? name : metricName(name, subName);
+
+        MetricRegistry mset = mreg.withPrefix(statType.metricGroupName(), fullName);
+
+        if (mset == null)
+            return null;
+
+        Map<String, Long> res;
+
+        switch (statType) {
+            case CACHE_GROUP:
+                return ((LongMetric)mset.findMetric(PHYSICAL_READS)).value();
+
+            case HASH_INDEX:
+            case SORTED_INDEX:
+                long leaf = ((LongMetric)mset.findMetric(PHYSICAL_READS_LEAF)).value();
+                long inner = ((LongMetric)mset.findMetric(PHYSICAL_READS_INNER)).value();
+
+                return leaf + inner;
+
+            default:
+                return null;
+        }
+    }
 }
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/metric/JmxExporterSpiTest.java b/modules/core/src/test/java/org/apache/ignite/internal/metric/JmxExporterSpiTest.java
new file mode 100644
index 0000000..0ccbf32
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/metric/JmxExporterSpiTest.java
@@ -0,0 +1,126 @@
+/*
+ * 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.metric;
+
+import java.lang.management.ManagementFactory;
+import java.util.Set;
+import javax.management.DynamicMBean;
+import javax.management.MBeanFeatureInfo;
+import javax.management.MBeanServer;
+import javax.management.MBeanServerInvocationHandler;
+import javax.management.MalformedObjectNameException;
+import javax.management.ObjectName;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.configuration.DataRegionConfiguration;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.spi.metric.jmx.JmxExporterSpi;
+import org.apache.ignite.testframework.GridTestUtils.RunnableX;
+import org.junit.Test;
+
+import static java.util.Arrays.stream;
+import static java.util.stream.Collectors.toSet;
+import static org.apache.ignite.testframework.GridTestUtils.assertThrowsWithCause;
+
+/** */
+public class JmxExporterSpiTest extends AbstractExporterSpiTest {
+    /** */
+    private static IgniteEx ignite;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        cfg.setDataStorageConfiguration(new DataStorageConfiguration()
+            .setDefaultDataRegionConfiguration(
+                new DataRegionConfiguration()
+                    .setPersistenceEnabled(true)));
+
+        JmxExporterSpi jmxSpi = new JmxExporterSpi();
+
+        jmxSpi.setExportFilter(m -> !m.name().startsWith(FILTERED_PREFIX));
+
+        cfg.setMetricExporterSpi(jmxSpi);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        cleanPersistenceDir();
+
+        ignite = startGrid(0);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids(true);
+
+        cleanPersistenceDir();
+    }
+
+    /** */
+    @Test
+    public void testDataRegionJmxMetrics() throws Exception {
+        DynamicMBean dataRegionMBean = metricSet("io", "dataregion.default");
+
+        Set<String> res = stream(dataRegionMBean.getMBeanInfo().getAttributes())
+            .map(MBeanFeatureInfo::getName)
+            .collect(toSet());
+
+        assertTrue(res.containsAll(EXPECTED_ATTRIBUTES));
+
+        for (String metricName : res)
+            assertNotNull(metricName, dataRegionMBean.getAttribute(metricName));
+    }
+
+    /** */
+    @Test
+    public void testFilterAndExport() throws Exception {
+        createAdditionalMetrics(ignite);
+
+        assertThrowsWithCause(new RunnableX() {
+            @Override public void runx() throws Exception {
+                metricSet("filtered", "metric");
+            }
+        }, IgniteException.class);
+
+        DynamicMBean bean1 = metricSet("other", "prefix");
+
+        assertEquals(42L, bean1.getAttribute("test"));
+        assertEquals(43L, bean1.getAttribute("test2"));
+
+        DynamicMBean bean2 = metricSet("other", "prefix2");
+
+        assertEquals(44L, bean2.getAttribute("test3"));
+    }
+
+    /** */
+    public DynamicMBean metricSet(String grp, String name) throws MalformedObjectNameException {
+        ObjectName mbeanName = U.makeMBeanName(ignite.name(), grp, name);
+
+        MBeanServer mbeanSrv = ManagementFactory.getPlatformMBeanServer();
+
+        if (!mbeanSrv.isRegistered(mbeanName))
+            throw new IgniteException("MBean not registered.");
+
+        return MBeanServerInvocationHandler.newProxyInstance(mbeanSrv, mbeanName, DynamicMBean.class, false);
+    }
+}
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/metric/LogExporterSpiTest.java b/modules/core/src/test/java/org/apache/ignite/internal/metric/LogExporterSpiTest.java
new file mode 100644
index 0000000..7629937
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/metric/LogExporterSpiTest.java
@@ -0,0 +1,113 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.metric;
+
+import java.util.Set;
+import org.apache.ignite.configuration.DataRegionConfiguration;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.util.GridConcurrentHashSet;
+import org.apache.ignite.spi.metric.log.LogExporterSpi;
+import org.apache.ignite.testframework.ListeningTestLogger;
+import org.junit.Test;
+
+import static java.util.Arrays.asList;
+import static org.apache.ignite.testframework.GridTestUtils.waitForCondition;
+
+/** */
+public class LogExporterSpiTest extends AbstractExporterSpiTest {
+    /** Test logger. */
+    private final ListeningTestLogger log = new ListeningTestLogger(false, super.log);
+
+    /** */
+    private IgniteEx ignite;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        cfg.setDataStorageConfiguration(new DataStorageConfiguration()
+            .setDefaultDataRegionConfiguration(
+                new DataRegionConfiguration()
+                    .setPersistenceEnabled(true)))
+            .setGridLogger(log);
+
+        LogExporterSpi logSpi = new LogExporterSpi();
+
+        logSpi.setPeriod(EXPORT_TIMEOUT);
+
+        logSpi.setExportFilter(m -> !m.name().startsWith(FILTERED_PREFIX));
+
+        cfg.setMetricExporterSpi(logSpi);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids(true);
+
+        cleanPersistenceDir();
+    }
+
+    /** */
+    @Test
+    public void testLogSpi() throws Exception {
+        cleanPersistenceDir();
+
+        Set<String> expectedAttributes = new GridConcurrentHashSet<>(EXPECTED_ATTRIBUTES);
+
+        log.registerListener(s -> {
+            for (String attr : expectedAttributes) {
+                if (s.contains(attr))
+                    expectedAttributes.remove(attr);
+            }
+        });
+
+        ignite = startGrid(0);
+
+        boolean res = waitForCondition(expectedAttributes::isEmpty, EXPORT_TIMEOUT * 10);
+
+        assertTrue(res);
+
+        log.registerListener(s -> {
+            if(s.contains(FILTERED_PREFIX))
+                fail("Filtered prefix shouldn't export.");
+        });
+
+        Set<String> expectedMetrics = new GridConcurrentHashSet<>(asList(
+            "other.prefix.test = 42",
+            "other.prefix.test2 = 43",
+            "other.prefix2.test3 = 44"
+        ));
+
+        log.registerListener(s -> {
+            for (String metric : expectedMetrics) {
+                if (s.contains(metric))
+                    expectedMetrics.remove(metric);
+            }
+        });
+
+        createAdditionalMetrics(ignite);
+
+        res = waitForCondition(expectedMetrics::isEmpty, EXPORT_TIMEOUT * 10);
+
+        assertTrue(res);
+    }
+}
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/metric/MetricsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/metric/MetricsSelfTest.java
new file mode 100644
index 0000000..42f4ca5
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/metric/MetricsSelfTest.java
@@ -0,0 +1,393 @@
+/*
+ * 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.metric;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.processors.metric.MetricRegistryImpl;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.spi.metric.BooleanMetric;
+import org.apache.ignite.spi.metric.DoubleMetric;
+import org.apache.ignite.spi.metric.IntMetric;
+import org.apache.ignite.spi.metric.LongMetric;
+import org.apache.ignite.spi.metric.Metric;
+import org.apache.ignite.internal.processors.metric.MetricRegistry;
+import org.apache.ignite.spi.metric.ObjectMetric;
+import org.apache.ignite.internal.processors.metric.impl.DoubleMetricImpl;
+import org.apache.ignite.internal.processors.metric.impl.IntMetricImpl;
+import org.apache.ignite.internal.processors.metric.impl.LongAdderMetricImpl;
+import org.apache.ignite.internal.processors.metric.impl.LongMetricImpl;
+import org.apache.ignite.internal.processors.metric.impl.BooleanMetricImpl;
+import org.apache.ignite.internal.processors.metric.impl.HistogramMetric;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static java.util.Arrays.asList;
+import static java.util.stream.Collectors.toSet;
+import static junit.framework.TestCase.assertNull;
+import static junit.framework.TestCase.assertTrue;
+import static org.apache.ignite.internal.processors.metric.impl.MetricUtils.metricName;
+import static org.apache.ignite.testframework.GridTestUtils.runAsync;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+
+/** */
+@RunWith(Parameterized.class)
+public class MetricsSelfTest {
+    /** Metrics prefix. */
+    @Parameterized.Parameters(name = "Prefix {0}")
+    public static Iterable<String[]> data() {
+        return asList(
+            new String[] {""},
+            new String[] {"test.prefix"});
+    }
+
+    /** Prefix. */
+    @Parameterized.Parameter
+    public String prefix;
+
+    /** */
+    private MetricRegistry mreg;
+
+    /** */
+    @Before
+    public void setUp() throws Exception {
+        mreg = new MetricRegistryImpl();
+
+        if (!F.isEmpty(prefix))
+            mreg = mreg.withPrefix(prefix);
+    }
+
+    /** */
+    @Test
+    public void testLongCounter() throws Exception {
+        LongMetricImpl l = mreg.metric("ltest", "test");
+
+        run(l::increment, 100);
+
+        assertEquals(100*100, l.value());
+
+        l.reset();
+
+        assertEquals(0, l.value());
+    }
+
+    /** */
+    @Test
+    public void testLongAdderCounter() throws Exception {
+        LongAdderMetricImpl l = mreg.longAdderMetric("latest", "test");
+
+        run(l::increment, 100);
+
+        assertEquals(100*100, l.value());
+
+        l.reset();
+
+        assertEquals(0, l.value());
+    }
+
+    /** */
+    @Test
+    public void testDoubleCounter() throws Exception {
+        DoubleMetricImpl l = mreg.doubleMetric("dtest", "test");
+
+        run(() -> l.add(1), 100);
+
+        assertEquals(100*100f, l.value(), .000001);
+
+        l.reset();
+
+        assertEquals(0, l.value(), .000001);
+    }
+
+    /** */
+    @Test
+    public void testIntCounter() throws Exception {
+        IntMetricImpl l = mreg.intMetric("itest", "test");
+
+        run(() -> l.add(1), 100);
+
+        assertEquals(100*100, l.value());
+
+        l.reset();
+
+        assertEquals(0, l.value());
+    }
+
+    /** */
+    @Test
+    public void testRegister() throws Exception {
+        LongMetricImpl l = new LongMetricImpl(testMetricName("rtest"), "test");
+
+        mreg.register(l);
+
+        assertEquals(l, mreg.findMetric("rtest"));
+
+        l.reset();
+
+        assertEquals(0, l.value());
+    }
+
+    /** */
+    @Test
+    public void testBooleanMetric() throws Exception {
+        final boolean[] v = new boolean[1];
+
+        mreg.register("bmtest", () -> v[0], "test");
+
+        BooleanMetric m = (BooleanMetric)mreg.findMetric("bmtest");
+
+        assertEquals(v[0], m.value());
+
+        v[0] = true;
+
+        assertEquals(v[0], m.value());
+    }
+
+    /** */
+    @Test
+    public void testDoubleMetric() throws Exception {
+        final double[] v = new double[] {42};
+
+        mreg.register("dmtest", () -> v[0], "test");
+
+        DoubleMetric m = (DoubleMetric)mreg.findMetric("dmtest");
+
+        assertEquals(v[0], m.value(), 0);
+
+        v[0] = 1;
+
+        assertEquals(v[0], m.value(), 0);
+    }
+
+    /** */
+    @Test
+    public void testIntMetric() throws Exception {
+        final int[] v = new int[] {42};
+
+        mreg.register("imtest", () -> v[0], "test");
+
+        IntMetric m = (IntMetric)mreg.findMetric("imtest");
+
+        assertEquals(v[0], m.value());
+
+        v[0] = 1;
+
+        assertEquals(v[0], m.value());
+    }
+
+    /** */
+    @Test
+    public void testLongMetric() throws Exception {
+        final long[] v = new long[] {42};
+
+        mreg.register("lmtest", () -> v[0], "test");
+
+        LongMetric m = (LongMetric)mreg.findMetric("lmtest");
+
+        assertEquals(v[0], m.value());
+
+        v[0] = 1;
+
+        assertEquals(v[0], m.value());
+    }
+
+    /** */
+    @Test
+    public void testObjectMetric() throws Exception {
+        final String[] v = new String[] {"42"};
+
+        mreg.register("omtest", () -> v[0], String.class, "test");
+
+        ObjectMetric<String> m = (ObjectMetric<String>)mreg.findMetric("omtest");
+
+        assertEquals(v[0], m.value());
+
+        v[0] = "1";
+
+        assertEquals(v[0], m.value());
+    }
+
+    /** */
+    @Test
+    public void testBooleanGauges() throws Exception {
+        BooleanMetricImpl bg = mreg.booleanMetric("bg", "test");
+
+        bg.value(true);
+
+        assertTrue(bg.value());
+
+        bg.reset();
+
+        assertFalse(bg.value());
+    }
+
+    /** */
+    @Test
+    public void testHistogram() throws Exception {
+        HistogramMetric h = mreg.histogram("hmtest", new long[] {10, 100, 500}, "test");
+
+        List<IgniteInternalFuture> futs = new ArrayList<>();
+
+        int cnt = 10;
+
+        futs.add(runAsync(() -> {
+            for (int i = 0; i < cnt; i++)
+                h.value(9);
+        }));
+
+        futs.add(runAsync(() -> {
+            for (int i = 0; i < cnt*2; i++)
+                h.value(99);
+        }));
+
+        futs.add(runAsync(() -> {
+            for (int i = 0; i < cnt*3; i++)
+                h.value(500);
+        }));
+
+        futs.add(runAsync(() -> {
+            for (int i = 0; i < cnt*4; i++)
+                h.value(501);
+        }));
+
+        for (IgniteInternalFuture fut : futs)
+            fut.get();
+
+        long[] res = h.value();
+
+        assertEquals(cnt, res[0]);
+        assertEquals(cnt*2, res[1]);
+        assertEquals(cnt*3, res[2]);
+        assertEquals(cnt*4, res[3]);
+    }
+
+    /** */
+    @Test
+    public void testGetMetrics() throws Exception {
+        MetricRegistry mreg = newMetricRegistry();
+
+        mreg.metric("test1", "");
+        mreg.metric("test2", "");
+        mreg.metric("test3", "");
+        mreg.metric("test4", "");
+        mreg.metric("test5", "");
+
+        Set<String> names = new HashSet<>(asList(
+            testMetricName("test1"),
+            testMetricName("test2"),
+            testMetricName("test3"),
+            testMetricName("test4"),
+            testMetricName("test5")));
+
+        Set<String> res = mreg.getMetrics().stream()
+            .map(Metric::name)
+            .collect(toSet());
+
+        assertEquals(names, res);
+    }
+
+    /** */
+    @Test
+    public void testCreationListener() throws Exception {
+        MetricRegistry mreg = newMetricRegistry();
+
+        mreg.metric("test0", "");
+
+        Set<String> res = new HashSet<>();
+
+        mreg.addMetricCreationListener(m -> res.add(m.name()));
+
+        mreg.metric("test1", null);
+        mreg.metric("test2", null);
+        mreg.metric("test3", null);
+        mreg.metric("test4", null);
+        mreg.metric("test5", null);
+
+        Set<String> names = new HashSet<>(asList(
+            testMetricName("test1"),
+            testMetricName("test2"),
+            testMetricName("test3"),
+            testMetricName("test4"),
+            testMetricName("test5")));
+
+        assertEquals(names, res);
+    }
+
+    /** */
+    @Test
+    public void testRemove() throws Exception {
+        MetricRegistry mreg = newMetricRegistry();
+
+        LongMetricImpl cntr = mreg.metric("my.name", null);
+        LongMetricImpl cntr2 = mreg.metric("my.name.x", null);
+
+        assertNotNull(cntr);
+        assertNotNull(cntr2);
+
+        assertNotNull(mreg.findMetric("my.name"));
+        assertNotNull(mreg.findMetric("my.name.x"));
+
+        mreg.remove("my.name");
+
+        assertNull(mreg.findMetric("my.name"));
+        assertNotNull(mreg.findMetric("my.name.x"));
+
+        cntr = mreg.metric("my.name", null);
+
+        assertNotNull(mreg.findMetric("my.name"));
+    }
+
+    /** */
+    private MetricRegistry newMetricRegistry() {
+        MetricRegistry mreg = new MetricRegistryImpl();
+
+        if (!F.isEmpty(prefix))
+            mreg = mreg.withPrefix(prefix);
+        return mreg;
+    }
+
+    /** */
+    private void run(Runnable r, int cnt) throws org.apache.ignite.IgniteCheckedException {
+        List<IgniteInternalFuture> futs = new ArrayList<>();
+
+        for (int i=0; i<cnt; i++) {
+            futs.add(runAsync(() -> {
+                for (int j = 0; j < cnt; j++)
+                    r.run();
+            }));
+        }
+
+        for (IgniteInternalFuture fut : futs)
+            fut.get();
+    }
+
+    /** */
+    private String testMetricName(String name) {
+        if (prefix.isEmpty())
+            return name;
+
+        return metricName(prefix, name);
+    }
+}
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalIteratorSwitchSegmentTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalIteratorSwitchSegmentTest.java
index 230cb5c..b7d4e40 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalIteratorSwitchSegmentTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalIteratorSwitchSegmentTest.java
@@ -53,11 +53,13 @@ import org.apache.ignite.internal.processors.cache.persistence.wal.reader.Standa
 import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordSerializer;
 import org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordSerializerFactoryImpl;
 import org.apache.ignite.internal.processors.cacheobject.IgniteCacheObjectProcessor;
+import org.apache.ignite.internal.processors.metric.GridMetricManager;
 import org.apache.ignite.internal.processors.subscription.GridInternalSubscriptionProcessor;
 import org.apache.ignite.internal.util.typedef.T2;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.spi.eventstorage.NoopEventStorageSpi;
+import org.apache.ignite.spi.metric.noop.NoopMetricExporterSpi;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.junit.Assert;
@@ -440,6 +442,7 @@ public class IgniteWalIteratorSwitchSegmentTest extends GridCommonAbstractTest {
                 );
 
                 cfg.setEventStorageSpi(new NoopEventStorageSpi());
+                cfg.setMetricExporterSpi(new NoopMetricExporterSpi());
 
                 return cfg;
             }
@@ -451,6 +454,10 @@ public class IgniteWalIteratorSwitchSegmentTest extends GridCommonAbstractTest {
             @Override public GridEventStorageManager event() {
                 return new GridEventStorageManager(this);
             }
+
+            @Override public GridMetricManager metric() {
+                return new GridMetricManager(this);
+            }
         };
 
         IgniteWriteAheadLogManager walMgr = new FileWriteAheadLogManager(kctx);
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/BPlusTreePageMemoryImplTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/BPlusTreePageMemoryImplTest.java
index 7f19dde..a2dbaf4 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/BPlusTreePageMemoryImplTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/BPlusTreePageMemoryImplTest.java
@@ -31,10 +31,12 @@ import org.apache.ignite.internal.processors.cache.persistence.CheckpointWritePr
 import org.apache.ignite.internal.processors.cache.persistence.DataRegionMetricsImpl;
 import org.apache.ignite.internal.processors.cache.persistence.IgniteCacheDatabaseSharedManager;
 import org.apache.ignite.internal.processors.database.BPlusTreeSelfTest;
+import org.apache.ignite.internal.processors.metric.GridMetricManager;
 import org.apache.ignite.internal.processors.plugin.IgnitePluginProcessor;
 import org.apache.ignite.internal.processors.subscription.GridInternalSubscriptionProcessor;
 import org.apache.ignite.internal.util.typedef.CIX3;
 import org.apache.ignite.spi.encryption.noop.NoopEncryptionSpi;
+import org.apache.ignite.spi.metric.noop.NoopMetricExporterSpi;
 import org.apache.ignite.testframework.junits.GridTestKernalContext;
 import org.mockito.Mockito;
 
@@ -56,12 +58,14 @@ public class BPlusTreePageMemoryImplTest extends BPlusTreeSelfTest {
         IgniteConfiguration cfg = new IgniteConfiguration();
 
         cfg.setEncryptionSpi(new NoopEncryptionSpi());
+        cfg.setMetricExporterSpi(new NoopMetricExporterSpi());
 
         GridTestKernalContext cctx = new GridTestKernalContext(log, cfg);
 
         cctx.add(new IgnitePluginProcessor(cctx, cfg, Collections.emptyList()));
         cctx.add(new GridInternalSubscriptionProcessor(cctx));
         cctx.add(new GridEncryptionManager(cctx));
+        cctx.add(new GridMetricManager(cctx));
 
         GridCacheSharedContext<Object, Object> sharedCtx = new GridCacheSharedContext<>(
             cctx,
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/BPlusTreeReuseListPageMemoryImplTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/BPlusTreeReuseListPageMemoryImplTest.java
index 5414064..cf70333 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/BPlusTreeReuseListPageMemoryImplTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/BPlusTreeReuseListPageMemoryImplTest.java
@@ -30,10 +30,12 @@ import org.apache.ignite.internal.processors.cache.persistence.CheckpointWritePr
 import org.apache.ignite.internal.processors.cache.persistence.DataRegionMetricsImpl;
 import org.apache.ignite.internal.processors.cache.persistence.IgniteCacheDatabaseSharedManager;
 import org.apache.ignite.internal.processors.database.BPlusTreeReuseSelfTest;
+import org.apache.ignite.internal.processors.metric.GridMetricManager;
 import org.apache.ignite.internal.processors.plugin.IgnitePluginProcessor;
 import org.apache.ignite.internal.processors.subscription.GridInternalSubscriptionProcessor;
 import org.apache.ignite.internal.util.lang.GridInClosure3X;
 import org.apache.ignite.spi.encryption.noop.NoopEncryptionSpi;
+import org.apache.ignite.spi.metric.noop.NoopMetricExporterSpi;
 import org.apache.ignite.testframework.junits.GridTestKernalContext;
 import org.mockito.Mockito;
 
@@ -55,12 +57,14 @@ public class BPlusTreeReuseListPageMemoryImplTest extends BPlusTreeReuseSelfTest
         IgniteConfiguration cfg = new IgniteConfiguration();
 
         cfg.setEncryptionSpi(new NoopEncryptionSpi());
+        cfg.setMetricExporterSpi(new NoopMetricExporterSpi());
 
         GridTestKernalContext cctx = new GridTestKernalContext(log, cfg);
 
         cctx.add(new IgnitePluginProcessor(cctx, cfg, Collections.emptyList()));
         cctx.add(new GridInternalSubscriptionProcessor(cctx));
         cctx.add(new GridEncryptionManager(cctx));
+        cctx.add(new GridMetricManager(cctx));
 
         GridCacheSharedContext<Object, Object> sharedCtx = new GridCacheSharedContext<>(
             cctx,
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/IgnitePageMemReplaceDelayedWriteUnitTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/IgnitePageMemReplaceDelayedWriteUnitTest.java
index af09f9a..a468f5a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/IgnitePageMemReplaceDelayedWriteUnitTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/IgnitePageMemReplaceDelayedWriteUnitTest.java
@@ -41,11 +41,13 @@ import org.apache.ignite.internal.processors.cache.persistence.CheckpointWritePr
 import org.apache.ignite.internal.processors.cache.persistence.DataRegionMetricsImpl;
 import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
 import org.apache.ignite.internal.processors.cache.persistence.IgniteCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.metric.GridMetricManager;
 import org.apache.ignite.internal.processors.subscription.GridInternalSubscriptionProcessor;
 import org.apache.ignite.internal.util.GridMultiCollectionWrapper;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.logger.NullLogger;
 import org.apache.ignite.spi.encryption.noop.NoopEncryptionSpi;
+import org.apache.ignite.spi.metric.noop.NoopMetricExporterSpi;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.jetbrains.annotations.NotNull;
 import org.junit.Rule;
@@ -238,6 +240,12 @@ public class IgnitePageMemReplaceDelayedWriteUnitTest {
                 return new GridEncryptionManager(kernalCtx);
             }
         });
+        when(kernalCtx.metric()).thenAnswer(new Answer<Object>() {
+            @Override public Object answer(InvocationOnMock mock) throws Throwable {
+                return new GridMetricManager(kernalCtx);
+            }
+        });
+
         when(sctx.kernalContext()).thenReturn(kernalCtx);
 
         DataRegionConfiguration regCfg = cfg.getDataStorageConfiguration().getDefaultDataRegionConfiguration();
@@ -264,6 +272,7 @@ public class IgnitePageMemReplaceDelayedWriteUnitTest {
         IgniteConfiguration cfg = new IgniteConfiguration();
 
         cfg.setEncryptionSpi(new NoopEncryptionSpi());
+        cfg.setMetricExporterSpi(new NoopMetricExporterSpi());
 
         cfg.setDataStorageConfiguration(
             new DataStorageConfiguration()
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/IndexStoragePageMemoryImplTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/IndexStoragePageMemoryImplTest.java
index 96f56c0..4e69127 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/IndexStoragePageMemoryImplTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/IndexStoragePageMemoryImplTest.java
@@ -32,11 +32,13 @@ import org.apache.ignite.internal.processors.cache.persistence.CheckpointWritePr
 import org.apache.ignite.internal.processors.cache.persistence.DataRegionMetricsImpl;
 import org.apache.ignite.internal.processors.cache.persistence.IgniteCacheDatabaseSharedManager;
 import org.apache.ignite.internal.processors.database.IndexStorageSelfTest;
+import org.apache.ignite.internal.processors.metric.GridMetricManager;
 import org.apache.ignite.internal.processors.plugin.IgnitePluginProcessor;
 import org.apache.ignite.internal.processors.subscription.GridInternalSubscriptionProcessor;
 import org.apache.ignite.internal.util.lang.GridInClosure3X;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.spi.encryption.noop.NoopEncryptionSpi;
+import org.apache.ignite.spi.metric.noop.NoopMetricExporterSpi;
 import org.apache.ignite.testframework.junits.GridTestKernalContext;
 import org.mockito.Mockito;
 
@@ -71,12 +73,14 @@ public class IndexStoragePageMemoryImplTest extends IndexStorageSelfTest {
         IgniteConfiguration cfg = new IgniteConfiguration();
 
         cfg.setEncryptionSpi(new NoopEncryptionSpi());
+        cfg.setMetricExporterSpi(new NoopMetricExporterSpi());
 
         GridTestKernalContext cctx = new GridTestKernalContext(log, cfg);
 
         cctx.add(new IgnitePluginProcessor(cctx, cfg, Collections.emptyList()));
         cctx.add(new GridInternalSubscriptionProcessor(cctx));
         cctx.add(new GridEncryptionManager(cctx));
+        cctx.add(new GridMetricManager(cctx));
 
         GridCacheSharedContext<Object, Object> sharedCtx = new GridCacheSharedContext<>(
             cctx,
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImplNoLoadTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImplNoLoadTest.java
index 1e91643..cc508cf 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImplNoLoadTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImplNoLoadTest.java
@@ -32,11 +32,13 @@ import org.apache.ignite.internal.processors.cache.persistence.CheckpointLockSta
 import org.apache.ignite.internal.processors.cache.persistence.CheckpointWriteProgressSupplier;
 import org.apache.ignite.internal.processors.cache.persistence.DataRegionMetricsImpl;
 import org.apache.ignite.internal.processors.cache.persistence.IgniteCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.metric.GridMetricManager;
 import org.apache.ignite.internal.processors.plugin.IgnitePluginProcessor;
 import org.apache.ignite.internal.processors.subscription.GridInternalSubscriptionProcessor;
 import org.apache.ignite.internal.util.lang.GridInClosure3X;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.spi.encryption.noop.NoopEncryptionSpi;
+import org.apache.ignite.spi.metric.noop.NoopMetricExporterSpi;
 import org.apache.ignite.testframework.junits.GridTestKernalContext;
 import org.junit.Test;
 import org.mockito.Mockito;
@@ -61,12 +63,14 @@ public class PageMemoryImplNoLoadTest extends PageMemoryNoLoadSelfTest {
         IgniteConfiguration cfg = new IgniteConfiguration();
 
         cfg.setEncryptionSpi(new NoopEncryptionSpi());
+        cfg.setMetricExporterSpi(new NoopMetricExporterSpi());
 
         GridTestKernalContext cctx = new GridTestKernalContext(log, cfg);
 
         cctx.add(new IgnitePluginProcessor(cctx, cfg, Collections.emptyList()));
         cctx.add(new GridInternalSubscriptionProcessor(cctx));
         cctx.add(new GridEncryptionManager(cctx));
+        cctx.add(new GridMetricManager(cctx));
 
         GridCacheSharedContext<Object, Object> sharedCtx = new GridCacheSharedContext<>(
             cctx,
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImplTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImplTest.java
index 13b3305..da27930 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImplTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PageMemoryImplTest.java
@@ -44,11 +44,13 @@ import org.apache.ignite.internal.processors.cache.persistence.DataRegionMetrics
 import org.apache.ignite.internal.processors.cache.persistence.IgniteCacheDatabaseSharedManager;
 import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
 import org.apache.ignite.internal.processors.failure.FailureProcessor;
+import org.apache.ignite.internal.processors.metric.GridMetricManager;
 import org.apache.ignite.internal.processors.plugin.IgnitePluginProcessor;
 import org.apache.ignite.internal.processors.subscription.GridInternalSubscriptionProcessor;
 import org.apache.ignite.internal.util.lang.GridInClosure3X;
 import org.apache.ignite.plugin.PluginProvider;
 import org.apache.ignite.spi.encryption.noop.NoopEncryptionSpi;
+import org.apache.ignite.spi.metric.noop.NoopMetricExporterSpi;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.GridTestKernalContext;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
@@ -281,12 +283,14 @@ public class PageMemoryImplTest extends GridCommonAbstractTest {
         igniteCfg.setDataStorageConfiguration(new DataStorageConfiguration());
         igniteCfg.setFailureHandler(new NoOpFailureHandler());
         igniteCfg.setEncryptionSpi(new NoopEncryptionSpi());
+        igniteCfg.setMetricExporterSpi(new NoopMetricExporterSpi());
 
         GridTestKernalContext kernalCtx = new GridTestKernalContext(new GridTestLog4jLogger(), igniteCfg);
 
         kernalCtx.add(new IgnitePluginProcessor(kernalCtx, igniteCfg, Collections.<PluginProvider>emptyList()));
         kernalCtx.add(new GridInternalSubscriptionProcessor(kernalCtx));
         kernalCtx.add(new GridEncryptionManager(kernalCtx));
+        kernalCtx.add(new GridMetricManager(kernalCtx));
 
         FailureProcessor failureProc = new FailureProcessor(kernalCtx);
 
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteThrottleSandboxTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteThrottleSandboxTest.java
index 743fc3b..0343bed 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteThrottleSandboxTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteThrottleSandboxTest.java
@@ -35,7 +35,7 @@ import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.configuration.WALMode;
 import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.processors.cache.persistence.CheckpointWriteProgressSupplier;
-import org.apache.ignite.internal.processors.cache.ratemetrics.HitRateMetrics;
+import org.apache.ignite.internal.processors.metric.impl.HitRateMetric;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.testframework.GridTestUtils;
@@ -114,7 +114,7 @@ public class PagesWriteThrottleSandboxTest extends GridCommonAbstractTest {
 
             final AtomicBoolean run = new AtomicBoolean(true);
 
-            final HitRateMetrics getRate = new HitRateMetrics(5000, 5);
+            final HitRateMetric getRate = new HitRateMetric("getRate", "", 5000, 5);
 
             GridTestUtils.runMultiThreadedAsync(new Callable<Object>() {
                 @Override public Object call() throws Exception {
@@ -125,14 +125,14 @@ public class PagesWriteThrottleSandboxTest extends GridCommonAbstractTest {
 
                         ignite(0).cache(CACHE_NAME).get(key);
 
-                        getRate.onHit();
+                        getRate.increment();
                     }
 
                     return null;
                 }
             }, 2, "read-loader");
 
-            final HitRateMetrics putRate = new HitRateMetrics(1000, 5);
+            final HitRateMetric putRate = new HitRateMetric("putRate", "", 1000, 5);
 
             GridTestUtils.runAsync(new Runnable() {
                 @Override public void run() {
@@ -160,7 +160,7 @@ public class PagesWriteThrottleSandboxTest extends GridCommonAbstractTest {
                             e.printStackTrace();
                         }
 
-                        System.out.println("@@@ putsPerSec=," + (putRate.getRate()) + ", getsPerSec=," + (getRate.getRate()) + ", dirtyPages=," + dirtyPages + ", cpWrittenPages=," + cpWrittenPages + ", cpBufPages=," + cpBufPages);
+                        System.out.println("@@@ putsPerSec=," + (putRate.value()) + ", getsPerSec=," + (getRate.value()) + ", dirtyPages=," + dirtyPages + ", cpWrittenPages=," + cpWrittenPages + ", cpBufPages=," + cpBufPages);
 
                         try {
                             Thread.sleep(1000);
@@ -179,7 +179,7 @@ public class PagesWriteThrottleSandboxTest extends GridCommonAbstractTest {
                     ds.addData(ThreadLocalRandom.current().nextInt(keyCnt), new TestValue(ThreadLocalRandom.current().nextInt(),
                         ThreadLocalRandom.current().nextInt()));
 
-                    putRate.onHit();
+                    putRate.increment();
                 }
             }
 
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteThrottleSmokeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteThrottleSmokeTest.java
index cdb6e01..4f442e1 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteThrottleSmokeTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/PagesWriteThrottleSmokeTest.java
@@ -41,7 +41,7 @@ import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
 import org.apache.ignite.internal.processors.cache.persistence.file.FileIODecorator;
 import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
 import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
-import org.apache.ignite.internal.processors.cache.ratemetrics.HitRateMetrics;
+import org.apache.ignite.internal.processors.metric.impl.HitRateMetric;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.testframework.GridTestUtils;
@@ -128,9 +128,9 @@ public class PagesWriteThrottleSmokeTest extends GridCommonAbstractTest {
 
             final AtomicBoolean zeroDropdown = new AtomicBoolean(false);
 
-            final HitRateMetrics putRate10secs = new HitRateMetrics(10_000, 20);
+            final HitRateMetric putRate10secs = new HitRateMetric("putRate10secs", "", 10_000, 20);
 
-            final HitRateMetrics putRate1sec = new HitRateMetrics(1_000, 20);
+            final HitRateMetric putRate1sec = new HitRateMetric("putRate1sec", "", 1_000, 20);
 
             GridTestUtils.runAsync(new Runnable() {
                 @Override public void run() {
@@ -139,10 +139,10 @@ public class PagesWriteThrottleSmokeTest extends GridCommonAbstractTest {
 
                         while (run.get()) {
                             System.out.println(
-                                "Put rate over last 10 seconds: " + (putRate10secs.getRate() / 10) +
-                                    " puts/sec, over last 1 second: " + putRate1sec.getRate());
+                                "Put rate over last 10 seconds: " + (putRate10secs.value() / 10) +
+                                    " puts/sec, over last 1 second: " + putRate1sec.value());
 
-                            if (putRate10secs.getRate() == 0) {
+                            if (putRate10secs.value() == 0) {
                                 zeroDropdown.set(true);
 
                                 run.set(false);
@@ -173,9 +173,9 @@ public class PagesWriteThrottleSmokeTest extends GridCommonAbstractTest {
                         cache.put(ThreadLocalRandom.current().nextInt(keyCnt), new TestValue(ThreadLocalRandom.current().nextInt(),
                             ThreadLocalRandom.current().nextInt()));
 
-                        putRate10secs.onHit();
+                        putRate10secs.increment();
 
-                        putRate1sec.onHit();
+                        putRate1sec.increment();
                     }
 
                     run.set(false);
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TransactionIntegrityWithPrimaryIndexCorruptionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TransactionIntegrityWithPrimaryIndexCorruptionTest.java
index 9bde5f5..f33dda9 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TransactionIntegrityWithPrimaryIndexCorruptionTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TransactionIntegrityWithPrimaryIndexCorruptionTest.java
@@ -33,7 +33,7 @@ import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree;
 import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
 import org.apache.ignite.internal.processors.cache.persistence.tree.util.PageHandler;
 import org.apache.ignite.internal.processors.cache.tree.SearchRow;
-import org.apache.ignite.internal.stat.IoStatisticsHolder;
+import org.apache.ignite.internal.metric.IoStatisticsHolder;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.junit.Assume;
 import org.junit.Test;
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/CacheFreeListSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/CacheFreeListSelfTest.java
index b4089e9..12b804f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/CacheFreeListSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/CacheFreeListSelfTest.java
@@ -47,7 +47,7 @@ import org.apache.ignite.internal.processors.cache.persistence.freelist.CacheFre
 import org.apache.ignite.internal.processors.cache.persistence.freelist.FreeList;
 import org.apache.ignite.internal.processors.cache.persistence.tree.io.CacheVersionIO;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
-import org.apache.ignite.internal.stat.IoStatisticsHolderNoOp;
+import org.apache.ignite.internal.metric.IoStatisticsHolderNoOp;
 import org.apache.ignite.plugin.extensions.communication.MessageReader;
 import org.apache.ignite.plugin.extensions.communication.MessageWriter;
 import org.apache.ignite.testframework.GridTestUtils;
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/DataRegionMetricsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/DataRegionMetricsSelfTest.java
index 313d6c8..6c27bc1 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/DataRegionMetricsSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/DataRegionMetricsSelfTest.java
@@ -21,7 +21,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.ignite.DataRegionMetrics;
 import org.apache.ignite.configuration.DataRegionConfiguration;
 import org.apache.ignite.internal.processors.cache.persistence.DataRegionMetricsImpl;
-import org.apache.ignite.internal.processors.cache.ratemetrics.HitRateMetrics;
+import org.apache.ignite.internal.processors.metric.impl.HitRateMetric;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.junit.Test;
 
@@ -185,7 +185,7 @@ public class DataRegionMetricsSelfTest extends GridCommonAbstractTest {
     }
 
     /**
-     * As rate metrics {@link HitRateMetrics implementation} is tied to absolute time ticks
+     * As rate metrics {@link HitRateMetric implementation} is tied to absolute time ticks
      * (not related to the first hit) all tests need to align start time with this sequence of ticks.
      *
      * @param rateTimeInterval Rate time interval.
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/stat/IoStatisticsMetricsLocalMXBeanImplSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/stat/IoStatisticsMetricsLocalMXBeanImplSelfTest.java
deleted file mode 100644
index 3fd320d..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/stat/IoStatisticsMetricsLocalMXBeanImplSelfTest.java
+++ /dev/null
@@ -1,220 +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.stat;
-
-import java.lang.management.ManagementFactory;
-import java.time.format.DateTimeFormatter;
-import javax.management.MBeanServer;
-import javax.management.MBeanServerInvocationHandler;
-import javax.management.ObjectName;
-import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.IgniteEx;
-import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.mxbean.IoStatisticsMetricsMXBean;
-import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-import org.junit.Assert;
-import org.junit.Test;
-
-import static org.apache.ignite.internal.stat.IoStatisticsHolderIndex.HASH_PK_IDX_NAME;
-
-/**
- * Test of local node IO statistics MX bean.
- */
-public class IoStatisticsMetricsLocalMXBeanImplSelfTest extends GridCommonAbstractTest {
-    /** */
-    private static IgniteEx ignite;
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String name) throws Exception {
-        final IgniteConfiguration cfg = super.getConfiguration(name);
-
-        final CacheConfiguration cCfg = new CacheConfiguration()
-            .setName(DEFAULT_CACHE_NAME);
-
-        cfg.setCacheConfiguration(cCfg);
-
-        return cfg;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        super.beforeTestsStarted();
-
-        ignite = startGrid(0);
-    }
-
-    /**
-     * Simple test JMX bean for indexes IO stats.
-     *
-     * @throws Exception In case of failure.
-     */
-    @Test
-    public void testIndexBasic() throws Exception {
-        IoStatisticsMetricsMXBean bean = ioStatMXBean();
-
-        IoStatisticsManager ioStatMgr = ignite.context().ioStats();
-
-        Assert.assertEquals(ioStatMgr.startTime().toEpochSecond(), bean.getStartTime());
-
-        Assert.assertEquals(ioStatMgr.startTime().format(DateTimeFormatter.ISO_DATE_TIME), bean.getStartTimeLocal());
-
-        bean.reset();
-
-        Assert.assertEquals(ioStatMgr.startTime().toEpochSecond(), bean.getStartTime());
-
-        Assert.assertEquals(ioStatMgr.startTime().format(DateTimeFormatter.ISO_DATE_TIME), bean.getStartTimeLocal());
-
-        int cnt = 100;
-
-        populateCache(cnt);
-
-        long idxLeafLogicalCnt = bean.getIndexLeafLogicalReads(DEFAULT_CACHE_NAME, HASH_PK_IDX_NAME);
-
-        Assert.assertEquals(cnt, idxLeafLogicalCnt);
-
-        long idxLeafPhysicalCnt = bean.getIndexLeafPhysicalReads(DEFAULT_CACHE_NAME, HASH_PK_IDX_NAME);
-
-        Assert.assertEquals(0, idxLeafPhysicalCnt);
-
-        long idxInnerLogicalCnt = bean.getIndexInnerLogicalReads(DEFAULT_CACHE_NAME, HASH_PK_IDX_NAME);
-
-        Assert.assertEquals(0, idxInnerLogicalCnt);
-
-        long idxInnerPhysicalCnt = bean.getIndexInnerPhysicalReads(DEFAULT_CACHE_NAME, HASH_PK_IDX_NAME);
-
-        Assert.assertEquals(0, idxInnerPhysicalCnt);
-
-        Long aggregatedIdxLogicalReads = bean.getIndexLogicalReads(DEFAULT_CACHE_NAME, HASH_PK_IDX_NAME);
-
-        Assert.assertNotNull(aggregatedIdxLogicalReads);
-
-        Assert.assertEquals(aggregatedIdxLogicalReads.longValue(), idxLeafLogicalCnt + idxLeafPhysicalCnt +
-            idxInnerLogicalCnt + idxInnerPhysicalCnt);
-
-        Long aggregatedIdxPhysicalReads = bean.getIndexPhysicalReads(DEFAULT_CACHE_NAME, HASH_PK_IDX_NAME);
-
-        Assert.assertNotNull(aggregatedIdxPhysicalReads);
-
-        Assert.assertEquals(0, aggregatedIdxPhysicalReads.longValue());
-
-        String formatted = bean.getIndexStatistics(DEFAULT_CACHE_NAME, HASH_PK_IDX_NAME);
-
-        Assert.assertEquals("HASH_INDEX default.HASH_PK [LOGICAL_READS_LEAF=100, LOGICAL_READS_INNER=0, " +
-            "PHYSICAL_READS_INNER=0, PHYSICAL_READS_LEAF=0]", formatted);
-
-        String unexistedStats = bean.getIndexStatistics("unknownCache", "unknownIdx");
-
-        Assert.assertEquals("SORTED_INDEX unknownCache.unknownIdx []", unexistedStats);
-    }
-
-    /**
-     * Simple test JMX bean for caches IO stats.
-     *
-     * @throws Exception In case of failure.
-     */
-    @Test
-    public void testCacheBasic() throws Exception {
-        IoStatisticsMetricsMXBean bean = ioStatMXBean();
-
-        IoStatisticsManager ioStatMgr = ignite.context().ioStats();
-
-        Assert.assertEquals(ioStatMgr.startTime().toEpochSecond(), bean.getStartTime());
-
-        Assert.assertEquals(ioStatMgr.startTime().format(DateTimeFormatter.ISO_DATE_TIME), bean.getStartTimeLocal());
-
-        bean.reset();
-
-        Assert.assertEquals(ioStatMgr.startTime().toEpochSecond(), bean.getStartTime());
-
-        Assert.assertEquals(ioStatMgr.startTime().format(DateTimeFormatter.ISO_DATE_TIME), bean.getStartTimeLocal());
-
-        int cnt = 100;
-
-        warmUpMemmory(bean, cnt);
-
-        populateCache(cnt);
-
-        Long cacheLogicalReadsCnt = bean.getCacheGroupLogicalReads(DEFAULT_CACHE_NAME);
-
-        Assert.assertNotNull(cacheLogicalReadsCnt);
-
-        Assert.assertEquals(cnt, cacheLogicalReadsCnt.longValue());
-
-        Long cachePhysicalReadsCnt = bean.getCacheGroupPhysicalReads(DEFAULT_CACHE_NAME);
-
-        Assert.assertNotNull(cachePhysicalReadsCnt);
-
-        Assert.assertEquals(0, cachePhysicalReadsCnt.longValue());
-
-        String formatted = bean.getCacheGroupStatistics(DEFAULT_CACHE_NAME);
-
-        Assert.assertEquals("CACHE_GROUP default [LOGICAL_READS=100, PHYSICAL_READS=0]", formatted);
-
-        String unexistedStats = bean.getCacheGroupStatistics("unknownCache");
-
-        Assert.assertEquals("CACHE_GROUP unknownCache []", unexistedStats);
-    }
-
-    /**
-     * Warm up memmory to allocate partitions cache pages related to inserting keys.
-     *
-     * @param bean JMX bean.
-     * @param cnt Number of inserting elements.
-     */
-    private void warmUpMemmory(IoStatisticsMetricsMXBean bean, int cnt) {
-        populateCache(cnt);
-
-        clearCache(cnt);
-
-        bean.reset();
-    }
-
-    /**
-     * @param cnt Number of inserting elements.
-     */
-    private void populateCache(int cnt) {
-        for (int i = 0; i < cnt; i++)
-            ignite.cache(DEFAULT_CACHE_NAME).put(i, i);
-    }
-
-    /**
-     * @param cnt Number of removing elements.
-     */
-    private void clearCache(int cnt) {
-        for (int i = 0; i < cnt; i++)
-            ignite.cache(DEFAULT_CACHE_NAME).remove(i);
-    }
-
-    /**
-     * @return IO statistics MX bean for node with given index.
-     * @throws Exception In case of failure.
-     */
-    private IoStatisticsMetricsMXBean ioStatMXBean() throws Exception {
-        ObjectName mbeanName = U.makeMBeanName(getTestIgniteInstanceName(0), "IOMetrics",
-            IoStatisticsMetricsLocalMXBeanImpl.class.getSimpleName());
-
-        MBeanServer mbeanSrv = ManagementFactory.getPlatformMBeanServer();
-
-        if (!mbeanSrv.isRegistered(mbeanName))
-            fail("MBean is not registered: " + mbeanName.getCanonicalName());
-
-        return MBeanServerInvocationHandler.newProxyInstance(mbeanSrv, mbeanName, IoStatisticsMetricsMXBean.class, false);
-    }
-}
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheMvccTestSuite9.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheMvccTestSuite9.java
index a034541..243fb4b 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheMvccTestSuite9.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheMvccTestSuite9.java
@@ -29,8 +29,8 @@ import org.apache.ignite.internal.processors.cache.distributed.IgniteCachePrimar
 import org.apache.ignite.internal.processors.cache.distributed.IgniteTxCachePrimarySyncTest;
 import org.apache.ignite.internal.processors.cache.distributed.IgniteTxConcurrentRemoveObjectsTest;
 import org.apache.ignite.internal.processors.cache.transactions.TxPartitionCounterStateConsistencyHistoryRebalanceTest;
-import org.apache.ignite.internal.stat.IoStatisticsCachePersistenceSelfTest;
-import org.apache.ignite.internal.stat.IoStatisticsCacheSelfTest;
+import org.apache.ignite.internal.metric.IoStatisticsCachePersistenceSelfTest;
+import org.apache.ignite.internal.metric.IoStatisticsCacheSelfTest;
 import org.apache.ignite.testframework.junits.DynamicSuite;
 import org.apache.ignite.internal.processors.cache.transactions.TxPartitionCounterStateOnePrimaryOneBackupHistoryRebalanceTest;
 import org.apache.ignite.internal.processors.cache.transactions.TxPartitionCounterStateOnePrimaryOneBackupTest;
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite9.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite9.java
index 38a8fa9..d9f15c3 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite9.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite9.java
@@ -20,6 +20,14 @@ package org.apache.ignite.testsuites;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
+import org.apache.ignite.internal.metric.CacheMetricsAddRemoveTest;
+import org.apache.ignite.internal.metric.IoStatisticsCachePersistenceSelfTest;
+import org.apache.ignite.internal.metric.IoStatisticsCacheSelfTest;
+import org.apache.ignite.internal.metric.IoStatisticsMetricsLocalMXBeanImplSelfTest;
+import org.apache.ignite.internal.metric.IoStatisticsSelfTest;
+import org.apache.ignite.internal.metric.JmxExporterSpiTest;
+import org.apache.ignite.internal.metric.LogExporterSpiTest;
+import org.apache.ignite.internal.metric.MetricsSelfTest;
 import org.apache.ignite.internal.processors.cache.CachePutIfAbsentTest;
 import org.apache.ignite.internal.processors.cache.GridCacheLongRunningTransactionDiagnosticsTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheGetCustomCollectionsSelfTest;
@@ -33,19 +41,15 @@ import org.apache.ignite.internal.processors.cache.distributed.IgniteTxConcurren
 import org.apache.ignite.internal.processors.cache.transactions.PartitionUpdateCounterTest;
 import org.apache.ignite.internal.processors.cache.transactions.TxDataConsistencyOnCommitFailureTest;
 import org.apache.ignite.internal.processors.cache.transactions.TxPartitionCounterStateConsistencyHistoryRebalanceTest;
-import org.apache.ignite.internal.processors.cache.transactions.TxPartitionCounterStateOnePrimaryTwoBackupsFailAllHistoryRebalanceTest;
-import org.apache.ignite.internal.stat.IoStatisticsCachePersistenceSelfTest;
-import org.apache.ignite.internal.stat.IoStatisticsCacheSelfTest;
-import org.apache.ignite.internal.stat.IoStatisticsManagerSelfTest;
-import org.apache.ignite.internal.stat.IoStatisticsMetricsLocalMXBeanImplSelfTest;
+import org.apache.ignite.internal.processors.cache.transactions.TxPartitionCounterStateConsistencyTest;
 import org.apache.ignite.internal.processors.cache.transactions.TxPartitionCounterStateOnePrimaryOneBackupHistoryRebalanceTest;
 import org.apache.ignite.internal.processors.cache.transactions.TxPartitionCounterStateOnePrimaryOneBackupTest;
+import org.apache.ignite.internal.processors.cache.transactions.TxPartitionCounterStateOnePrimaryTwoBackupsFailAllHistoryRebalanceTest;
 import org.apache.ignite.internal.processors.cache.transactions.TxPartitionCounterStateOnePrimaryTwoBackupsFailAllTest;
 import org.apache.ignite.internal.processors.cache.transactions.TxPartitionCounterStateOnePrimaryTwoBackupsHistoryRebalanceTest;
 import org.apache.ignite.internal.processors.cache.transactions.TxPartitionCounterStateOnePrimaryTwoBackupsTest;
 import org.apache.ignite.internal.processors.cache.transactions.TxPartitionCounterStatePutTest;
 import org.apache.ignite.internal.processors.cache.transactions.TxPartitionCounterStateTwoPrimaryTwoBackupsTest;
-import org.apache.ignite.internal.processors.cache.transactions.TxPartitionCounterStateConsistencyTest;
 import org.apache.ignite.internal.processors.cache.transactions.TxPartitionCounterStateWithFilterTest;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.DynamicSuite;
@@ -102,8 +106,12 @@ public class IgniteCacheTestSuite9 {
         // IO statistics
         GridTestUtils.addTestIfNeeded(suite, IoStatisticsCachePersistenceSelfTest.class, ignoredTests);
         GridTestUtils.addTestIfNeeded(suite, IoStatisticsCacheSelfTest.class, ignoredTests);
-        GridTestUtils.addTestIfNeeded(suite, IoStatisticsManagerSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, IoStatisticsSelfTest.class, ignoredTests);
         GridTestUtils.addTestIfNeeded(suite, IoStatisticsMetricsLocalMXBeanImplSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, MetricsSelfTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, CacheMetricsAddRemoveTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, JmxExporterSpiTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, LogExporterSpiTest.class, ignoredTests);
 
         GridTestUtils.addTestIfNeeded(suite, GridCacheLongRunningTransactionDiagnosticsTest.class, ignoredTests);
 
diff --git a/modules/ignored-tests/pom.xml b/modules/ignored-tests/pom.xml
index b9b4823..f6293d9 100644
--- a/modules/ignored-tests/pom.xml
+++ b/modules/ignored-tests/pom.xml
@@ -171,7 +171,7 @@
         <dependency>
             <groupId>commons-io</groupId>
             <artifactId>commons-io</artifactId>
-            <version>2.4</version>
+            <version>${commons.io.version}</version>
             <scope>test</scope>
         </dependency>
 
diff --git a/modules/indexing/pom.xml b/modules/indexing/pom.xml
index 829b039..9770bd9 100644
--- a/modules/indexing/pom.xml
+++ b/modules/indexing/pom.xml
@@ -89,7 +89,7 @@
         <dependency>
             <groupId>commons-io</groupId>
             <artifactId>commons-io</artifactId>
-            <version>2.4</version>
+            <version>${commons.io.version}</version>
             <scope>test</scope>
         </dependency>
 
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 885fa7a..01c508a 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
@@ -33,6 +33,7 @@ import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.IgniteSystemProperties;
 import org.apache.ignite.cache.query.annotations.QuerySqlFunction;
... 1733 lines suppressed ...