You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sb...@apache.org on 2015/05/12 15:18:23 UTC

[8/8] incubator-ignite git commit: ignite-37 Improve offheap metrics for cache

ignite-37 Improve offheap metrics for cache


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

Branch: refs/heads/ignite-37
Commit: c14eb5637d2b289f0278098e687e3ffd16adff7f
Parents: 08360c9
Author: agura <ag...@gridgain.com>
Authored: Thu Apr 30 20:53:53 2015 +0300
Committer: agura <ag...@gridgain.com>
Committed: Tue May 12 16:17:49 2015 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/cache/CacheMetrics.java   | 187 +++++++--
 .../internal/managers/GridManagerAdapter.java   |  59 +--
 .../swapspace/GridSwapSpaceManager.java         | 147 ++++---
 .../processors/cache/CacheMetricsImpl.java      | 305 +++++++++++++-
 .../cache/CacheMetricsMXBeanImpl.java           | 100 +++++
 .../processors/cache/CacheMetricsSnapshot.java  | 380 +++++++++++++----
 .../processors/cache/GridCacheSwapManager.java  | 152 +++----
 .../offheap/GridOffHeapProcessor.java           | 105 +++--
 .../ignite/mxbean/CacheMetricsMXBean.java       |  80 ++++
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |  35 +-
 .../org/apache/ignite/spi/IgniteSpiContext.java |  47 ---
 .../spi/swapspace/file/FileSwapSpaceSpi.java    |   8 +-
 .../swapspace/GridSwapSpaceManagerSelfTest.java |  44 +-
 ...CacheLocalOffHeapAndSwapMetricsSelfTest.java | 412 +++++++++++++++++++
 .../testframework/GridSpiTestContext.java       |  25 +-
 .../IgniteCacheMetricsSelfTestSuite.java        |   1 +
 16 files changed, 1616 insertions(+), 471 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c14eb563/modules/core/src/main/java/org/apache/ignite/cache/CacheMetrics.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/CacheMetrics.java b/modules/core/src/main/java/org/apache/ignite/cache/CacheMetrics.java
index 0d87326..799aace 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/CacheMetrics.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/CacheMetrics.java
@@ -30,21 +30,21 @@ public interface CacheMetrics {
     /**
      * The number of get requests that were satisfied by the cache.
      *
-     * @return the number of hits
+     * @return The number of hits.
      */
     public long getCacheHits();
 
     /**
      * This is a measure of cache efficiency.
      *
-     * @return the percentage of successful hits, as a decimal e.g 75.
+     * @return The percentage of successful hits, as a decimal e.g 75.
      */
     public float getCacheHitPercentage();
 
     /**
      * A miss is a get request that is not satisfied.
      *
-     * @return the number of misses
+     * @return The number of misses.
      */
     public long getCacheMisses();
 
@@ -52,7 +52,7 @@ public interface CacheMetrics {
      * Returns the percentage of cache accesses that did not find a requested entry
      * in the cache.
      *
-     * @return the percentage of accesses that failed to find anything
+     * @return The percentage of accesses that failed to find anything.
      */
     public float getCacheMissPercentage();
 
@@ -60,14 +60,14 @@ public interface CacheMetrics {
      * The total number of requests to the cache. This will be equal to the sum of
      * the hits and misses.
      *
-     * @return the number of gets
+     * @return The number of gets.
      */
     public long getCacheGets();
 
     /**
      * The total number of puts to the cache.
      *
-     * @return the number of puts
+     * @return The number of puts.
      */
     public long getCachePuts();
 
@@ -75,7 +75,7 @@ public interface CacheMetrics {
      * The total number of removals from the cache. This does not include evictions,
      * where the cache itself initiates the removal to make space.
      *
-     * @return the number of removals
+     * @return The number of removals.
      */
     public long getCacheRemovals();
 
@@ -84,28 +84,28 @@ public interface CacheMetrics {
      * initiated by the cache itself to free up space. An eviction is not treated as
      * a removal and does not appear in the removal counts.
      *
-     * @return the number of evictions
+     * @return The number of evictions.
      */
     public long getCacheEvictions();
 
     /**
      * The mean time to execute gets.
      *
-     * @return the time in µs
+     * @return The time in µs.
      */
     public float getAverageGetTime();
 
     /**
      * The mean time to execute puts.
      *
-     * @return the time in µs
+     * @return The time in µs.
      */
     public float getAveragePutTime();
 
     /**
      * The mean time to execute removes.
      *
-     * @return the time in µs
+     * @return The time in µs.
      */
     public float getAverageRemoveTime();
 
@@ -113,7 +113,7 @@ public interface CacheMetrics {
     /**
      * The mean time to execute tx commit.
      *
-     * @return the time in µs
+     * @return The time in µs.
      */
     public float getAverageTxCommitTime();
 
@@ -124,7 +124,6 @@ public interface CacheMetrics {
      */
     public float getAverageTxRollbackTime();
 
-
     /**
      * Gets total number of transaction commits.
      *
@@ -154,6 +153,62 @@ public interface CacheMetrics {
     public long getOverflowSize();
 
     /**
+     * The total number of get requests to the off-heap memory.
+     *
+     * @return The number of gets.
+     */
+    public long getOffHeapGets();
+
+    /**
+     * The total number of put requests to the off-heap memory.
+     *
+     * @return The number of puts.
+     */
+    public long getOffHeapPuts();
+
+    /**
+     * The total number of removals from the off-heap memory. This does not include evictions.
+     *
+     * @return The number of removals.
+     */
+    public long getOffHeapRemovals();
+
+    /**
+     * The total number of evictions from the off-heap memory.
+     *
+     * @return The number of evictions.
+     */
+    public long getOffHeapEvictions();
+
+    /**
+     * The number of get requests that were satisfied by the off-heap memory.
+     *
+     * @return The off-heap hits number.
+     */
+    public long getOffHeapHits();
+
+    /**
+     * Gets the percentage of hits on off-heap memory.
+     *
+     * @return The percentage of hits on off-heap memory.
+     */
+    public float getOffHeapHitPercentage();
+
+    /**
+     * A miss is a get request that is not satisfied by off-heap memory.
+     *
+     * @return The off-heap misses number.
+     */
+    public long getOffHeapMisses();
+
+    /**
+     * Gets the percentage of misses on off-heap memory.
+     *
+     * @return The percentage of misses on off-heap memory.
+     */
+    public float getOffHeapMissPercentage();
+
+    /**
      * Gets number of entries stored in off-heap memory.
      *
      * @return Number of entries stored in off-heap memory.
@@ -161,6 +216,20 @@ public interface CacheMetrics {
     public long getOffHeapEntriesCount();
 
     /**
+     * Gets number of primary entries stored in off-heap memory.
+     *
+     * @return Number of primary entries stored in off-heap memory.
+     */
+    public long getOffHeapPrimaryEntriesCount();
+
+    /**
+     * Gets number of backup entries stored in off-heap memory.
+     *
+     * @return Number of backup entries stored in off-heap memory.
+     */
+    public long getOffHeapBackupEntriesCount();
+
+    /**
      * Gets memory size allocated in off-heap.
      *
      * @return Memory size allocated in off-heap.
@@ -168,6 +237,76 @@ public interface CacheMetrics {
     public long getOffHeapAllocatedSize();
 
     /**
+     * Gets off-heap memory maximum size.
+     *
+     * @return Off-heap memory maximum size.
+     */
+    public long getOffHeapMaxSize();
+
+    /**
+     * The total number of get requests to the swap.
+     *
+     * @return The number of gets.
+     */
+    public long getSwapGets();
+
+    /**
+     * The total number of put requests to the swap.
+     *
+     * @return The number of puts.
+     */
+    public long getSwapPuts();
+
+    /**
+     * The total number of removals from the swap.
+     *
+     * @return The number of removals.
+     */
+    public long getSwapRemovals();
+
+    /**
+     * The number of get requests that were satisfied by the swap.
+     *
+     * @return The swap hits number.
+     */
+    public long getSwapHits();
+
+    /**
+     * A miss is a get request that is not satisfied by swap.
+     *
+     * @return The swap misses number.
+     */
+    public long getSwapMisses();
+
+    /**
+     * Gets number of entries stored in swap.
+     *
+     * @return Number of entries stored in swap.
+     */
+    public long getSwapEntriesCount();
+
+    /**
+     * Gets size of swap.
+     *
+     * @return Size of swap.
+     */
+    public long getSwapSize();
+
+    /**
+     * Gets the percentage of hits on swap.
+     *
+     * @return The percentage of hits on swap.
+     */
+    public float getSwapHitPercentage();
+
+    /**
+     * Gets the percentage of misses on swap.
+     *
+     * @return The percentage of misses on swap.
+     */
+    public float getSwapMissPercentage();
+
+    /**
      * Gets number of non-{@code null} values in the cache.
      *
      * @return Number of non-{@code null} values in the cache.
@@ -184,7 +323,7 @@ public interface CacheMetrics {
     /**
      * Returns {@code true} if this cache is empty.
      *
-     * @return {@code true} if this cache is empty.
+     * @return {@code True} if this cache is empty.
      */
     public boolean isEmpty();
 
@@ -294,7 +433,7 @@ public interface CacheMetrics {
     public int getTxDhtRolledbackVersionsSize();
 
     /**
-     * Returns {@code True} if write-behind is enabled.
+     * Returns {@code true} if write-behind is enabled.
      *
      * @return {@code True} if write-behind is enabled.
      */
@@ -372,16 +511,16 @@ public interface CacheMetrics {
     /**
      * Determines the required type of keys for this {@link Cache}, if any.
      *
-     * @return the fully qualified class name of the key type,
-     * or "java.lang.Object" if the type is undefined.
+     * @return The fully qualified class name of the key type,
+     * or {@code "java.lang.Object"} if the type is undefined.
      */
     public String getKeyType();
 
     /**
      * Determines the required type of values for this {@link Cache}, if any.
      *
-     * @return the fully qualified class name of the value type,
-     *         or "java.lang.Object" if the type is undefined.
+     * @return The fully qualified class name of the value type,
+     *         or {@code "java.lang.Object"} if the type is undefined.
      */
     public String getValueType();
 
@@ -407,7 +546,7 @@ public interface CacheMetrics {
      * <p>
      * The default value is {@code true}.
      *
-     * @return true if the cache is store by value
+     * @return {@code True} if the cache is store by value.
      */
     public boolean isStoreByValue();
 
@@ -416,7 +555,7 @@ public interface CacheMetrics {
      * <p>
      * The default value is {@code false}.
      *
-     * @return true if statistics collection is enabled
+     * @return {@code True} if statistics collection is enabled.
      */
     public boolean isStatisticsEnabled();
 
@@ -425,7 +564,7 @@ public interface CacheMetrics {
      * <p>
      * The default value is {@code false}.
      *
-     * @return true if management is enabled
+     * @return {@code true} if management is enabled.
      */
     public boolean isManagementEnabled();
 
@@ -434,7 +573,7 @@ public interface CacheMetrics {
      * <p>
      * The default value is {@code false}
      *
-     * @return {@code true} when a {@link Cache} is in
+     * @return {@code True} when a {@link Cache} is in
      *         "read-through" mode.
      * @see CacheLoader
      */
@@ -448,7 +587,7 @@ public interface CacheMetrics {
      * <p>
      * The default value is {@code false}
      *
-     * @return {@code true} when a {@link Cache} is in "write-through" mode.
+     * @return {@code True} when a {@link Cache} is in "write-through" mode.
      * @see CacheWriter
      */
     public boolean isWriteThrough();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c14eb563/modules/core/src/main/java/org/apache/ignite/internal/managers/GridManagerAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/GridManagerAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/GridManagerAdapter.java
index c93c059..1eb7143 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/GridManagerAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/GridManagerAdapter.java
@@ -23,7 +23,6 @@ import org.apache.ignite.events.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.managers.communication.*;
 import org.apache.ignite.internal.managers.eventstorage.*;
-import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.util.tostring.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
@@ -31,7 +30,7 @@ import org.apache.ignite.lang.*;
 import org.apache.ignite.plugin.extensions.communication.*;
 import org.apache.ignite.plugin.security.*;
 import org.apache.ignite.spi.*;
-import org.apache.ignite.spi.swapspace.*;
+
 import org.jetbrains.annotations.*;
 
 import javax.cache.expiry.*;
@@ -439,46 +438,10 @@ public abstract class GridManagerAdapter<T extends IgniteSpi> implements GridMan
                         return ctx.cache().cache(cacheName).containsKey(key);
                     }
 
-                    @Override public void writeToSwap(String spaceName, Object key, @Nullable Object val,
-                        @Nullable ClassLoader ldr) {
-                        assert ctx.swap().enabled();
-
-                        try {
-                            ctx.swap().write(spaceName, key, val, ldr);
-                        }
-                        catch (IgniteCheckedException e) {
-                            throw U.convertException(e);
-                        }
-                    }
-
-                    @SuppressWarnings({"unchecked"})
-                    @Nullable @Override public <T> T readFromSwap(String spaceName, SwapKey key,
-                        @Nullable ClassLoader ldr) {
-                        try {
-                            assert ctx.swap().enabled();
-
-                            return ctx.swap().readValue(spaceName, key, ldr);
-                        }
-                        catch (IgniteCheckedException e) {
-                            throw U.convertException(e);
-                        }
-                    }
-
                     @Override public int partition(String cacheName, Object key) {
                         return ctx.cache().cache(cacheName).affinity().partition(key);
                     }
 
-                    @Override public void removeFromSwap(String spaceName, Object key, @Nullable ClassLoader ldr) {
-                        try {
-                            assert ctx.swap().enabled();
-
-                            ctx.swap().remove(spaceName, key, null, ldr);
-                        }
-                        catch (IgniteCheckedException e) {
-                            throw U.convertException(e);
-                        }
-                    }
-
                     @Override public IgniteNodeValidationResult validateNode(ClusterNode node) {
                         for (GridComponent comp : ctx) {
                             IgniteNodeValidationResult err = comp.validateNode(node);
@@ -508,26 +471,6 @@ public abstract class GridManagerAdapter<T extends IgniteSpi> implements GridMan
                         }
                     }
 
-                    @SuppressWarnings("unchecked")
-                    @Nullable @Override public <V> V readValueFromOffheapAndSwap(@Nullable String spaceName,
-                        Object key, @Nullable ClassLoader ldr) {
-                        try {
-                            IgniteInternalCache<Object, V> cache = ctx.cache().cache(spaceName);
-
-                            GridCacheContext cctx = cache.context();
-
-                            if (cctx.isNear())
-                                cctx = cctx.near().dht().context();
-
-                            GridCacheSwapEntry e = cctx.swap().read(cctx.toCacheKeyObject(key), true, true);
-
-                            return e != null ? CU.<V>value(e.value(), cctx, true) : null;
-                        }
-                        catch (IgniteCheckedException e) {
-                            throw U.convertException(e);
-                        }
-                    }
-
                     @Override public MessageFormatter messageFormatter() {
                         return ctx.io().formatter();
                     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c14eb563/modules/core/src/main/java/org/apache/ignite/internal/managers/swapspace/GridSwapSpaceManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/swapspace/GridSwapSpaceManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/swapspace/GridSwapSpaceManager.java
index e62e43f..cf8747d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/swapspace/GridSwapSpaceManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/swapspace/GridSwapSpaceManager.java
@@ -21,6 +21,7 @@ import org.apache.ignite.*;
 import org.apache.ignite.events.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.managers.*;
+import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.util.*;
 import org.apache.ignite.internal.util.lang.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
@@ -138,20 +139,28 @@ public class GridSwapSpaceManager extends GridManagerAdapter<SwapSpaceSpi> {
      * Reads value from swap.
      *
      * @param spaceName Space name.
+     * @param part Partition.
      * @param key Key.
-     * @param ldr Class loader (optional).
+     * @param cctx Cache context.
      * @return Value.
      * @throws IgniteCheckedException If failed.
      */
-    @Nullable public byte[] read(@Nullable String spaceName, SwapKey key, @Nullable ClassLoader ldr)
+    @Nullable public byte[] read(@Nullable String spaceName, int part, KeyCacheObject key, GridCacheContext cctx)
         throws IgniteCheckedException {
         assert key != null;
 
         try {
-            return getSpi().read(spaceName, key, context(ldr));
+            byte[] bytes = getSpi().read(spaceName, new SwapKey(key.value(cctx.cacheObjectContext(), false), part,
+                key.valueBytes(cctx.cacheObjectContext())), context(cctx.deploy().globalLoader()));
+
+            if (cctx.config().isStatisticsEnabled())
+                cctx.cache().metrics0().onSwapRead(bytes != null);
+
+            return bytes;
         }
         catch (IgniteSpiException e) {
-            throw new IgniteCheckedException("Failed to read from swap space [space=" + spaceName + ", key=" + key + ']', e);
+            throw new IgniteCheckedException("Failed to read from swap space [space=" + spaceName +
+                ", key=" + key + ']', e);
         }
     }
 
@@ -159,35 +168,43 @@ public class GridSwapSpaceManager extends GridManagerAdapter<SwapSpaceSpi> {
      * Reads value from swap.
      *
      * @param spaceName Space name.
-     * @param key Swap key.
-     * @param ldr Class loader (optional).
+     * @param part Partition.
+     * @param key Key.
+     * @param cctx Cache context.
      * @return Value.
      * @throws IgniteCheckedException If failed.
      */
     @SuppressWarnings({"unchecked"})
-    @Nullable public <T> T readValue(@Nullable String spaceName, SwapKey key, @Nullable ClassLoader ldr)
+    @Nullable public <T> T readValue(@Nullable String spaceName, int part, KeyCacheObject key, GridCacheContext cctx)
         throws IgniteCheckedException {
         assert key != null;
 
-        return unmarshal(read(spaceName, key, ldr), ldr);
+        return unmarshal(read(spaceName, part, key, cctx), cctx.deploy().globalLoader());
     }
 
     /**
      * Writes value to swap.
      *
      * @param spaceName Space name.
+     * @param part Partition.
      * @param key Key.
      * @param val Value.
-     * @param ldr Class loader (optional).
+     * @param cctx Cache context.
      * @throws IgniteCheckedException If failed.
      */
-    public void write(@Nullable String spaceName, SwapKey key, byte[] val, @Nullable ClassLoader ldr)
+    public void write(@Nullable String spaceName, int part, KeyCacheObject key, byte[] val, GridCacheContext cctx)
         throws IgniteCheckedException {
         assert key != null;
         assert val != null;
 
         try {
-            getSpi().store(spaceName, key, val, context(ldr));
+            SwapKey swapKey = new SwapKey(key.value(cctx.cacheObjectContext(), false), part,
+                key.valueBytes(cctx.cacheObjectContext()));
+
+            getSpi().store(spaceName, swapKey, val, context(cctx.deploy().globalLoader()));
+
+            if (cctx.config().isStatisticsEnabled())
+                cctx.cache().metrics0().onSwapWrite();
         }
         catch (IgniteSpiException e) {
             throw new IgniteCheckedException("Failed to write to swap space [space=" + spaceName + ", key=" + key +
@@ -199,51 +216,81 @@ public class GridSwapSpaceManager extends GridManagerAdapter<SwapSpaceSpi> {
      * Writes batch to swap.
      *
      * @param spaceName Space name.
-     * @param batch Swapped entries.
-     * @param ldr Class loader (optional).
+     * @param swapped Swapped entries.
+     * @param cctx Cache context.
      * @throws IgniteCheckedException If failed.
      */
-    public <K, V> void writeAll(String spaceName, Map<SwapKey, byte[]> batch,
-        @Nullable ClassLoader ldr) throws IgniteCheckedException {
-        getSpi().storeAll(spaceName, batch, context(ldr));
+    public void writeAll(String spaceName, Iterable<GridCacheBatchSwapEntry> swapped, GridCacheContext cctx)
+        throws IgniteCheckedException {
+        Map<SwapKey, byte[]> batch = new LinkedHashMap<>();
+
+        int cnt = 0;
+
+        for (GridCacheBatchSwapEntry entry : swapped) {
+            SwapKey swapKey = new SwapKey(entry.key().value(cctx.cacheObjectContext(), false),
+                entry.partition(),
+                entry.key().valueBytes(cctx.cacheObjectContext()));
+
+            batch.put(swapKey, entry.marshal());
+
+            cnt++;
+        }
+
+        getSpi().storeAll(spaceName, batch, context(cctx.deploy().globalLoader()));
+
+        if (cctx.config().isStatisticsEnabled())
+            cctx.cache().metrics0().onSwapWrite(cnt);
     }
 
     /**
      * Writes value to swap.
      *
      * @param spaceName Space name.
+     * @param part Partition.
      * @param key Key.
      * @param val Value.
-     * @param ldr Class loader (optional).
-     * @throws IgniteCheckedException If failed.
-     */
-    public void write(@Nullable String spaceName, Object key, @Nullable Object val, @Nullable ClassLoader ldr)
-        throws IgniteCheckedException {
+     * @param cctx Cache context.
+     * */
+    public void write(@Nullable String spaceName, int part, KeyCacheObject key, @Nullable Object val,
+        GridCacheContext cctx) throws IgniteCheckedException {
         assert key != null;
 
-        write(spaceName, new SwapKey(key), marshal(val), ldr);
+        write(spaceName, part, key, marshal(val), cctx);
     }
 
     /**
      * Removes value from swap.
      *
      * @param spaceName Space name.
+     * @param part Partition.
      * @param key Key.
+     * @param cctx Cache context.
      * @param c Optional closure that takes removed value and executes after actual
      *      removing. If there was no value in storage the closure is executed given
      *      {@code null} value as parameter.
-     * @param ldr Class loader (optional).
      * @throws IgniteCheckedException If failed.
      */
-    public void remove(@Nullable String spaceName, SwapKey key, @Nullable IgniteInClosure<byte[]> c,
-        @Nullable ClassLoader ldr) throws IgniteCheckedException {
+    public void remove(@Nullable String spaceName, int part, KeyCacheObject key,
+        final GridCacheContext cctx, @Nullable final IgniteInClosure<byte[]> c) throws IgniteCheckedException {
         assert key != null;
 
         try {
-            getSpi().remove(spaceName, key, c, context(ldr));
+            SwapKey swapKey = new SwapKey(key.value(cctx.cacheObjectContext(), false), part,
+                key.valueBytes(cctx.cacheObjectContext()));
+
+            getSpi().remove(spaceName, swapKey, new IgniteInClosure<byte[]>() {
+                @Override public void apply(byte[] bytes) {
+                    if (c != null)
+                        c.apply(bytes);
+
+                    if (bytes != null && cctx.config().isStatisticsEnabled())
+                        cctx.cache().metrics0().onSwapRemove();
+                }
+            }, context(cctx.deploy().globalLoader()));
         }
         catch (IgniteSpiException e) {
-            throw new IgniteCheckedException("Failed to remove from swap space [space=" + spaceName + ", key=" + key + ']', e);
+            throw new IgniteCheckedException("Failed to remove from swap space [space=" + spaceName +
+                ", key=" + key + ']', e);
         }
     }
 
@@ -251,19 +298,32 @@ public class GridSwapSpaceManager extends GridManagerAdapter<SwapSpaceSpi> {
      * Removes value from swap.
      *
      * @param spaceName Space name.
-     * @param keys Collection of keys.
+     * @param keys Keys.
+     * @param cctx Cache context.
      * @param c Optional closure that takes removed value and executes after actual
      *      removing. If there was no value in storage the closure is executed given
      *      {@code null} value as parameter.
-     * @param ldr Class loader (optional).
      * @throws IgniteCheckedException If failed.
      */
-    public void removeAll(@Nullable String spaceName, Collection<SwapKey> keys,
-        IgniteBiInClosure<SwapKey, byte[]> c, @Nullable ClassLoader ldr) throws IgniteCheckedException {
+    public void removeAll(@Nullable String spaceName, Collection<KeyCacheObject> keys, GridCacheContext cctx,
+        IgniteBiInClosure<SwapKey, byte[]> c) throws IgniteCheckedException {
         assert keys != null;
 
+        Collection<SwapKey> swapKeys = new ArrayList<>(keys.size());
+
+        for (KeyCacheObject key : keys) {
+            SwapKey swapKey = new SwapKey(key.value(cctx.cacheObjectContext(), false),
+                cctx.affinity().partition(key),
+                key.valueBytes(cctx.cacheObjectContext()));
+
+            swapKeys.add(swapKey);
+        }
+
         try {
-            getSpi().removeAll(spaceName, keys, c, context(ldr));
+            getSpi().removeAll(spaceName, swapKeys, c, context(cctx.deploy().globalLoader()));
+
+            if (cctx.config().isStatisticsEnabled())
+                cctx.cache().metrics0().onSwapRemove();
         }
         catch (IgniteSpiException e) {
             throw new IgniteCheckedException("Failed to remove from swap space [space=" + spaceName + ", " +
@@ -272,24 +332,6 @@ public class GridSwapSpaceManager extends GridManagerAdapter<SwapSpaceSpi> {
     }
 
     /**
-     * Removes value from swap.
-     *
-     * @param spaceName Space name.
-     * @param key Key.
-     * @param c Optional closure that takes removed value and executes after actual
-     *      removing. If there was no value in storage the closure is executed given
-     *      {@code null} value as parameter.
-     * @param ldr Class loader (optional).
-     * @throws IgniteCheckedException If failed.
-     */
-    public void remove(@Nullable String spaceName, Object key, @Nullable IgniteInClosure<byte[]> c,
-        @Nullable ClassLoader ldr) throws IgniteCheckedException {
-        assert key != null;
-
-        remove(spaceName, new SwapKey(key), c, ldr);
-    }
-
-    /**
      * Gets size in bytes for swap space.
      *
      * @param spaceName Space name.
@@ -356,7 +398,7 @@ public class GridSwapSpaceManager extends GridManagerAdapter<SwapSpaceSpi> {
      *
      * @param spaceName Space name.
      * @return Iterator over space entries or {@code null} if space is unknown.
-     * @throws org.apache.ignite.spi.IgniteSpiException If failed.
+     * @throws IgniteSpiException If failed.
      */
     @Nullable public GridCloseableIterator<Map.Entry<byte[], byte[]>> rawIterator(@Nullable String spaceName)
         throws IgniteCheckedException {
@@ -397,7 +439,7 @@ public class GridSwapSpaceManager extends GridManagerAdapter<SwapSpaceSpi> {
      * @param spaceName Space name.
      * @param ldr Class loader.
      * @return Iterator over space entries or {@code null} if space is unknown.
-     * @throws org.apache.ignite.spi.IgniteSpiException If failed.
+     * @throws IgniteSpiException If failed.
      */
     @Nullable public <K> GridCloseableIterator<K> keysIterator(@Nullable String spaceName,
         @Nullable ClassLoader ldr) throws IgniteCheckedException {
@@ -417,7 +459,6 @@ public class GridSwapSpaceManager extends GridManagerAdapter<SwapSpaceSpi> {
      * @return Unmarshalled value.
      * @throws IgniteCheckedException If failed.
      */
-    @SuppressWarnings({"unchecked"})
     private <T> T unmarshal(byte[] swapBytes, @Nullable ClassLoader ldr) throws IgniteCheckedException {
         if (swapBytes == null)
             return null;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c14eb563/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java
index 560de97..3dcda3c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java
@@ -25,6 +25,8 @@ import org.apache.ignite.internal.util.typedef.internal.*;
 
 import java.util.concurrent.atomic.*;
 
+import static org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion.*;
+
 /**
  * Adapter for cache metrics.
  */
@@ -63,7 +65,7 @@ public class CacheMetricsImpl implements CacheMetrics {
     private AtomicLong getTimeNanos = new AtomicLong();
 
     /** Remove time taken nanos. */
-    private AtomicLong removeTimeNanos = new AtomicLong();
+    private AtomicLong rmvTimeNanos = new AtomicLong();
 
     /** Commit transaction time taken nanos. */
     private AtomicLong commitTimeNanos = new AtomicLong();
@@ -71,6 +73,39 @@ public class CacheMetricsImpl implements CacheMetrics {
     /** Commit transaction time taken nanos. */
     private AtomicLong rollbackTimeNanos = new AtomicLong();
 
+    /** Number of reads from off-heap memory. */
+    private AtomicLong offHeapGets = new AtomicLong();
+
+    /** Number of writes to off-heap memory. */
+    private AtomicLong offHeapPuts = new AtomicLong();
+
+    /** Number of removed entries from off-heap memory. */
+    private AtomicLong offHeapRemoves = new AtomicLong();
+
+    /** Number of evictions from off-heap memory. */
+    private AtomicLong offHeapEvicts = new AtomicLong();
+
+    /** Number of off-heap hits. */
+    private AtomicLong offHeapHits = new AtomicLong();
+
+    /** Number of off-heap misses. */
+    private AtomicLong offHeapMisses = new AtomicLong();
+
+    /** Number of reads from swap. */
+    private AtomicLong swapGets = new AtomicLong();
+
+    /** Number of writes to swap. */
+    private AtomicLong swapPuts = new AtomicLong();
+
+    /** Number of removed entries from swap. */
+    private AtomicLong swapRemoves = new AtomicLong();
+
+    /** Number of swap hits. */
+    private AtomicLong swapHits = new AtomicLong();
+
+    /** Number of swap misses. */
+    private AtomicLong swapMisses = new AtomicLong();
+
     /** Cache metrics. */
     @GridToStringExclude
     private transient CacheMetricsImpl delegate;
@@ -126,16 +161,160 @@ public class CacheMetricsImpl implements CacheMetrics {
     }
 
     /** {@inheritDoc} */
+    @Override public long getOffHeapGets() {
+        return offHeapGets.get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getOffHeapPuts() {
+        return offHeapPuts.get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getOffHeapRemovals() {
+        return offHeapRemoves.get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getOffHeapEvictions() {
+        return offHeapEvicts.get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getOffHeapHits() {
+        return offHeapHits.get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public float getOffHeapHitPercentage() {
+        long hits0 = offHeapHits.get();
+        long gets0 = offHeapGets.get();
+
+        if (hits0 == 0)
+            return 0;
+
+        return (float) hits0 / gets0 * 100.0f;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getOffHeapMisses() {
+        return offHeapMisses.get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public float getOffHeapMissPercentage() {
+        long misses0 = offHeapMisses.get();
+        long reads0 = offHeapGets.get();
+
+        if (misses0 == 0)
+            return 0;
+
+        return (float) misses0 / reads0 * 100.0f;
+    }
+
+    /** {@inheritDoc} */
     @Override public long getOffHeapEntriesCount() {
         return cctx.cache().offHeapEntriesCount();
     }
 
     /** {@inheritDoc} */
+    @Override public long getOffHeapPrimaryEntriesCount() {
+        try {
+            return cctx.swap().offheapEntriesCount(true, false, NONE);
+        }
+        catch (IgniteCheckedException e) {
+            return 0;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getOffHeapBackupEntriesCount() {
+        try {
+            return cctx.swap().offheapEntriesCount(false, true, NONE);
+        }
+        catch (IgniteCheckedException e) {
+            return 0;
+        }
+    }
+
+    /** {@inheritDoc} */
     @Override public long getOffHeapAllocatedSize() {
         return cctx.cache().offHeapAllocatedSize();
     }
 
     /** {@inheritDoc} */
+    @Override public long getOffHeapMaxSize() {
+        return cctx.config().getOffHeapMaxMemory();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getSwapGets() {
+        return swapGets.get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getSwapPuts() {
+        return swapPuts.get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getSwapRemovals() {
+        return swapRemoves.get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getSwapHits() {
+        return swapHits.get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getSwapMisses() {
+        return swapMisses.get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getSwapEntriesCount() {
+        try {
+            return cctx.cache().swapKeys();
+        }
+        catch (IgniteCheckedException e) {
+            return 0;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getSwapSize() {
+        try {
+            return cctx.cache().swapSize();
+        }
+        catch (IgniteCheckedException e) {
+            return 0;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public float getSwapHitPercentage() {
+        long hits0 = swapHits.get();
+        long gets0 = swapGets.get();
+
+        if (hits0 == 0)
+            return 0;
+
+        return (float) hits0 / gets0 * 100.0f;
+    }
+
+    /** {@inheritDoc} */
+    @Override public float getSwapMissPercentage() {
+        long misses0 = swapMisses.get();
+        long reads0 = swapGets.get();
+
+        if (misses0 == 0)
+            return 0;
+
+        return (float) misses0 / reads0 * 100.0f;
+    }
+
+    /** {@inheritDoc} */
     @Override public int getSize() {
         return cctx.cache().size();
     }
@@ -317,11 +496,24 @@ public class CacheMetricsImpl implements CacheMetrics {
         txCommits.set(0);
         txRollbacks.set(0);
         putTimeNanos.set(0);
-        removeTimeNanos.set(0);
+        rmvTimeNanos.set(0);
         getTimeNanos.set(0);
         commitTimeNanos.set(0);
         rollbackTimeNanos.set(0);
 
+        offHeapGets.set(0);
+        offHeapPuts.set(0);
+        offHeapRemoves.set(0);
+        offHeapHits.set(0);
+        offHeapMisses.set(0);
+        offHeapEvicts.set(0);
+
+        swapGets.set(0);
+        swapPuts.set(0);
+        swapRemoves.set(0);
+        swapHits.set(0);
+        swapMisses.set(0);
+
         if (delegate != null)
             delegate.clear();
     }
@@ -402,7 +594,7 @@ public class CacheMetricsImpl implements CacheMetrics {
 
     /** {@inheritDoc} */
     @Override public float getAverageRemoveTime() {
-        long timeNanos = removeTimeNanos.get();
+        long timeNanos = rmvTimeNanos.get();
         long removesCnt = rmCnt.get();
 
         if (timeNanos == 0 || removesCnt == 0)
@@ -483,7 +675,6 @@ public class CacheMetricsImpl implements CacheMetrics {
             delegate.onTxRollback(duration);
     }
 
-
     /**
      * Increments the get time accumulator.
      *
@@ -514,7 +705,7 @@ public class CacheMetricsImpl implements CacheMetrics {
      * @param duration the time taken in nanoseconds.
      */
     public void addRemoveTimeNanos(long duration) {
-        removeTimeNanos.addAndGet(duration);
+        rmvTimeNanos.addAndGet(duration);
 
         if (delegate != null)
             delegate.addRemoveTimeNanos(duration);
@@ -526,7 +717,7 @@ public class CacheMetricsImpl implements CacheMetrics {
      * @param duration the time taken in nanoseconds.
      */
     public void addRemoveAndGetTimeNanos(long duration) {
-        removeTimeNanos.addAndGet(duration);
+        rmvTimeNanos.addAndGet(duration);
         getTimeNanos.addAndGet(duration);
 
         if (delegate != null)
@@ -581,6 +772,108 @@ public class CacheMetricsImpl implements CacheMetrics {
         return cctx.config().isManagementEnabled();
     }
 
+    /**
+     * Off-heap read callback.
+     *
+     * @param hit Hit or miss flag.
+     */
+    public void onOffHeapRead(boolean hit) {
+        offHeapGets.incrementAndGet();
+
+        if (hit)
+            offHeapHits.incrementAndGet();
+        else
+            offHeapMisses.incrementAndGet();
+
+        if (delegate != null)
+            delegate.onOffHeapRead(hit);
+    }
+
+    /**
+     * Off-heap write callback.
+     */
+    public void onOffHeapWrite() {
+        offHeapPuts.incrementAndGet();
+
+        if (delegate != null)
+            delegate.onOffHeapWrite();
+    }
+
+    /**
+     * Off-heap remove callback.
+     */
+    public void onOffHeapRemove() {
+        offHeapRemoves.incrementAndGet();
+
+        if (delegate != null)
+            delegate.onOffHeapRemove();
+    }
+
+    /**
+     * Off-heap evict callback.
+     */
+    public void onOffHeapEvict() {
+        offHeapEvicts.incrementAndGet();
+
+        if (delegate != null)
+            delegate.onOffHeapRemove();
+    }
+
+    /**
+     * Swap read callback.
+     *
+     * @param hit Hit or miss flag.
+     */
+    public void onSwapRead(boolean hit) {
+        swapGets.incrementAndGet();
+
+        if (hit)
+            swapHits.incrementAndGet();
+        else
+            swapMisses.incrementAndGet();
+
+        if (delegate != null)
+            delegate.onSwapRead(hit);
+    }
+
+    /**
+     * Swap write callback.
+     */
+    public void onSwapWrite() {
+        onSwapWrite(1);
+    }
+
+    /**
+     * Swap write callback.
+     *
+     * @param cnt Amount of entries.
+     */
+    public void onSwapWrite(int cnt) {
+        swapPuts.addAndGet(cnt);
+
+        if (delegate != null)
+            delegate.onSwapWrite(cnt);
+    }
+
+    /**
+     * Swap remove callback.
+     */
+    public void onSwapRemove() {
+        onSwapRemove(1);
+    }
+
+    /**
+     * Swap remove callback.
+     *
+     * @param cnt Amount of entries.
+     */
+    public void onSwapRemove(int cnt) {
+        swapRemoves.addAndGet(cnt);
+
+        if (delegate != null)
+            delegate.onSwapRemove(cnt);
+    }
+
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(CacheMetricsImpl.class, this);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c14eb563/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsMXBeanImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsMXBeanImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsMXBeanImpl.java
index e9d547c..966027a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsMXBeanImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsMXBeanImpl.java
@@ -49,16 +49,116 @@ class CacheMetricsMXBeanImpl implements CacheMetricsMXBean {
     }
 
     /** {@inheritDoc} */
+    @Override public long getOffHeapGets() {
+        return cache.metrics0().getOffHeapGets();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getOffHeapPuts() {
+        return cache.metrics0().getOffHeapPuts();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getOffHeapRemovals() {
+        return cache.metrics0().getOffHeapRemovals();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getOffHeapEvictions() {
+        return cache.metrics0().getOffHeapEvictions();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getOffHeapHits() {
+        return cache.metrics0().getOffHeapHits();
+    }
+
+    /** {@inheritDoc} */
+    @Override public float getOffHeapHitPercentage() {
+        return cache.metrics0().getOffHeapHitPercentage();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getOffHeapMisses() {
+        return cache.metrics0().getOffHeapMisses();
+    }
+
+    /** {@inheritDoc} */
+    @Override public float getOffHeapMissPercentage() {
+        return cache.metrics0().getOffHeapMissPercentage();
+    }
+
+    /** {@inheritDoc} */
     @Override public long getOffHeapEntriesCount() {
         return cache.metrics0().getOffHeapEntriesCount();
     }
 
     /** {@inheritDoc} */
+    @Override public long getOffHeapPrimaryEntriesCount() {
+        return cache.metrics0().getOffHeapPrimaryEntriesCount();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getOffHeapBackupEntriesCount() {
+        return cache.metrics0().getOffHeapBackupEntriesCount();
+    }
+
+    /** {@inheritDoc} */
     @Override public long getOffHeapAllocatedSize() {
         return cache.metrics0().getOffHeapAllocatedSize();
     }
 
     /** {@inheritDoc} */
+    @Override public long getOffHeapMaxSize() {
+        return cache.metrics0().getOffHeapMaxSize();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getSwapGets() {
+        return cache.metrics0().getSwapGets();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getSwapPuts() {
+        return cache.metrics0().getSwapPuts();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getSwapRemovals() {
+        return cache.metrics0().getSwapRemovals();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getSwapHits() {
+        return cache.metrics0().getSwapHits();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getSwapMisses() {
+        return cache.metrics0().getSwapMisses();
+    }
+
+    /** {@inheritDoc} */
+    @Override public float getSwapHitPercentage() {
+        return cache.metrics0().getSwapHitPercentage();
+    }
+
+    /** {@inheritDoc} */
+    @Override public float getSwapMissPercentage() {
+        return cache.metrics0().getSwapMissPercentage();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getSwapEntriesCount() {
+        return cache.metrics0().getSwapEntriesCount();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getSwapSize() {
+        return cache.metrics0().getSwapSize();
+    }
+
+    /** {@inheritDoc} */
     @Override public int getSize() {
         return cache.metrics0().getSize();
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c14eb563/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsSnapshot.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsSnapshot.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsSnapshot.java
index 4fe152a..cf16d9d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsSnapshot.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsSnapshot.java
@@ -61,7 +61,7 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
     private float getAvgTimeNanos = 0;
 
     /** Remove time taken nanos. */
-    private float removeAvgTimeNanos = 0;
+    private float rmvAvgTimeNanos = 0;
 
     /** Commit transaction time taken nanos. */
     private float commitAvgTimeNanos = 0;
@@ -75,12 +75,60 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
     /** Number of entries that was swapped to disk. */
     private long overflowSize;
 
+    /** Number of reads from off-heap. */
+    private long offHeapGets;
+
+    /** Number of writes to off-heap. */
+    private long offHeapPuts;
+
+    /** Number of removed entries from off-heap. */
+    private long offHeapRemoves;
+
+    /** Number of evictions from off-heap. */
+    private long offHeapEvicts;
+
+    /** Off-heap hits number. */
+    private long offHeapHits;
+
+    /** Off-heap misses number. */
+    private long offHeapMisses;
+
     /** Number of entries stored in off-heap memory. */
-    private long offHeapEntriesCount;
+    private long offHeapEntriesCnt;
+
+    /** Number of primary entries stored in off-heap memory. */
+    private long offHeapPrimaryEntriesCnt;
+
+    /** Number of backup entries stored in off-heap memory. */
+    private long offHeapBackupEntriesCnt;
 
     /** Memory size allocated in off-heap. */
     private long offHeapAllocatedSize;
 
+    /** Off-heap memory maximum size*/
+    private long offHeapMaxSize;
+
+    /** Number of reads from swap. */
+    private long swapGets;
+
+    /** Number of writes to swap. */
+    private long swapPuts;
+
+    /** Number of removed entries from swap. */
+    private long swapRemoves;
+
+    /** Number of entries stored in swap. */
+    private long swapEntriesCnt;
+
+    /** Swap hits number. */
+    private long swapHits;
+
+    /** Swap misses number. */
+    private long swapMisses;
+
+    /** Swap size. */
+    private long swapSize;
+
     /** Number of non-{@code null} values in the cache. */
     private int size;
 
@@ -91,7 +139,7 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
     private boolean isEmpty;
 
     /** Gets current size of evict queue used to batch up evictions. */
-    private int dhtEvictQueueCurrentSize;
+    private int dhtEvictQueueCurrSize;
 
     /** Transaction per-thread map size. */
     private int txThreadMapSize;
@@ -106,7 +154,7 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
     private int txPrepareQueueSize;
 
     /** Start version counts map size. */
-    private int txStartVersionCountsSize;
+    private int txStartVerCountsSize;
 
     /** Number of cached committed transaction IDs. */
     private int txCommittedVersionsSize;
@@ -127,7 +175,7 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
     private int txDhtPrepareQueueSize;
 
     /** DHT start version counts map size. */
-    private int txDhtStartVersionCountsSize;
+    private int txDhtStartVerCountsSize;
 
     /** Number of cached committed DHT transaction IDs. */
     private int txDhtCommittedVersionsSize;
@@ -142,34 +190,34 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
     private int writeBehindFlushSize;
 
     /** Count of worker threads. */
-    private int writeBehindFlushThreadCount;
+    private int writeBehindFlushThreadCnt;
 
     /** Flush frequency in milliseconds. */
-    private long writeBehindFlushFrequency;
+    private long writeBehindFlushFreq;
 
     /** Maximum size of batch. */
     private int writeBehindStoreBatchSize;
 
     /** Count of cache overflow events since start. */
-    private int writeBehindTotalCriticalOverflowCount;
+    private int writeBehindTotalCriticalOverflowCnt;
 
     /** Count of cache overflow events since start. */
-    private int writeBehindCriticalOverflowCount;
+    private int writeBehindCriticalOverflowCnt;
 
     /** Count of entries in store-retry state. */
-    private int writeBehindErrorRetryCount;
+    private int writeBehindErrorRetryCnt;
 
     /** Total count of entries in cache store internal buffer. */
-    private int writeBehindBufferSize;
+    private int writeBehindBufSize;
 
     /** */
     private String keyType;
 
     /** */
-    private String valueType;
+    private String valType;
 
     /** */
-    private boolean isStoreByValue;
+    private boolean isStoreByVal;
 
     /** */
     private boolean isStatisticsEnabled;
@@ -207,45 +255,64 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
 
         putAvgTimeNanos = m.getAveragePutTime();
         getAvgTimeNanos = m.getAverageGetTime();
-        removeAvgTimeNanos = m.getAverageRemoveTime();
+        rmvAvgTimeNanos = m.getAverageRemoveTime();
         commitAvgTimeNanos = m.getAverageTxCommitTime();
         rollbackAvgTimeNanos = m.getAverageTxRollbackTime();
 
         cacheName = m.name();
         overflowSize = m.getOverflowSize();
-        offHeapEntriesCount = m.getOffHeapEntriesCount();
+
+        offHeapGets = m.getOffHeapGets();
+        offHeapPuts = m.getOffHeapPuts();
+        offHeapRemoves = m.getOffHeapRemovals();
+        offHeapEvicts = m.getOffHeapEvictions();
+        offHeapHits = m.getOffHeapHits();
+        offHeapMisses = m.getOffHeapMisses();
+        offHeapEntriesCnt = m.getOffHeapEntriesCount();
+        offHeapPrimaryEntriesCnt = m.getOffHeapPrimaryEntriesCount();
+        offHeapBackupEntriesCnt = m.getOffHeapBackupEntriesCount();
         offHeapAllocatedSize = m.getOffHeapAllocatedSize();
+        offHeapMaxSize = m.getOffHeapMaxSize();
+
+        swapGets = m.getSwapGets();
+        swapPuts = m.getSwapPuts();
+        swapRemoves = m.getSwapRemovals();
+        swapHits = m.getSwapHits();
+        swapMisses = m.getSwapMisses();
+        swapEntriesCnt = m.getSwapEntriesCount();
+        swapSize = m.getSwapSize();
+
         size = m.getSize();
         keySize = m.getKeySize();
         isEmpty = m.isEmpty();
-        dhtEvictQueueCurrentSize = m.getDhtEvictQueueCurrentSize();
+        dhtEvictQueueCurrSize = m.getDhtEvictQueueCurrentSize();
         txThreadMapSize = m.getTxThreadMapSize();
         txXidMapSize = m.getTxXidMapSize();
         txCommitQueueSize = m.getTxCommitQueueSize();
         txPrepareQueueSize = m.getTxPrepareQueueSize();
-        txStartVersionCountsSize = m.getTxStartVersionCountsSize();
+        txStartVerCountsSize = m.getTxStartVersionCountsSize();
         txCommittedVersionsSize = m.getTxCommittedVersionsSize();
         txRolledbackVersionsSize = m.getTxRolledbackVersionsSize();
         txDhtThreadMapSize = m.getTxDhtThreadMapSize();
         txDhtXidMapSize = m.getTxDhtXidMapSize();
         txDhtCommitQueueSize = m.getTxDhtCommitQueueSize();
         txDhtPrepareQueueSize = m.getTxDhtPrepareQueueSize();
-        txDhtStartVersionCountsSize = m.getTxDhtStartVersionCountsSize();
+        txDhtStartVerCountsSize = m.getTxDhtStartVersionCountsSize();
         txDhtCommittedVersionsSize = m.getTxDhtCommittedVersionsSize();
         txDhtRolledbackVersionsSize = m.getTxDhtRolledbackVersionsSize();
         isWriteBehindEnabled = m.isWriteBehindEnabled();
         writeBehindFlushSize = m.getWriteBehindFlushSize();
-        writeBehindFlushThreadCount = m.getWriteBehindFlushThreadCount();
-        writeBehindFlushFrequency = m.getWriteBehindFlushFrequency();
+        writeBehindFlushThreadCnt = m.getWriteBehindFlushThreadCount();
+        writeBehindFlushFreq = m.getWriteBehindFlushFrequency();
         writeBehindStoreBatchSize = m.getWriteBehindStoreBatchSize();
-        writeBehindTotalCriticalOverflowCount = m.getWriteBehindTotalCriticalOverflowCount();
-        writeBehindCriticalOverflowCount = m.getWriteBehindCriticalOverflowCount();
-        writeBehindErrorRetryCount = m.getWriteBehindErrorRetryCount();
-        writeBehindBufferSize = m.getWriteBehindBufferSize();
+        writeBehindTotalCriticalOverflowCnt = m.getWriteBehindTotalCriticalOverflowCount();
+        writeBehindCriticalOverflowCnt = m.getWriteBehindCriticalOverflowCount();
+        writeBehindErrorRetryCnt = m.getWriteBehindErrorRetryCount();
+        writeBehindBufSize = m.getWriteBehindBufferSize();
 
         keyType = m.getKeyType();
-        valueType = m.getValueType();
-        isStoreByValue = m.isStoreByValue();
+        valType = m.getValueType();
+        isStoreByVal = m.isStoreByValue();
         isStatisticsEnabled = m.isStatisticsEnabled();
         isManagementEnabled = m.isManagementEnabled();
         isReadThrough = m.isReadThrough();
@@ -263,21 +330,23 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
         isEmpty = loc.isEmpty();
         isWriteBehindEnabled = loc.isWriteBehindEnabled();
         writeBehindFlushSize = loc.getWriteBehindFlushSize();
-        writeBehindFlushThreadCount = loc.getWriteBehindFlushThreadCount();
-        writeBehindFlushFrequency = loc.getWriteBehindFlushFrequency();
+        writeBehindFlushThreadCnt = loc.getWriteBehindFlushThreadCount();
+        writeBehindFlushFreq = loc.getWriteBehindFlushFrequency();
         writeBehindStoreBatchSize = loc.getWriteBehindStoreBatchSize();
-        writeBehindBufferSize = loc.getWriteBehindBufferSize();
+        writeBehindBufSize = loc.getWriteBehindBufferSize();
         size = loc.getSize();
         keySize = loc.getKeySize();
 
         keyType = loc.getKeyType();
-        valueType = loc.getValueType();
-        isStoreByValue = loc.isStoreByValue();
+        valType = loc.getValueType();
+        isStoreByVal = loc.isStoreByValue();
         isStatisticsEnabled = loc.isStatisticsEnabled();
         isManagementEnabled = loc.isManagementEnabled();
         isReadThrough = loc.isReadThrough();
         isWriteThrough = loc.isWriteThrough();
 
+        offHeapMaxSize = loc.getOffHeapMaxSize();
+
         for (CacheMetrics e : metrics) {
             reads += e.getCacheGets();
             puts += e.getCachePuts();
@@ -290,7 +359,7 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
 
             putAvgTimeNanos += e.getAveragePutTime();
             getAvgTimeNanos += e.getAverageGetTime();
-            removeAvgTimeNanos += e.getAverageRemoveTime();
+            rmvAvgTimeNanos += e.getAverageRemoveTime();
             commitAvgTimeNanos += e.getAverageTxCommitTime();
             rollbackAvgTimeNanos += e.getAverageTxRollbackTime();
 
@@ -299,19 +368,35 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
             else
                 overflowSize = -1;
 
-            offHeapEntriesCount += e.getOffHeapEntriesCount();
+            offHeapGets += e.getOffHeapGets();
+            offHeapPuts += e.getOffHeapPuts();
+            offHeapRemoves += e.getOffHeapRemovals();
+            offHeapEvicts += e.getOffHeapEvictions();
+            offHeapHits += e.getOffHeapHits();
+            offHeapMisses += e.getOffHeapMisses();
+            offHeapEntriesCnt += e.getOffHeapEntriesCount();
+            offHeapPrimaryEntriesCnt += e.getOffHeapPrimaryEntriesCount();
+            offHeapBackupEntriesCnt += e.getOffHeapBackupEntriesCount();
             offHeapAllocatedSize += e.getOffHeapAllocatedSize();
 
+            swapGets += e.getSwapGets();
+            swapPuts += e.getSwapPuts();
+            swapRemoves += e.getSwapRemovals();
+            swapHits += e.getSwapHits();
+            swapMisses += e.getSwapMisses();
+            swapEntriesCnt += e.getSwapEntriesCount();
+            swapSize += e.getSwapSize();
+
             if (e.getDhtEvictQueueCurrentSize() > -1)
-                dhtEvictQueueCurrentSize += e.getDhtEvictQueueCurrentSize();
+                dhtEvictQueueCurrSize += e.getDhtEvictQueueCurrentSize();
             else
-                dhtEvictQueueCurrentSize = -1;
+                dhtEvictQueueCurrSize = -1;
 
             txThreadMapSize += e.getTxThreadMapSize();
             txXidMapSize += e.getTxXidMapSize();
             txCommitQueueSize += e.getTxCommitQueueSize();
             txPrepareQueueSize += e.getTxPrepareQueueSize();
-            txStartVersionCountsSize += e.getTxStartVersionCountsSize();
+            txStartVerCountsSize += e.getTxStartVersionCountsSize();
             txCommittedVersionsSize += e.getTxCommittedVersionsSize();
             txRolledbackVersionsSize += e.getTxRolledbackVersionsSize();
 
@@ -336,9 +421,9 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
                 txDhtPrepareQueueSize = -1;
 
             if (e.getTxDhtStartVersionCountsSize() > -1)
-                txDhtStartVersionCountsSize += e.getTxDhtStartVersionCountsSize();
+                txDhtStartVerCountsSize += e.getTxDhtStartVersionCountsSize();
             else
-                txDhtStartVersionCountsSize = -1;
+                txDhtStartVerCountsSize = -1;
 
             if (e.getTxDhtCommittedVersionsSize() > -1)
                 txDhtCommittedVersionsSize += e.getTxDhtCommittedVersionsSize();
@@ -351,19 +436,19 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
                 txDhtRolledbackVersionsSize = -1;
 
             if (e.getWriteBehindTotalCriticalOverflowCount() > -1)
-                writeBehindTotalCriticalOverflowCount += e.getWriteBehindTotalCriticalOverflowCount();
+                writeBehindTotalCriticalOverflowCnt += e.getWriteBehindTotalCriticalOverflowCount();
             else
-                writeBehindTotalCriticalOverflowCount = -1;
+                writeBehindTotalCriticalOverflowCnt = -1;
 
             if (e.getWriteBehindCriticalOverflowCount() > -1)
-                writeBehindCriticalOverflowCount += e.getWriteBehindCriticalOverflowCount();
+                writeBehindCriticalOverflowCnt += e.getWriteBehindCriticalOverflowCount();
             else
-                writeBehindCriticalOverflowCount = -1;
+                writeBehindCriticalOverflowCnt = -1;
 
             if (e.getWriteBehindErrorRetryCount() > -1)
-                writeBehindErrorRetryCount += e.getWriteBehindErrorRetryCount();
+                writeBehindErrorRetryCnt += e.getWriteBehindErrorRetryCount();
             else
-                writeBehindErrorRetryCount = -1;
+                writeBehindErrorRetryCnt = -1;
         }
 
         int size = metrics.size();
@@ -371,7 +456,7 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
         if (size > 1) {
             putAvgTimeNanos /= size;
             getAvgTimeNanos /= size;
-            removeAvgTimeNanos /= size;
+            rmvAvgTimeNanos /= size;
             commitAvgTimeNanos /= size;
             rollbackAvgTimeNanos /= size;
         }
@@ -435,7 +520,7 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
 
     /** {@inheritDoc} */
     @Override public float getAverageRemoveTime() {
-        return removeAvgTimeNanos;
+        return rmvAvgTimeNanos;
     }
 
     /** {@inheritDoc} */
@@ -469,8 +554,63 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
     }
 
     /** {@inheritDoc} */
+    @Override public long getOffHeapGets() {
+        return offHeapGets;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getOffHeapPuts() {
+        return offHeapPuts;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getOffHeapRemovals() {
+        return offHeapRemoves;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getOffHeapEvictions() {
+        return offHeapEvicts;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getOffHeapHits() {
+        return offHeapHits;
+    }
+
+    /** {@inheritDoc} */
+    @Override public float getOffHeapHitPercentage() {
+        if (offHeapHits == 0 || offHeapGets == 0)
+            return 0;
+
+        return (float) offHeapHits / offHeapGets * 100.0f;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getOffHeapMisses() {
+        return offHeapMisses;
+    }
+
+    /** {@inheritDoc} */
+    @Override public float getOffHeapMissPercentage() {
+        if (offHeapMisses == 0 || offHeapGets == 0)
+            return 0;
+
+        return (float) offHeapMisses / offHeapGets * 100.0f;
+    }
+    /** {@inheritDoc} */
     @Override public long getOffHeapEntriesCount() {
-        return offHeapEntriesCount;
+        return offHeapEntriesCnt;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getOffHeapPrimaryEntriesCount() {
+        return offHeapPrimaryEntriesCnt;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getOffHeapBackupEntriesCount() {
+        return offHeapBackupEntriesCnt;
     }
 
     /** {@inheritDoc} */
@@ -479,6 +619,62 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
     }
 
     /** {@inheritDoc} */
+    @Override public long getOffHeapMaxSize() {
+        return offHeapMaxSize;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getSwapGets() {
+        return swapGets;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getSwapPuts() {
+        return swapPuts;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getSwapRemovals() {
+        return swapRemoves;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getSwapHits() {
+        return swapHits;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getSwapMisses() {
+        return swapMisses;
+    }
+
+    /** {@inheritDoc} */
+    @Override public float getSwapHitPercentage() {
+        if (swapHits == 0 || swapGets == 0)
+            return 0;
+
+        return (float) swapHits / swapGets * 100.0f;
+    }
+
+    /** {@inheritDoc} */
+    @Override public float getSwapMissPercentage() {
+        if (swapMisses == 0 || swapGets == 0)
+            return 0;
+
+        return (float) swapMisses / swapGets * 100.0f;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getSwapEntriesCount() {
+        return swapEntriesCnt;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getSwapSize() {
+        return swapSize;
+    }
+
+    /** {@inheritDoc} */
     @Override public int getSize() {
         return size;
     }
@@ -495,7 +691,7 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
 
     /** {@inheritDoc} */
     @Override public int getDhtEvictQueueCurrentSize() {
-        return dhtEvictQueueCurrentSize;
+        return dhtEvictQueueCurrSize;
     }
 
     /** {@inheritDoc} */
@@ -520,7 +716,7 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
 
     /** {@inheritDoc} */
     @Override public int getTxStartVersionCountsSize() {
-        return txStartVersionCountsSize;
+        return txStartVerCountsSize;
     }
 
     /** {@inheritDoc} */
@@ -555,7 +751,7 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
 
     /** {@inheritDoc} */
     @Override public int getTxDhtStartVersionCountsSize() {
-        return txDhtStartVersionCountsSize;
+        return txDhtStartVerCountsSize;
     }
 
     /** {@inheritDoc} */
@@ -580,12 +776,12 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
 
     /** {@inheritDoc} */
     @Override public int getWriteBehindFlushThreadCount() {
-        return writeBehindFlushThreadCount;
+        return writeBehindFlushThreadCnt;
     }
 
     /** {@inheritDoc} */
     @Override public long getWriteBehindFlushFrequency() {
-        return writeBehindFlushFrequency;
+        return writeBehindFlushFreq;
     }
 
     /** {@inheritDoc} */
@@ -595,22 +791,22 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
 
     /** {@inheritDoc} */
     @Override public int getWriteBehindTotalCriticalOverflowCount() {
-        return writeBehindTotalCriticalOverflowCount;
+        return writeBehindTotalCriticalOverflowCnt;
     }
 
     /** {@inheritDoc} */
     @Override public int getWriteBehindCriticalOverflowCount() {
-        return writeBehindCriticalOverflowCount;
+        return writeBehindCriticalOverflowCnt;
     }
 
     /** {@inheritDoc} */
     @Override public int getWriteBehindErrorRetryCount() {
-        return writeBehindErrorRetryCount;
+        return writeBehindErrorRetryCnt;
     }
 
     /** {@inheritDoc} */
     @Override public int getWriteBehindBufferSize() {
-        return writeBehindBufferSize;
+        return writeBehindBufSize;
     }
 
     /** {@inheritDoc} */
@@ -620,12 +816,12 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
 
     /** {@inheritDoc} */
     @Override public String getValueType() {
-        return valueType;
+        return valType;
     }
 
     /** {@inheritDoc} */
     @Override public boolean isStoreByValue() {
-        return isStoreByValue;
+        return isStoreByVal;
     }
 
     /** {@inheritDoc} */
@@ -666,31 +862,49 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
 
         out.writeFloat(putAvgTimeNanos);
         out.writeFloat(getAvgTimeNanos);
-        out.writeFloat(removeAvgTimeNanos);
+        out.writeFloat(rmvAvgTimeNanos);
         out.writeFloat(commitAvgTimeNanos);
         out.writeFloat(rollbackAvgTimeNanos);
 
         out.writeLong(overflowSize);
-        out.writeLong(offHeapEntriesCount);
+        out.writeLong(offHeapGets);
+        out.writeLong(offHeapPuts);
+        out.writeLong(offHeapRemoves);
+        out.writeLong(offHeapEvicts);
+        out.writeLong(offHeapHits);
+        out.writeLong(offHeapMisses);
+        out.writeLong(offHeapEntriesCnt);
+        out.writeLong(offHeapPrimaryEntriesCnt);
+        out.writeLong(offHeapBackupEntriesCnt);
         out.writeLong(offHeapAllocatedSize);
-        out.writeInt(dhtEvictQueueCurrentSize);
+        out.writeLong(offHeapMaxSize);
+
+        out.writeLong(swapGets);
+        out.writeLong(swapPuts);
+        out.writeLong(swapRemoves);
+        out.writeLong(swapHits);
+        out.writeLong(swapMisses);
+        out.writeLong(swapEntriesCnt);
+        out.writeLong(swapSize);
+
+        out.writeInt(dhtEvictQueueCurrSize);
         out.writeInt(txThreadMapSize);
         out.writeInt(txXidMapSize);
         out.writeInt(txCommitQueueSize);
         out.writeInt(txPrepareQueueSize);
-        out.writeInt(txStartVersionCountsSize);
+        out.writeInt(txStartVerCountsSize);
         out.writeInt(txCommittedVersionsSize);
         out.writeInt(txRolledbackVersionsSize);
         out.writeInt(txDhtThreadMapSize);
         out.writeInt(txDhtXidMapSize);
         out.writeInt(txDhtCommitQueueSize);
         out.writeInt(txDhtPrepareQueueSize);
-        out.writeInt(txDhtStartVersionCountsSize);
+        out.writeInt(txDhtStartVerCountsSize);
         out.writeInt(txDhtCommittedVersionsSize);
         out.writeInt(txDhtRolledbackVersionsSize);
-        out.writeInt(writeBehindTotalCriticalOverflowCount);
-        out.writeInt(writeBehindCriticalOverflowCount);
-        out.writeInt(writeBehindErrorRetryCount);
+        out.writeInt(writeBehindTotalCriticalOverflowCnt);
+        out.writeInt(writeBehindCriticalOverflowCnt);
+        out.writeInt(writeBehindErrorRetryCnt);
     }
 
     /** {@inheritDoc} */
@@ -706,30 +920,48 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
 
         putAvgTimeNanos = in.readFloat();
         getAvgTimeNanos = in.readFloat();
-        removeAvgTimeNanos = in.readFloat();
+        rmvAvgTimeNanos = in.readFloat();
         commitAvgTimeNanos = in.readFloat();
         rollbackAvgTimeNanos = in.readFloat();
 
         overflowSize = in.readLong();
-        offHeapEntriesCount = in.readLong();
+        offHeapGets = in.readLong();
+        offHeapPuts = in.readLong();
+        offHeapRemoves = in.readLong();
+        offHeapEvicts = in.readLong();
+        offHeapHits = in.readLong();
+        offHeapMisses = in.readLong();
+        offHeapEntriesCnt = in.readLong();
+        offHeapPrimaryEntriesCnt = in.readLong();
+        offHeapBackupEntriesCnt = in.readLong();
         offHeapAllocatedSize = in.readLong();
-        dhtEvictQueueCurrentSize = in.readInt();
+        offHeapMaxSize = in.readLong();
+
+        swapGets = in.readLong();
+        swapPuts = in.readLong();
+        swapRemoves = in.readLong();
+        swapHits = in.readLong();
+        swapMisses = in.readLong();
+        swapEntriesCnt = in.readLong();
+        swapSize = in.readLong();
+
+        dhtEvictQueueCurrSize = in.readInt();
         txThreadMapSize = in.readInt();
         txXidMapSize = in.readInt();
         txCommitQueueSize = in.readInt();
         txPrepareQueueSize = in.readInt();
-        txStartVersionCountsSize = in.readInt();
+        txStartVerCountsSize = in.readInt();
         txCommittedVersionsSize = in.readInt();
         txRolledbackVersionsSize = in.readInt();
         txDhtThreadMapSize = in.readInt();
         txDhtXidMapSize = in.readInt();
         txDhtCommitQueueSize = in.readInt();
         txDhtPrepareQueueSize = in.readInt();
-        txDhtStartVersionCountsSize = in.readInt();
+        txDhtStartVerCountsSize = in.readInt();
         txDhtCommittedVersionsSize = in.readInt();
         txDhtRolledbackVersionsSize = in.readInt();
-        writeBehindTotalCriticalOverflowCount = in.readInt();
-        writeBehindCriticalOverflowCount = in.readInt();
-        writeBehindErrorRetryCount = in.readInt();
+        writeBehindTotalCriticalOverflowCnt = in.readInt();
+        writeBehindCriticalOverflowCnt = in.readInt();
+        writeBehindErrorRetryCnt = in.readInt();
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c14eb563/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapManager.java
index eb82218..f442f03 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapManager.java
@@ -121,6 +121,9 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
                         warnFirstEvict();
 
                     writeToSwap(part, cctx.toCacheKeyObject(kb), vb);
+
+                    if (cctx.config().isStatisticsEnabled())
+                        cctx.cache().metrics0().onOffHeapEvict();
                 }
                 catch (IgniteCheckedException e) {
                     log.error("Failed to unmarshal off-heap entry [part=" + part + ", hash=" + hash + ']', e);
@@ -395,8 +398,7 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
      * @return Reconstituted swap entry or {@code null} if entry is obsolete.
      * @throws IgniteCheckedException If failed.
      */
-    @Nullable private <X extends GridCacheSwapEntry> X swapEntry(X e) throws IgniteCheckedException
-    {
+    @Nullable private <X extends GridCacheSwapEntry> X swapEntry(X e) throws IgniteCheckedException {
         assert e != null;
 
         checkIteratorQueue();
@@ -425,16 +427,13 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
         int part = cctx.affinity().partition(key);
 
         // First check off-heap store.
-        if (offheapEnabled)
-            if (offheap.contains(spaceName, part, key, key.valueBytes(cctx.cacheObjectContext())))
-                return true;
+        if (offheapEnabled && offheap.contains(spaceName, part, key, cctx))
+            return true;
 
         if (swapEnabled) {
             assert key != null;
 
-            byte[] valBytes = swapMgr.read(spaceName,
-                new SwapKey(key.value(cctx.cacheObjectContext(), false), part, key.valueBytes(cctx.cacheObjectContext())),
-                cctx.deploy().globalLoader());
+            byte[] valBytes = swapMgr.read(spaceName, part, key, cctx);
 
             return valBytes != null;
         }
@@ -444,7 +443,6 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
 
     /**
      * @param key Key to read.
-     * @param keyBytes Key bytes.
      * @param part Key partition.
      * @param entryLocked {@code True} if cache entry is locked.
      * @param readOffheap Read offheap flag.
@@ -454,7 +452,6 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
      */
     @SuppressWarnings({"unchecked"})
     @Nullable private GridCacheSwapEntry read(KeyCacheObject key,
-        byte[] keyBytes,
         int part,
         boolean entryLocked,
         boolean readOffheap,
@@ -479,7 +476,7 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
 
             // First check off-heap store.
             if (readOffheap && offheapEnabled) {
-                byte[] bytes = offheap.get(spaceName, part, key, keyBytes);
+                byte[] bytes = offheap.get(spaceName, part, key, cctx);
 
                 if (bytes != null)
                     return swapEntry(unmarshalSwapEntry(bytes));
@@ -490,9 +487,7 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
 
             assert key != null;
 
-            byte[] bytes = swapMgr.read(spaceName,
-                new SwapKey(key.value(cctx.cacheObjectContext(), false), part, keyBytes),
-                cctx.deploy().globalLoader());
+            byte[] bytes = swapMgr.read(spaceName, part, key, cctx);
 
             if (bytes == null && lsnr != null)
                 return lsnr.entry;
@@ -522,7 +517,10 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
 
         // First try removing from offheap.
         if (offheapEnabled) {
-            byte[] entryBytes = offheap.remove(spaceName, part, key, key.valueBytes(cctx.cacheObjectContext()));
+            byte[] entryBytes = offheap.remove(spaceName, part, key, cctx);
+
+            if (cctx.config().isStatisticsEnabled())
+                cctx.cache().metrics0().onOffHeapRead(entryBytes != null);
 
             if (entryBytes != null) {
                 GridCacheSwapEntry entry = swapEntry(unmarshalSwapEntry(entryBytes));
@@ -567,8 +565,7 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
      * @return Value from swap or {@code null}.
      * @throws IgniteCheckedException If failed.
      */
-    @Nullable private GridCacheSwapEntry readAndRemoveSwap(final KeyCacheObject key,
-        final int part)
+    @Nullable private GridCacheSwapEntry readAndRemoveSwap(final KeyCacheObject key, final int part)
         throws IgniteCheckedException {
         if (!swapEnabled)
             return null;
@@ -576,12 +573,11 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
         final GridTuple<GridCacheSwapEntry> t = F.t1();
         final GridTuple<IgniteCheckedException> err = F.t1();
 
-        SwapKey swapKey = new SwapKey(key.value(cctx.cacheObjectContext(), false),
-            part,
-            key.valueBytes(cctx.cacheObjectContext()));
-
-        swapMgr.remove(spaceName, swapKey, new CI1<byte[]>() {
+        swapMgr.remove(spaceName, part, key, cctx, new CI1<byte[]>() {
             @Override public void apply(byte[] rmv) {
+                if (cctx.config().isStatisticsEnabled())
+                    cctx.cache().metrics0().onSwapRead(rmv != null);
+
                 if (rmv != null) {
                     try {
                         GridCacheSwapEntry entry = swapEntry(unmarshalSwapEntry(rmv));
@@ -624,7 +620,7 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
                     }
                 }
             }
-        }, cctx.deploy().globalLoader());
+        });
 
         if (err.get() != null)
             throw err.get();
@@ -649,12 +645,7 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
         if (!offheapEnabled && !swapEnabled)
             return null;
 
-        return read(entry.key(),
-            entry.key().valueBytes(cctx.cacheObjectContext()),
-            entry.partition(),
-            locked,
-            readOffheap,
-            readSwap);
+        return read(entry.key(), entry.partition(), locked, readOffheap, readSwap);
     }
 
     /**
@@ -670,8 +661,7 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
 
         int part = cctx.affinity().partition(key);
 
-        IgniteBiTuple<Long, Integer> ptr =
-            offheap.valuePointer(spaceName, part, key, key.valueBytes(cctx.cacheObjectContext()));
+        IgniteBiTuple<Long, Integer> ptr = offheap.valuePointer(spaceName, part, key, cctx);
 
         if (ptr != null) {
             assert ptr.get1() != null;
@@ -700,7 +690,7 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
 
         int part = cctx.affinity().partition(key);
 
-        return read(key, key.valueBytes(cctx.cacheObjectContext()), part, false, readOffheap, readSwap);
+        return read(key, part, false, readOffheap, readSwap);
     }
 
     /**
@@ -729,7 +719,8 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
 
         final GridCacheQueryManager qryMgr = cctx.queries();
 
-        Collection<SwapKey> unprocessedKeys = null;
+        Collection<KeyCacheObject> unprocessedKeys = null;
+
         final Collection<GridCacheBatchSwapEntry> res = new ArrayList<>(keys.size());
 
         // First try removing from offheap.
@@ -738,7 +729,7 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
                 int part = cctx.affinity().partition(key);
 
                 byte[] entryBytes =
-                    offheap.remove(spaceName, part, key, key.valueBytes(cctx.cacheObjectContext()));
+                    offheap.remove(spaceName, part, key, cctx);
 
                 if (entryBytes != null) {
                     GridCacheSwapEntry entry = swapEntry(unmarshalSwapEntry(entryBytes));
@@ -775,11 +766,7 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
                     if (unprocessedKeys == null)
                         unprocessedKeys = new ArrayList<>(keys.size());
 
-                    SwapKey swapKey = new SwapKey(key.value(cctx.cacheObjectContext(), false),
-                        cctx.affinity().partition(key),
-                        key.valueBytes(cctx.cacheObjectContext()));
-
-                    unprocessedKeys.add(swapKey);
+                    unprocessedKeys.add(key);
                 }
             }
 
@@ -789,13 +776,8 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
         else {
             unprocessedKeys = new ArrayList<>(keys.size());
 
-            for (KeyCacheObject key : keys) {
-                SwapKey swapKey = new SwapKey(key.value(cctx.cacheObjectContext(), false),
-                    cctx.affinity().partition(key),
-                    key.valueBytes(cctx.cacheObjectContext()));
-
-                unprocessedKeys.add(swapKey);
-            }
+            for (KeyCacheObject key : keys)
+                unprocessedKeys.add(key);
         }
 
         assert swapEnabled;
@@ -804,9 +786,7 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
         // Swap is enabled.
         final GridTuple<IgniteCheckedException> err = F.t1();
 
-        swapMgr.removeAll(spaceName,
-            unprocessedKeys,
-            new IgniteBiInClosure<SwapKey, byte[]>() {
+        swapMgr.removeAll(spaceName, unprocessedKeys, cctx, new IgniteBiInClosure<SwapKey, byte[]>() {
                 @Override public void apply(SwapKey swapKey, byte[] rmv) {
                     if (rmv != null) {
                         try {
@@ -859,8 +839,8 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
                         }
                     }
                 }
-            },
-            cctx.deploy().globalLoader());
+            }
+        );
 
         if (err.get() != null)
             throw err.get();
@@ -880,7 +860,7 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
 
         int part = cctx.affinity().partition(key);
 
-        return offheap.removex(spaceName, part, key, key.valueBytes(cctx.cacheObjectContext()));
+        return offheap.removex(spaceName, part, key, cctx);
     }
 
     /**
@@ -904,7 +884,7 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
 
         int part = cctx.affinity().partition(key);
 
-        offheap.enableEviction(spaceName, part, key, key.valueBytes(cctx.cacheObjectContext()));
+        offheap.enableEviction(spaceName, part, key, cctx);
     }
 
     /**
@@ -942,10 +922,7 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
 
         // First try offheap.
         if (offheapEnabled) {
-            byte[] val = offheap.remove(spaceName,
-                part,
-                key.value(cctx.cacheObjectContext(), false),
-                key.valueBytes(cctx.cacheObjectContext()));
+            byte[] val = offheap.remove(spaceName, part, key, cctx);
 
             if (val != null) {
                 if (c != null)
@@ -955,16 +932,8 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
             }
         }
 
-        if (swapEnabled) {
-            SwapKey swapKey = new SwapKey(key.value(cctx.cacheObjectContext(), false),
-                part,
-                key.valueBytes(cctx.cacheObjectContext()));
-
-            swapMgr.remove(spaceName,
-                swapKey,
-                c,
-                cctx.deploy().globalLoader());
-        }
+        if (swapEnabled)
+            swapMgr.remove(spaceName, part, key, cctx, c);
     }
 
     /**
@@ -1004,13 +973,8 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
             keyClsLdrId,
             valClsLdrId);
 
-        if (offheapEnabled) {
-            offheap.put(spaceName, part, key, key.valueBytes(cctx.cacheObjectContext()), entry.marshal());
-
-            if (cctx.events().isRecordable(EVT_CACHE_OBJECT_TO_OFFHEAP))
-                cctx.events().addEvent(part, key, cctx.nodeId(), (IgniteUuid)null, null,
-                    EVT_CACHE_OBJECT_TO_OFFHEAP, null, false, null, true, null, null, null);
-        }
+        if (offheapEnabled)
+            offheap.put(spaceName, part, key, entry.marshal(), cctx);
         else if (swapEnabled)
             writeToSwap(part, key, entry.marshal());
 
@@ -1035,32 +999,14 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
 
         if (offheapEnabled) {
             for (GridCacheBatchSwapEntry swapEntry : swapped) {
-                offheap.put(spaceName,
-                    swapEntry.partition(),
-                    swapEntry.key(),
-                    swapEntry.key().valueBytes(cctx.cacheObjectContext()),
-                    swapEntry.marshal());
-
-                if (cctx.events().isRecordable(EVT_CACHE_OBJECT_TO_OFFHEAP))
-                    cctx.events().addEvent(swapEntry.partition(), swapEntry.key(), cctx.nodeId(),
-                        (IgniteUuid)null, null, EVT_CACHE_OBJECT_TO_OFFHEAP, null, false, null, true, null, null, null);
+                offheap.put(spaceName, swapEntry.partition(), swapEntry.key(), swapEntry.marshal(), cctx);
 
                 if (qryMgr != null)
                     qryMgr.onSwap(swapEntry.key());
             }
         }
         else {
-            Map<SwapKey, byte[]> batch = new LinkedHashMap<>();
-
-            for (GridCacheBatchSwapEntry entry : swapped) {
-                SwapKey swapKey = new SwapKey(entry.key().value(cctx.cacheObjectContext(), false),
-                    entry.partition(),
-                    entry.key().valueBytes(cctx.cacheObjectContext()));
-
-                batch.put(swapKey, entry.marshal());
-            }
-
-            swapMgr.writeAll(spaceName, batch, cctx.deploy().globalLoader());
+            swapMgr.writeAll(spaceName, swapped, cctx);
 
             if (cctx.events().isRecordable(EVT_CACHE_OBJECT_SWAPPED)) {
                 for (GridCacheBatchSwapEntry batchSwapEntry : swapped) {
@@ -1082,17 +1028,10 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
      * @param entry Entry bytes.
      * @throws IgniteCheckedException If failed.
      */
-    private void writeToSwap(int part,
-        KeyCacheObject key,
-        byte[] entry)
-        throws IgniteCheckedException
-    {
+    private void writeToSwap(int part, KeyCacheObject key, byte[] entry) throws IgniteCheckedException {
         checkIteratorQueue();
 
-        swapMgr.write(spaceName,
-            new SwapKey(key.value(cctx.cacheObjectContext(), false), part, key.valueBytes(cctx.cacheObjectContext())),
-            entry,
-            cctx.deploy().globalLoader());
+        swapMgr.write(spaceName, part, key, entry, cctx);
 
         if (cctx.events().isRecordable(EVT_CACHE_OBJECT_SWAPPED))
             cctx.events().addEvent(part, key, cctx.nodeId(), (IgniteUuid) null, null,
@@ -1274,7 +1213,7 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
 
                     int part = cctx.affinity().partition(key);
 
-                    offheap.removex(spaceName, part, key, key.valueBytes(cctx.cacheObjectContext()));
+                    offheap.removex(spaceName, part, key, cctx);
                 }
                 else
                     it.removeX();
@@ -1432,6 +1371,7 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
                 return it.hasNext();
             }
 
+            @SuppressWarnings("unchecked")
             @Override protected void onRemove() throws IgniteCheckedException {
                 if (cur == null)
                     throw new IllegalStateException("Method next() has not yet been called, or the remove() method " +
@@ -1616,7 +1556,7 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
 
                     int part = cctx.affinity().partition(key);
 
-                    offheap.removex(spaceName, part, key, key.valueBytes(cctx.cacheObjectContext()));
+                    offheap.removex(spaceName, part, key, cctx);
                 }
 
                 @Override protected void onClose() throws IgniteCheckedException {
@@ -1646,7 +1586,7 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
 
                 int part = cctx.affinity().partition(key);
 
-                offheap.removex(spaceName, part, key, key.valueBytes(cctx.cacheObjectContext()));
+                offheap.removex(spaceName, part, key, cctx);
             }
         };
     }