You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ag...@apache.org on 2017/04/07 12:41:57 UTC

[7/9] ignite git commit: IGNITE-4536 - Update metrics for memory policies.

http://git-wip-us.apache.org/repos/asf/ignite/blob/3f90a9c0/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/LocalCacheOffHeapAndSwapMetricsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/LocalCacheOffHeapAndSwapMetricsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/LocalCacheOffHeapAndSwapMetricsSelfTest.java
index f6becd0..a9fd6cb 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/LocalCacheOffHeapAndSwapMetricsSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/LocalCacheOffHeapAndSwapMetricsSelfTest.java
@@ -185,77 +185,6 @@ public class LocalCacheOffHeapAndSwapMetricsSelfTest extends GridCommonAbstractT
     /**
      * @throws Exception if failed.
      */
-    public void testSwapMetrics() throws Exception {
-        fail("https://issues.apache.org/jira/browse/IGNITE-4536");
-
-        createCache(CacheMemoryMode.ONHEAP_TIERED, -1, true);
-
-        for (int i = 0; i < KEYS_CNT; i++)
-            cache.put(i, i);
-
-        printStat();
-
-        assertEquals(cache.localMetrics().getCacheEvictions(), cache.localMetrics().getSwapPuts());
-        assertEquals(KEYS_CNT, cache.localMetrics().getSwapGets());
-        assertEquals(0, cache.localMetrics().getSwapHits());
-        assertEquals(0f, cache.localMetrics().getSwapHitPercentage());
-        assertEquals(KEYS_CNT, cache.localMetrics().getSwapMisses());
-        assertEquals(100f, cache.localMetrics().getSwapMissPercentage());
-        assertEquals(0, cache.localMetrics().getSwapRemovals());
-
-        assertEquals(cache.localMetrics().getCacheEvictions(), cache.localMetrics().getSwapEntriesCount());
-
-        for (int i = 0; i < KEYS_CNT; i++)
-            cache.get(i);
-
-        printStat();
-
-        assertEquals(cache.localMetrics().getCacheEvictions(), cache.localMetrics().getSwapPuts());
-        assertEquals(KEYS_CNT * 2, cache.localMetrics().getSwapGets());
-        assertEquals(KEYS_CNT, cache.localMetrics().getSwapHits());
-        assertEquals(100 * KEYS_CNT / (KEYS_CNT * 2.0), cache.localMetrics().getSwapHitPercentage(), 0.1);
-        assertEquals(KEYS_CNT, cache.localMetrics().getSwapMisses());
-        assertEquals(100 * KEYS_CNT / (KEYS_CNT * 2.0), cache.localMetrics().getSwapMissPercentage(), 0.1);
-        assertEquals(KEYS_CNT, cache.localMetrics().getSwapRemovals());
-
-        assertEquals(KEYS_CNT - MAX_SIZE, cache.localMetrics().getSwapEntriesCount());
-
-        for (int i = KEYS_CNT; i < KEYS_CNT * 2; i++)
-            cache.get(i);
-
-        printStat();
-
-        assertEquals(cache.localMetrics().getCacheEvictions(), cache.localMetrics().getSwapPuts());
-        assertEquals(KEYS_CNT * 3, cache.localMetrics().getSwapGets());
-        assertEquals(KEYS_CNT, cache.localMetrics().getSwapHits());
-        assertEquals(100 / 3.0, cache.localMetrics().getSwapHitPercentage(), 0.1);
-        assertEquals(KEYS_CNT * 2, cache.localMetrics().getSwapMisses());
-        assertEquals(100 - (100 / 3.0), cache.localMetrics().getSwapMissPercentage(), 0.1);
-        assertEquals(KEYS_CNT, cache.localMetrics().getSwapRemovals());
-
-        assertEquals(KEYS_CNT - MAX_SIZE, cache.localMetrics().getSwapEntriesCount());
-
-        for (int i = 0; i < KEYS_CNT; i++)
-            cache.remove(i);
-
-        printStat();
-
-        assertEquals(cache.localMetrics().getCacheEvictions(), cache.localMetrics().getSwapPuts());
-        assertEquals(KEYS_CNT * 4 - MAX_SIZE, cache.localMetrics().getSwapGets());
-        assertEquals(KEYS_CNT * 2 - MAX_SIZE, cache.localMetrics().getSwapHits());
-        assertEquals(100 * (KEYS_CNT * 2.0 - MAX_SIZE) / (KEYS_CNT * 4.0 - MAX_SIZE),
-            cache.localMetrics().getSwapHitPercentage(), 0.1);
-        assertEquals(KEYS_CNT * 2, cache.localMetrics().getSwapMisses());
-        assertEquals(100 * KEYS_CNT * 2.0 / (KEYS_CNT * 4.0 - MAX_SIZE),
-            cache.localMetrics().getSwapMissPercentage(), 0.1);
-        assertEquals(KEYS_CNT * 2 - MAX_SIZE, cache.localMetrics().getSwapRemovals());
-
-        assertEquals(0, cache.localMetrics().getSwapEntriesCount());
-    }
-
-    /**
-     * @throws Exception if failed.
-     */
     public void testOffHeapAndSwapMetrics() throws Exception {
         fail("https://issues.apache.org/jira/browse/IGNITE-4536");
 
@@ -279,16 +208,6 @@ public class LocalCacheOffHeapAndSwapMetricsSelfTest extends GridCommonAbstractT
         assertEquals(OFFHEAP_MAX_CNT, cache.localMetrics().getOffHeapPrimaryEntriesCount());
         assertEquals(0, cache.localMetrics().getOffHeapBackupEntriesCount());
 
-        assertEquals(cache.localMetrics().getOffHeapEvictions(), cache.localMetrics().getSwapPuts());
-        assertEquals(KEYS_CNT, cache.localMetrics().getSwapGets());
-        assertEquals(0, cache.localMetrics().getSwapHits());
-        assertEquals(0f, cache.localMetrics().getSwapHitPercentage());
-        assertEquals(KEYS_CNT, cache.localMetrics().getSwapMisses());
-        assertEquals(100f, cache.localMetrics().getSwapMissPercentage());
-        assertEquals(0, cache.localMetrics().getSwapRemovals());
-
-        assertEquals(cache.localMetrics().getOffHeapEvictions(), cache.localMetrics().getSwapEntriesCount());
-
         for (int i = 0; i < KEYS_CNT; i++)
             cache.get(i);
 
@@ -307,16 +226,6 @@ public class LocalCacheOffHeapAndSwapMetricsSelfTest extends GridCommonAbstractT
         assertEquals(OFFHEAP_MAX_CNT, cache.localMetrics().getOffHeapPrimaryEntriesCount());
         assertEquals(0, cache.localMetrics().getOffHeapBackupEntriesCount());
 
-        assertEquals(cache.localMetrics().getOffHeapEvictions(), cache.localMetrics().getSwapPuts());
-        assertEquals(KEYS_CNT * 2, cache.localMetrics().getSwapGets());
-        assertEquals(KEYS_CNT, cache.localMetrics().getSwapHits());
-        assertEquals(100 * KEYS_CNT / (KEYS_CNT * 2.0), cache.localMetrics().getSwapHitPercentage(), 0.1);
-        assertEquals(KEYS_CNT, cache.localMetrics().getSwapMisses());
-        assertEquals(100 * KEYS_CNT / (KEYS_CNT * 2.0), cache.localMetrics().getSwapMissPercentage(), 0.1);
-        assertEquals(KEYS_CNT, cache.localMetrics().getSwapRemovals());
-
-        assertEquals(KEYS_CNT - MAX_SIZE - OFFHEAP_MAX_CNT, cache.localMetrics().getSwapEntriesCount());
-
         for (int i = KEYS_CNT; i < KEYS_CNT * 2; i++)
             cache.get(i);
 
@@ -335,16 +244,6 @@ public class LocalCacheOffHeapAndSwapMetricsSelfTest extends GridCommonAbstractT
         assertEquals(OFFHEAP_MAX_CNT, cache.localMetrics().getOffHeapPrimaryEntriesCount());
         assertEquals(0, cache.localMetrics().getOffHeapBackupEntriesCount());
 
-        assertEquals(cache.localMetrics().getOffHeapEvictions(), cache.localMetrics().getSwapPuts());
-        assertEquals(KEYS_CNT * 3, cache.localMetrics().getSwapGets());
-        assertEquals(KEYS_CNT, cache.localMetrics().getSwapHits());
-        assertEquals(100 / 3.0, cache.localMetrics().getSwapHitPercentage(), 0.1);
-        assertEquals(KEYS_CNT * 2, cache.localMetrics().getSwapMisses());
-        assertEquals(100 - (100 / 3.0), cache.localMetrics().getSwapMissPercentage(), 0.1);
-        assertEquals(KEYS_CNT, cache.localMetrics().getSwapRemovals());
-
-        assertEquals(KEYS_CNT - MAX_SIZE - OFFHEAP_MAX_CNT, cache.localMetrics().getSwapEntriesCount());
-
         for (int i = 0; i < KEYS_CNT; i++)
             cache.remove(i);
 
@@ -364,18 +263,6 @@ public class LocalCacheOffHeapAndSwapMetricsSelfTest extends GridCommonAbstractT
         assertEquals(0, cache.localMetrics().getOffHeapEntriesCount());
         assertEquals(0, cache.localMetrics().getOffHeapPrimaryEntriesCount());
         assertEquals(0, cache.localMetrics().getOffHeapBackupEntriesCount());
-
-        assertEquals(cache.localMetrics().getOffHeapEvictions(), cache.localMetrics().getSwapPuts());
-        assertEquals(KEYS_CNT * 4 - MAX_SIZE - OFFHEAP_MAX_CNT, cache.localMetrics().getSwapGets());
-        assertEquals(KEYS_CNT * 2 - MAX_SIZE - OFFHEAP_MAX_CNT, cache.localMetrics().getSwapHits());
-        assertEquals(100 * (KEYS_CNT * 2.0 - MAX_SIZE - OFFHEAP_MAX_CNT) / (KEYS_CNT * 4.0 - MAX_SIZE - OFFHEAP_MAX_CNT),
-            cache.localMetrics().getSwapHitPercentage(), 0.1);
-        assertEquals(KEYS_CNT * 2, cache.localMetrics().getSwapMisses());
-        assertEquals(100 * KEYS_CNT * 2.0 / (KEYS_CNT * 4.0 - MAX_SIZE - OFFHEAP_MAX_CNT),
-            cache.localMetrics().getSwapMissPercentage(), 0.1);
-        assertEquals(KEYS_CNT * 2 - MAX_SIZE - OFFHEAP_MAX_CNT, cache.localMetrics().getSwapRemovals());
-
-        assertEquals(0, cache.localMetrics().getSwapEntriesCount());
     }
 
     /**
@@ -487,16 +374,6 @@ public class LocalCacheOffHeapAndSwapMetricsSelfTest extends GridCommonAbstractT
         assertEquals(OFFHEAP_MAX_CNT, cache.metrics().getOffHeapPrimaryEntriesCount());
         assertEquals(0, cache.metrics().getOffHeapBackupEntriesCount());
 
-        assertEquals(cache.metrics().getOffHeapEvictions(), cache.metrics().getSwapPuts());
-        assertEquals(KEYS_CNT, cache.metrics().getSwapGets());
-        assertEquals(0, cache.metrics().getSwapHits());
-        assertEquals(0f, cache.metrics().getSwapHitPercentage());
-        assertEquals(KEYS_CNT, cache.metrics().getSwapMisses());
-        assertEquals(100f, cache.metrics().getSwapMissPercentage());
-        assertEquals(0, cache.metrics().getSwapRemovals());
-
-        assertEquals(cache.metrics().getOffHeapEvictions(), cache.metrics().getSwapEntriesCount());
-
         for (int i = 0; i < KEYS_CNT; i++)
             cache.get(i);
 
@@ -515,16 +392,6 @@ public class LocalCacheOffHeapAndSwapMetricsSelfTest extends GridCommonAbstractT
         assertEquals(OFFHEAP_MAX_CNT, cache.metrics().getOffHeapPrimaryEntriesCount());
         assertEquals(0, cache.metrics().getOffHeapBackupEntriesCount());
 
-        assertEquals(cache.metrics().getOffHeapEvictions(), cache.metrics().getSwapPuts());
-        assertEquals(KEYS_CNT * 2, cache.metrics().getSwapGets());
-        assertEquals(KEYS_CNT, cache.metrics().getSwapHits());
-        assertEquals(100 * KEYS_CNT / (KEYS_CNT * 2.0), cache.metrics().getSwapHitPercentage(), 0.1);
-        assertEquals(KEYS_CNT, cache.metrics().getSwapMisses());
-        assertEquals(100 * KEYS_CNT / (KEYS_CNT * 2.0), cache.metrics().getSwapMissPercentage(), 0.1);
-        assertEquals(KEYS_CNT, cache.metrics().getSwapRemovals());
-
-        assertEquals(KEYS_CNT - OFFHEAP_MAX_CNT, cache.metrics().getSwapEntriesCount());
-
         for (int i = KEYS_CNT; i < KEYS_CNT * 2; i++)
             cache.get(i);
 
@@ -544,16 +411,6 @@ public class LocalCacheOffHeapAndSwapMetricsSelfTest extends GridCommonAbstractT
         assertEquals(OFFHEAP_MAX_CNT, cache.metrics().getOffHeapPrimaryEntriesCount());
         assertEquals(0, cache.metrics().getOffHeapBackupEntriesCount());
 
-        assertEquals(cache.metrics().getOffHeapEvictions(), cache.metrics().getSwapPuts());
-        assertEquals(KEYS_CNT * 3, cache.metrics().getSwapGets());
-        assertEquals(KEYS_CNT, cache.metrics().getSwapHits());
-        assertEquals(100 / 3.0, cache.metrics().getSwapHitPercentage(), 0.1);
-        assertEquals(KEYS_CNT * 2, cache.metrics().getSwapMisses());
-        assertEquals(100 - (100 / 3.0), cache.metrics().getSwapMissPercentage(), 0.1);
-        assertEquals(KEYS_CNT, cache.metrics().getSwapRemovals());
-
-        assertEquals(KEYS_CNT - OFFHEAP_MAX_CNT, cache.metrics().getSwapEntriesCount());
-
         for (int i = 0; i < KEYS_CNT; i++)
             cache.remove(i);
 
@@ -573,18 +430,6 @@ public class LocalCacheOffHeapAndSwapMetricsSelfTest extends GridCommonAbstractT
         assertEquals(0, cache.metrics().getOffHeapEntriesCount());
         assertEquals(0, cache.metrics().getOffHeapPrimaryEntriesCount());
         assertEquals(0, cache.metrics().getOffHeapBackupEntriesCount());
-
-        assertEquals(cache.metrics().getOffHeapEvictions(), cache.metrics().getSwapPuts());
-        assertEquals(KEYS_CNT * 4 - OFFHEAP_MAX_CNT, cache.metrics().getSwapGets());
-        assertEquals(KEYS_CNT * 2 - OFFHEAP_MAX_CNT, cache.metrics().getSwapHits());
-        assertEquals(100 * (KEYS_CNT * 2.0 - OFFHEAP_MAX_CNT) / (KEYS_CNT * 4.0 - OFFHEAP_MAX_CNT),
-            cache.metrics().getSwapHitPercentage(), 0.1);
-        assertEquals(KEYS_CNT * 2, cache.metrics().getSwapMisses());
-        assertEquals(100 * KEYS_CNT * 2.0 / (KEYS_CNT * 4.0 - OFFHEAP_MAX_CNT),
-            cache.metrics().getSwapMissPercentage(), 0.1);
-        assertEquals(KEYS_CNT * 2 - OFFHEAP_MAX_CNT, cache.metrics().getSwapRemovals());
-
-        assertEquals(0, cache.metrics().getSwapEntriesCount());
     }
 
     /**
@@ -593,35 +438,26 @@ public class LocalCacheOffHeapAndSwapMetricsSelfTest extends GridCommonAbstractT
     protected void printStat() {
         System.out.println("!!! -------------------------------------------------------");
         System.out.println("!!! Puts: cache = " + cache.localMetrics().getCachePuts() +
-            ", offheap = " + cache.localMetrics().getOffHeapPuts() +
-            ", swap = " + cache.localMetrics().getSwapPuts());
+            ", offheap = " + cache.localMetrics().getOffHeapPuts());
         System.out.println("!!! Gets: cache = " + cache.localMetrics().getCacheGets() +
-            ", offheap = " + cache.localMetrics().getOffHeapGets() +
-            ", swap = " + cache.localMetrics().getSwapGets());
+            ", offheap = " + cache.localMetrics().getOffHeapGets());
         System.out.println("!!! Removes: cache = " + cache.localMetrics().getCacheRemovals() +
-            ", offheap = " + cache.localMetrics().getOffHeapRemovals() +
-            ", swap = " + cache.localMetrics().getSwapRemovals());
+            ", offheap = " + cache.localMetrics().getOffHeapRemovals());
         System.out.println("!!! Evictions: cache = " + cache.localMetrics().getCacheEvictions() +
             ", offheap = " + cache.localMetrics().getOffHeapEvictions() +
             ", swap = none" );
         System.out.println("!!! Hits: cache = " + cache.localMetrics().getCacheHits() +
-            ", offheap = " + cache.localMetrics().getOffHeapHits() +
-            ", swap = " + cache.localMetrics().getSwapHits());
+            ", offheap = " + cache.localMetrics().getOffHeapHits());
         System.out.println("!!! Hit(%): cache = " + cache.localMetrics().getCacheHitPercentage() +
-            ", offheap = " + cache.localMetrics().getOffHeapHitPercentage() +
-            ", swap = " + cache.localMetrics().getSwapHitPercentage());
+            ", offheap = " + cache.localMetrics().getOffHeapHitPercentage());
         System.out.println("!!! Misses: cache = " + cache.localMetrics().getCacheMisses() +
-            ", offheap = " + cache.localMetrics().getOffHeapMisses() +
-            ", swap = " + cache.localMetrics().getSwapMisses());
+            ", offheap = " + cache.localMetrics().getOffHeapMisses());
         System.out.println("!!! Miss(%): cache = " + cache.localMetrics().getCacheMissPercentage() +
-            ", offheap = " + cache.localMetrics().getOffHeapMissPercentage() +
-            ", swap = " + cache.localMetrics().getSwapMissPercentage());
+            ", offheap = " + cache.localMetrics().getOffHeapMissPercentage());
         System.out.println("!!! Entries: cache = " + cache.localMetrics().getSize() +
-            ", offheap = " + cache.localMetrics().getOffHeapEntriesCount() +
-            ", swap = " + cache.localMetrics().getSwapEntriesCount());
+            ", offheap = " + cache.localMetrics().getOffHeapEntriesCount());
         System.out.println("!!! Size: cache = none" +
-            ", offheap = " + cache.localMetrics().getOffHeapAllocatedSize() +
-            ", swap = " + cache.localMetrics().getSwapSize());
+            ", offheap = " + cache.localMetrics().getOffHeapAllocatedSize());
         System.out.println();
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/3f90a9c0/modules/core/src/test/java/org/apache/ignite/internal/processors/database/BPlusTreeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/BPlusTreeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/BPlusTreeSelfTest.java
index 4c6fe4c..9e5ca70 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/BPlusTreeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/BPlusTreeSelfTest.java
@@ -39,6 +39,7 @@ import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.pagemem.PageUtils;
 import org.apache.ignite.internal.pagemem.impl.PageMemoryNoStoreImpl;
 import org.apache.ignite.internal.processors.cache.database.DataStructure;
+import org.apache.ignite.internal.processors.cache.database.MemoryMetricsImpl;
 import org.apache.ignite.internal.processors.cache.database.tree.BPlusTree;
 import org.apache.ignite.internal.processors.cache.database.tree.io.BPlusIO;
 import org.apache.ignite.internal.processors.cache.database.tree.io.BPlusInnerIO;
@@ -1677,7 +1678,7 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
         for (int i = 0; i < sizes.length; i++)
             sizes[i] = 1024 * MB / CPUS;
 
-        PageMemory pageMem = new PageMemoryNoStoreImpl(log, new UnsafeMemoryProvider(sizes), null, PAGE_SIZE, true);
+        PageMemory pageMem = new PageMemoryNoStoreImpl(log, new UnsafeMemoryProvider(sizes), null, PAGE_SIZE, new MemoryMetricsImpl(null), true);
 
         pageMem.start();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/3f90a9c0/modules/core/src/test/java/org/apache/ignite/internal/processors/database/FreeListImplSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/FreeListImplSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/FreeListImplSelfTest.java
index 471a05c..7fccef1 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/FreeListImplSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/FreeListImplSelfTest.java
@@ -38,6 +38,7 @@ import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.CacheObjectContext;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 import org.apache.ignite.internal.processors.cache.database.CacheDataRow;
+import org.apache.ignite.internal.processors.cache.database.MemoryMetricsImpl;
 import org.apache.ignite.internal.processors.cache.database.freelist.FreeList;
 import org.apache.ignite.internal.processors.cache.database.freelist.FreeListImpl;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
@@ -316,7 +317,7 @@ public class FreeListImplSelfTest extends GridCommonAbstractTest {
         for (int i = 0; i < sizes.length; i++)
             sizes[i] = 1024 * MB / CPUS;
 
-        PageMemory pageMem = new PageMemoryNoStoreImpl(log, new UnsafeMemoryProvider(sizes), null, pageSize, true);
+        PageMemory pageMem = new PageMemoryNoStoreImpl(log, new UnsafeMemoryProvider(sizes), null, pageSize, new MemoryMetricsImpl(null), true);
 
         pageMem.start();
 
@@ -333,7 +334,7 @@ public class FreeListImplSelfTest extends GridCommonAbstractTest {
 
         long metaPageId = pageMem.allocatePage(1, 1, PageIdAllocator.FLAG_DATA);
 
-        return new FreeListImpl(1, "freelist", pageMem, null, null, metaPageId, true);
+        return new FreeListImpl(1, "freelist", pageMem, new MemoryMetricsImpl(null), null, null, metaPageId, true);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/3f90a9c0/modules/core/src/test/java/org/apache/ignite/internal/processors/database/MemoryMetricsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/MemoryMetricsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/MemoryMetricsSelfTest.java
new file mode 100644
index 0000000..5347a23
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/MemoryMetricsSelfTest.java
@@ -0,0 +1,313 @@
+/*
+ * 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.database;
+
+import java.util.concurrent.CountDownLatch;
+import org.apache.ignite.MemoryMetrics;
+import org.apache.ignite.internal.processors.cache.database.MemoryMetricsImpl;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+import static java.lang.Thread.sleep;
+
+/**
+ *
+ */
+public class MemoryMetricsSelfTest extends GridCommonAbstractTest {
+    /** */
+    private MemoryMetricsImpl memMetrics;
+
+    /** */
+    private int threadsCnt = 1;
+
+    /** */
+    private Thread[] allocationThreads;
+
+    /** */
+    private Thread watcherThread;
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        memMetrics = new MemoryMetricsImpl(null);
+
+        memMetrics.enableMetrics();
+    }
+
+    /**
+     * Test for allocationRate metric in single-threaded mode.
+     * @throws Exception if any happens during test.
+     */
+    public void testAllocationRateSingleThreaded() throws Exception {
+        threadsCnt = 1;
+        memMetrics.rateTimeInterval(10);
+
+        CountDownLatch startLatch = new CountDownLatch(1);
+
+        startAllocationThreads(startLatch, 340, 50);
+        AllocationRateWatcher watcher = startWatcherThread(startLatch, 20);
+
+        startLatch.countDown();
+
+        joinAllThreads();
+
+        assertEquals(4, watcher.rateDropsCntr);
+    }
+
+    /**
+     * Test for allocationRate metric in multi-threaded mode with short silent period in the middle of the test.
+     * @throws Exception if any happens during test.
+     */
+    public void testAllocationRateMultiThreaded() throws Exception {
+        threadsCnt = 4;
+        memMetrics.rateTimeInterval(5);
+
+        CountDownLatch startLatch = new CountDownLatch(1);
+
+        startAllocationThreads(startLatch, 7_800, 1);
+
+        AllocationRateWatcher watcher = startWatcherThread(startLatch, 20);
+
+        startLatch.countDown();
+
+        joinAllocationThreads();
+
+        assertEquals(4, watcher.rateDropsCntr);
+
+        sleep(3);
+
+        threadsCnt = 8;
+
+        CountDownLatch restartLatch = new CountDownLatch(1);
+
+        startAllocationThreads(restartLatch, 8_000, 1);
+
+        restartLatch.countDown();
+
+        joinAllThreads();
+
+        assertTrue(watcher.rateDropsCntr > 4);
+    }
+
+    /**
+     * Test verifies that allocationRate calculation algorithm survives setting new values to rateTimeInterval parameter.
+     * @throws Exception if any happens during test.
+     */
+    public void testAllocationRateTimeIntervalConcurrentChange() throws Exception {
+        threadsCnt = 5;
+        memMetrics.rateTimeInterval(5);
+
+        CountDownLatch startLatch = new CountDownLatch(1);
+
+        startAllocationThreads(startLatch, 10_000, 1);
+
+        AllocationRateWatcher watcher = startWatcherThread(startLatch, 20);
+
+        startLatch.countDown();
+
+        for (int i = 0; i < 10; i++) {
+            Thread.sleep(25);
+
+            memMetrics.rateTimeInterval((2 + i * 5) % 3 + 1);
+        }
+
+        joinAllThreads();
+
+        assertTrue(watcher.rateDropsCntr > 4);
+    }
+
+    /**
+     *
+     * @throws Exception if any happens during test.
+     */
+    public void testAllocationRateSubintervalsConcurrentChange() throws Exception {
+        threadsCnt = 5;
+        memMetrics.rateTimeInterval(5);
+
+        CountDownLatch startLatch = new CountDownLatch(1);
+
+        startAllocationThreads(startLatch, 10_000, 1);
+
+        AllocationRateWatcher watcher = startWatcherThread(startLatch, 20);
+
+        startLatch.countDown();
+
+        for (int i = 0; i < 10; i++) {
+            Thread.sleep(25);
+
+            memMetrics.subIntervals((2 + i * 5) % 3 + 1);
+        }
+
+        joinAllThreads();
+
+        assertTrue(watcher.rateDropsCntr > 4);
+    }
+
+    /**
+     * @param startLatch Start latch.
+     * @param watchingDelay Watching delay.
+     */
+    private AllocationRateWatcher startWatcherThread(CountDownLatch startLatch, int watchingDelay) {
+        AllocationRateWatcher watcher = new AllocationRateWatcher(startLatch, memMetrics, watchingDelay);
+
+        watcherThread = new Thread(watcher);
+
+        watcherThread.start();
+
+        return watcher;
+    }
+
+    /**
+     * @param startLatch Start latch.
+     * @param iterationsCnt Iterations count.
+     * @param allocationsDelay Allocations delay.
+     */
+    private void startAllocationThreads(CountDownLatch startLatch, int iterationsCnt, int allocationsDelay) {
+        assert threadsCnt > 0;
+
+        allocationThreads = new Thread[threadsCnt];
+
+        for (int i = 0; i < threadsCnt; i++) {
+            AllocationsIncrementer inc = new AllocationsIncrementer(startLatch, memMetrics, iterationsCnt, allocationsDelay);
+
+            Thread incThread = new Thread(inc);
+            incThread.start();
+
+            allocationThreads[i] = incThread;
+        }
+    }
+
+    /**
+     *
+     */
+    private void joinAllThreads() throws Exception {
+        joinAllocationThreads();
+
+        watcherThread.interrupt();
+        watcherThread.join();
+    }
+
+    /**
+     *
+     */
+    private void joinAllocationThreads() throws Exception {
+        assert allocationThreads != null;
+        assert allocationThreads.length > 0;
+
+        for (Thread allocationThread : allocationThreads)
+            allocationThread.join();
+    }
+
+    /**
+     *
+     */
+    private static class AllocationsIncrementer implements Runnable {
+        /** */
+        private final CountDownLatch startLatch;
+
+        /** */
+        private final MemoryMetricsImpl memMetrics;
+
+        /** */
+        private final int iterationsCnt;
+
+        /** */
+        private final int delay;
+
+        /**
+         * @param startLatch Start latch.
+         * @param memMetrics Mem metrics.
+         * @param iterationsCnt Iterations count.
+         * @param delay Delay.
+         */
+        private AllocationsIncrementer(CountDownLatch startLatch, MemoryMetricsImpl memMetrics, int iterationsCnt, int delay) {
+            this.startLatch = startLatch;
+            this.memMetrics = memMetrics;
+            this.iterationsCnt = iterationsCnt;
+            this.delay = delay;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void run() {
+            try {
+                startLatch.await();
+
+                for (int i = 0; i < iterationsCnt; i++) {
+                    memMetrics.incrementTotalAllocatedPages();
+
+                    sleep(delay);
+                }
+            }
+            catch (InterruptedException ignore) {
+                // No-op.
+            }
+            catch (Exception e) {
+                e.printStackTrace();
+            }
+        }
+    }
+
+    /**
+     *
+     */
+    private static class AllocationRateWatcher implements Runnable {
+        /** */
+        private volatile int rateDropsCntr;
+
+        /** */
+        private final CountDownLatch startLatch;
+
+        /** */
+        private final MemoryMetrics memMetrics;
+
+        /** */
+        private final int delay;
+
+        /**
+         * @param startLatch Start latch.
+         * @param memMetrics Mem metrics.
+         * @param delay Delay.
+         */
+        private AllocationRateWatcher(CountDownLatch startLatch, MemoryMetrics memMetrics, int delay) {
+            this.startLatch = startLatch;
+            this.memMetrics = memMetrics;
+            this.delay = delay;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void run() {
+            try {
+                startLatch.await();
+
+                float prevRate = 0;
+
+                while (!Thread.currentThread().isInterrupted()) {
+                    if (prevRate > memMetrics.getAllocationRate())
+                        rateDropsCntr++;
+
+                    prevRate = memMetrics.getAllocationRate();
+
+                    sleep(delay);
+                }
+            }
+            catch (InterruptedException ignore) {
+                // No-op.
+            }
+            catch (Exception e) {
+                e.printStackTrace();
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/3f90a9c0/modules/core/src/test/java/org/apache/ignite/internal/processors/database/MetadataStorageSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/MetadataStorageSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/MetadataStorageSelfTest.java
index 7eeda4b..8b49d03 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/MetadataStorageSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/MetadataStorageSelfTest.java
@@ -21,6 +21,7 @@ import org.apache.ignite.internal.mem.DirectMemoryProvider;
 import org.apache.ignite.internal.pagemem.FullPageId;
 import org.apache.ignite.internal.pagemem.PageIdAllocator;
 import org.apache.ignite.internal.pagemem.impl.PageMemoryNoStoreImpl;
+import org.apache.ignite.internal.processors.cache.database.MemoryMetricsImpl;
 import org.apache.ignite.internal.processors.cache.database.MetadataStorage;
 import org.apache.ignite.internal.mem.file.MappedFileMemoryProvider;
 import org.apache.ignite.internal.pagemem.PageMemory;
@@ -160,6 +161,6 @@ public class MetadataStorageSelfTest extends GridCommonAbstractTest {
 
         DirectMemoryProvider provider = new MappedFileMemoryProvider(log(), allocationPath, clean, sizes);
 
-        return new PageMemoryNoStoreImpl(log, provider, null, PAGE_SIZE, true);
+        return new PageMemoryNoStoreImpl(log, provider, null, PAGE_SIZE, new MemoryMetricsImpl(null), true);
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/3f90a9c0/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsIgniteMock.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsIgniteMock.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsIgniteMock.java
index 2a522fa..cb988e3 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsIgniteMock.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsIgniteMock.java
@@ -38,6 +38,7 @@ import org.apache.ignite.IgniteSemaphore;
 import org.apache.ignite.IgniteServices;
 import org.apache.ignite.IgniteSet;
 import org.apache.ignite.IgniteTransactions;
+import org.apache.ignite.MemoryMetrics;
 import org.apache.ignite.cache.affinity.Affinity;
 import org.apache.ignite.cluster.ClusterGroup;
 import org.apache.ignite.cluster.ClusterNode;
@@ -528,6 +529,13 @@ public class IgfsIgniteMock implements IgniteEx {
         throwUnsupported();
     }
 
+    /** {@inheritDoc} */
+    @Override public Collection<MemoryMetrics> memoryMetrics() {
+        throwUnsupported();
+
+        return null;
+    }
+
     /**
      * Throw {@link UnsupportedOperationException}.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/3f90a9c0/modules/core/src/test/java/org/apache/ignite/platform/PlatformCacheWriteMetricsTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/platform/PlatformCacheWriteMetricsTask.java b/modules/core/src/test/java/org/apache/ignite/platform/PlatformCacheWriteMetricsTask.java
index 593f26d..da013bd 100644
--- a/modules/core/src/test/java/org/apache/ignite/platform/PlatformCacheWriteMetricsTask.java
+++ b/modules/core/src/test/java/org/apache/ignite/platform/PlatformCacheWriteMetricsTask.java
@@ -175,128 +175,78 @@ public class PlatformCacheWriteMetricsTask extends ComputeTaskAdapter<Long, Obje
         }
 
         /** {@inheritDoc} */
-        @Override public long getOverflowSize() {
-            return 16;
-        }
-
-        /** {@inheritDoc} */
         @Override public long getOffHeapGets() {
-            return 17;
+            return 16;
         }
 
         /** {@inheritDoc} */
         @Override public long getOffHeapPuts() {
-            return 18;
+            return 17;
         }
 
         /** {@inheritDoc} */
         @Override public long getOffHeapRemovals() {
-            return 19;
+            return 18;
         }
 
         /** {@inheritDoc} */
         @Override public long getOffHeapEvictions() {
-            return 20;
+            return 19;
         }
 
         /** {@inheritDoc} */
         @Override public long getOffHeapHits() {
-            return 21;
+            return 20;
         }
 
         /** {@inheritDoc} */
         @Override public float getOffHeapHitPercentage() {
-            return 22;
+            return 21;
         }
 
         /** {@inheritDoc} */
         @Override public long getOffHeapMisses() {
-            return 23;
+            return 22;
         }
 
         /** {@inheritDoc} */
         @Override public float getOffHeapMissPercentage() {
-            return 24;
+            return 23;
         }
 
         /** {@inheritDoc} */
         @Override public long getOffHeapEntriesCount() {
-            return 25;
+            return 24;
         }
 
         /** {@inheritDoc} */
         @Override public long getOffHeapPrimaryEntriesCount() {
-            return 26;
+            return 25;
         }
 
         /** {@inheritDoc} */
         @Override public long getOffHeapBackupEntriesCount() {
-            return 27;
+            return 26;
         }
 
         /** {@inheritDoc} */
         @Override public long getOffHeapAllocatedSize() {
-            return 28;
+            return 27;
         }
 
         /** {@inheritDoc} */
         @Override public long getOffHeapMaxSize() {
-            return 29;
-        }
-
-        /** {@inheritDoc} */
-        @Override public long getSwapGets() {
-            return 30;
-        }
-
-        /** {@inheritDoc} */
-        @Override public long getSwapPuts() {
-            return 31;
-        }
-
-        /** {@inheritDoc} */
-        @Override public long getSwapRemovals() {
-            return 32;
-        }
-
-        /** {@inheritDoc} */
-        @Override public long getSwapHits() {
-            return 33;
-        }
-
-        /** {@inheritDoc} */
-        @Override public long getSwapMisses() {
-            return 34;
-        }
-
-        /** {@inheritDoc} */
-        @Override public long getSwapEntriesCount() {
-            return 35;
-        }
-
-        /** {@inheritDoc} */
-        @Override public long getSwapSize() {
-            return 36;
-        }
-
-        /** {@inheritDoc} */
-        @Override public float getSwapHitPercentage() {
-            return 37;
-        }
-
-        /** {@inheritDoc} */
-        @Override public float getSwapMissPercentage() {
-            return 38;
+            return 28;
         }
 
         /** {@inheritDoc} */
         @Override public int getSize() {
-            return 39;
+            return 29;
         }
 
         /** {@inheritDoc} */
         @Override public int getKeySize() {
-            return 40;
+            return 30;
         }
 
         /** {@inheritDoc} */
@@ -306,77 +256,77 @@ public class PlatformCacheWriteMetricsTask extends ComputeTaskAdapter<Long, Obje
 
         /** {@inheritDoc} */
         @Override public int getDhtEvictQueueCurrentSize() {
-            return 41;
+            return 31;
         }
 
         /** {@inheritDoc} */
         @Override public int getTxThreadMapSize() {
-            return 42;
+            return 32;
         }
 
         /** {@inheritDoc} */
         @Override public int getTxXidMapSize() {
-            return 43;
+            return 33;
         }
 
         /** {@inheritDoc} */
         @Override public int getTxCommitQueueSize() {
-            return 44;
+            return 34;
         }
 
         /** {@inheritDoc} */
         @Override public int getTxPrepareQueueSize() {
-            return 45;
+            return 35;
         }
 
         /** {@inheritDoc} */
         @Override public int getTxStartVersionCountsSize() {
-            return 46;
+            return 36;
         }
 
         /** {@inheritDoc} */
         @Override public int getTxCommittedVersionsSize() {
-            return 47;
+            return 37;
         }
 
         /** {@inheritDoc} */
         @Override public int getTxRolledbackVersionsSize() {
-            return 48;
+            return 38;
         }
 
         /** {@inheritDoc} */
         @Override public int getTxDhtThreadMapSize() {
-            return 49;
+            return 39;
         }
 
         /** {@inheritDoc} */
         @Override public int getTxDhtXidMapSize() {
-            return 50;
+            return 40;
         }
 
         /** {@inheritDoc} */
         @Override public int getTxDhtCommitQueueSize() {
-            return 51;
+            return 41;
         }
 
         /** {@inheritDoc} */
         @Override public int getTxDhtPrepareQueueSize() {
-            return 52;
+            return 42;
         }
 
         /** {@inheritDoc} */
         @Override public int getTxDhtStartVersionCountsSize() {
-            return 53;
+            return 43;
         }
 
         /** {@inheritDoc} */
         @Override public int getTxDhtCommittedVersionsSize() {
-            return 54;
+            return 44;
         }
 
         /** {@inheritDoc} */
         @Override public int getTxDhtRolledbackVersionsSize() {
-            return 55;
+            return 45;
         }
 
         /** {@inheritDoc} */
@@ -386,42 +336,42 @@ public class PlatformCacheWriteMetricsTask extends ComputeTaskAdapter<Long, Obje
 
         /** {@inheritDoc} */
         @Override public int getWriteBehindFlushSize() {
-            return 56;
+            return 46;
         }
 
         /** {@inheritDoc} */
         @Override public int getWriteBehindFlushThreadCount() {
-            return 57;
+            return 47;
         }
 
         /** {@inheritDoc} */
         @Override public long getWriteBehindFlushFrequency() {
-            return 58;
+            return 48;
         }
 
         /** {@inheritDoc} */
         @Override public int getWriteBehindStoreBatchSize() {
-            return 59;
+            return 49;
         }
 
         /** {@inheritDoc} */
         @Override public int getWriteBehindTotalCriticalOverflowCount() {
-            return 60;
+            return 50;
         }
 
         /** {@inheritDoc} */
         @Override public int getWriteBehindCriticalOverflowCount() {
-            return 61;
+            return 51;
         }
 
         /** {@inheritDoc} */
         @Override public int getWriteBehindErrorRetryCount() {
-            return 62;
+            return 52;
         }
 
         /** {@inheritDoc} */
         @Override public int getWriteBehindBufferSize() {
-            return 63;
+            return 53;
         }
 
         /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/3f90a9c0/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteMock.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteMock.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteMock.java
index bf3f17e..ab918e5 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteMock.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteMock.java
@@ -45,6 +45,7 @@ import org.apache.ignite.IgniteScheduler;
 import org.apache.ignite.IgniteServices;
 import org.apache.ignite.IgniteSet;
 import org.apache.ignite.IgniteTransactions;
+import org.apache.ignite.MemoryMetrics;
 import org.apache.ignite.binary.BinaryObjectBuilder;
 import org.apache.ignite.binary.BinaryObjectException;
 import org.apache.ignite.cache.affinity.Affinity;
@@ -436,6 +437,11 @@ public class IgniteMock implements Ignite {
         // No-op.
     }
 
+    /** {@inheritDoc} */
+    @Override public Collection<MemoryMetrics> memoryMetrics() {
+        return null;
+    }
+
     /**
      * @param staticCfg Configuration.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/3f90a9c0/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
index 9d58481..ca65890 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
@@ -48,6 +48,7 @@ import org.apache.ignite.IgniteSemaphore;
 import org.apache.ignite.IgniteServices;
 import org.apache.ignite.IgniteSet;
 import org.apache.ignite.IgniteTransactions;
+import org.apache.ignite.MemoryMetrics;
 import org.apache.ignite.cache.affinity.Affinity;
 import org.apache.ignite.cluster.ClusterGroup;
 import org.apache.ignite.cluster.ClusterNode;
@@ -648,6 +649,11 @@ public class IgniteProcessProxy implements IgniteEx {
     }
 
     /** {@inheritDoc} */
+    @Override public Collection<MemoryMetrics> memoryMetrics() {
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
+    }
+
+    /** {@inheritDoc} */
     @Override public void close() throws IgniteException {
         if (locJvmGrid != null) {
             final CountDownLatch rmtNodeStoppedLatch = new CountDownLatch(1);

http://git-wip-us.apache.org/repos/asf/ignite/blob/3f90a9c0/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
index 4a29ec9..70084f2 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
@@ -20,6 +20,7 @@ package org.apache.ignite.testsuites;
 import java.util.Set;
 import junit.framework.TestSuite;
 import org.apache.ignite.GridSuppressedExceptionSelfTest;
+import org.apache.ignite.internal.processors.database.MemoryMetricsSelfTest;
 import org.apache.ignite.util.AttributeNodeFilterSelfTest;
 import org.apache.ignite.internal.ClusterGroupHostsSelfTest;
 import org.apache.ignite.internal.ClusterGroupSelfTest;
@@ -167,6 +168,7 @@ public class IgniteBasicTestSuite extends TestSuite {
         suite.addTestSuite(BPlusTreeReuseSelfTest.class);
         suite.addTestSuite(MetadataStorageSelfTest.class);
         suite.addTestSuite(FreeListImplSelfTest.class);
+        suite.addTestSuite(MemoryMetricsSelfTest.class);
 
         return suite;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/3f90a9c0/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/database/InlineIndexHelperTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/database/InlineIndexHelperTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/database/InlineIndexHelperTest.java
index 2e6ec34..361bad3 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/database/InlineIndexHelperTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/database/InlineIndexHelperTest.java
@@ -28,6 +28,7 @@ import org.apache.ignite.internal.mem.unsafe.UnsafeMemoryProvider;
 import org.apache.ignite.internal.pagemem.PageIdAllocator;
 import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.pagemem.impl.PageMemoryNoStoreImpl;
+import org.apache.ignite.internal.processors.cache.database.MemoryMetricsImpl;
 import org.apache.ignite.logger.java.JavaLogger;
 import org.h2.result.SortOrder;
 import org.h2.value.CompareMode;
@@ -139,6 +140,7 @@ public class InlineIndexHelperTest extends TestCase {
             new UnsafeMemoryProvider(sizes),
             null,
             PAGE_SIZE,
+            new MemoryMetricsImpl(null),
             false);
 
         pageMem.start();
@@ -184,6 +186,7 @@ public class InlineIndexHelperTest extends TestCase {
             new UnsafeMemoryProvider(sizes),
             null,
             PAGE_SIZE,
+            new MemoryMetricsImpl(null),
             false);
 
         pageMem.start();
@@ -305,6 +308,7 @@ public class InlineIndexHelperTest extends TestCase {
             new UnsafeMemoryProvider(sizes),
             null,
             PAGE_SIZE,
+            new MemoryMetricsImpl(null),
             false);
 
         pageMem.start();

http://git-wip-us.apache.org/repos/asf/ignite/blob/3f90a9c0/modules/platforms/cpp/core-test/config/cache-test.xml
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core-test/config/cache-test.xml b/modules/platforms/cpp/core-test/config/cache-test.xml
index 7fe7ac6..43ed8d2 100644
--- a/modules/platforms/cpp/core-test/config/cache-test.xml
+++ b/modules/platforms/cpp/core-test/config/cache-test.xml
@@ -30,5 +30,5 @@
         http://www.springframework.org/schema/util/spring-util.xsd">
     <import resource="cache-test-default.xml"/>
 
-        <bean parent="grid.cfg"/>
+    <bean parent="grid.cfg"/>
 </beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/3f90a9c0/modules/platforms/cpp/core-test/src/cache_test.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core-test/src/cache_test.cpp b/modules/platforms/cpp/core-test/src/cache_test.cpp
index 91e28f7..a9ee47e 100644
--- a/modules/platforms/cpp/core-test/src/cache_test.cpp
+++ b/modules/platforms/cpp/core-test/src/cache_test.cpp
@@ -225,7 +225,7 @@ BOOST_AUTO_TEST_CASE(TestPutAllIterMap)
 
     for (int i = 0; i < 100; i++)
         map[i] = i + 1;
-    
+
     cache::Cache<int, int> cache = Cache();
 
     cache.PutAll(map.begin(), map.end());

http://git-wip-us.apache.org/repos/asf/ignite/blob/3f90a9c0/modules/spring/src/main/java/org/apache/ignite/IgniteSpringBean.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/main/java/org/apache/ignite/IgniteSpringBean.java b/modules/spring/src/main/java/org/apache/ignite/IgniteSpringBean.java
index 9010f23..b1994ac 100644
--- a/modules/spring/src/main/java/org/apache/ignite/IgniteSpringBean.java
+++ b/modules/spring/src/main/java/org/apache/ignite/IgniteSpringBean.java
@@ -272,6 +272,13 @@ public class IgniteSpringBean implements Ignite, DisposableBean, InitializingBea
     }
 
     /** {@inheritDoc} */
+    @Override public Collection<MemoryMetrics> memoryMetrics() {
+        checkIgnite();
+
+        return g.memoryMetrics();
+    }
+
+    /** {@inheritDoc} */
     @Override public <K, V> IgniteCache<K, V> cache(@Nullable String name) {
         checkIgnite();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/3f90a9c0/modules/tools/src/main/java/org/apache/ignite/tools/ant/beautifier/GridJavadocAntTask.java
----------------------------------------------------------------------
diff --git a/modules/tools/src/main/java/org/apache/ignite/tools/ant/beautifier/GridJavadocAntTask.java b/modules/tools/src/main/java/org/apache/ignite/tools/ant/beautifier/GridJavadocAntTask.java
index d34b4e4..611af41 100644
--- a/modules/tools/src/main/java/org/apache/ignite/tools/ant/beautifier/GridJavadocAntTask.java
+++ b/modules/tools/src/main/java/org/apache/ignite/tools/ant/beautifier/GridJavadocAntTask.java
@@ -117,6 +117,8 @@ public class GridJavadocAntTask extends MatchingTask {
 
         boolean fail = false;
 
+        ArrayList<String> errMsgs = new ArrayList<>();
+
         for (String fileName : scanner.getIncludedFiles()) {
             String file = dir.getAbsolutePath() + '/' + fileName;
 
@@ -129,12 +131,26 @@ public class GridJavadocAntTask extends MatchingTask {
             catch (IllegalArgumentException e) {
                 System.err.println("JavaDoc error: " + e.getMessage());
 
+                errMsgs.add(e.getMessage());
+
                 fail = true;
             }
         }
 
         if (fail)
-            throw new BuildException("Execution failed due to previous errors.");
+            throw new BuildException("Execution failed due to: " + prepareErrorSummary(errMsgs));
+    }
+
+    /**
+     * @param errMsgs Err msgs.
+     */
+    private String prepareErrorSummary(ArrayList<String> errMsgs) {
+        StringBuilder strBdr = new StringBuilder();
+
+        for (String errMsg : errMsgs)
+            strBdr.append(errMsg).append(System.lineSeparator());
+
+        return strBdr.toString();
     }
 
     /**