You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by av...@apache.org on 2018/08/03 09:58:27 UTC

[14/50] [abbrv] ignite git commit: IGNITE-6846 Add metrics for entry processor invocations. - Fixes #3148.

http://git-wip-us.apache.org/repos/asf/ignite/blob/5e8669af/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java
index d930c6b..8f3f2ef 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java
@@ -1506,6 +1506,17 @@ public class PlatformCache extends PlatformAbstractTarget {
         writer.writeLong(metrics.getCacheSize());
         writer.writeLong(metrics.getRebalancedKeys());
         writer.writeLong(metrics.getEstimatedRebalancingKeys());
+        writer.writeLong(metrics.getEntryProcessorPuts());
+        writer.writeFloat(metrics.getEntryProcessorAverageInvocationTime());
+        writer.writeLong(metrics.getEntryProcessorInvocations());
+        writer.writeFloat(metrics.getEntryProcessorMaxInvocationTime());
+        writer.writeFloat(metrics.getEntryProcessorMinInvocationTime());
+        writer.writeLong(metrics.getEntryProcessorReadOnlyInvocations());
+        writer.writeFloat(metrics.getEntryProcessorHitPercentage());
+        writer.writeLong(metrics.getEntryProcessorHits());
+        writer.writeLong(metrics.getEntryProcessorMisses());
+        writer.writeFloat(metrics.getEntryProcessorMissPercentage());
+        writer.writeLong(metrics.getEntryProcessorRemovals());
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/5e8669af/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractMetricsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractMetricsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractMetricsSelfTest.java
index 721989b..eb4d2d5 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractMetricsSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractMetricsSelfTest.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.processors.cache;
 
+import com.google.common.collect.ImmutableMap;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.LinkedHashSet;
@@ -27,9 +28,14 @@ import java.util.concurrent.TimeUnit;
 import javax.cache.expiry.Duration;
 import javax.cache.expiry.ExpiryPolicy;
 import javax.cache.expiry.TouchedExpiryPolicy;
+import javax.cache.processor.EntryProcessorException;
+import javax.cache.processor.EntryProcessorResult;
+import javax.cache.processor.MutableEntry;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheEntryProcessor;
 import org.apache.ignite.cache.CacheMetrics;
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.configuration.CacheConfiguration;
@@ -49,6 +55,51 @@ public abstract class GridCacheAbstractMetricsSelfTest extends GridCacheAbstract
     /** */
     private static final int KEY_CNT = 500;
 
+    /** Entry processor, performing removal. */
+    private final CacheEntryProcessor<Integer, Integer, Object> removingProcessor =
+        new CacheEntryProcessor<Integer, Integer, Object>() {
+            @Override
+            public Object process(
+                    MutableEntry<Integer, Integer> entry,
+                    Object... arguments
+            ) throws EntryProcessorException {
+
+                entry.remove();
+
+                return null;
+            }
+        };
+
+    /** Entry processor, performing reading. */
+    private final CacheEntryProcessor<Integer, Integer, Object> readingProcessor =
+        new CacheEntryProcessor<Integer, Integer, Object>() {
+            @Override
+            public Object process(
+                    MutableEntry<Integer, Integer> entry,
+                    Object... arguments
+            ) throws EntryProcessorException {
+
+                entry.getValue();
+
+                return null;
+            }
+        };
+
+    /** Entry processor, performing updating. */
+    private final CacheEntryProcessor<Integer, Integer, Object> updatingProcessor =
+        new CacheEntryProcessor<Integer, Integer, Object>() {
+            @Override
+            public Object process(
+                    MutableEntry<Integer, Integer> entry,
+                    Object... arguments
+            ) throws EntryProcessorException {
+
+                entry.setValue(1);
+
+                return null;
+            }
+        };
+
     /** {@inheritDoc} */
     @Override protected boolean swapEnabled() {
         return false;
@@ -134,6 +185,47 @@ public abstract class GridCacheAbstractMetricsSelfTest extends GridCacheAbstract
         for (int i = 0; i < KEY_CNT; i++)
             jcache.put(i, i);
 
+        // Invoke update on cache.
+        for (int i = 0; i < KEY_CNT; i++)
+            jcache.invoke(i, new CacheEntryProcessor<Object, Object, Object>() {
+                @Override
+                public Object process(MutableEntry<Object, Object> entry,
+                                      Object... arguments) throws EntryProcessorException {
+
+                    Object key = entry.getKey();
+
+                    entry.setValue(key);
+
+                    return null;
+                }
+            });
+
+        // Read-only invoke on cache.
+        for (int i = 0; i < KEY_CNT; i++)
+            jcache.invoke(i, new CacheEntryProcessor<Object, Object, Object>() {
+                @Override
+                public Object process(MutableEntry<Object, Object> entry,
+                                      Object... arguments) throws EntryProcessorException {
+
+                    entry.getKey();
+
+                    return null;
+                }
+            });
+
+        // Remove invoke on cache.
+        for (int i = 0; i < KEY_CNT; i++)
+            jcache.invoke(i, new CacheEntryProcessor<Object, Object, Object>() {
+                @Override
+                public Object process(MutableEntry<Object, Object> entry,
+                                      Object... arguments) throws EntryProcessorException {
+
+                    entry.remove();
+
+                    return null;
+                }
+            });
+
         // Get from cache.
         for (int i = 0; i < KEY_CNT; i++)
             jcache.get(i);
@@ -156,6 +248,14 @@ public abstract class GridCacheAbstractMetricsSelfTest extends GridCacheAbstract
             assertEquals(m.getAveragePutTime(), 0f);
             assertEquals(m.getAverageTxCommitTime(), 0f);
             assertEquals(m.getAverageTxRollbackTime(), 0f);
+
+            assertEquals(m.getEntryProcessorPuts(), 0);
+            assertEquals(m.getEntryProcessorRemovals(), 0);
+            assertEquals(m.getEntryProcessorReadOnlyInvocations(), 0);
+            assertEquals(m.getEntryProcessorMinInvocationTime(), 0f);
+            assertEquals(m.getEntryProcessorMaxInvocationTime(), 0f);
+            assertEquals(m.getEntryProcessorAverageInvocationTime(), 0f);
+            assertEquals(m.getEntryProcessorInvocations(), 0);
         }
     }
 
@@ -934,4 +1034,321 @@ public abstract class GridCacheAbstractMetricsSelfTest extends GridCacheAbstract
 
         assertEquals(0, entry.expireTime());
     }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testInvocationRemovesOnEmptyCache() throws IgniteCheckedException {
+        testInvocationRemoves(true);
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testInvocationRemoves() throws IgniteCheckedException {
+        testInvocationRemoves(false);
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    private void testInvocationRemoves(boolean emptyCache) throws IgniteCheckedException {
+        IgniteCache<Integer, Integer> cache0 = grid(0).cache(DEFAULT_CACHE_NAME);
+
+        final Integer key = primaryKey(cache0);
+
+        if (emptyCache)
+            cache0.remove(key);
+        else
+            cache0.put(key, 0);
+
+        cache0.invoke(key, removingProcessor);
+
+        assertEquals(1, cache0.localMetrics().getEntryProcessorRemovals());
+
+        if (emptyCache) {
+            assertEquals(1, cache0.localMetrics().getEntryProcessorMisses());
+
+            assertEquals(100f, cache0.localMetrics().getEntryProcessorMissPercentage());
+            assertEquals(0f, cache0.localMetrics().getEntryProcessorHitPercentage());
+        }
+        else {
+            assertEquals(1, cache0.localMetrics().getEntryProcessorHits());
+
+            assertEquals(0f, cache0.localMetrics().getEntryProcessorMissPercentage());
+            assertEquals(100f, cache0.localMetrics().getEntryProcessorHitPercentage());
+        }
+
+        for (int i = 1; i < gridCount(); i++) {
+            Ignite ignite = ignite(i);
+
+            IgniteCache<Integer, Integer> cache = ignite.cache(DEFAULT_CACHE_NAME);
+
+            if (affinity(cache).isPrimaryOrBackup(ignite.cluster().localNode(), key))
+                assertEquals(1, cache.localMetrics().getEntryProcessorRemovals());
+        }
+
+        assertEquals(1, cache0.localMetrics().getEntryProcessorInvocations());
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testUpdateInvocationsOnEmptyCache() throws IgniteCheckedException {
+        testUpdateInvocations(true);
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testUpdateInvocations() throws IgniteCheckedException {
+        testUpdateInvocations(false);
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    private void testUpdateInvocations(final boolean emptyCache) throws IgniteCheckedException {
+        IgniteCache<Integer, Integer> cache0 = grid(0).cache(DEFAULT_CACHE_NAME);
+
+        final Integer key = primaryKey(jcache(0));
+
+        if (emptyCache)
+            cache0.remove(key);
+        else
+            cache0.put(key, 0);
+
+        cache0.invoke(key, updatingProcessor);
+
+        assertEquals(1, cache0.localMetrics().getEntryProcessorPuts());
+
+        if (emptyCache) {
+            assertEquals(1, cache0.localMetrics().getEntryProcessorMisses());
+
+            assertEquals(100f, cache0.localMetrics().getEntryProcessorMissPercentage());
+            assertEquals(0f, cache0.localMetrics().getEntryProcessorHitPercentage());
+        }
+        else {
+            assertEquals(1, cache0.localMetrics().getEntryProcessorHits());
+
+            assertEquals(0f, cache0.localMetrics().getEntryProcessorMissPercentage());
+            assertEquals(100f, cache0.localMetrics().getEntryProcessorHitPercentage());
+        }
+
+        for (int i = 1; i < gridCount(); i++) {
+            Ignite ignite = ignite(i);
+
+            IgniteCache<Integer, Integer> cache = ignite.cache(DEFAULT_CACHE_NAME);
+
+            if (affinity(cache).isPrimaryOrBackup(ignite.cluster().localNode(), key))
+                assertEquals(1, cache.localMetrics().getEntryProcessorPuts());
+        }
+
+        assertEquals(1, cache0.localMetrics().getEntryProcessorInvocations());
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testReadOnlyInvocationsOnEmptyCache() throws IgniteCheckedException {
+        testReadOnlyInvocations(true);
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testReadOnlyInvocations() throws IgniteCheckedException {
+        testReadOnlyInvocations(false);
+    }
+
+    /**
+     * @param emptyCache Empty cache.
+     * @throws IgniteCheckedException If failed.
+     */
+    private void testReadOnlyInvocations(final boolean emptyCache) throws IgniteCheckedException {
+        IgniteCache<Integer, Integer> cache0 = grid(0).cache(DEFAULT_CACHE_NAME);
+
+        final Integer key = primaryKey(jcache(0));
+
+        if (emptyCache)
+            cache0.remove(key);
+        else
+            cache0.put(key, 0);
+
+        cache0.invoke(key, readingProcessor);
+
+        assertEquals(1, cache0.localMetrics().getEntryProcessorReadOnlyInvocations());
+
+        if (emptyCache) {
+            assertEquals(1, cache0.localMetrics().getEntryProcessorMisses());
+
+            assertEquals(100f, cache0.localMetrics().getEntryProcessorMissPercentage());
+            assertEquals(0f, cache0.localMetrics().getEntryProcessorHitPercentage());
+        }
+        else {
+            assertEquals(1, cache0.localMetrics().getEntryProcessorHits());
+
+            assertEquals(0f, cache0.localMetrics().getEntryProcessorMissPercentage());
+            assertEquals(100f, cache0.localMetrics().getEntryProcessorHitPercentage());
+        }
+
+        for (int i = 1; i < gridCount(); i++)
+            assertEquals(0, jcache(i).localMetrics().getEntryProcessorReadOnlyInvocations());
+
+        assertEquals(1, cache0.localMetrics().getEntryProcessorInvocations());
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testInvokeAvgTime() throws IgniteCheckedException {
+        IgniteCache<Integer, Integer> cache0 = grid(0).cache(DEFAULT_CACHE_NAME);
+
+        float averageTime = cache0.localMetrics().getEntryProcessorAverageInvocationTime();
+
+        assertEquals(0.0, averageTime, 0.001f);
+
+        final Integer key = primaryKey(cache0);
+
+        cache0.invoke(key, new CacheEntryProcessor<Integer, Integer, Object>() {
+            @Override
+            public Object process(MutableEntry<Integer, Integer> entry,
+                                  Object... arguments) throws EntryProcessorException {
+
+                entry.setValue(1);
+
+                try {
+                    Thread.sleep(100);
+                } catch (InterruptedException e) {
+                    throw new EntryProcessorException(e);
+                }
+
+                return null;
+            }
+        });
+
+        averageTime = cache0.localMetrics().getEntryProcessorAverageInvocationTime();
+
+        assertTrue(averageTime > 0.0);
+
+        float maxTime = cache0.localMetrics().getEntryProcessorMaxInvocationTime();
+        float minTime = cache0.localMetrics().getEntryProcessorMinInvocationTime();
+
+        assertTrue(maxTime > 0.0);
+        assertEquals(maxTime, minTime, 0.001f);
+
+        cache0.invoke(key, new CacheEntryProcessor<Integer, Integer, Object>() {
+            @Override
+            public Object process(MutableEntry<Integer, Integer> entry,
+                                  Object... arguments) throws EntryProcessorException {
+
+                entry.setValue(1);
+
+                try {
+                    Thread.sleep(200);
+                } catch (InterruptedException e) {
+                    throw new EntryProcessorException(e);
+                }
+
+                return null;
+            }
+        });
+
+        maxTime = cache0.localMetrics().getEntryProcessorMaxInvocationTime();
+        minTime = cache0.localMetrics().getEntryProcessorMinInvocationTime();
+        averageTime = cache0.localMetrics().getEntryProcessorAverageInvocationTime();
+
+        assertTrue(maxTime > averageTime && averageTime > minTime);
+        assertEquals(2, cache0.localMetrics().getEntryProcessorInvocations());
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testInvokeAsyncAvgTime() throws IgniteCheckedException {
+        IgniteCache<Integer, Integer> cache = grid(0).cache(DEFAULT_CACHE_NAME);
+
+        assertEquals(0.0, cache.localMetrics().getEntryProcessorAverageInvocationTime(), 0.001f);
+
+        final Integer key = primaryKey(cache);
+
+        cache.invokeAsync(key, updatingProcessor).get();
+
+        U.sleep(100);
+
+        assertTrue(cache.localMetrics().getEntryProcessorAverageInvocationTime() > 0.0);
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testInvokeAllAvgTime() throws IgniteCheckedException {
+        IgniteCache<Integer, Integer> cache = grid(0).cache(DEFAULT_CACHE_NAME);
+
+        assertEquals(0.0, cache.localMetrics().getEntryProcessorAverageInvocationTime(), 0.001f);
+
+        cache.invokeAll(ImmutableMap.of(0, updatingProcessor,
+            1, readingProcessor,
+            2, removingProcessor));
+
+        U.sleep(100);
+
+        assertTrue(cache.localMetrics().getEntryProcessorAverageInvocationTime() > 0.0);
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testInvokeAllAsyncAvgTime() throws IgniteCheckedException {
+        IgniteCache<Integer, Integer> cache = grid(0).cache(DEFAULT_CACHE_NAME);
+
+        assertEquals(0.0, cache.localMetrics().getEntryProcessorAverageInvocationTime(), 0.001f);
+
+        Map<Integer, EntryProcessorResult<Object>> invokeFut = cache.invokeAllAsync(
+                ImmutableMap.of(0, updatingProcessor,
+                        1, readingProcessor,
+                        2, removingProcessor)).get();
+
+        U.sleep(100);
+
+        assertTrue(cache.localMetrics().getEntryProcessorAverageInvocationTime() > 0.0);
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testInvokeAllMultipleKeysAvgTime() throws IgniteCheckedException {
+        IgniteCache<Integer, Integer> cache = grid(0).cache(DEFAULT_CACHE_NAME);
+
+        Set<Integer> keys = new HashSet<>();
+        keys.add(1);
+        keys.add(2);
+
+        assertEquals(0.0, cache.localMetrics().getEntryProcessorAverageInvocationTime(), 0.001f);
+
+        cache.invokeAll(keys, updatingProcessor);
+
+        U.sleep(100);
+
+        assertTrue(cache.localMetrics().getEntryProcessorAverageInvocationTime() > 0.0);
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testInvokeAllAsyncMultipleKeysAvgTime() throws IgniteCheckedException {
+        IgniteCache<Integer, Integer> cache = grid(0).cache(DEFAULT_CACHE_NAME);
+
+        Set<Integer> keys = new HashSet<>();
+        keys.add(1);
+        keys.add(2);
+
+        assertEquals(0.0, cache.localMetrics().getEntryProcessorAverageInvocationTime(), 0.001f);
+
+        cache.invokeAllAsync(keys, updatingProcessor).get();
+
+        U.sleep(100);
+
+        assertTrue(cache.localMetrics().getEntryProcessorAverageInvocationTime() > 0.0);
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/5e8669af/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
index daf96c4..99abd70 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
@@ -478,7 +478,8 @@ public class GridCacheTestEntryEx extends GridMetadataAwareAdapter implements Gr
         @Nullable CacheEntryPredicate[] filter,
         boolean intercept,
         UUID subjId,
-        String taskName)
+        String taskName,
+        boolean transformOp)
         throws IgniteCheckedException, GridCacheEntryRemovedException {
         return new GridTuple3<>(false, null, null);
     }
@@ -512,8 +513,9 @@ public class GridCacheTestEntryEx extends GridMetadataAwareAdapter implements Gr
         String taskName,
         @Nullable CacheObject prevVal,
         @Nullable Long updateCntr,
-        @Nullable GridDhtAtomicAbstractUpdateFuture fut) throws IgniteCheckedException,
-        GridCacheEntryRemovedException {
+        @Nullable GridDhtAtomicAbstractUpdateFuture fut,
+        boolean transformOp)
+        throws IgniteCheckedException, GridCacheEntryRemovedException {
         assert false;
 
         return null;

http://git-wip-us.apache.org/repos/asf/ignite/blob/5e8669af/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearAtomicMetricsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearAtomicMetricsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearAtomicMetricsSelfTest.java
new file mode 100644
index 0000000..70cebff
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearAtomicMetricsSelfTest.java
@@ -0,0 +1,35 @@
+/*
+ * 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.distributed.near;
+
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.configuration.CacheConfiguration;
+
+/**
+ * Atomic cache metrics test.
+ */
+public class GridCacheNearAtomicMetricsSelfTest extends GridCacheNearMetricsSelfTest {
+    /** {@inheritDoc} */
+    @Override protected CacheConfiguration cacheConfiguration(String igniteInstanceName) throws Exception {
+        CacheConfiguration ccfg = super.cacheConfiguration(igniteInstanceName);
+
+        ccfg.setAtomicityMode(CacheAtomicityMode.ATOMIC);
+
+        return ccfg;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5e8669af/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearMetricsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearMetricsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearMetricsSelfTest.java
index 341b1fe..70b06f3 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearMetricsSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearMetricsSelfTest.java
@@ -17,8 +17,11 @@
 
 package org.apache.ignite.internal.processors.cache.distributed.near;
 
+import javax.cache.processor.EntryProcessorException;
+import javax.cache.processor.MutableEntry;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.CacheEntryProcessor;
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.cache.CacheWriteSynchronizationMode;
 import org.apache.ignite.cluster.ClusterNode;
@@ -446,4 +449,389 @@ public class GridCacheNearMetricsSelfTest extends GridCacheAbstractSelfTest {
             }
         }
     }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCreateReadRemoveInvokesFromPrimary() throws Exception {
+        Ignite g0 = grid(0);
+
+        IgniteCache<Integer, Integer> cache0 = g0.cache(DEFAULT_CACHE_NAME);
+
+        int key = primaryKey(cache0);
+
+        setValue1ByEntryProcessor(cache0, key);
+
+        readKeyByEntryProcessor(cache0, key);
+
+        removeKeyByEntryProcessor(cache0, key);
+
+        for (int j = 0; j < gridCount(); j++) {
+            Ignite g = grid(j);
+
+            IgniteCache<Object, Object> jcache = g.cache(DEFAULT_CACHE_NAME);
+
+            if (affinity(jcache).isPrimaryOrBackup(g.cluster().localNode(), key))
+                assertEquals(1, jcache.localMetrics().getEntryProcessorPuts());
+            else
+                assertEquals(0, jcache.localMetrics().getEntryProcessorPuts());
+
+            if (affinity(jcache).isPrimary(g.cluster().localNode(), key)) {
+                assertEquals(1, jcache.localMetrics().getEntryProcessorPuts());
+                assertEquals(1, jcache.localMetrics().getEntryProcessorRemovals());
+                assertEquals(1, jcache.localMetrics().getEntryProcessorReadOnlyInvocations());
+                assertEquals(3, jcache.localMetrics().getEntryProcessorInvocations());
+
+                assertEquals(1, jcache.localMetrics().getEntryProcessorMisses());
+                assertEquals(2, jcache.localMetrics().getEntryProcessorHits());
+
+                assertEquals((float) 1 / 3 * 100.0f, jcache.localMetrics().getEntryProcessorMissPercentage(), 0.001f);
+                assertEquals((float) 2 / 3 * 100.0f, jcache.localMetrics().getEntryProcessorHitPercentage(), 0.001f);
+            }
+            else if (affinity(jcache).isBackup(g.cluster().localNode(), key)) {
+                assertEquals(1, jcache.localMetrics().getEntryProcessorPuts());
+                assertEquals(1, jcache.localMetrics().getEntryProcessorRemovals());
+                assertEquals(0, jcache.localMetrics().getEntryProcessorReadOnlyInvocations());
+                assertEquals(2, jcache.localMetrics().getEntryProcessorInvocations());
+
+                assertEquals(1, jcache.localMetrics().getEntryProcessorMisses());
+                assertEquals(1, jcache.localMetrics().getEntryProcessorHits());
+
+                assertEquals(50.0f, jcache.localMetrics().getEntryProcessorMissPercentage(), 0.001f);
+                assertEquals(50.0f, jcache.localMetrics().getEntryProcessorHitPercentage(), 0.001f);
+            }
+            else
+                assertNoMetricsChanged(jcache);
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCreateReadRemoveInvokesFromBackup() throws Exception {
+        Ignite g0 = grid(0);
+
+        IgniteCache<Integer, Integer> cache0 = g0.cache(DEFAULT_CACHE_NAME);
+
+        int key = backupKey(cache0);
+
+        setValue1ByEntryProcessor(cache0, key);
+
+        readKeyByEntryProcessor(cache0, key);
+
+        removeKeyByEntryProcessor(cache0, key);
+
+        for (int j = 0; j < gridCount(); j++) {
+            Ignite g = grid(j);
+
+            IgniteCache<Object, Object> jcache = g.cache(DEFAULT_CACHE_NAME);
+
+            if (affinity(jcache).isPrimaryOrBackup(g.cluster().localNode(), key))
+                assertEquals(1, jcache.localMetrics().getEntryProcessorPuts());
+            else
+                assertEquals(0, jcache.localMetrics().getEntryProcessorPuts());
+
+            if (affinity(jcache).isPrimary(g.cluster().localNode(), key)) {
+                assertEquals(1, jcache.localMetrics().getEntryProcessorPuts());
+                assertEquals(1, jcache.localMetrics().getEntryProcessorRemovals());
+                assertEquals(1, jcache.localMetrics().getEntryProcessorReadOnlyInvocations());
+                assertEquals(3, jcache.localMetrics().getEntryProcessorInvocations());
+
+                assertEquals(1, jcache.localMetrics().getEntryProcessorMisses());
+                assertEquals(2, jcache.localMetrics().getEntryProcessorHits());
+
+                assertEquals((float) 1 / 3 * 100.0f, jcache.localMetrics().getEntryProcessorMissPercentage(), 0.001f);
+                assertEquals((float) 2 / 3 * 100.0f, jcache.localMetrics().getEntryProcessorHitPercentage(), 0.001f);
+            }
+            else if (affinity(jcache).isBackup(g.cluster().localNode(), key)) {
+                assertEquals(1, jcache.localMetrics().getEntryProcessorPuts());
+                assertEquals(1, jcache.localMetrics().getEntryProcessorRemovals());
+                assertEquals(0, jcache.localMetrics().getEntryProcessorReadOnlyInvocations());
+                assertEquals(2, jcache.localMetrics().getEntryProcessorInvocations());
+
+                assertEquals(1, jcache.localMetrics().getEntryProcessorMisses());
+                assertEquals(1, jcache.localMetrics().getEntryProcessorHits());
+
+                assertEquals(50.0f, jcache.localMetrics().getEntryProcessorMissPercentage(), 0.001f);
+                assertEquals(50.0f, jcache.localMetrics().getEntryProcessorHitPercentage(), 0.001f);
+            }
+            else
+                assertNoMetricsChanged(jcache);
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCreateReadRemoveInvokesFromNear() throws Exception {
+        Ignite g0 = grid(0);
+
+        IgniteCache<Integer, Integer> cache0 = g0.cache(DEFAULT_CACHE_NAME);
+
+        int key;
+
+        for (int i = 0; ; i++) {
+            if (!affinity(cache0).isPrimaryOrBackup(g0.cluster().localNode(), i)) {
+                setValue1ByEntryProcessor(cache0, i);
+
+                readKeyByEntryProcessor(cache0, i);
+
+                removeKeyByEntryProcessor(cache0, i);
+
+                key = i;
+
+                break;
+            }
+        }
+
+        for (int j = 0; j < gridCount(); j++) {
+            Ignite g = grid(j);
+
+            IgniteCache<Object, Object> jcache = g.cache(DEFAULT_CACHE_NAME);
+
+            assertEquals(1, jcache.localMetrics().getEntryProcessorPuts());
+            assertEquals(1, jcache.localMetrics().getEntryProcessorRemovals());
+            assertEquals(1, jcache.localMetrics().getEntryProcessorMisses());
+
+            if (affinity(jcache).isPrimary(g.cluster().localNode(), key)) {
+                assertEquals(1, jcache.localMetrics().getEntryProcessorReadOnlyInvocations());
+                assertEquals(1, jcache.localMetrics().getEntryProcessorRemovals());
+                assertEquals(3, jcache.localMetrics().getEntryProcessorInvocations());
+
+                assertEquals(2, jcache.localMetrics().getEntryProcessorHits());
+
+                assertEquals((float) 1 / 3 * 100.0f, jcache.localMetrics().getEntryProcessorMissPercentage(), 0.001f);
+                assertEquals((float) 2 / 3 * 100.0f, jcache.localMetrics().getEntryProcessorHitPercentage(), 0.001f);
+            } else {
+                assertEquals(0, jcache.localMetrics().getEntryProcessorReadOnlyInvocations());
+                assertEquals(1, jcache.localMetrics().getEntryProcessorRemovals());
+                assertEquals(2, jcache.localMetrics().getEntryProcessorInvocations());
+
+                assertEquals(1, jcache.localMetrics().getEntryProcessorHits());
+
+                assertEquals(50.0f, jcache.localMetrics().getEntryProcessorMissPercentage(), 0.001f);
+                assertEquals(50.0f, jcache.localMetrics().getEntryProcessorHitPercentage(), 0.001f);
+            }
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testReadRemoveInvokesFromPrimary() throws Exception {
+        Ignite g0 = grid(0);
+
+        IgniteCache<Integer, Integer> cache0 = g0.cache(DEFAULT_CACHE_NAME);
+
+        int key = primaryKey(cache0);
+
+        readKeyByEntryProcessor(cache0, key);
+
+        removeKeyByEntryProcessor(cache0, key);
+
+        for (int j = 0; j < gridCount(); j++) {
+            Ignite g = grid(j);
+
+            IgniteCache<Object, Object> jcache = g.cache(DEFAULT_CACHE_NAME);
+
+            if (affinity(jcache).isPrimary(g.cluster().localNode(), key)) {
+                assertEquals(0, jcache.localMetrics().getEntryProcessorPuts());
+                assertEquals(1, jcache.localMetrics().getEntryProcessorReadOnlyInvocations());
+                assertEquals(1, jcache.localMetrics().getEntryProcessorRemovals());
+
+                assertEquals(2, jcache.localMetrics().getEntryProcessorInvocations());
+
+                assertEquals(2, jcache.localMetrics().getEntryProcessorMisses());
+
+                assertEquals(100.0f, jcache.localMetrics().getEntryProcessorMissPercentage(), 0.001f);
+            }
+            else if (affinity(jcache).isBackup(g.cluster().localNode(), key)) {
+                assertEquals(0, jcache.localMetrics().getEntryProcessorPuts());
+                assertEquals(0, jcache.localMetrics().getEntryProcessorReadOnlyInvocations());
+
+                assertEquals(1, jcache.localMetrics().getEntryProcessorRemovals());
+                assertEquals(1, jcache.localMetrics().getEntryProcessorInvocations());
+
+                assertEquals(1, jcache.localMetrics().getEntryProcessorMisses());
+
+                assertEquals(100.0f, jcache.localMetrics().getEntryProcessorMissPercentage(), 0.001f);
+            }
+            else
+                assertNoMetricsChanged(jcache);
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testReadRemoveInvokesFromBackup() throws Exception {
+        Ignite g0 = grid(0);
+
+        IgniteCache<Integer, Integer> cache0 = g0.cache(DEFAULT_CACHE_NAME);
+
+        int key = backupKey(cache0);
+
+        readKeyByEntryProcessor(cache0, key);
+
+        removeKeyByEntryProcessor(cache0, key);
+
+        for (int j = 0; j < gridCount(); j++) {
+            Ignite g = grid(j);
+
+            IgniteCache<Object, Object> jcache = g.cache(DEFAULT_CACHE_NAME);
+
+            if (affinity(jcache).isPrimary(g.cluster().localNode(), key)) {
+                assertEquals(0, jcache.localMetrics().getEntryProcessorPuts());
+                assertEquals(1, jcache.localMetrics().getEntryProcessorRemovals());
+                assertEquals(1, jcache.localMetrics().getEntryProcessorReadOnlyInvocations());
+                assertEquals(2, jcache.localMetrics().getEntryProcessorInvocations());
+
+                assertEquals(2, jcache.localMetrics().getEntryProcessorMisses());
+
+                assertEquals(100.0f, jcache.localMetrics().getEntryProcessorMissPercentage(), 0.001f);
+            }
+            else if (affinity(jcache).isBackup(g.cluster().localNode(), key)) {
+
+                assertEquals(0, jcache.localMetrics().getEntryProcessorPuts());
+                assertEquals(1, jcache.localMetrics().getEntryProcessorRemovals());
+                assertEquals(0, jcache.localMetrics().getEntryProcessorReadOnlyInvocations());
+                assertEquals(1, jcache.localMetrics().getEntryProcessorInvocations());
+
+                assertEquals(1, jcache.localMetrics().getEntryProcessorMisses());
+
+                assertEquals(100.0f, jcache.localMetrics().getEntryProcessorMissPercentage(), 0.001f);
+            }
+            else
+                assertNoMetricsChanged(jcache);
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testReadRemoveInvokesFromNear() throws Exception {
+        Ignite g0 = grid(0);
+
+        IgniteCache<Integer, Integer> cache0 = g0.cache(DEFAULT_CACHE_NAME);
+
+        int key;
+
+        for (int i = 0; ; i++) {
+            if (!affinity(cache0).isPrimaryOrBackup(g0.cluster().localNode(), i)) {
+                readKeyByEntryProcessor(cache0, i);
+
+                removeKeyByEntryProcessor(cache0, i);
+
+                key = i;
+
+                break;
+            }
+        }
+
+        for (int j = 0; j < gridCount(); j++) {
+            Ignite g = grid(j);
+
+            IgniteCache<Object, Object> jcache = g.cache(DEFAULT_CACHE_NAME);
+
+            if (affinity(jcache).isPrimary(g.cluster().localNode(), key)) {
+                assertEquals(0, jcache.localMetrics().getEntryProcessorPuts());
+                assertEquals(1, jcache.localMetrics().getEntryProcessorRemovals());
+                assertEquals(1, jcache.localMetrics().getEntryProcessorReadOnlyInvocations());
+                assertEquals(2, jcache.localMetrics().getEntryProcessorInvocations());
+
+                assertEquals(2, jcache.localMetrics().getEntryProcessorMisses());
+
+                assertEquals(100.0f, jcache.localMetrics().getEntryProcessorMissPercentage(), 0.001f);
+            }
+            else if (affinity(jcache).isBackup(g.cluster().localNode(), key)) {
+                assertEquals(0, jcache.localMetrics().getEntryProcessorPuts());
+                assertEquals(1, jcache.localMetrics().getEntryProcessorRemovals());
+                assertEquals(0, jcache.localMetrics().getEntryProcessorReadOnlyInvocations());
+                assertEquals(1, jcache.localMetrics().getEntryProcessorInvocations());
+
+                assertEquals(1, jcache.localMetrics().getEntryProcessorMisses());
+
+                assertEquals(100.0f, jcache.localMetrics().getEntryProcessorMissPercentage(), 0.001f);
+            }
+            else {
+                assertEquals(0, jcache.localMetrics().getEntryProcessorPuts());
+                assertEquals(1, jcache.localMetrics().getEntryProcessorRemovals());
+                assertEquals(0, jcache.localMetrics().getEntryProcessorReadOnlyInvocations());
+                assertEquals(1, jcache.localMetrics().getEntryProcessorInvocations());
+
+                assertEquals(1, jcache.localMetrics().getEntryProcessorMisses());
+
+                assertEquals(100.0f, jcache.localMetrics().getEntryProcessorMissPercentage(), 0.001f);
+            }
+        }
+    }
+
+    /**
+     * Checks no metrics changed in cache.
+     *
+     * @param jcache Cache to be checked.
+     */
+    private void assertNoMetricsChanged(IgniteCache<Object, Object> jcache) {
+        assertEquals(0, jcache.localMetrics().getEntryProcessorPuts());
+        assertEquals(0, jcache.localMetrics().getEntryProcessorRemovals());
+        assertEquals(0, jcache.localMetrics().getEntryProcessorReadOnlyInvocations());
+        assertEquals(0, jcache.localMetrics().getEntryProcessorInvocations());
+
+        assertEquals(0, jcache.localMetrics().getEntryProcessorMisses());
+        assertEquals(0, jcache.localMetrics().getEntryProcessorHits());
+
+        assertEquals(0, jcache.localMetrics().getEntryProcessorMissPercentage(), 0.001f);
+        assertEquals(0, jcache.localMetrics().getEntryProcessorHitPercentage(), 0.001f);
+    }
+
+    /**
+     * Invokes entry processor, which removes key from cache.
+     *
+     * @param cache Cache.
+     * @param key Key.
+     */
+    private void removeKeyByEntryProcessor(IgniteCache<Integer, Integer> cache, int key) {
+        cache.invoke(key, new CacheEntryProcessor<Integer, Integer, Object>() {
+            @Override public Object process(MutableEntry<Integer, Integer> entry,
+                Object... arguments) throws EntryProcessorException {
+                entry.remove();
+
+                return null;
+            }
+        });
+    }
+
+    /**
+     * Invokes entry processor, which reads key from cache.
+     *
+     * @param cache Cache.
+     * @param key Key.
+     */
+    private void readKeyByEntryProcessor(IgniteCache<Integer, Integer> cache, int key) {
+        cache.invoke(key, new CacheEntryProcessor<Integer, Integer, Object>() {
+            @Override public Object process(MutableEntry<Integer, Integer> entry,
+                Object... arguments) throws EntryProcessorException {
+                entry.getValue();
+
+                return null;
+            }
+        });
+    }
+
+    /**
+     * Invokes entry processor, which sets value "1" for key into cache.
+     *
+     * @param cache Cache.
+     * @param key Key.
+     */
+    private void setValue1ByEntryProcessor(IgniteCache<Integer, Integer> cache, int key) {
+        cache.invoke(key, new CacheEntryProcessor<Integer, Integer, Object>() {
+            @Override public Object process(MutableEntry<Integer, Integer> entry,
+                Object... arguments) throws EntryProcessorException {
+                entry.setValue(1);
+
+                return null;
+            }
+        });
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/5e8669af/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 fe61e35..f4c1642 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
@@ -483,5 +483,60 @@ public class PlatformCacheWriteMetricsTask extends ComputeTaskAdapter<Long, Obje
         @Override public long getEstimatedRebalancingKeys() {
             return 67;
         }
+
+        /** {@inheritDoc} */
+        @Override public long getEntryProcessorPuts() {
+            return 68;
+        }
+
+        /** {@inheritDoc} */
+        @Override public long getEntryProcessorReadOnlyInvocations() {
+            return 69;
+        }
+
+        /** {@inheritDoc} */
+        @Override public long getEntryProcessorInvocations() {
+            return 70;
+        }
+
+        /** {@inheritDoc} */
+        @Override public long getEntryProcessorHits() {
+            return 71;
+        }
+
+        /** {@inheritDoc} */
+        @Override public float getEntryProcessorHitPercentage() {
+            return 72;
+        }
+
+        /** {@inheritDoc} */
+        @Override public float getEntryProcessorMissPercentage() {
+            return 73;
+        }
+
+        /** {@inheritDoc} */
+        @Override public long getEntryProcessorMisses() {
+            return 74;
+        }
+
+        /** {@inheritDoc} */
+        @Override public float getEntryProcessorAverageInvocationTime() {
+            return 75;
+        }
+
+        /** {@inheritDoc} */
+        @Override public float getEntryProcessorMinInvocationTime() {
+            return 76;
+        }
+
+        /** {@inheritDoc} */
+        @Override public float getEntryProcessorMaxInvocationTime() {
+            return 77;
+        }
+
+        /** {@inheritDoc} */
+        @Override public long getEntryProcessorRemovals() {
+            return 78;
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/5e8669af/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheMetricsSelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheMetricsSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheMetricsSelfTestSuite.java
index ac4b512..6f62ffb 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheMetricsSelfTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheMetricsSelfTestSuite.java
@@ -29,6 +29,7 @@ import org.apache.ignite.internal.processors.cache.GridEvictionPolicyMBeansTest;
 import org.apache.ignite.internal.processors.cache.OffheapCacheMetricsForClusterGroupSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicPartitionedMetricsSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicPartitionedTckMetricsSelfTestImpl;
+import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheNearAtomicMetricsSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheNearMetricsSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedHitsAndMissesSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedMetricsSelfTest;
@@ -54,6 +55,7 @@ public class IgniteCacheMetricsSelfTestSuite extends TestSuite {
         suite.addTestSuite(GridCacheLocalMetricsSelfTest.class);
         suite.addTestSuite(GridCacheLocalAtomicMetricsNoReadThroughSelfTest.class);
         suite.addTestSuite(GridCacheNearMetricsSelfTest.class);
+        suite.addTestSuite(GridCacheNearAtomicMetricsSelfTest.class);
         suite.addTestSuite(GridCacheReplicatedMetricsSelfTest.class);
         suite.addTestSuite(GridCachePartitionedMetricsSelfTest.class);
         suite.addTestSuite(GridCachePartitionedHitsAndMissesSelfTest.class);

http://git-wip-us.apache.org/repos/asf/ignite/blob/5e8669af/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheMetricsTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheMetricsTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheMetricsTest.cs
index 129b4b5..d5f8d9e 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheMetricsTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheMetricsTest.cs
@@ -219,6 +219,17 @@ namespace Apache.Ignite.Core.Tests.Cache
                 Assert.AreEqual(true, metrics.IsWriteThrough);
                 Assert.AreEqual(true, metrics.IsValidForReading);
                 Assert.AreEqual(true, metrics.IsValidForWriting);
+                Assert.AreEqual(68, metrics.EntryProcessorPuts);
+                Assert.AreEqual(69, metrics.EntryProcessorReadOnlyInvocations);
+                Assert.AreEqual(70, metrics.EntryProcessorInvocations);
+                Assert.AreEqual(71, metrics.EntryProcessorHits);
+                Assert.AreEqual(72, metrics.EntryProcessorHitPercentage);
+                Assert.AreEqual(73, metrics.EntryProcessorMissPercentage);
+                Assert.AreEqual(74, metrics.EntryProcessorMisses);
+                Assert.AreEqual(75, metrics.EntryProcessorAverageInvocationTime);
+                Assert.AreEqual(76, metrics.EntryProcessorMinInvocationTime);
+                Assert.AreEqual(77, metrics.EntryProcessorMaxInvocationTime);
+                Assert.AreEqual(78, metrics.EntryProcessorRemovals);
             }
         }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/5e8669af/modules/platforms/dotnet/Apache.Ignite.Core/Cache/ICacheMetrics.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/ICacheMetrics.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/ICacheMetrics.cs
index e0e7301..53be9ef 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/ICacheMetrics.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/ICacheMetrics.cs
@@ -672,5 +672,93 @@ namespace Apache.Ignite.Core.Cache
         /// Number of estimated keys to rebalance.
         /// </returns>
         long EstimatedRebalancingKeys { get; }
+
+        /// <summary>
+        /// The total number of cache invocations, caused update.
+        /// </summary>
+        /// <returns>
+        /// The number of invocation updates.
+        /// </returns>
+        long EntryProcessorPuts { get; }
+
+        /// <summary>
+        /// The total number of cache invocations, caused removal.
+        /// </summary>
+        /// <returns>
+        /// The number of invocation removals.
+        /// </returns>
+        long EntryProcessorRemovals { get; }
+
+        /// <summary>
+        /// The total number of cache invocations, caused no updates.
+        /// </summary>
+        /// <returns>
+        /// The number of read-only invocations.
+        /// </returns>
+        long EntryProcessorReadOnlyInvocations { get; }
+
+        /// <summary>
+        /// The total number of cache invocations.
+        /// </summary>
+        /// <returns>
+        /// The number of cache invocations.
+        /// </returns>
+        long EntryProcessorInvocations { get; }
+
+        /// <summary>
+        /// The total number of invocations on keys, which exist in cache.
+        /// </summary>
+        /// <returns>
+        /// The number of cache invocation hits.
+        /// </returns>
+        long EntryProcessorHits { get; }
+
+        /// <summary>
+        /// The percentage of invocations on keys, which exist in cache.
+        /// </summary>
+        /// <returns>
+        /// The percentage of successful invocation hits.
+        /// </returns>
+        float EntryProcessorHitPercentage { get; }
+
+        /// <summary>
+        /// The total number of invocations on keys, which don't exist in cache.
+        /// </summary>
+        /// <returns>
+        /// The number of cache invocation misses.
+        /// </returns>
+        long EntryProcessorMisses { get; }
+
+        /// <summary>
+        /// The percentage of invocations on keys, which don't exist in cache.
+        /// </summary>
+        /// <returns>
+        /// The percentage of invocation misses.
+        /// </returns>
+        float EntryProcessorMissPercentage { get; }
+
+        /// <summary>
+        /// The mean time to execute cache invokes.
+        /// </summary>
+        /// <returns>
+        /// The time in µs.
+        /// </returns>
+        float EntryProcessorAverageInvocationTime { get; }
+
+        /// <summary>
+        /// So far, the minimum time to execute cache invokes.
+        /// </summary>
+        /// <returns>
+        /// The time in µs.
+        /// </returns>
+        float EntryProcessorMinInvocationTime { get; }
+
+        /// <summary>
+        /// So far, the maximum time to execute cache invokes.
+        /// </summary>
+        /// <returns>
+        /// The time in µs.
+        /// </returns>
+        float EntryProcessorMaxInvocationTime { get; }
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5e8669af/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheMetricsImpl.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheMetricsImpl.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheMetricsImpl.cs
index be6980d..f34fa3e 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheMetricsImpl.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheMetricsImpl.cs
@@ -253,6 +253,39 @@ namespace Apache.Ignite.Core.Impl.Cache
         /** */
         private readonly long _estimatedRebalancedKeys;
 
+        /** */
+        private readonly long _entryProcessorPuts;
+
+        /** */
+        private readonly float _entryProcessorAverageInvocationTime;
+
+        /** */
+        private readonly long _entryProcessorInvocations;
+
+        /** */
+        private readonly float _entryProcessorMaxInvocationTime;
+
+        /** */
+        private readonly float _entryProcessorMinInvocationTime;
+
+        /** */
+        private readonly long _entryProcessorReadOnlyInvocations;
+
+        /** */
+        private readonly float _entryProcessorHitPercentage;
+
+        /** */
+        private readonly long _entryProcessorHits;
+
+        /** */
+        private readonly long _entryProcessorMisses;
+
+        /** */
+        private readonly float _entryProcessorMissPercentage;
+
+        /** */
+        private readonly long _entryProcessorRemovals;
+
         /// <summary>
         /// Initializes a new instance of the <see cref="CacheMetricsImpl"/> class.
         /// </summary>
@@ -335,6 +368,17 @@ namespace Apache.Ignite.Core.Impl.Cache
             _cacheSize = reader.ReadLong();
             _rebalancedKeys = reader.ReadLong();
             _estimatedRebalancedKeys = reader.ReadLong();
+            _entryProcessorPuts = reader.ReadLong();
+            _entryProcessorAverageInvocationTime = reader.ReadFloat();
+            _entryProcessorInvocations = reader.ReadLong();
+            _entryProcessorMaxInvocationTime = reader.ReadFloat();
+            _entryProcessorMinInvocationTime = reader.ReadFloat();
+            _entryProcessorReadOnlyInvocations = reader.ReadLong();
+            _entryProcessorHitPercentage = reader.ReadFloat();
+            _entryProcessorHits = reader.ReadLong();
+            _entryProcessorMisses = reader.ReadLong();
+            _entryProcessorMissPercentage = reader.ReadFloat();
+            _entryProcessorRemovals = reader.ReadLong();
         }
 
         /** <inheritDoc /> */
@@ -561,8 +605,41 @@ namespace Apache.Ignite.Core.Impl.Cache
 
         /** <inheritDoc /> */
         public long RebalancedKeys { get { return _rebalancedKeys; } }
-        
+
         /** <inheritDoc /> */
         public long EstimatedRebalancingKeys { get { return _estimatedRebalancedKeys; } }
+
+        /** <inheritDoc /> */
+        public long EntryProcessorPuts { get { return _entryProcessorPuts; } }
+
+        /** <inheritDoc /> */
+        public float EntryProcessorAverageInvocationTime { get { return _entryProcessorAverageInvocationTime; } }
+
+        /** <inheritDoc /> */
+        public long EntryProcessorInvocations { get { return _entryProcessorInvocations; } }
+
+        /** <inheritDoc /> */
+        public float EntryProcessorMaxInvocationTime { get { return _entryProcessorMaxInvocationTime; } }
+
+        /** <inheritDoc /> */
+        public float EntryProcessorMinInvocationTime { get { return _entryProcessorMinInvocationTime; } }
+
+        /** <inheritDoc /> */
+        public long EntryProcessorReadOnlyInvocations { get { return _entryProcessorReadOnlyInvocations; } }
+
+        /** <inheritDoc /> */
+        public float EntryProcessorHitPercentage { get { return _entryProcessorHitPercentage; } }
+
+        /** <inheritDoc /> */
+        public long EntryProcessorHits { get { return _entryProcessorHits; } }
+
+        /** <inheritDoc /> */
+        public long EntryProcessorMisses { get { return _entryProcessorMisses; } }
+
+        /** <inheritDoc /> */
+        public float EntryProcessorMissPercentage { get { return _entryProcessorMissPercentage; } }
+
+        /** <inheritDoc /> */
+        public long EntryProcessorRemovals { get { return _entryProcessorRemovals; } }
     }
 }
\ No newline at end of file