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/05/25 15:42:14 UTC

ignite git commit: IGNITE-5124 - Memory metrics API Improvements - Fixes #1988.

Repository: ignite
Updated Branches:
  refs/heads/master e8f5af333 -> db7d776ad


IGNITE-5124 - Memory metrics API Improvements - Fixes #1988.

Signed-off-by: Alexey Goncharuk <al...@gmail.com>


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/db7d776a
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/db7d776a
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/db7d776a

Branch: refs/heads/master
Commit: db7d776ad53edee819dcde9ab3abef48219fe514
Parents: e8f5af3
Author: Sergey Chugunov <se...@gmail.com>
Authored: Thu May 25 18:40:00 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Thu May 25 18:40:00 2017 +0300

----------------------------------------------------------------------
 .../src/main/java/org/apache/ignite/Ignite.java | 18 +++-
 .../MemoryPolicyConfiguration.java              | 96 +++++++++++++++++++-
 .../apache/ignite/internal/IgniteKernal.java    | 12 +++
 .../IgniteCacheDatabaseSharedManager.java       | 40 ++++++++
 .../cache/database/MemoryMetricsImpl.java       | 31 ++++---
 .../cache/MemoryPolicyConfigValidationTest.java | 62 ++++++++++++-
 .../processors/igfs/IgfsIgniteMock.java         |  7 ++
 .../ignite/testframework/junits/IgniteMock.java |  5 +
 .../junits/multijvm/IgniteProcessProxy.java     |  5 +
 .../org/apache/ignite/IgniteSpringBean.java     |  7 ++
 10 files changed, 266 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/db7d776a/modules/core/src/main/java/org/apache/ignite/Ignite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/Ignite.java b/modules/core/src/main/java/org/apache/ignite/Ignite.java
index 671efca..8d5e967 100644
--- a/modules/core/src/main/java/org/apache/ignite/Ignite.java
+++ b/modules/core/src/main/java/org/apache/ignite/Ignite.java
@@ -27,6 +27,8 @@ import org.apache.ignite.cluster.ClusterGroup;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.CollectionConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.MemoryConfiguration;
+import org.apache.ignite.configuration.MemoryPolicyConfiguration;
 import org.apache.ignite.configuration.NearCacheConfiguration;
 import org.apache.ignite.internal.util.typedef.G;
 import org.apache.ignite.lang.IgniteProductVersion;
@@ -618,8 +620,22 @@ public interface Ignite extends AutoCloseable {
     /**
      * Returns a collection of {@link MemoryMetrics} that reflects page memory usage on this Apache Ignite node
      * instance.
+     * Returns the collection that contains the latest snapshots for each memory region
+     * configured with {@link MemoryPolicyConfiguration configuration} on this Ignite node instance.
      *
-     * @return Collection of {@link MemoryMetrics}
+     * @return Collection of {@link MemoryMetrics} snapshots.
      */
     public Collection<MemoryMetrics> memoryMetrics();
+
+    /**
+     * Returns the latest {@link MemoryMetrics} snapshot for the memory region of the given name.
+     *
+     * To get the metrics for the default memory region use
+     * {@link MemoryConfiguration#DFLT_MEM_PLC_DEFAULT_NAME} as the name
+     * or a custom name if the default memory region has been renamed.
+     *
+     * @param memPlcName Name of memory region configured with {@link MemoryPolicyConfiguration config}.
+     * @return {@link MemoryMetrics} snapshot or {@code null} if no memory region is configured under specified name.
+     */
+    @Nullable public MemoryMetrics memoryMetrics(String memPlcName);
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/db7d776a/modules/core/src/main/java/org/apache/ignite/configuration/MemoryPolicyConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/MemoryPolicyConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/MemoryPolicyConfiguration.java
index 2716d8e..be0877f 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/MemoryPolicyConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/MemoryPolicyConfiguration.java
@@ -17,7 +17,9 @@
 package org.apache.ignite.configuration;
 
 import java.io.Serializable;
+import org.apache.ignite.MemoryMetrics;
 import org.apache.ignite.internal.mem.IgniteOutOfMemoryException;
+import org.apache.ignite.mxbean.MemoryMetricsMXBean;
 
 import static org.apache.ignite.configuration.MemoryConfiguration.DFLT_MEM_PLC_DEFAULT_NAME;
 
@@ -66,6 +68,12 @@ public final class MemoryPolicyConfiguration implements Serializable {
     /** Default metrics enabled flag. */
     public static final boolean DFLT_METRICS_ENABLED = false;
 
+    /** Default amount of sub intervals to calculate {@link MemoryMetrics#getAllocationRate()} metric. */
+    public static final int DFLT_SUB_INTERVALS = 5;
+
+    /** Default length of interval over which {@link MemoryMetrics#getAllocationRate()} metric is calculated. */
+    public static final int DFLT_RATE_TIME_INTERVAL_SEC = 60;
+
     /** Memory policy name. */
     private String name = DFLT_MEM_PLC_DEFAULT_NAME;
 
@@ -90,9 +98,28 @@ public final class MemoryPolicyConfiguration implements Serializable {
     /** Minimum number of empty pages in reuse lists. */
     private int emptyPagesPoolSize = 100;
 
-    /** */
+    /**
+     * Flag to enable the memory metrics collection for this memory policy.
+     */
     private boolean metricsEnabled = DFLT_METRICS_ENABLED;
 
+    /** Number of sub-intervals the whole {@link #setRateTimeInterval(int)} will be split into to calculate
+     * {@link MemoryMetrics#getAllocationRate()} and {@link MemoryMetrics#getEvictionRate()} rates (5 by default).
+     * <p>
+     * Setting it to a bigger value will result in more precise calculation and smaller drops of
+     * {@link MemoryMetrics#getAllocationRate()} metric when next sub-interval has to be recycled but introduces bigger
+     * calculation overhead. */
+    private int subIntervals = DFLT_SUB_INTERVALS;
+
+    /**
+     * Time interval for {@link MemoryMetrics#getAllocationRate()}
+     * and {@link MemoryMetrics#getEvictionRate()} monitoring purposes.
+     * <p>
+     * For instance, after setting the interval to 60 seconds, subsequent calls to {@link MemoryMetrics#getAllocationRate()}
+     * will return average allocation rate (pages per second) for the last minute.
+     */
+    private int rateTimeInterval = DFLT_RATE_TIME_INTERVAL_SEC;
+
     /**
      * Gets memory policy name.
      *
@@ -263,7 +290,7 @@ public final class MemoryPolicyConfiguration implements Serializable {
 
     /**
      * Gets whether memory metrics are enabled by default on node startup. Memory metrics can be enabled and disabled
-     * at runtime via memory metrics MX bean.
+     * at runtime via memory metrics {@link MemoryMetricsMXBean MX bean}.
      *
      * @return Metrics enabled flag.
      */
@@ -273,7 +300,7 @@ public final class MemoryPolicyConfiguration implements Serializable {
 
     /**
      * Sets memory metrics enabled flag. If this flag is {@code true}, metrics will be enabled on node startup.
-     * Memory metrics can be enabled and disabled at runtime via memory metrics MX bean.
+     * Memory metrics can be enabled and disabled at runtime via memory metrics {@link MemoryMetricsMXBean MX bean}.
      *
      * @param metricsEnabled Metrics enabled flag.
      * @return {@code this} for chaining.
@@ -283,4 +310,67 @@ public final class MemoryPolicyConfiguration implements Serializable {
 
         return this;
     }
+
+    /**
+     * Gets time interval for {@link MemoryMetrics#getAllocationRate()}
+     * and {@link MemoryMetrics#getEvictionRate()} monitoring purposes.
+     * <p>
+     * For instance, after setting the interval to 60 seconds,
+     * subsequent calls to {@link MemoryMetrics#getAllocationRate()}
+     * will return average allocation rate (pages per second) for the last minute.
+     *
+     * @return Time interval over which allocation rate is calculated.
+     */
+    public int getRateTimeInterval() {
+        return rateTimeInterval;
+    }
+
+    /**
+     * Sets time interval for {@link MemoryMetrics#getAllocationRate()}
+     * and {@link MemoryMetrics#getEvictionRate()} monitoring purposes.
+     * <p>
+     * For instance, after setting the interval to 60 seconds,
+     * subsequent calls to {@link MemoryMetrics#getAllocationRate()}
+     * will return average allocation rate (pages per second) for the last minute.
+     *
+     * @param rateTimeInterval Time interval used for allocation and eviction rates calculations.
+     * @return {@code this} for chaining.
+     */
+    public MemoryPolicyConfiguration setRateTimeInterval(int rateTimeInterval) {
+        this.rateTimeInterval = rateTimeInterval;
+
+        return this;
+    }
+
+    /**
+     * Gets a number of sub-intervals the whole {@link #setRateTimeInterval(int)}
+     * will be split into to calculate {@link MemoryMetrics#getAllocationRate()}
+     * and {@link MemoryMetrics#getEvictionRate()} rates (5 by default).
+     * <p>
+     * Setting it to a bigger value will result in more precise calculation and smaller drops of
+     * {@link MemoryMetrics#getAllocationRate()} metric when next sub-interval has to be recycled but introduces bigger
+     * calculation overhead.
+     *
+     * @return number of sub intervals.
+     */
+    public int getSubIntervals() {
+        return subIntervals;
+    }
+
+    /**
+     * Sets a number of sub-intervals the whole {@link #setRateTimeInterval(int)} will be split into to calculate
+     * {@link MemoryMetrics#getAllocationRate()} and {@link MemoryMetrics#getEvictionRate()} rates (5 by default).
+     * <p>
+     * Setting it to a bigger value will result in more precise calculation and smaller drops of
+     * {@link MemoryMetrics#getAllocationRate()} metric when next sub-interval has to be recycled but introduces bigger
+     * calculation overhead.
+     *
+     * @param subIntervals A number of sub-intervals.
+     * @return {@code this} for chaining.
+     */
+    public MemoryPolicyConfiguration setSubIntervals(int subIntervals) {
+        this.subIntervals = subIntervals;
+
+        return this;
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/db7d776a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------
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 0f43169..6393e9f 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
@@ -3426,6 +3426,18 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
     }
 
     /** {@inheritDoc} */
+    @Nullable @Override public MemoryMetrics memoryMetrics(String memPlcName) {
+        guard();
+
+        try {
+            return ctx.cache().context().database().memoryMetrics(memPlcName);
+        }
+        finally {
+            unguard();
+        }
+    }
+
+    /** {@inheritDoc} */
     @Nullable @Override public IgniteAtomicSequence atomicSequence(String name, long initVal, boolean create) {
         guard();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/db7d776a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/IgniteCacheDatabaseSharedManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/IgniteCacheDatabaseSharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/IgniteCacheDatabaseSharedManager.java
index 17c16b8..e356f35 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/IgniteCacheDatabaseSharedManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/IgniteCacheDatabaseSharedManager.java
@@ -340,6 +340,8 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
 
                 checkPolicySize(plcCfg);
 
+                checkMetricsProperties(plcCfg);
+
                 checkPolicyEvictionProperties(plcCfg, memCfg);
             }
         }
@@ -351,6 +353,26 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
     }
 
     /**
+     * @param plcCfg Memory policy config.
+     *
+     * @throws IgniteCheckedException if validation of memory metrics properties fails.
+     */
+    private static void checkMetricsProperties(MemoryPolicyConfiguration plcCfg) throws IgniteCheckedException {
+        if (plcCfg.getRateTimeInterval() <= 0)
+            throw new IgniteCheckedException("Rate time interval must be greater than zero " +
+                "(use MemoryPolicyConfiguration.rateTimeInterval property to adjust the interval) " +
+                "[name=" + plcCfg.getName() +
+                ", rateTimeInterval=" + plcCfg.getRateTimeInterval() + "]"
+            );
+        if (plcCfg.getSubIntervals() <= 0)
+            throw new IgniteCheckedException("Sub intervals must be greater than zero " +
+                "(use MemoryPolicyConfiguration.subIntervals property to adjust the sub intervals) " +
+                "[name=" + plcCfg.getName() +
+                ", subIntervals=" + plcCfg.getSubIntervals() + "]"
+            );
+    }
+
+    /**
      * @param sysCacheInitSize System cache initial size.
      * @param sysCacheMaxSize System cache max size.
      *
@@ -541,6 +563,24 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
     }
 
     /**
+     * @param memPlcName Name of {@link MemoryPolicy} to obtain {@link MemoryMetrics} for.
+     * @return {@link MemoryMetrics} snapshot for specified {@link MemoryPolicy} or {@code null} if
+     * no {@link MemoryPolicy} is configured for specified name.
+     */
+    @Nullable public MemoryMetrics memoryMetrics(String memPlcName) {
+        if (!F.isEmpty(memMetricsMap)) {
+            MemoryMetrics memMetrics = memMetricsMap.get(memPlcName);
+
+            if (memMetrics == null)
+                return null;
+            else
+                return new MemoryMetricsSnapshot(memMetrics);
+        }
+        else
+            return null;
+    }
+
+    /**
      * @param memPlcName Memory policy name.
      * @return {@link MemoryPolicy} instance associated with a given {@link MemoryPolicyConfiguration}.
      * @throws IgniteCheckedException in case of request for unknown MemoryPolicy.

http://git-wip-us.apache.org/repos/asf/ignite/blob/db7d776a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/MemoryMetricsImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/MemoryMetricsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/MemoryMetricsImpl.java
index ee356a1..3e3b690 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/MemoryMetricsImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/MemoryMetricsImpl.java
@@ -44,10 +44,10 @@ public class MemoryMetricsImpl implements MemoryMetrics {
     private volatile boolean metricsEnabled;
 
     /** */
-    private volatile int subInts = 5;
+    private volatile int subInts;
 
     /** */
-    private volatile LongAdder8[] allocRateCounters = new LongAdder8[subInts];
+    private volatile LongAdder8[] allocRateCounters;
 
     /** */
     private final AtomicInteger counterIdx = new AtomicInteger(0);
@@ -58,9 +58,8 @@ public class MemoryMetricsImpl implements MemoryMetrics {
     /** */
     private final MemoryPolicyConfiguration memPlcCfg;
 
-    /** Time interval (in seconds) when allocations/evictions are counted to calculate rate.
-     * Default value is 60 seconds. */
-    private volatile int rateTimeInterval = 60;
+    /** Time interval (in seconds) when allocations/evictions are counted to calculate rate. */
+    private volatile int rateTimeInterval;
 
     /**
      * @param memPlcCfg MemoryPolicyConfiguration.
@@ -70,6 +69,12 @@ public class MemoryMetricsImpl implements MemoryMetrics {
 
         metricsEnabled = memPlcCfg.isMetricsEnabled();
 
+        rateTimeInterval = memPlcCfg.getRateTimeInterval();
+
+        subInts = memPlcCfg.getSubIntervals();
+
+        allocRateCounters = new LongAdder8[subInts];
+
         for (int i = 0; i < subInts; i++)
             allocRateCounters[i] = new LongAdder8();
     }
@@ -140,9 +145,6 @@ public class MemoryMetricsImpl implements MemoryMetrics {
      *
      */
     private void updateAllocationRateMetrics() {
-        if (subInts != allocRateCounters.length)
-            return;
-
         long lastUpdT = lastUpdTime.get();
         long currT = IgniteUtils.currentTimeMillis();
 
@@ -154,11 +156,16 @@ public class MemoryMetricsImpl implements MemoryMetrics {
 
         LongAdder8[] rateCntrs = allocRateCounters;
 
+        if (subInts != rateCntrs.length)
+            return;
+
+        int cntrIdx = counterIdx.get();
+
         for (int i = 1; i <= subInts; i++) {
             if (deltaT < subInt(i)) {
                 if (i > 1) {
                     if (!lastUpdTime.compareAndSet(lastUpdT, currT)) {
-                        rateCntrs[counterIdx.get()].increment();
+                        rateCntrs[cntrIdx].increment();
 
                         break;
                     }
@@ -174,7 +181,7 @@ public class MemoryMetricsImpl implements MemoryMetrics {
                     break;
                 }
                 else {
-                    rateCntrs[counterIdx.get()].increment();
+                    rateCntrs[cntrIdx].increment();
 
                     break;
                 }
@@ -182,8 +189,8 @@ public class MemoryMetricsImpl implements MemoryMetrics {
             else if (i == subInts && lastUpdTime.compareAndSet(lastUpdT, currT))
                 resetAll();
 
-            if (currIdx != counterIdx.get()) {
-                rateCntrs[counterIdx.get()].increment();
+            if (currIdx != cntrIdx) {
+                rateCntrs[cntrIdx].increment();
 
                 break;
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/db7d776a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/MemoryPolicyConfigValidationTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/MemoryPolicyConfigValidationTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/MemoryPolicyConfigValidationTest.java
index a1a05eb..1d8174b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/MemoryPolicyConfigValidationTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/MemoryPolicyConfigValidationTest.java
@@ -92,6 +92,16 @@ public class MemoryPolicyConfigValidationTest extends GridCommonAbstractTest {
 
                 break;
 
+            case LTE_ZERO_RATE_TIME_INTERVAL:
+                plcs = createRateTimeIntervalIsNegative();
+
+                break;
+
+            case LTE_ZERO_SUB_INTERVALS:
+                plcs = createSubIntervalsIsNegative();
+
+                break;
+
             default:
                 fail("Violation type was not configured: " + violationType);
         }
@@ -106,6 +116,30 @@ public class MemoryPolicyConfigValidationTest extends GridCommonAbstractTest {
     /**
      *
      */
+    private MemoryPolicyConfiguration[] createSubIntervalsIsNegative() {
+        MemoryPolicyConfiguration[] res = new MemoryPolicyConfiguration[1];
+
+        res[0] = createMemoryPolicy(VALID_DEFAULT_MEM_PLC_NAME, 100 * 1024 * 1024, 100 * 1024 * 1024);
+        res[0].setSubIntervals(-10);
+
+        return res;
+    }
+
+    /**
+     *
+     */
+    private MemoryPolicyConfiguration[] createRateTimeIntervalIsNegative() {
+        MemoryPolicyConfiguration[] res = new MemoryPolicyConfiguration[1];
+
+        res[0] = createMemoryPolicy(VALID_DEFAULT_MEM_PLC_NAME, 100 * 1024 * 1024, 100 * 1024 * 1024);
+        res[0].setRateTimeInterval(-10);
+
+        return res;
+    }
+
+    /**
+     *
+     */
     private MemoryPolicyConfiguration[] createValidUserDefault() {
         MemoryPolicyConfiguration[] res = new MemoryPolicyConfiguration[1];
 
@@ -277,6 +311,24 @@ public class MemoryPolicyConfigValidationTest extends GridCommonAbstractTest {
     }
 
     /**
+     *
+     */
+    public void testRateTimeIntervalPropertyIsNegative() throws Exception {
+        violationType = ValidationViolationType.LTE_ZERO_RATE_TIME_INTERVAL;
+
+        doTest(violationType);
+    }
+
+    /**
+     *
+     */
+    public void testSubIntervalsPropertyIsNegative() throws Exception {
+        violationType = ValidationViolationType.LTE_ZERO_SUB_INTERVALS;
+
+        doTest(violationType);
+    }
+
+    /**
      * Tries to start ignite node with invalid configuration and checks that corresponding exception is thrown.
      *
      * @param violationType Configuration violation type.
@@ -323,7 +375,15 @@ public class MemoryPolicyConfigValidationTest extends GridCommonAbstractTest {
         TOO_SMALL_USER_DEFINED_DFLT_MEM_PLC_SIZE("User-defined default MemoryPolicy size is less than 1MB."),
 
         /** */
-        MAX_SIZE_IS_SMALLER_THAN_INITIAL_SIZE("MemoryPolicy maxSize must not be smaller than initialSize");
+        MAX_SIZE_IS_SMALLER_THAN_INITIAL_SIZE("MemoryPolicy maxSize must not be smaller than initialSize"),
+
+        /** Case when rateTimeInterval property of MemoryPolicyConfiguration is less than or equals zero. */
+        LTE_ZERO_RATE_TIME_INTERVAL("Rate time interval must be greater than zero " +
+            "(use MemoryPolicyConfiguration.rateTimeInterval property to adjust the interval)"),
+
+        /** Case when subIntervals property of MemoryPolicyConfiguration is less than or equals zero. */
+        LTE_ZERO_SUB_INTERVALS("Sub intervals must be greater than zero " +
+            "(use MemoryPolicyConfiguration.subIntervals property to adjust the sub intervals)");
 
         /**
          * @param violationMsg Violation message.

http://git-wip-us.apache.org/repos/asf/ignite/blob/db7d776a/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 cc058b1..ed0af3c 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
@@ -546,6 +546,13 @@ public class IgfsIgniteMock implements IgniteEx {
         return null;
     }
 
+    /** {@inheritDoc} */
+    @Nullable @Override public MemoryMetrics memoryMetrics(String memPlcName) {
+        throwUnsupported();
+
+        return null;
+    }
+
     /**
      * Throw {@link UnsupportedOperationException}.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/db7d776a/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 ea53345..1f95dd9 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
@@ -446,6 +446,11 @@ public class IgniteMock implements Ignite {
         return null;
     }
 
+    /** {@inheritDoc} */
+    @Nullable @Override public MemoryMetrics memoryMetrics(String memPlcName) {
+        return null;
+    }
+
     /**
      * @param staticCfg Configuration.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/db7d776a/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 34ca22f..f459e57 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
@@ -662,6 +662,11 @@ public class IgniteProcessProxy implements IgniteEx {
     }
 
     /** {@inheritDoc} */
+    @Nullable @Override public MemoryMetrics memoryMetrics(String memPlcName) {
+        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/db7d776a/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 b1994ac..094c2dc 100644
--- a/modules/spring/src/main/java/org/apache/ignite/IgniteSpringBean.java
+++ b/modules/spring/src/main/java/org/apache/ignite/IgniteSpringBean.java
@@ -279,6 +279,13 @@ public class IgniteSpringBean implements Ignite, DisposableBean, InitializingBea
     }
 
     /** {@inheritDoc} */
+    @Nullable @Override public MemoryMetrics memoryMetrics(String memPlcName) {
+        checkIgnite();
+
+        return g.memoryMetrics(memPlcName);
+    }
+
+    /** {@inheritDoc} */
     @Override public <K, V> IgniteCache<K, V> cache(@Nullable String name) {
         checkIgnite();