You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by yz...@apache.org on 2015/11/25 22:35:54 UTC

[06/13] ignite git commit: IGNITE-1544: Make sure objects number stored in a cache can be bigger than Integer.MAX_VALUE. This closes #252

IGNITE-1544: Make sure objects number stored in a cache can be bigger than Integer.MAX_VALUE. This closes #252


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

Branch: refs/heads/ignite-1.5-cache-get-opts
Commit: 8a9cc5408f99713fb4a1a9036940fc76aef7b050
Parents: c23cda1
Author: Andrey Gura <ag...@gridgain.com>
Authored: Wed Nov 25 18:21:48 2015 +0300
Committer: Denis Magda <dm...@gridgain.com>
Committed: Wed Nov 25 18:21:48 2015 +0300

----------------------------------------------------------------------
 .../java/org/apache/ignite/IgniteCache.java     |  23 +++
 .../processors/cache/GridCacheAdapter.java      | 156 ++++++++++++++++++-
 .../processors/cache/GridCacheProxyImpl.java    |  61 ++++++++
 .../processors/cache/IgniteCacheProxy.java      |  42 +++++
 .../processors/cache/IgniteInternalCache.java   |  47 +++++-
 .../distributed/dht/GridDhtCacheAdapter.java    |   7 +-
 .../distributed/near/GridNearCacheAdapter.java  |  10 ++
 .../CacheDeferredDeleteSanitySelfTest.java      |  98 ++++++++++++
 .../multijvm/IgniteCacheProcessProxy.java       |  38 +++++
 .../ignite/testsuites/IgniteCacheTestSuite.java |   2 +
 ...niteCacheP2pUnmarshallingQueryErrorTest.java |   7 +-
 .../ignite-int-max-values-offheap-config.xml    |  89 +++++++++++
 .../ignite-int-max-values-onheap-config.xml     |  84 ++++++++++
 .../ignite-int-max-values-swap-config.xml       |  93 +++++++++++
 .../test-max-int-values-offheap.properties      |  70 +++++++++
 .../test-max-int-values-onheap.properties       |  70 +++++++++
 .../config/test-max-int-values-swap.properties  |  69 ++++++++
 .../yardstick/cache/IntMaxValueEntriesTest.java | 135 ++++++++++++++++
 18 files changed, 1094 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/8a9cc540/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteCache.java b/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
index c9de3f1..c575865 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
@@ -348,6 +348,19 @@ public interface IgniteCache<K, V> extends javax.cache.Cache<K, V>, IgniteAsyncS
     public int size(CachePeekMode... peekModes) throws CacheException;
 
     /**
+     * Gets the number of all entries cached across all nodes as a long value. By default, if {@code peekModes} value
+     * isn't defined, only size of primary copies across all nodes will be returned. This behavior is identical to
+     * calling this method with {@link CachePeekMode#PRIMARY} peek mode.
+     * <p>
+     * NOTE: this operation is distributed and will query all participating nodes for their cache sizes.
+     *
+     * @param peekModes Optional peek modes. If not provided, then total cache size is returned.
+     * @return Cache size across all nodes.
+     */
+    @IgniteAsyncSupported
+    public long sizeLong(CachePeekMode... peekModes) throws CacheException;
+
+    /**
      * Gets the number of all entries cached on this node. By default, if {@code peekModes} value isn't defined,
      * only size of primary copies will be returned. This behavior is identical to calling this method with
      * {@link CachePeekMode#PRIMARY} peek mode.
@@ -358,6 +371,16 @@ public interface IgniteCache<K, V> extends javax.cache.Cache<K, V>, IgniteAsyncS
     public int localSize(CachePeekMode... peekModes);
 
     /**
+     * Gets the number of all entries cached on this node as a long value. By default, if {@code peekModes} value isn't
+     * defined, only size of primary copies will be returned. This behavior is identical to calling this method with
+     * {@link CachePeekMode#PRIMARY} peek mode.
+     *
+     * @param peekModes Optional peek modes. If not provided, then total cache size is returned.
+     * @return Cache size on this node.
+     */
+    public long localSizeLong(CachePeekMode... peekModes);
+
+    /**
      * @param map Map containing keys and entry processors to be applied to values.
      * @param args Additional arguments to pass to the {@link EntryProcessor}.
      * @return The map of {@link EntryProcessorResult}s of the processing per key,

http://git-wip-us.apache.org/repos/asf/ignite/blob/8a9cc540/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index 26a3acd..d9713e1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
@@ -303,7 +303,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      * @param ctx Cache context.
      * @param map Concurrent map.
      */
-    @SuppressWarnings("OverriddenMethodCallDuringObjectConstruction")
+    @SuppressWarnings({"OverriddenMethodCallDuringObjectConstruction", "deprecation"})
     protected GridCacheAdapter(final GridCacheContext<K, V> ctx, GridCacheConcurrentMap map) {
         assert ctx != null;
 
@@ -821,6 +821,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      * @throws GridCacheEntryRemovedException If entry removed.
      * @throws IgniteCheckedException If failed.
      */
+    @SuppressWarnings("ConstantConditions")
     @Nullable private CacheObject localCachePeek0(KeyCacheObject key,
         boolean heap,
         boolean offheap,
@@ -3601,6 +3602,14 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     }
 
     /** {@inheritDoc} */
+    @Override public long sizeLong(CachePeekMode[] peekModes) throws IgniteCheckedException {
+        if (isLocal())
+            return localSizeLong(peekModes);
+
+        return sizeLongAsync(peekModes).get();
+    }
+
+    /** {@inheritDoc} */
     @Override public IgniteInternalFuture<Integer> sizeAsync(final CachePeekMode[] peekModes) {
         assert peekModes != null;
 
@@ -3622,11 +3631,36 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     }
 
     /** {@inheritDoc} */
-    @SuppressWarnings("ForLoopReplaceableByForEach")
+    @Override public IgniteInternalFuture<Long> sizeLongAsync(final CachePeekMode[] peekModes) {
+        assert peekModes != null;
+
+        PeekModes modes = parsePeekModes(peekModes, true);
+
+        IgniteClusterEx cluster = ctx.grid().cluster();
+
+        ClusterGroup grp = modes.near ? cluster.forCacheNodes(name(), true, true, false) : cluster.forDataNodes(name());
+
+        Collection<ClusterNode> nodes = grp.nodes();
+
+        if (nodes.isEmpty())
+            return new GridFinishedFuture<>(0L);
+
+        ctx.kernalContext().task().setThreadContext(TC_SUBGRID, nodes);
+
+        return ctx.kernalContext().task().execute(
+            new SizeLongTask(ctx.name(), ctx.affinity().affinityTopologyVersion(), peekModes), null);
+    }
+
+    /** {@inheritDoc} */
     @Override public int localSize(CachePeekMode[] peekModes) throws IgniteCheckedException {
+        return (int)localSizeLong(peekModes);
+    }
+
+    /** {@inheritDoc} */
+    @Override public long localSizeLong(CachePeekMode[] peekModes) throws IgniteCheckedException {
         PeekModes modes = parsePeekModes(peekModes, true);
 
-        int size = 0;
+        long size = 0;
 
         if (ctx.isLocal()) {
             modes.primary = true;
@@ -3676,6 +3710,11 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     }
 
     /** {@inheritDoc} */
+    @Override public long sizeLong() {
+        return map.publicSize();
+    }
+
+    /** {@inheritDoc} */
     @Override public int nearSize() {
         return 0;
     }
@@ -3686,6 +3725,11 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     }
 
     /** {@inheritDoc} */
+    @Override public long primarySizeLong() {
+        return map.publicSize();
+    }
+
+    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(GridCacheAdapter.class, this, "name", name(), "size", size());
     }
@@ -5233,6 +5277,47 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     }
 
     /**
+     * Internal callable for global size calculation.
+     */
+    @GridInternal
+    private static class SizeLongJob extends TopologyVersionAwareJob {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** Peek modes. */
+        private final CachePeekMode[] peekModes;
+
+        /**
+         * @param cacheName Cache name.
+         * @param topVer Affinity topology version.
+         * @param peekModes Cache peek modes.
+         */
+        private SizeLongJob(String cacheName, AffinityTopologyVersion topVer, CachePeekMode[] peekModes) {
+            super(cacheName, topVer);
+
+            this.peekModes = peekModes;
+        }
+
+        /** {@inheritDoc} */
+        @Nullable @Override public Object localExecute(@Nullable IgniteInternalCache cache) {
+            if (cache == null)
+                return 0;
+
+            try {
+                return cache.localSizeLong(peekModes);
+            }
+            catch (IgniteCheckedException e) {
+                throw U.convertException(e);
+            }
+        }
+
+        /** {@inheritDoc} */
+        public String toString() {
+            return S.toString(SizeLongJob.class, this);
+        }
+    }
+
+    /**
      * Holder for last async operation future.
      */
     protected static class FutureHolder {
@@ -5942,6 +6027,71 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     }
 
     /**
+     * Size task.
+     */
+    private static class SizeLongTask extends ComputeTaskAdapter<Object, Long> {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** Cache name. */
+        private final String cacheName;
+
+        /** Affinity topology version. */
+        private final AffinityTopologyVersion topVer;
+
+        /** Peek modes. */
+        private final CachePeekMode[] peekModes;
+
+        /**
+         * @param cacheName Cache name.
+         * @param topVer Affinity topology version.
+         * @param peekModes Cache peek modes.
+         */
+        public SizeLongTask(String cacheName, AffinityTopologyVersion topVer, CachePeekMode[] peekModes) {
+            this.cacheName = cacheName;
+            this.topVer = topVer;
+            this.peekModes = peekModes;
+        }
+
+        /** {@inheritDoc} */
+        @Nullable @Override public Map<? extends ComputeJob, ClusterNode> map(List<ClusterNode> subgrid,
+            @Nullable Object arg) throws IgniteException {
+            Map<ComputeJob, ClusterNode> jobs = new HashMap();
+
+            for (ClusterNode node : subgrid)
+                jobs.put(new SizeLongJob(cacheName, topVer, peekModes), node);
+
+            return jobs;
+        }
+
+        /** {@inheritDoc} */
+        @Override public ComputeJobResultPolicy result(ComputeJobResult res, List<ComputeJobResult> rcvd) {
+            IgniteException e = res.getException();
+
+            if (e != null) {
+                if (e instanceof ClusterTopologyException)
+                    return ComputeJobResultPolicy.WAIT;
+
+                throw new IgniteException("Remote job threw exception.", e);
+            }
+
+            return ComputeJobResultPolicy.WAIT;
+        }
+
+        /** {@inheritDoc} */
+        @Nullable @Override public Long reduce(List<ComputeJobResult> results) throws IgniteException {
+            long size = 0;
+
+            for (ComputeJobResult res : results) {
+                if (res != null && res.getException() == null)
+                    size += res.<Long>getData();
+            }
+
+            return size;
+        }
+    }
+
+    /**
      * Clear task.
      */
     private static class ClearTask<K> extends ComputeTaskAdapter<Object, Object> {

http://git-wip-us.apache.org/repos/asf/ignite/blob/8a9cc540/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProxyImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProxyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProxyImpl.java
index cca6d6a..e82c422 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProxyImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProxyImpl.java
@@ -228,6 +228,7 @@ public class GridCacheProxyImpl<K, V> implements IgniteInternalCache<K, V>, Exte
     }
 
     /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
     @Override public <K1, V1> GridCacheProxyImpl<K1, V1> keepPortable() {
         if (opCtx != null && opCtx.isKeepBinary())
             return (GridCacheProxyImpl<K1, V1>)this;
@@ -1352,6 +1353,18 @@ public class GridCacheProxyImpl<K, V> implements IgniteInternalCache<K, V>, Exte
     }
 
     /** {@inheritDoc} */
+    @Override public long sizeLong() {
+        CacheOperationContext prev = gate.enter(opCtx);
+
+        try {
+            return delegate.sizeLong();
+        }
+        finally {
+            gate.leave(prev);
+        }
+    }
+
+    /** {@inheritDoc} */
     @Override public int size(CachePeekMode[] peekModes) throws IgniteCheckedException {
         CacheOperationContext prev = gate.enter(opCtx);
 
@@ -1364,6 +1377,18 @@ public class GridCacheProxyImpl<K, V> implements IgniteInternalCache<K, V>, Exte
     }
 
     /** {@inheritDoc} */
+    @Override public long sizeLong(CachePeekMode[] peekModes) throws IgniteCheckedException {
+        CacheOperationContext prev = gate.enter(opCtx);
+
+        try {
+            return delegate.sizeLong(peekModes);
+        }
+        finally {
+            gate.leave(prev);
+        }
+    }
+
+    /** {@inheritDoc} */
     @Override public IgniteInternalFuture<Integer> sizeAsync(CachePeekMode[] peekModes) {
         CacheOperationContext prev = gate.enter(opCtx);
 
@@ -1376,6 +1401,18 @@ public class GridCacheProxyImpl<K, V> implements IgniteInternalCache<K, V>, Exte
     }
 
     /** {@inheritDoc} */
+    @Override public IgniteInternalFuture<Long> sizeLongAsync(CachePeekMode[] peekModes) {
+        CacheOperationContext prev = gate.enter(opCtx);
+
+        try {
+            return delegate.sizeLongAsync(peekModes);
+        }
+        finally {
+            gate.leave(prev);
+        }
+    }
+
+    /** {@inheritDoc} */
     @Override public int localSize(CachePeekMode[] peekModes) throws IgniteCheckedException {
         CacheOperationContext prev = gate.enter(opCtx);
 
@@ -1388,6 +1425,18 @@ public class GridCacheProxyImpl<K, V> implements IgniteInternalCache<K, V>, Exte
     }
 
     /** {@inheritDoc} */
+    @Override public long localSizeLong(CachePeekMode[] peekModes) throws IgniteCheckedException {
+        CacheOperationContext prev = gate.enter(opCtx);
+
+        try {
+            return delegate.localSizeLong(peekModes);
+        }
+        finally {
+            gate.leave(prev);
+        }
+    }
+
+    /** {@inheritDoc} */
     @Override public int nearSize() {
         CacheOperationContext prev = gate.enter(opCtx);
 
@@ -1412,6 +1461,18 @@ public class GridCacheProxyImpl<K, V> implements IgniteInternalCache<K, V>, Exte
     }
 
     /** {@inheritDoc} */
+    @Override public long primarySizeLong() {
+        CacheOperationContext prev = gate.enter(opCtx);
+
+        try {
+            return delegate.primarySizeLong();
+        }
+        finally {
+            gate.leave(prev);
+        }
+    }
+
+    /** {@inheritDoc} */
     @Override public void promoteAll(@Nullable Collection<? extends K> keys) throws IgniteCheckedException {
         CacheOperationContext prev = gate.enter(opCtx);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/8a9cc540/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
index cb36432..2bab152 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
@@ -279,6 +279,7 @@ public class IgniteCacheProxy<K, V> extends AsyncSupportAdapter<IgniteCache<K, V
     }
 
     /** {@inheritDoc} */
+    @SuppressWarnings("deprecation")
     @Nullable @Override public Cache.Entry<K, V> randomEntry() {
         GridKernalContext kctx = ctx.kernalContext();
 
@@ -784,6 +785,30 @@ public class IgniteCacheProxy<K, V> extends AsyncSupportAdapter<IgniteCache<K, V
     }
 
     /** {@inheritDoc} */
+    @Override public long sizeLong(CachePeekMode... peekModes) throws CacheException {
+        GridCacheGateway<K, V> gate = this.gate;
+
+        CacheOperationContext prev = onEnter(gate, opCtx);
+
+        try {
+            if (isAsync()) {
+                setFuture(delegate.sizeLongAsync(peekModes));
+
+                return 0;
+            }
+            else
+                return delegate.sizeLong(peekModes);
+        }
+        catch (IgniteCheckedException e) {
+            throw cacheException(e);
+        }
+        finally {
+            onLeave(gate, prev);
+        }
+    }
+
+
+    /** {@inheritDoc} */
     @Override public int localSize(CachePeekMode... peekModes) {
         GridCacheGateway<K, V> gate = this.gate;
 
@@ -801,6 +826,23 @@ public class IgniteCacheProxy<K, V> extends AsyncSupportAdapter<IgniteCache<K, V
     }
 
     /** {@inheritDoc} */
+    @Override public long localSizeLong(CachePeekMode... peekModes) {
+        GridCacheGateway<K, V> gate = this.gate;
+
+        CacheOperationContext prev = onEnter(gate, opCtx);
+
+        try {
+            return delegate.localSizeLong(peekModes);
+        }
+        catch (IgniteCheckedException e) {
+            throw cacheException(e);
+        }
+        finally {
+            onLeave(gate, prev);
+        }
+    }
+
+    /** {@inheritDoc} */
     @Override public V get(K key) {
         try {
             GridCacheGateway<K, V> gate = this.gate;

http://git-wip-us.apache.org/repos/asf/ignite/blob/8a9cc540/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteInternalCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteInternalCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteInternalCache.java
index 4155706..1055cf4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteInternalCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteInternalCache.java
@@ -1342,6 +1342,17 @@ public interface IgniteInternalCache<K, V> extends Iterable<Cache.Entry<K, V>> {
     public int size();
 
     /**
+     * Gets the number of all entries cached on this node as a long value. This method will return the count of
+     * all cache entries and has O(1) complexity on base {@link IgniteInternalCache}. It is essentially the
+     * size of cache key set and is semantically identical to {{@code Cache.keySet().size()}.
+     * <p>
+     * NOTE: this operation is not distributed and returns only the number of entries cached on this node.
+     *
+     * @return Size of cache on this node.
+     */
+    public long sizeLong();
+
+    /**
      * @param peekModes Peek modes.
      * @return Local cache size.
      * @throws IgniteCheckedException If failed.
@@ -1350,6 +1361,13 @@ public interface IgniteInternalCache<K, V> extends Iterable<Cache.Entry<K, V>> {
 
     /**
      * @param peekModes Peek modes.
+     * @return Local cache size as a long value.
+     * @throws IgniteCheckedException If failed.
+     */
+    public long localSizeLong(CachePeekMode[] peekModes) throws IgniteCheckedException;
+
+    /**
+     * @param peekModes Peek modes.
      * @return Global cache size.
      * @throws IgniteCheckedException If failed.
      */
@@ -1357,11 +1375,24 @@ public interface IgniteInternalCache<K, V> extends Iterable<Cache.Entry<K, V>> {
 
     /**
      * @param peekModes Peek modes.
+     * @return Global cache size as a long value.
+     * @throws IgniteCheckedException If failed.
+     */
+    public long sizeLong(CachePeekMode[] peekModes) throws IgniteCheckedException;
+
+    /**
+     * @param peekModes Peek modes.
      * @return Future.
      */
     public IgniteInternalFuture<Integer> sizeAsync(CachePeekMode[] peekModes);
 
     /**
+     * @param peekModes Peek modes.
+     * @return Future.
+     */
+    public IgniteInternalFuture<Long> sizeLongAsync(CachePeekMode[] peekModes);
+
+    /**
      * Gets size of near cache key set. This method will return count of all entries in near
      * cache and has O(1) complexity on base cache projection.
      * <p>
@@ -1386,6 +1417,20 @@ public interface IgniteInternalCache<K, V> extends Iterable<Cache.Entry<K, V>> {
     public int primarySize();
 
     /**
+     * Gets the number of all primary entries cached on this node as a long value. For {@link CacheMode#LOCAL}
+     * non-distributed cache mode, this method is identical to {@link #size()}.
+     * <p>
+     * For {@link CacheMode#PARTITIONED} and {@link CacheMode#REPLICATED} modes, this method will
+     * return number of primary entries cached on this node (excluding any backups). The complexity of
+     * this method is O(P), where P is the total number of partitions.
+     * <p>
+     * NOTE: this operation is not distributed and returns only the number of primary entries cached on this node.
+     *
+     * @return Number of primary entries in cache.
+     */
+    public long primarySizeLong();
+
+    /**
      * This method unswaps cache entries by given keys, if any, from swap storage
      * into memory.
      * <h2 class="header">Transactions</h2>
@@ -1491,7 +1536,7 @@ public interface IgniteInternalCache<K, V> extends Iterable<Cache.Entry<K, V>> {
      * @param subjId Client ID.
      * @return Internal projection.
      */
-    IgniteInternalCache<K, V> forSubjectId(UUID subjId);
+    public IgniteInternalCache<K, V> forSubjectId(UUID subjId);
 
     /**
      * Store DR data.

http://git-wip-us.apache.org/repos/asf/ignite/blob/8a9cc540/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
index 8537357..de11235 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
@@ -593,7 +593,12 @@ public abstract class GridDhtCacheAdapter<K, V> extends GridDistributedCacheAdap
 
     /** {@inheritDoc} */
     @Override public int primarySize() {
-        int sum = 0;
+        return (int)primarySizeLong();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long primarySizeLong() {
+        long sum = 0;
 
         AffinityTopologyVersion topVer = ctx.affinity().affinityTopologyVersion();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/8a9cc540/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java
index cc9d007..b471486 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java
@@ -310,11 +310,21 @@ public abstract class GridNearCacheAdapter<K, V> extends GridDistributedCacheAda
     }
 
     /** {@inheritDoc} */
+    @Override public long sizeLong() {
+        return nearEntries().size() + dht().sizeLong();
+    }
+
+    /** {@inheritDoc} */
     @Override public int primarySize() {
         return dht().primarySize();
     }
 
     /** {@inheritDoc} */
+    @Override public long primarySizeLong() {
+        return dht().primarySizeLong();
+    }
+
+    /** {@inheritDoc} */
     @Override public int nearSize() {
         return nearEntries().size();
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/8a9cc540/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheDeferredDeleteSanitySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheDeferredDeleteSanitySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheDeferredDeleteSanitySelfTest.java
new file mode 100644
index 0000000..5040172
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheDeferredDeleteSanitySelfTest.java
@@ -0,0 +1,98 @@
+/*
+ * 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;
+
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.NearCacheConfiguration;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheMode.LOCAL;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.cache.CacheMode.REPLICATED;
+
+/**
+ * Sanity tests of deferred delete for different cache configurations.
+ */
+public class CacheDeferredDeleteSanitySelfTest extends GridCommonAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        startGrid(0);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If fails.
+     */
+    public void testDeferredDelete() throws Exception {
+        testDeferredDelete(LOCAL, ATOMIC, false, false);
+        testDeferredDelete(LOCAL, TRANSACTIONAL, false, false);
+
+        testDeferredDelete(PARTITIONED, ATOMIC, false, true);
+        testDeferredDelete(PARTITIONED, TRANSACTIONAL, false, true);
+
+        testDeferredDelete(REPLICATED, ATOMIC, false, true);
+        testDeferredDelete(REPLICATED, TRANSACTIONAL, false, true);
+
+        // Near
+        testDeferredDelete(LOCAL, ATOMIC, true, false);
+        testDeferredDelete(LOCAL, TRANSACTIONAL, true, false);
+
+        testDeferredDelete(PARTITIONED, ATOMIC, true, true);
+        testDeferredDelete(PARTITIONED, TRANSACTIONAL, true, false);
+
+        testDeferredDelete(REPLICATED, ATOMIC, true, true);
+        testDeferredDelete(REPLICATED, TRANSACTIONAL, true, true);
+    }
+
+    /**
+     * @param mode Mode.
+     * @param atomicityMode Atomicity mode.
+     * @param near Near cache enabled.
+     * @param expVal Expected deferred delete value.
+     */
+    @SuppressWarnings("unchecked")
+    private void testDeferredDelete(CacheMode mode, CacheAtomicityMode atomicityMode, boolean near, boolean expVal) {
+        CacheConfiguration ccfg = new CacheConfiguration()
+            .setCacheMode(mode)
+            .setAtomicityMode(atomicityMode);
+
+        if (near)
+            ccfg.setNearConfiguration(new NearCacheConfiguration());
+
+        IgniteCache cache = null;
+
+        try {
+            cache = grid(0).getOrCreateCache(ccfg);
+
+            assertEquals(expVal, ((IgniteCacheProxy)grid(0).cache(null)).context().deferredDelete());
+        }
+        finally {
+            if (cache != null)
+                cache.destroy();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8a9cc540/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteCacheProcessProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteCacheProcessProxy.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteCacheProcessProxy.java
index d2e17d8..035f1b0 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteCacheProcessProxy.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteCacheProcessProxy.java
@@ -196,11 +196,21 @@ public class IgniteCacheProcessProxy<K, V> implements IgniteCache<K, V> {
     }
 
     /** {@inheritDoc} */
+    @Override public long sizeLong(CachePeekMode... peekModes) throws CacheException {
+        return compute.call(new SizeLongTask(cacheName, isAsync, peekModes, false));
+    }
+
+    /** {@inheritDoc} */
     @Override public int localSize(CachePeekMode... peekModes) {
         return compute.call(new SizeTask(cacheName, isAsync, peekModes, true));
     }
 
     /** {@inheritDoc} */
+    @Override public long localSizeLong(CachePeekMode... peekModes) {
+        return compute.call(new SizeLongTask(cacheName, isAsync, peekModes, true));
+    }
+
+    /** {@inheritDoc} */
     @Override  public <T> Map<K, EntryProcessorResult<T>> invokeAll(
         Map<? extends K, ? extends EntryProcessor<K, V, T>> map,
         Object... args)
@@ -649,6 +659,34 @@ public class IgniteCacheProcessProxy<K, V> implements IgniteCache<K, V> {
     /**
      *
      */
+    private static class SizeLongTask extends CacheTaskAdapter<Void, Void, Long> {
+        /** Peek modes. */
+        private final CachePeekMode[] peekModes;
+
+        /** Local. */
+        private final boolean loc;
+
+        /**
+         * @param cacheName Cache name.
+         * @param async Async.
+         * @param peekModes Peek modes.
+         * @param loc Local.
+         */
+        public SizeLongTask(String cacheName, boolean async, CachePeekMode[] peekModes, boolean loc) {
+            super(cacheName, async);
+            this.loc = loc;
+            this.peekModes = peekModes;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Long call() throws Exception {
+            return loc ? cache().localSizeLong(peekModes) : cache().sizeLong(peekModes);
+        }
+    }
+
+    /**
+     *
+     */
     private static class GetTask<K, V> extends CacheTaskAdapter<K, V, V> {
         /** Key. */
         private final K key;

http://git-wip-us.apache.org/repos/asf/ignite/blob/8a9cc540/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
index 8af9443..2c9ba93 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
@@ -36,6 +36,7 @@ import org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStorePortableMarshallerSe
 import org.apache.ignite.cache.store.jdbc.GridCacheJdbcBlobStoreMultithreadedSelfTest;
 import org.apache.ignite.cache.store.jdbc.GridCacheJdbcBlobStoreSelfTest;
 import org.apache.ignite.internal.processors.cache.CacheAffinityCallSelfTest;
+import org.apache.ignite.internal.processors.cache.CacheDeferredDeleteSanitySelfTest;
 import org.apache.ignite.internal.processors.cache.CacheFutureExceptionSelfTest;
 import org.apache.ignite.internal.processors.cache.CacheNamesSelfTest;
 import org.apache.ignite.internal.processors.cache.GridCacheAffinityApiSelfTest;
@@ -262,6 +263,7 @@ public class IgniteCacheTestSuite extends TestSuite {
         suite.addTestSuite(GridCachePartitionedOffHeapLocalStoreSelfTest.class);
         suite.addTestSuite(GridCacheTxPartitionedLocalStoreSelfTest.class);
         suite.addTestSuite(IgniteCacheSystemTransactionsSelfTest.class);
+        suite.addTestSuite(CacheDeferredDeleteSanitySelfTest.class);
 
         suite.addTest(IgniteCacheTcpClientDiscoveryTestSuite.suite());
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/8a9cc540/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingQueryErrorTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingQueryErrorTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingQueryErrorTest.java
index 4d9456a..d4a4c62 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingQueryErrorTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingQueryErrorTest.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.cache;
 
 import java.io.IOException;
 import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
 import javax.cache.CacheException;
 import org.apache.ignite.cache.query.ScanQuery;
 import org.apache.ignite.cache.query.SqlQuery;
@@ -68,8 +69,6 @@ public class IgniteCacheP2pUnmarshallingQueryErrorTest extends IgniteCacheP2pUnm
     public void testResponseMessageOnRequestUnmarshallingFailed() throws Exception {
         readCnt.set(Integer.MAX_VALUE);
 
-        jcache(0).put(new TestKey(String.valueOf(++key)), "");
-
         try {
             jcache().query(new ScanQuery<>(new IgniteBiPredicate<TestKey, String>() {
                 @Override public boolean apply(TestKey key, String val) {
@@ -79,6 +78,10 @@ public class IgniteCacheP2pUnmarshallingQueryErrorTest extends IgniteCacheP2pUnm
                 private void readObject(ObjectInputStream is) throws IOException {
                     throw new IOException();
                 }
+
+                private void writeObject(ObjectOutputStream os) throws IOException {
+                    throw new IOException();
+                }
             })).getAll();
 
             assertTrue("Request unmarshalling failed, but error response was not sent.", portableMarshaller());

http://git-wip-us.apache.org/repos/asf/ignite/blob/8a9cc540/modules/yardstick/config/ignite-int-max-values-offheap-config.xml
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/ignite-int-max-values-offheap-config.xml b/modules/yardstick/config/ignite-int-max-values-offheap-config.xml
new file mode 100644
index 0000000..1aaab67
--- /dev/null
+++ b/modules/yardstick/config/ignite-int-max-values-offheap-config.xml
@@ -0,0 +1,89 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  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.
+-->
+
+<!--
+    Ignite Spring configuration file.
+-->
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xsi:schemaLocation="
+        http://www.springframework.org/schema/beans
+        http://www.springframework.org/schema/beans/spring-beans.xsd">
+    <bean id="grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
+        <property name="peerClassLoadingEnabled" value="true"/>
+
+        <property name="marshaller">
+            <bean class="org.apache.ignite.marshaller.optimized.OptimizedMarshaller">
+                <property name="requireSerializable" value="false"/>
+            </bean>
+        </property>
+
+<!--
+        <property name="marshaller">
+            <bean class="org.apache.ignite.internal.portable.api.PortableMarshaller"/>
+        </property>
+-->
+
+        <property name="networkTimeout" value="600000"/>
+
+        <property name="discoverySpi">
+            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
+                <property name="networkTimeout" value="600000"/>
+
+                <property name="socketTimeout" value="600000"/>
+
+                <property name="ackTimeout" value="60000"/>
+
+                <property name="ipFinder">
+                    <!-- <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder"> -->
+                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.multicast.TcpDiscoveryMulticastIpFinder">
+                        <!--
+                                                <property name="addresses">
+                                                    <list>
+                                                        &lt;!&ndash; In distributed environment, replace with actual host IP address. &ndash;&gt;
+                                                        <value>127.0.0.1:47500..47509</value>
+                                                    </list>
+                                                </property>
+                        -->
+                    </bean>
+                </property>
+            </bean>
+        </property>
+
+        <property name="cacheConfiguration">
+            <list>
+                <bean class="org.apache.ignite.configuration.CacheConfiguration">
+                    <property name="name" value="int-max-value-cache"/>
+                    <property name="cacheMode" value="PARTITIONED"/>
+                    <property name="backups" value="0"/>
+                    <property name="atomicityMode" value="ATOMIC"/>
+                    <property name="memoryMode" value="ONHEAP_TIERED"/>
+                    <property name="offHeapMaxMemory" value="#{32 * 1024 * 1024 * 1024L}"/>
+                    <property name="swapEnabled" value="false"/>
+                    <property name="evictionPolicy">
+                        <bean class="org.apache.ignite.cache.eviction.fifo.FifoEvictionPolicy">
+                            <property name="maxSize" value="4000000"/>
+                            <property name="batchSize" value="500000"/>
+                        </bean>
+                    </property>
+                </bean>
+            </list>
+        </property>
+    </bean>
+</beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/8a9cc540/modules/yardstick/config/ignite-int-max-values-onheap-config.xml
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/ignite-int-max-values-onheap-config.xml b/modules/yardstick/config/ignite-int-max-values-onheap-config.xml
new file mode 100644
index 0000000..d1359b3
--- /dev/null
+++ b/modules/yardstick/config/ignite-int-max-values-onheap-config.xml
@@ -0,0 +1,84 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  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.
+-->
+
+<!--
+    Ignite Spring configuration file.
+-->
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xsi:schemaLocation="
+        http://www.springframework.org/schema/beans
+        http://www.springframework.org/schema/beans/spring-beans.xsd">
+    <bean id="grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
+        <property name="peerClassLoadingEnabled" value="true"/>
+
+        <property name="marshaller">
+            <bean class="org.apache.ignite.marshaller.optimized.OptimizedMarshaller">
+                <property name="requireSerializable" value="false"/>
+            </bean>
+        </property>
+
+<!--
+        <property name="marshaller">
+            <bean class="org.apache.ignite.internal.portable.api.PortableMarshaller"/>
+        </property>
+-->
+
+        <property name="networkTimeout" value="600000"/>
+
+        <property name="discoverySpi">
+            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
+                <property name="networkTimeout" value="600000"/>
+
+                <property name="socketTimeout" value="600000"/>
+
+                <property name="ackTimeout" value="60000"/>
+
+
+                <property name="ipFinder">
+                    <!-- <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder"> -->
+                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.multicast.TcpDiscoveryMulticastIpFinder">
+                        <!--
+                                                <property name="addresses">
+                                                    <list>
+                                                        &lt;!&ndash; In distributed environment, replace with actual host IP address. &ndash;&gt;
+                                                        <value>127.0.0.1:47500..47509</value>
+                                                    </list>
+                                                </property>
+                        -->
+                    </bean>
+                </property>
+            </bean>
+        </property>
+
+        <property name="cacheConfiguration">
+            <list>
+                <bean class="org.apache.ignite.configuration.CacheConfiguration">
+                    <property name="name" value="int-max-value-cache"/>
+                    <property name="cacheMode" value="PARTITIONED"/>
+                    <property name="backups" value="0"/>
+                    <property name="atomicityMode" value="ATOMIC"/>
+                    <property name="memoryMode" value="ONHEAP_TIERED"/>
+                    <property name="offHeapMaxMemory" value="-1"/>
+                    <property name="swapEnabled" value="false"/>
+                </bean>
+            </list>
+        </property>
+    </bean>
+</beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/8a9cc540/modules/yardstick/config/ignite-int-max-values-swap-config.xml
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/ignite-int-max-values-swap-config.xml b/modules/yardstick/config/ignite-int-max-values-swap-config.xml
new file mode 100644
index 0000000..8634b4f
--- /dev/null
+++ b/modules/yardstick/config/ignite-int-max-values-swap-config.xml
@@ -0,0 +1,93 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  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.
+-->
+
+<!--
+    Ignite Spring configuration file.
+-->
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xsi:schemaLocation="
+        http://www.springframework.org/schema/beans
+        http://www.springframework.org/schema/beans/spring-beans.xsd">
+    <bean id="grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
+        <property name="peerClassLoadingEnabled" value="true"/>
+
+        <property name="marshaller">
+            <bean class="org.apache.ignite.marshaller.optimized.OptimizedMarshaller">
+                <property name="requireSerializable" value="false"/>
+            </bean>
+        </property>
+
+<!--
+        <property name="marshaller">
+            <bean class="org.apache.ignite.internal.portable.api.PortableMarshaller"/>
+        </property>
+-->
+
+        <property name="networkTimeout" value="600000"/>
+
+        <property name="discoverySpi">
+            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
+                <property name="networkTimeout" value="600000"/>
+
+                <property name="socketTimeout" value="600000"/>
+
+                <property name="ackTimeout" value="60000"/>
+
+                <property name="ipFinder">
+                    <!-- <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder"> -->
+                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.multicast.TcpDiscoveryMulticastIpFinder">
+                        <!--
+                                                <property name="addresses">
+                                                    <list>
+                                                        &lt;!&ndash; In distributed environment, replace with actual host IP address. &ndash;&gt;
+                                                        <value>127.0.0.1:47500..47509</value>
+                                                    </list>
+                                                </property>
+                        -->
+                    </bean>
+                </property>
+            </bean>
+        </property>
+
+        <property name="cacheConfiguration">
+            <list>
+                <bean class="org.apache.ignite.configuration.CacheConfiguration">
+                    <property name="name" value="int-max-value-cache"/>
+                    <property name="cacheMode" value="PARTITIONED"/>
+                    <property name="backups" value="0"/>
+                    <property name="atomicityMode" value="ATOMIC"/>
+                    <property name="memoryMode" value="ONHEAP_TIERED"/>
+                    <property name="offHeapMaxMemory" value="#{32 * 1024 * 1024 * 1024L}"/>
+                    <property name="swapEnabled" value="true"/>
+                    <property name="evictionPolicy">
+                        <bean class="org.apache.ignite.cache.eviction.fifo.FifoEvictionPolicy">
+                            <property name="maxSize" value="4000000"/>
+                            <property name="batchSize" value="500000"/>
+                        </bean>
+                    </property>
+                </bean>
+            </list>
+        </property>
+
+        <property name="swapSpaceSpi">
+            <bean class="org.apache.ignite.spi.swapspace.file.FileSwapSpaceSpi"/>
+        </property>
+    </bean>
+</beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/8a9cc540/modules/yardstick/config/test-max-int-values-offheap.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/test-max-int-values-offheap.properties b/modules/yardstick/config/test-max-int-values-offheap.properties
new file mode 100644
index 0000000..5b4da20
--- /dev/null
+++ b/modules/yardstick/config/test-max-int-values-offheap.properties
@@ -0,0 +1,70 @@
+# 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.
+
+#
+# Contains benchmarks for SQL queries.
+#
+
+# JVM options.
+# JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false"
+
+# Uncomment to enable concurrent garbage collection (GC) if you encounter long GC pauses.
+JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false \
+  -Xloggc:./gc${now0}.log \
+  -XX:+PrintGCDetails \
+  -XX:-PrintGCTimeStamps \
+  -verbose:gc \
+  -Xmx8g \
+  -Xms8g \
+  -XX:+UseParNewGC \
+  -XX:+UseConcMarkSweepGC \
+  -XX:+UseTLAB \
+  -XX:NewSize=1g \
+  -XX:MaxNewSize=1g \
+  -XX:MaxTenuringThreshold=0 \
+  -XX:SurvivorRatio=1024 \
+  -XX:+UseCMSInitiatingOccupancyOnly \
+  -XX:CMSInitiatingOccupancyFraction=60 \
+"
+
+# List of default probes.
+# Add DStatProbe or VmStatProbe if your OS supports it (e.g. if running on Linux).
+# BENCHMARK_DEFAULT_PROBES=ThroughputLatencyProbe,PercentileProbe
+
+# Packages where the specified benchmark is searched by reflection mechanism.
+BENCHMARK_PACKAGES=org.yardstickframework,org.apache.ignite.yardstick
+
+# Probe point writer class name.
+# BENCHMARK_WRITER=
+
+# Comma-separated list of the hosts to run BenchmarkServers on. 2 nodes on local host are enabled by default.
+SERVER_HOSTS=fosters-215,fosters-216,fosters-217,fosters-226,fosters-219,fosters-221,fosters-222,fosters-223
+
+# Comma-separated list of the hosts to run BenchmarkDrivers on. 1 node on local host is enabled by default.
+DRIVER_HOSTS=fosters-218
+
+# Remote username.
+# REMOTE_USER=
+
+# Number of nodes, used to wait for the specified number of nodes to start.
+nodesNum=$((`echo ${SERVER_HOSTS} | tr ',' '\n' | wc -l` + `echo ${DRIVER_HOSTS} | tr ',' '\n' | wc -l`))
+
+# Run configuration.
+# Note that each benchmark is set to run for 300 seconds (5 mins) with warm-up set to 60 seconds (1 minute).
+CONFIGS="\
+-cfg ${SCRIPT_DIR}/../config/ignite-int-max-values-offheap-config.xml -nn ${nodesNum} -b 0 -w 0 -d 9999999 -t 1 -sm PRIMARY_SYNC -dn IntMaxValueEntriesTest -sn IgniteNode -ds int-max-values-offheap\
+"
+
+

http://git-wip-us.apache.org/repos/asf/ignite/blob/8a9cc540/modules/yardstick/config/test-max-int-values-onheap.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/test-max-int-values-onheap.properties b/modules/yardstick/config/test-max-int-values-onheap.properties
new file mode 100644
index 0000000..d29800a
--- /dev/null
+++ b/modules/yardstick/config/test-max-int-values-onheap.properties
@@ -0,0 +1,70 @@
+# 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.
+
+#
+# Contains benchmarks for SQL queries.
+#
+
+# JVM options.
+# JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false"
+
+# Uncomment to enable concurrent garbage collection (GC) if you encounter long GC pauses.
+JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false \
+  -Xloggc:./gc${now0}.log \
+  -XX:+PrintGCDetails \
+  -XX:-PrintGCTimeStamps \
+  -verbose:gc \
+  -Xmx92g \
+  -Xms32g \
+  -XX:+UseParNewGC \
+  -XX:+UseConcMarkSweepGC \
+  -XX:+UseTLAB \
+  -XX:NewSize=4g \
+  -XX:MaxNewSize=4g \
+  -XX:MaxTenuringThreshold=0 \
+  -XX:SurvivorRatio=1024 \
+  -XX:+UseCMSInitiatingOccupancyOnly \
+  -XX:CMSInitiatingOccupancyFraction=60 \
+"
+
+# List of default probes.
+# Add DStatProbe or VmStatProbe if your OS supports it (e.g. if running on Linux).
+# BENCHMARK_DEFAULT_PROBES=ThroughputLatencyProbe,PercentileProbe
+
+# Packages where the specified benchmark is searched by reflection mechanism.
+BENCHMARK_PACKAGES=org.yardstickframework,org.apache.ignite.yardstick
+
+# Probe point writer class name.
+# BENCHMARK_WRITER=
+
+# Comma-separated list of the hosts to run BenchmarkServers on. 2 nodes on local host are enabled by default.
+SERVER_HOSTS=fosters-215,fosters-216,fosters-217,fosters-226,fosters-219,fosters-221,fosters-222,fosters-223
+
+# Comma-separated list of the hosts to run BenchmarkDrivers on. 1 node on local host is enabled by default.
+DRIVER_HOSTS=fosters-218
+
+# Remote username.
+# REMOTE_USER=
+
+# Number of nodes, used to wait for the specified number of nodes to start.
+nodesNum=$((`echo ${SERVER_HOSTS} | tr ',' '\n' | wc -l` + `echo ${DRIVER_HOSTS} | tr ',' '\n' | wc -l`))
+
+# Run configuration.
+# Note that each benchmark is set to run for 300 seconds (5 mins) with warm-up set to 60 seconds (1 minute).
+CONFIGS="\
+-cfg ${SCRIPT_DIR}/../config/ignite-int-max-values-onheap-config.xml -nn ${nodesNum} -b 0 -w 0 -d 9999999 -t 1 -sm PRIMARY_SYNC -dn IntMaxValueEntriesTest -sn IgniteNode -ds int-max-values-onheap\
+"
+
+

http://git-wip-us.apache.org/repos/asf/ignite/blob/8a9cc540/modules/yardstick/config/test-max-int-values-swap.properties
----------------------------------------------------------------------
diff --git a/modules/yardstick/config/test-max-int-values-swap.properties b/modules/yardstick/config/test-max-int-values-swap.properties
new file mode 100644
index 0000000..203f004
--- /dev/null
+++ b/modules/yardstick/config/test-max-int-values-swap.properties
@@ -0,0 +1,69 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#      http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+#
+# Contains benchmarks for SQL queries.
+#
+
+# JVM options.
+# JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false"
+
+# Uncomment to enable concurrent garbage collection (GC) if you encounter long GC pauses.
+JVM_OPTS=${JVM_OPTS}" -DIGNITE_QUIET=false \
+  -Xloggc:./gc${now0}.log \
+  -XX:+PrintGCDetails \
+  -XX:-PrintGCTimeStamps \
+  -verbose:gc \
+  -Xmx8g \
+  -Xms8g \
+  -XX:+UseParNewGC \
+  -XX:+UseConcMarkSweepGC \
+  -XX:+UseTLAB \
+  -XX:NewSize=1g \
+  -XX:MaxNewSize=1g \
+  -XX:MaxTenuringThreshold=0 \
+  -XX:SurvivorRatio=1024 \
+  -XX:+UseCMSInitiatingOccupancyOnly \
+  -XX:CMSInitiatingOccupancyFraction=60 \
+"
+
+# List of default probes.
+# Add DStatProbe or VmStatProbe if your OS supports it (e.g. if running on Linux).
+# BENCHMARK_DEFAULT_PROBES=ThroughputLatencyProbe,PercentileProbe
+
+# Packages where the specified benchmark is searched by reflection mechanism.
+BENCHMARK_PACKAGES=org.yardstickframework,org.apache.ignite.yardstick
+
+# Probe point writer class name.
+# BENCHMARK_WRITER=
+
+# Comma-separated list of the hosts to run BenchmarkServers on. 2 nodes on local host are enabled by default.
+SERVER_HOSTS=fosters-215,fosters-216,fosters-217,fosters-226,fosters-219,fosters-221,fosters-222,fosters-223
+
+# Comma-separated list of the hosts to run BenchmarkDrivers on. 1 node on local host is enabled by default.
+DRIVER_HOSTS=fosters-218
+
+# Remote username.
+# REMOTE_USER=
+
+# Number of nodes, used to wait for the specified number of nodes to start.
+nodesNum=$((`echo ${SERVER_HOSTS} | tr ',' '\n' | wc -l` + `echo ${DRIVER_HOSTS} | tr ',' '\n' | wc -l`))
+
+# Run configuration.
+# Note that each benchmark is set to run for 300 seconds (5 mins) with warm-up set to 60 seconds (1 minute).
+CONFIGS="\
+-cfg ${SCRIPT_DIR}/../config/ignite-int-max-values-swap-config.xml -nn ${nodesNum} -b 0 -w 0 -d 9999999 -t 1 -sm PRIMARY_SYNC -dn IntMaxValueEntriesTest -sn IgniteNode -ds int-max-values-swap\
+"
+

http://git-wip-us.apache.org/repos/asf/ignite/blob/8a9cc540/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IntMaxValueEntriesTest.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IntMaxValueEntriesTest.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IntMaxValueEntriesTest.java
new file mode 100644
index 0000000..06ca3b7
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IntMaxValueEntriesTest.java
@@ -0,0 +1,135 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.yardstick.cache;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ThreadLocalRandom;
+import javax.cache.Cache;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteDataStreamer;
+import org.jsr166.LongAdder8;
+
+import static org.yardstickframework.BenchmarkUtils.println;
+
+/**
+ *
+ */
+public class IntMaxValueEntriesTest extends IgniteCacheAbstractBenchmark {
+    /** Threads. */
+    private static final int THREADS = 16;
+
+    /** Keys lo. */
+    private static final int KEYS_LO = -100_000;
+
+    /** Keys hi. */
+    private static final long KEYS_HI = Integer.MAX_VALUE;
+
+    /** Report delta. */
+    private static final int REPORT_DELTA = 1_000_000;
+
+    /** Cache name. */
+    private static final String CACHE_NAME = "int-max-value-cache";
+
+    /** {@inheritDoc} */
+    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
+        final IgniteCache<Integer, Object> cache = cache();
+
+        final IgniteDataStreamer<Integer, Object> stmr = ignite().dataStreamer(cache.getName());
+
+        final List<Thread> threads = new ArrayList<>(THREADS);
+
+        final LongAdder8 addedCnt = new LongAdder8();
+
+        int delta = (int)((KEYS_HI + Math.abs(KEYS_LO)) / THREADS);
+
+        System.out.println("Delta: " + delta);
+
+        for (int i = 0; i < THREADS; i++) {
+            final int lo = i == 0 ? KEYS_LO : delta * i + 1;
+
+            final int hi = i == THREADS - 1 ? (int)KEYS_HI : (int)((long)delta * (i + 1));
+
+            Thread t = new Thread(new Runnable() {
+                @Override public void run() {
+                    ThreadLocalRandom rnd = ThreadLocalRandom.current();
+
+                    byte val = (byte)rnd.nextInt();
+
+                    println("Start from " + lo + " to " + hi);
+
+                    for (int j = lo, k = 0; j < hi; j++, k++) {
+                        stmr.addData(j, val++);
+
+                        addedCnt.increment();
+
+                        if (k % REPORT_DELTA == 0)
+                            println(addedCnt.sum() + " entries");
+                    }
+
+                    println("Thread finished. " + addedCnt.sum() + " entries.");
+                }
+            });
+
+            threads.add(t);
+            t.start();
+        }
+
+        for (Thread thread : threads)
+            thread.join();
+
+        println("All threads finished. " + addedCnt.sum() + " entries.");
+
+        println("Streamer flush");
+
+        stmr.flush();
+
+        println("Streamer flushed");
+
+        println("Calculating cache size");
+        println("Cache size: " + cache.size());
+
+        println("Calculating long cache size");
+        println("Cache size long: " + cache.sizeLong());
+
+        Thread.sleep(10000);
+
+        println("Iterating started");
+
+        long cnt = 0;
+
+        for (Cache.Entry<Integer, Object> ignored : cache) {
+            cnt++;
+
+            if (cnt > 0 && cnt % REPORT_DELTA == 0)
+                println("Iterated via " + cnt + " entries");
+        }
+
+        println("Iterated via " + cnt + " entries");
+
+        cache.destroy();
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteCache cache() {
+        return ignite().cache(CACHE_NAME);
+    }
+}