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/21 12:55:50 UTC

[5/5] incubator-ignite git commit: ignite-916 Eviction policy should evict cache entries when memory size limit is reached

ignite-916 Eviction policy should evict cache entries when memory size limit is reached


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

Branch: refs/heads/ignite-916
Commit: 965a7fe5597405d6b7cdef27094b03edfb2d4597
Parents: 7e64372
Author: agura <ag...@gridgain.com>
Authored: Mon May 18 21:01:36 2015 +0300
Committer: agura <ag...@gridgain.com>
Committed: Thu May 21 13:22:09 2015 +0300

----------------------------------------------------------------------
 .../ignite/cache/eviction/EvictableEntry.java   |    7 +
 .../ignite/cache/eviction/EvictionPolicy.java   |    2 +
 .../cache/eviction/fifo/FifoEvictionPolicy.java |  145 ++-
 .../eviction/fifo/FifoEvictionPolicyMBean.java  |   22 +
 .../cache/eviction/lru/LruEvictionPolicy.java   |  151 ++-
 .../eviction/lru/LruEvictionPolicyMBean.java    |   38 +
 .../eviction/random/RandomEvictionPolicy.java   |   25 +-
 .../eviction/sorted/SortedEvictionPolicy.java   |  159 ++-
 .../sorted/SortedEvictionPolicyMBean.java       |   22 +
 .../cache/CacheEvictableEntryImpl.java          |   31 +
 .../GridCacheConcurrentTxMultiNodeTest.java     |    8 +-
 ...idCacheConfigurationConsistencySelfTest.java |   14 +-
 .../cache/GridCacheMemoryModeSelfTest.java      |   23 +-
 .../processors/cache/GridCacheOffHeapTest.java  |    5 +-
 .../cache/GridCacheReloadSelfTest.java          |    6 +-
 ...gniteCacheP2pUnmarshallingNearErrorTest.java |    6 +-
 .../cache/IgniteCachePeekModesAbstractTest.java |    5 +-
 ...GridCacheDhtEvictionNearReadersSelfTest.java |   11 +-
 .../dht/GridCacheDhtEvictionSelfTest.java       |   11 +-
 .../dht/IgniteCacheMultiTxLockSelfTest.java     |    6 +-
 .../GridCachePartitionedEvictionSelfTest.java   |   11 +-
 ...ePartitionedMultiThreadedPutGetSelfTest.java |    6 +-
 .../cache/eviction/EvictionAbstractTest.java    | 1057 ++++++++++++++++++
 .../GridCacheBatchEvictUnswapSelfTest.java      |    5 +-
 ...heConcurrentEvictionConsistencySelfTest.java |   82 +-
 .../GridCacheConcurrentEvictionsSelfTest.java   |   29 +-
 .../GridCacheDistributedEvictionsSelfTest.java  |    5 +-
 .../GridCacheEmptyEntriesAbstractSelfTest.java  |   11 +-
 .../eviction/GridCacheEvictionAbstractTest.java |  484 --------
 .../GridCacheEvictionTouchSelfTest.java         |   22 +-
 .../cache/eviction/GridCacheMockEntry.java      |    5 +
 .../fifo/FifoEvictionPolicySelfTest.java        |  262 +++++
 ...ridCacheFifoBatchEvictionPolicySelfTest.java |  384 -------
 .../GridCacheFifoEvictionPolicySelfTest.java    |  372 ------
 .../lru/GridCacheLruEvictionPolicySelfTest.java |  417 -------
 .../GridCacheLruNearEvictionPolicySelfTest.java |  136 ---
 ...heNearOnlyLruNearEvictionPolicySelfTest.java |  171 ---
 .../eviction/lru/LruEvictionPolicySelfTest.java |  353 ++++++
 .../lru/LruNearEvictionPolicySelfTest.java      |  140 +++
 .../LruNearOnlyNearEvictionPolicySelfTest.java  |  175 +++
 .../GridCacheRandomEvictionPolicySelfTest.java  |  258 -----
 .../random/RandomEvictionPolicySelfTest.java    |  357 ++++++
 ...dCacheSortedBatchEvictionPolicySelfTest.java |  385 -------
 ...acheSortedEvictionPolicyPerformanceTest.java |  135 ---
 .../GridCacheSortedEvictionPolicySelfTest.java  |  373 ------
 .../SortedEvictionPolicyPerformanceTest.java    |  134 +++
 .../sorted/SortedEvictionPolicySelfTest.java    |  266 +++++
 .../loadtests/GridCacheMultiNodeLoadTest.java   |    5 +-
 .../GridCachePartitionedAtomicLongLoadTest.java |    6 +-
 .../swap/GridSwapEvictAllBenchmark.java         |    6 +-
 .../IgniteCacheEvictionSelfTestSuite.java       |   14 +-
 .../cache/GridIndexingWithNoopSwapSelfTest.java |    6 +-
 .../IgniteCacheQueryMultiThreadedSelfTest.java  |   11 +-
 .../cache/ttl/CacheTtlAbstractSelfTest.java     |    6 +-
 .../org/apache/ignite/yardstick/IgniteNode.java |    7 +-
 55 files changed, 3467 insertions(+), 3326 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/965a7fe5/modules/core/src/main/java/org/apache/ignite/cache/eviction/EvictableEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/eviction/EvictableEntry.java b/modules/core/src/main/java/org/apache/ignite/cache/eviction/EvictableEntry.java
index d87109f..9f1889a 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/eviction/EvictableEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/eviction/EvictableEntry.java
@@ -46,6 +46,13 @@ public interface EvictableEntry<K, V> extends Cache.Entry<K, V> {
     public boolean isCached();
 
     /**
+     * Returns entry size in bytes.
+     *
+     * @return entry size in bytes.
+     */
+    public int size();
+
+    /**
      * Gets metadata added by eviction policy.
      *
      * @return Metadata value or {@code null}.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/965a7fe5/modules/core/src/main/java/org/apache/ignite/cache/eviction/EvictionPolicy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/eviction/EvictionPolicy.java b/modules/core/src/main/java/org/apache/ignite/cache/eviction/EvictionPolicy.java
index f409e9b..07c269d 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/eviction/EvictionPolicy.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/eviction/EvictionPolicy.java
@@ -20,6 +20,7 @@ package org.apache.ignite.cache.eviction;
 import org.apache.ignite.cache.eviction.fifo.*;
 import org.apache.ignite.cache.eviction.lru.*;
 import org.apache.ignite.cache.eviction.random.*;
+import org.apache.ignite.cache.eviction.sorted.*;
 
 /**
  * Pluggable cache eviction policy. Usually, implementations will internally order
@@ -32,6 +33,7 @@ import org.apache.ignite.cache.eviction.random.*;
  * <li>{@link LruEvictionPolicy}</li>
  * <li>{@link RandomEvictionPolicy}</li>
  * <li>{@link FifoEvictionPolicy}</li>
+ * <li>{@link SortedEvictionPolicy}</li>
  * </ul>
  * <p>
  * The eviction policy thread-safety is ensured by Ignition. Implementations of this interface should

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/965a7fe5/modules/core/src/main/java/org/apache/ignite/cache/eviction/fifo/FifoEvictionPolicy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/eviction/fifo/FifoEvictionPolicy.java b/modules/core/src/main/java/org/apache/ignite/cache/eviction/fifo/FifoEvictionPolicy.java
index bf8cf0d..61426ab 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/eviction/fifo/FifoEvictionPolicy.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/eviction/fifo/FifoEvictionPolicy.java
@@ -18,18 +18,28 @@
 package org.apache.ignite.cache.eviction.fifo;
 
 import org.apache.ignite.cache.eviction.*;
-import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
+
 import org.jsr166.*;
 import org.jsr166.ConcurrentLinkedDeque8.*;
 
 import java.io.*;
 import java.util.*;
 
+import static org.apache.ignite.configuration.CacheConfiguration.*;
+
 /**
  * Eviction policy based on {@code First In First Out (FIFO)} algorithm and supports batch eviction.
  * <p>
- * The eviction starts when the cache size becomes {@code batchSize} elements greater than the maximum size.
+ * The eviction starts in the following cases:
+ * <ul>
+ *     <li>The cache size becomes {@code batchSize} elements greater than the maximum size.</li>
+ *     <li>
+ *         The size of cache entries in bytes becomes greater than the maximum memory size.
+ *         The size of cache entry calculates as sum of key size and value size.
+ *     </li>
+ * </ul>
+ * <b>Note:</b>Batch eviction is enabled only if maximum memory limit isn't set ({@code maxMemSize == 0}).
  * {@code batchSize} elements will be evicted in this case. The default {@code batchSize} value is {@code 1}.
  * <p>
  * This implementation is very efficient since it does not create any additional
@@ -41,48 +51,22 @@ public class FifoEvictionPolicy<K, V> implements EvictionPolicy<K, V>, FifoEvict
     private static final long serialVersionUID = 0L;
 
     /** Maximum size. */
-    private volatile int max = CacheConfiguration.DFLT_CACHE_SIZE;
+    private volatile int max = DFLT_CACHE_SIZE;
 
     /** Batch size. */
     private volatile int batchSize = 1;
 
+    /** Max memory size. */
+    private volatile long maxMemSize;
+
+    /** Memory size. */
+    private final LongAdder8 memSize = new LongAdder8();
+
     /** FIFO queue. */
     private final ConcurrentLinkedDeque8<EvictableEntry<K, V>> queue =
         new ConcurrentLinkedDeque8<>();
 
     /**
-     * Constructs FIFO eviction policy with all defaults.
-     */
-    public FifoEvictionPolicy() {
-        // No-op.
-    }
-
-    /**
-     * Constructs FIFO eviction policy with maximum size. Empty entries are allowed.
-     *
-     * @param max Maximum allowed size of cache before entry will start getting evicted.
-     */
-    public FifoEvictionPolicy(int max) {
-        A.ensure(max > 0, "max > 0");
-
-        this.max = max;
-    }
-
-    /**
-     * Constructs FIFO eviction policy with maximum size and given batch size. Empty entries are allowed.
-     *
-     * @param max Maximum allowed size of cache before entry will start getting evicted.
-     * @param batchSize Batch size.
-     */
-    public FifoEvictionPolicy(int max, int batchSize) {
-        A.ensure(max > 0, "max > 0");
-        A.ensure(batchSize > 0, "batchSize > 0");
-
-        this.max = max;
-        this.batchSize = batchSize;
-    }
-
-    /**
      * Gets maximum allowed size of cache before entry will start getting evicted.
      *
      * @return Maximum allowed size of cache before entry will start getting evicted.
@@ -97,7 +81,7 @@ public class FifoEvictionPolicy<K, V> implements EvictionPolicy<K, V>, FifoEvict
      * @param max Maximum allowed size of cache before entry will start getting evicted.
      */
     @Override public void setMaxSize(int max) {
-        A.ensure(max > 0, "max > 0");
+        A.ensure(max >= 0, "max >= 0");
 
         this.max = max;
     }
@@ -119,6 +103,23 @@ public class FifoEvictionPolicy<K, V> implements EvictionPolicy<K, V>, FifoEvict
         return queue.size();
     }
 
+    /** {@inheritDoc} */
+    @Override public long getMaxMemSize() {
+        return maxMemSize;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void setMaxMemSize(long maxMemSize) {
+        A.ensure(maxMemSize >= 0, "maxMemSize >= 0");
+
+        this.maxMemSize = maxMemSize;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getCurrentMemSize() {
+        return memSize.longValue();
+    }
+
     /**
      * Gets read-only view on internal {@code FIFO} queue in proper order.
      *
@@ -141,8 +142,11 @@ public class FifoEvictionPolicy<K, V> implements EvictionPolicy<K, V>, FifoEvict
         else {
             Node<EvictableEntry<K, V>> node = entry.removeMeta();
 
-            if (node != null)
+            if (node != null) {
                 queue.unlinkx(node);
+
+                memSize.add(-entry.size());
+            }
         }
     }
 
@@ -173,11 +177,18 @@ public class FifoEvictionPolicy<K, V> implements EvictionPolicy<K, V>, FifoEvict
                         return false;
                     }
 
+                    memSize.add(entry.size());
+
                     return true;
                 }
                 // If node was unlinked by concurrent shrink() call, we must repeat the whole cycle.
                 else if (!entry.removeMeta(node))
                     return false;
+                else {
+                    memSize.add(-entry.size());
+
+                    return true;
+                }
             }
         }
 
@@ -189,38 +200,74 @@ public class FifoEvictionPolicy<K, V> implements EvictionPolicy<K, V>, FifoEvict
      * Shrinks FIFO queue to maximum allowed size.
      */
     private void shrink() {
+        long maxMem = this.maxMemSize;
+
+        if (maxMem > 0) {
+            long startMemSize = memSize.longValue();
+
+            if (startMemSize >= maxMem)
+                for (long i = maxMem; i < startMemSize && memSize.longValue() > maxMem;) {
+                    int size = shrink0();
+
+                    if (size == -1)
+                        break;
+
+                    i += size;
+                }
+        }
+
         int max = this.max;
 
-        int batchSize = this.batchSize;
+        if (max > 0) {
+            int startSize = queue.sizex();
+
+            // Shrink only if queue is full.
+            if (startSize >= max + (maxMem > 0 ? 1 : this.batchSize))
+                for (int i = max; i < startSize && queue.sizex() > max; i++)
+                    if (shrink0() == -1)
+                        break;
+        }
+    }
+
+    /**
+     * Tries to remove one item from queue.
+     *
+     * @return number of bytes that was free. {@code -1} if queue is empty.
+     */
+    private int shrink0() {
+        EvictableEntry<K, V> entry = queue.poll();
 
-        int startSize = queue.sizex();
+        if (entry == null)
+            return -1;
 
-        // Shrink only if queue is full.
-        if (startSize >= max + batchSize) {
-            for (int i = max; i < startSize && queue.sizex() > max; i++) {
-                EvictableEntry<K, V> entry = queue.poll();
+        int size = 0;
 
-                if (entry == null)
-                    break;
+        Node<EvictableEntry<K, V>> meta = entry.removeMeta();
 
-                Node<EvictableEntry<K, V>> meta = entry.removeMeta();
+        if (meta != null) {
+            size = entry.size();
 
-                if (meta != null && !entry.evict())
-                    touch(entry);
-            }
+            memSize.add(-size);
+
+            if (!entry.evict())
+                touch(entry);
         }
+
+        return size;
     }
 
     /** {@inheritDoc} */
     @Override public void writeExternal(ObjectOutput out) throws IOException {
         out.writeInt(max);
         out.writeInt(batchSize);
+        out.writeLong(maxMemSize);
     }
 
     /** {@inheritDoc} */
     @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
         max = in.readInt();
         batchSize = in.readInt();
+        maxMemSize = in.readLong();
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/965a7fe5/modules/core/src/main/java/org/apache/ignite/cache/eviction/fifo/FifoEvictionPolicyMBean.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/eviction/fifo/FifoEvictionPolicyMBean.java b/modules/core/src/main/java/org/apache/ignite/cache/eviction/fifo/FifoEvictionPolicyMBean.java
index 63a413e..c9a09ec 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/eviction/fifo/FifoEvictionPolicyMBean.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/eviction/fifo/FifoEvictionPolicyMBean.java
@@ -63,4 +63,26 @@ public interface FifoEvictionPolicyMBean {
      */
     @MXBeanDescription("Current FIFO queue size.")
     public int getCurrentSize();
+
+    /**
+     * Gets maximum allowed cache size in bytes.
+     *
+     * @return maximum allowed cache size in bytes.
+     */
+    @MXBeanDescription("Maximum allowed cache size in bytes.")
+    public long getMaxMemSize();
+
+    /**
+     * Sets maximum allowed cache size in bytes.
+     */
+    @MXBeanDescription("Set maximum allowed cache size in bytes.")
+    public void setMaxMemSize(long maxMemSize);
+
+    /**
+     * Gets current queue size in bytes.
+     *
+     * @return current queue size in bytes.
+     */
+    @MXBeanDescription("Current FIFO queue size in bytes.")
+    public long getCurrentMemSize();
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/965a7fe5/modules/core/src/main/java/org/apache/ignite/cache/eviction/lru/LruEvictionPolicy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/eviction/lru/LruEvictionPolicy.java b/modules/core/src/main/java/org/apache/ignite/cache/eviction/lru/LruEvictionPolicy.java
index 309d577..9acb79f 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/eviction/lru/LruEvictionPolicy.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/eviction/lru/LruEvictionPolicy.java
@@ -18,48 +18,52 @@
 package org.apache.ignite.cache.eviction.lru;
 
 import org.apache.ignite.cache.eviction.*;
-import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
+
 import org.jsr166.*;
 import org.jsr166.ConcurrentLinkedDeque8.*;
 
 import java.io.*;
 import java.util.*;
 
+import static org.apache.ignite.configuration.CacheConfiguration.*;
+
 /**
- * Eviction policy based on {@code Least Recently Used (LRU)} algorithm. This
- * implementation is very efficient since it is lock-free and does not
- * create any additional table-like data structures. The {@code LRU} ordering
- * information is maintained by attaching ordering metadata to cache entries.
+ * Eviction policy based on {@code Least Recently Used (LRU)} algorithm and supports batch eviction.
+ * <p>
+ * The eviction starts in the following cases:
+ * <ul>
+ *     <li>The cache size becomes {@code batchSize} elements greater than the maximum size.</li>
+ *     <li>
+ *         The size of cache entries in bytes becomes greater than the maximum memory size.
+ *         The size of cache entry calculates as sum of key size and value size.
+ *     </li>
+ * </ul>
+ * <b>Note:</b>Batch eviction is enabled only if maximum memory limit isn't set ({@code maxMemSize == 0}).
+ * {@code batchSize} elements will be evicted in this case. The default {@code batchSize} value is {@code 1}.
+
+ * This implementation is very efficient since it is lock-free and does not create any additional table-like
+ * data structures. The {@code LRU} ordering information is maintained by attaching ordering metadata to cache entries.
  */
 public class LruEvictionPolicy<K, V> implements EvictionPolicy<K, V>, LruEvictionPolicyMBean, Externalizable {
     /** */
     private static final long serialVersionUID = 0L;
 
     /** Maximum size. */
-    private volatile int max = CacheConfiguration.DFLT_CACHE_SIZE;
+    private volatile int max = DFLT_CACHE_SIZE;
 
-    /** Queue. */
-    private final ConcurrentLinkedDeque8<EvictableEntry<K, V>> queue =
-        new ConcurrentLinkedDeque8<>();
+    /** Batch size. */
+    private volatile int batchSize = 1;
 
-    /**
-     * Constructs LRU eviction policy with all defaults.
-     */
-    public LruEvictionPolicy() {
-        // No-op.
-    }
+    /** Max memory size. */
+    private volatile long maxMemSize;
 
-    /**
-     * Constructs LRU eviction policy with maximum size.
-     *
-     * @param max Maximum allowed size of cache before entry will start getting evicted.
-     */
-    public LruEvictionPolicy(int max) {
-        A.ensure(max > 0, "max > 0");
+    /** Memory size. */
+    private final LongAdder8 memSize = new LongAdder8();
 
-        this.max = max;
-    }
+    /** Queue. */
+    private final ConcurrentLinkedDeque8<EvictableEntry<K, V>> queue =
+        new ConcurrentLinkedDeque8<>();
 
     /**
      * Gets maximum allowed size of cache before entry will start getting evicted.
@@ -76,16 +80,45 @@ public class LruEvictionPolicy<K, V> implements EvictionPolicy<K, V>, LruEvictio
      * @param max Maximum allowed size of cache before entry will start getting evicted.
      */
     @Override public void setMaxSize(int max) {
-        A.ensure(max > 0, "max > 0");
+        A.ensure(max >= 0, "max >= 0");
 
         this.max = max;
     }
 
     /** {@inheritDoc} */
+    @Override public int getBatchSize() {
+        return batchSize;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void setBatchSize(int batchSize) {
+        A.ensure(batchSize > 0, "batchSize > 0");
+
+        this.batchSize = batchSize;
+    }
+
+    /** {@inheritDoc} */
     @Override public int getCurrentSize() {
         return queue.size();
     }
 
+    /** {@inheritDoc} */
+    @Override public long getMaxMemSize() {
+        return maxMemSize;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void setMaxMemSize(long maxMemSize) {
+        A.ensure(maxMemSize >= 0, "maxMemSize >= 0");
+
+        this.maxMemSize = maxMemSize;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getCurrentMemSize() {
+        return memSize.longValue();
+    }
+
     /**
      * Gets read-only view on internal {@code FIFO} queue in proper order.
      *
@@ -107,8 +140,11 @@ public class LruEvictionPolicy<K, V> implements EvictionPolicy<K, V>, LruEvictio
         else {
             Node<EvictableEntry<K, V>> node = entry.removeMeta();
 
-            if (node != null)
+            if (node != null) {
                 queue.unlinkx(node);
+
+                memSize.add(-entry.size());
+            }
         }
     }
 
@@ -139,11 +175,18 @@ public class LruEvictionPolicy<K, V> implements EvictionPolicy<K, V>, LruEvictio
                         return false;
                     }
 
+                    memSize.add(entry.size());
+
                     return true;
                 }
                 // If node was unlinked by concurrent shrink() call, we must repeat the whole cycle.
                 else if (!entry.removeMeta(node))
                     return false;
+                else {
+                    memSize.add(-entry.size());
+
+                    return true;
+                }
             }
         }
         else if (queue.unlinkx(node)) {
@@ -163,31 +206,73 @@ public class LruEvictionPolicy<K, V> implements EvictionPolicy<K, V>, LruEvictio
      * Shrinks queue to maximum allowed size.
      */
     private void shrink() {
+        long maxMem = this.maxMemSize;
+
+        if (maxMem > 0) {
+            long startMemSize = memSize.longValue();
+
+            if (startMemSize >= maxMem)
+                for (long i = maxMem; i < startMemSize && memSize.longValue() > maxMem;) {
+                    int size = shrink0();
+
+                    if (size == -1)
+                        break;
+
+                    i += size;
+                }
+        }
+
         int max = this.max;
 
-        int startSize = queue.sizex();
+        if (max > 0) {
+            int startSize = queue.sizex();
+
+            if (startSize >= max + (maxMem > 0 ? 1 : this.batchSize))
+                for (int i = max; i < startSize && queue.sizex() > max; i++)
+                    if (shrink0() == -1)
+                        break;
+        }
+    }
+
+    /**
+     * Tries to remove one item from queue.
+     *
+     * @return number of bytes that was free. {@code -1} if queue is empty.
+     */
+    private int shrink0() {
+        EvictableEntry<K, V> entry = queue.poll();
 
-        for (int i = 0; i < startSize && queue.sizex() > max; i++) {
-            EvictableEntry<K, V> entry = queue.poll();
+        if (entry == null)
+            return -1;
 
-            if (entry == null)
-                break;
+        int size = 0;
 
-            Node<EvictableEntry<K, V>> meta = entry.removeMeta();
+        Node<EvictableEntry<K, V>> meta = entry.removeMeta();
 
-            if (meta != null && !entry.evict())
+        if (meta != null) {
+            size = entry.size();
+
+            memSize.add(-size);
+
+            if (!entry.evict())
                 touch(entry);
         }
+
+        return size;
     }
 
     /** {@inheritDoc} */
     @Override public void writeExternal(ObjectOutput out) throws IOException {
         out.writeInt(max);
+        out.writeInt(batchSize);
+        out.writeLong(maxMemSize);
     }
 
     /** {@inheritDoc} */
     @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
         max = in.readInt();
+        batchSize = in.readInt();
+        maxMemSize = in.readLong();
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/965a7fe5/modules/core/src/main/java/org/apache/ignite/cache/eviction/lru/LruEvictionPolicyMBean.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/eviction/lru/LruEvictionPolicyMBean.java b/modules/core/src/main/java/org/apache/ignite/cache/eviction/lru/LruEvictionPolicyMBean.java
index c243374..69347ee 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/eviction/lru/LruEvictionPolicyMBean.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/eviction/lru/LruEvictionPolicyMBean.java
@@ -41,10 +41,48 @@ public interface LruEvictionPolicyMBean {
     public void setMaxSize(int max);
 
     /**
+     * Gets batch size.
+     *
+     * @return batch size.
+     */
+    @MXBeanDescription("Batch size.")
+    public int getBatchSize();
+
+    /**
+     * Sets batch size.
+     *
+     * @param batchSize Batch size.
+     */
+    @MXBeanDescription("Set batch size.")
+    public void setBatchSize(int batchSize);
+
+    /**
      * Gets current queue size.
      *
      * @return Current queue size.
      */
     @MXBeanDescription("Current queue size.")
     public int getCurrentSize();
+
+    /**
+     * Gets maximum allowed cache size in bytes.
+     *
+     * @return maximum allowed cache size in bytes.
+     */
+    @MXBeanDescription("Maximum allowed cache size in bytes.")
+    public long getMaxMemSize();
+
+    /**
+     * Sets maximum allowed cache size in bytes.
+     */
+    @MXBeanDescription("Set maximum allowed cache size in bytes.")
+    public void setMaxMemSize(long maxMemSize);
+
+    /**
+     * Gets current queue size in bytes.
+     *
+     * @return current queue size in bytes.
+     */
+    @MXBeanDescription("Current queue size in  bytes.")
+    public long getCurrentMemSize();
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/965a7fe5/modules/core/src/main/java/org/apache/ignite/cache/eviction/random/RandomEvictionPolicy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/eviction/random/RandomEvictionPolicy.java b/modules/core/src/main/java/org/apache/ignite/cache/eviction/random/RandomEvictionPolicy.java
index c88b31d..616bb23 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/eviction/random/RandomEvictionPolicy.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/eviction/random/RandomEvictionPolicy.java
@@ -19,19 +19,20 @@ package org.apache.ignite.cache.eviction.random;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cache.eviction.*;
-import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 
 import javax.cache.*;
 import java.io.*;
 
+import static org.apache.ignite.configuration.CacheConfiguration.*;
+
 /**
  * Cache eviction policy which will select random cache entry for eviction if cache
  * size exceeds the {@link #getMaxSize()} parameter. This implementation is
  * extremely light weight, lock-free, and does not create any data structures to maintain
  * any order for eviction.
  * <p>
- * Random eviction will provide the best performance over any key set in which every
+ * Random eviction will provide the best performance over any key queue in which every
  * key has the same probability of being accessed.
  */
 public class RandomEvictionPolicy<K, V> implements EvictionPolicy<K, V>, RandomEvictionPolicyMBean, Externalizable {
@@ -39,25 +40,7 @@ public class RandomEvictionPolicy<K, V> implements EvictionPolicy<K, V>, RandomE
     private static final long serialVersionUID = 0L;
 
     /** Maximum size. */
-    private volatile int max = CacheConfiguration.DFLT_CACHE_SIZE;
-
-    /**
-     * Constructs random eviction policy with all defaults.
-     */
-    public RandomEvictionPolicy() {
-        // No-op.
-    }
-
-    /**
-     * Constructs random eviction policy with maximum size.
-     *
-     * @param max Maximum allowed size of cache before entry will start getting evicted.
-     */
-    public RandomEvictionPolicy(int max) {
-        A.ensure(max > 0, "max > 0");
-
-        this.max = max;
-    }
+    private volatile int max = DFLT_CACHE_SIZE;
 
     /**
      * Gets maximum allowed size of cache before entry will start getting evicted.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/965a7fe5/modules/core/src/main/java/org/apache/ignite/cache/eviction/sorted/SortedEvictionPolicy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/eviction/sorted/SortedEvictionPolicy.java b/modules/core/src/main/java/org/apache/ignite/cache/eviction/sorted/SortedEvictionPolicy.java
index 7965c97..7e3c4a8 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/eviction/sorted/SortedEvictionPolicy.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/eviction/sorted/SortedEvictionPolicy.java
@@ -34,7 +34,15 @@ import static org.apache.ignite.configuration.CacheConfiguration.*;
 /**
  * Cache eviction policy which will select the minimum cache entry for eviction.
  * <p>
- * The eviction starts when the cache size becomes {@code batchSize} elements greater than the maximum size.
+ * The eviction starts in the following cases:
+ * <ul>
+ *     <li>The cache size becomes {@code batchSize} elements greater than the maximum size.</li>
+ *     <li>
+ *         The size of cache entries in bytes becomes greater than the maximum memory size.
+ *         The size of cache entry calculates as sum of key size and value size.
+ *     </li>
+ * </ul>
+ * <b>Note:</b>Batch eviction is enabled only if maximum memory limit isn't set ({@code maxMemSize == 0}).
  * {@code batchSize} elements will be evicted in this case. The default {@code batchSize} value is {@code 1}.
  * <p>
  * Entries comparison based on {@link Comparator} instance if provided.
@@ -48,59 +56,39 @@ public class SortedEvictionPolicy<K, V> implements EvictionPolicy<K, V>, SortedE
     private static final long serialVersionUID = 0L;
 
     /** Maximum size. */
-    private volatile int max;
+    private volatile int max = DFLT_CACHE_SIZE;
 
     /** Batch size. */
     private volatile int batchSize = 1;
 
+    /** Max memory size. */
+    private volatile long maxMemSize;
+
+    /** Memory size. */
+    private final LongAdder8 memSize = new LongAdder8();
+
     /** Comparator. */
     private Comparator<Holder<K, V>> comp;
 
     /** Order. */
     private final AtomicLong orderCnt = new AtomicLong();
 
-    /** Backed sorted set. */
+    /** Backed sorted queue. */
     private final GridConcurrentSkipListSetEx<K, V> set;
 
     /**
      * Constructs sorted eviction policy with all defaults.
      */
     public SortedEvictionPolicy() {
-        this(DFLT_CACHE_SIZE, null);
-    }
-
-    /**
-     * Constructs sorted eviction policy with maximum size.
-     *
-     * @param max Maximum allowed size of cache before entry will start getting evicted.
-     */
-    public SortedEvictionPolicy(int max) {
-        this(max, null);
+        this(null);
     }
 
     /**
      * Constructs sorted eviction policy with given maximum size and given entry comparator.
      *
-     * @param max Maximum allowed size of cache before entry will start getting evicted.
-     * @param comp Entries comparator.
-     */
-    public SortedEvictionPolicy(int max, @Nullable Comparator<EvictableEntry<K, V>> comp) {
-        this(max, 1, comp);
-    }
-
-    /**
-     * Constructs sorted eviction policy with given maximum size, eviction batch size and entries comparator.
-     *
-     * @param max Maximum allowed size of cache before entry will start getting evicted.
-     * @param batchSize Batch size.
      * @param comp Entries comparator.
      */
-    public SortedEvictionPolicy(int max, int batchSize, @Nullable Comparator<EvictableEntry<K, V>> comp) {
-        A.ensure(max > 0, "max > 0");
-        A.ensure(batchSize > 0, "batchSize > 0");
-
-        this.max = max;
-        this.batchSize = batchSize;
+    public SortedEvictionPolicy(@Nullable Comparator<EvictableEntry<K, V>> comp) {
         this.comp = comp == null ? new DefaultHolderComparator<K, V>() : new HolderComparator<>(comp);
         this.set = new GridConcurrentSkipListSetEx<>(this.comp);
     }
@@ -120,7 +108,7 @@ public class SortedEvictionPolicy<K, V> implements EvictionPolicy<K, V>, SortedE
      * @param max Maximum allowed size of cache before entry will start getting evicted.
      */
     @Override public void setMaxSize(int max) {
-        A.ensure(max > 0, "max > 0");
+        A.ensure(max >= 0, "max >= 0");
 
         this.max = max;
     }
@@ -142,12 +130,29 @@ public class SortedEvictionPolicy<K, V> implements EvictionPolicy<K, V>, SortedE
         return set.sizex();
     }
 
+    /** {@inheritDoc} */
+    @Override public long getMaxMemSize() {
+        return maxMemSize;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void setMaxMemSize(long maxMemSize) {
+        A.ensure(maxMemSize >= 0, "maxMemSize >= 0");
+
+        this.maxMemSize = maxMemSize;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getCurrentMemSize() {
+        return memSize.longValue();
+    }
+
     /**
-     * Gets read-only view of backed set in proper order.
+     * Gets read-only view of backed queue in proper order.
      *
-     * @return Read-only view of backed set.
+     * @return Read-only view of backed queue.
      */
-    public Collection<EvictableEntry<K, V>> set() {
+    public Collection<EvictableEntry<K, V>> queue() {
         Set<EvictableEntry<K, V>> cp = new LinkedHashSet<>();
 
         for (Holder<K, V> holder : set)
@@ -168,19 +173,22 @@ public class SortedEvictionPolicy<K, V> implements EvictionPolicy<K, V>, SortedE
         else {
             Holder<K, V> holder = entry.removeMeta();
 
-            if (holder != null)
+            if (holder != null) {
                 removeHolder(holder);
+
+                memSize.add(-entry.size());
+            }
         }
     }
 
     /**
      * @param entry Entry to touch.
-     * @return {@code True} if backed set has been changed by this call.
+     * @return {@code True} if backed queue has been changed by this call.
      */
     private boolean touch(EvictableEntry<K, V> entry) {
         Holder<K, V> holder = entry.meta();
 
-        // Entry has not been add yet to backed set..
+        // Entry has not been add yet to backed queue..
         if (holder == null) {
             while (true) {
                 holder = new Holder<>(entry, orderCnt.incrementAndGet());
@@ -188,7 +196,7 @@ public class SortedEvictionPolicy<K, V> implements EvictionPolicy<K, V>, SortedE
                 set.add(holder);
 
                 if (entry.putMetaIfAbsent(holder) != null) {
-                    // Was concurrently added, need to remove it from set.
+                    // Was concurrently added, need to remove it from queue.
                     removeHolder(holder);
 
                     // Set has not been changed.
@@ -196,17 +204,24 @@ public class SortedEvictionPolicy<K, V> implements EvictionPolicy<K, V>, SortedE
                 }
                 else if (holder.order > 0) {
                     if (!entry.isCached()) {
-                        // Was concurrently evicted, need to remove it from set.
+                        // Was concurrently evicted, need to remove it from queue.
                         removeHolder(holder);
 
                         return false;
                     }
 
+                    memSize.add(entry.size());
+
                     return true;
                 }
                 // If holder was removed by concurrent shrink() call, we must repeat the whole cycle.
                 else if (!entry.removeMeta(holder))
                     return false;
+                else {
+                    memSize.add(-entry.size());
+
+                    return true;
+                }
             }
         }
 
@@ -215,34 +230,71 @@ public class SortedEvictionPolicy<K, V> implements EvictionPolicy<K, V>, SortedE
     }
 
     /**
-     * Shrinks backed set to maximum allowed size.
+     * Shrinks backed queue to maximum allowed size.
      */
     private void shrink() {
-        int max = this.max;
+        long maxMem = this.maxMemSize;
 
-        int batchSize = this.batchSize;
+        if (maxMem > 0) {
+            long startMemSize = memSize.longValue();
 
-        int startSize = set.sizex();
+            if (startMemSize >= maxMem)
+                for (long i = maxMem; i < startMemSize && memSize.longValue() > maxMem;) {
+                    int size = shrink0();
 
-        if (startSize >= max + batchSize) {
-            for (int i = max; i < startSize && set.sizex() > max; i++) {
-                Holder<K, V> h = set.pollFirst();
+                    if (size == -1)
+                        break;
 
-                if (h == null)
-                    break;
+                    i += size;
+                }
+        }
 
-                EvictableEntry<K, V> entry = h.entry;
+        int max = this.max;
+
+        if (max > 0) {
+            int startSize = set.sizex();
 
-                if (h.order > 0 && entry.removeMeta(h) && !entry.evict())
-                    touch(entry);
+            if (startSize >= max + (maxMem > 0 ? 1 : this.batchSize)) {
+                for (int i = max; i < startSize && set.sizex() > max; i++) {
+                    if (shrink0() == -1)
+                        break;
+                }
             }
         }
     }
 
+    /**
+     * Tries to remove one item from queue.
+     *
+     * @return number of bytes that was free. {@code -1} if queue is empty.
+     */
+    private int shrink0() {
+        Holder<K, V> h = set.pollFirst();
+
+        if (h == null)
+            return -1;
+
+        int size = 0;
+
+        EvictableEntry<K, V> entry = h.entry;
+
+        if (h.order > 0 && entry.removeMeta(h)) {
+            size = entry.size();
+
+            memSize.add(-size);
+
+            if (!entry.evict())
+                touch(entry);
+        }
+
+        return size;
+    }
+
     /** {@inheritDoc} */
     @Override public void writeExternal(ObjectOutput out) throws IOException {
         out.writeInt(max);
         out.writeInt(batchSize);
+        out.writeLong(maxMemSize);
         out.writeObject(comp);
     }
 
@@ -251,11 +303,12 @@ public class SortedEvictionPolicy<K, V> implements EvictionPolicy<K, V>, SortedE
     @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
         max = in.readInt();
         batchSize = in.readInt();
+        maxMemSize = in.readLong();
         comp = (Comparator<Holder<K, V>>)in.readObject();
     }
 
     /**
-     * Removes holder from backed set and marks holder as removed.
+     * Removes holder from backed queue and marks holder as removed.
      *
      * @param holder Holder.
      */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/965a7fe5/modules/core/src/main/java/org/apache/ignite/cache/eviction/sorted/SortedEvictionPolicyMBean.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/eviction/sorted/SortedEvictionPolicyMBean.java b/modules/core/src/main/java/org/apache/ignite/cache/eviction/sorted/SortedEvictionPolicyMBean.java
index bc696ff..aada1ce 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/eviction/sorted/SortedEvictionPolicyMBean.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/eviction/sorted/SortedEvictionPolicyMBean.java
@@ -63,4 +63,26 @@ public interface SortedEvictionPolicyMBean {
      */
     @MXBeanDescription("Current sorted key set size.")
     public int getCurrentSize();
+
+    /**
+     * Gets maximum allowed cache size in bytes.
+     *
+     * @return maximum allowed cache size in bytes.
+     */
+    @MXBeanDescription("Maximum allowed cache size in bytes.")
+    public long getMaxMemSize();
+
+    /**
+     * Sets maximum allowed cache size in bytes.
+     */
+    @MXBeanDescription("Set maximum allowed cache size in bytes.")
+    public void setMaxMemSize(long maxMemSize);
+
+    /**
+     * Gets current sorted entries queue size in bytes.
+     *
+     * @return current sorted entries queue size in bytes.
+     */
+    @MXBeanDescription("Current sorted entries set size in bytes.")
+    public long getCurrentMemSize();
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/965a7fe5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEvictableEntryImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEvictableEntryImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEvictableEntryImpl.java
index 5d6062e..7a3fbee 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEvictableEntryImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEvictableEntryImpl.java
@@ -23,6 +23,7 @@ import org.apache.ignite.internal.processors.cache.transactions.*;
 import org.apache.ignite.internal.util.lang.*;
 import org.apache.ignite.internal.util.tostring.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
+
 import org.jetbrains.annotations.*;
 
 import java.util.*;
@@ -91,6 +92,36 @@ public class CacheEvictableEntryImpl<K, V> implements EvictableEntry<K, V> {
     }
 
     /** {@inheritDoc} */
+    public int size() {
+        try {
+            GridCacheContext<Object, Object> cctx = cached.context();
+
+            KeyCacheObject key = cached.key();
+
+            byte[] keyBytes = key.valueBytes(cctx.cacheObjectContext());
+
+            byte[] valBytes = null;
+
+            if (cctx.useOffheapEntry())
+                valBytes = cctx.offheap().get(cctx.swap().spaceName(), cached.partition(), key, keyBytes);
+            else {
+                CacheObject cacheObj = cached.valueBytes();
+
+                if (cacheObj != null)
+                    valBytes = cacheObj.valueBytes(cctx.cacheObjectContext());
+            }
+
+            return valBytes == null ? keyBytes.length : keyBytes.length + valBytes.length;
+        }
+        catch (GridCacheEntryRemovedException e) {
+            return 0;
+        }
+        catch (IgniteCheckedException e) {
+            throw new IgniteException(e);
+        }
+    }
+
+    /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public V getValue() {
         try {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/965a7fe5/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentTxMultiNodeTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentTxMultiNodeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentTxMultiNodeTest.java
index 8a1ae78..bba4ad9 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentTxMultiNodeTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentTxMultiNodeTest.java
@@ -26,7 +26,6 @@ import org.apache.ignite.cluster.*;
 import org.apache.ignite.compute.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
-import org.apache.ignite.internal.processors.cache.query.*;
 import org.apache.ignite.internal.processors.cache.distributed.dht.*;
 import org.apache.ignite.internal.processors.cache.distributed.near.*;
 import org.apache.ignite.internal.util.*;
@@ -38,6 +37,7 @@ import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
 import org.apache.ignite.testframework.junits.common.*;
 import org.apache.ignite.transactions.*;
+
 import org.jetbrains.annotations.*;
 
 import java.io.*;
@@ -110,7 +110,11 @@ public class GridCacheConcurrentTxMultiNodeTest extends GridCommonAbstractTest {
             CacheConfiguration cc = defaultCacheConfiguration();
 
             cc.setCacheMode(mode);
-            cc.setEvictionPolicy(new LruEvictionPolicy(1000));
+
+            LruEvictionPolicy plc = new LruEvictionPolicy();
+            plc.setMaxSize(1000);
+
+            cc.setEvictionPolicy(plc);
             cc.setEvictSynchronized(false);
             cc.setSwapEnabled(false);
             cc.setWriteSynchronizationMode(FULL_SYNC);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/965a7fe5/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConfigurationConsistencySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConfigurationConsistencySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConfigurationConsistencySelfTest.java
index 44171a8..445a5e1 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConfigurationConsistencySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConfigurationConsistencySelfTest.java
@@ -442,7 +442,12 @@ public class GridCacheConfigurationConsistencySelfTest extends GridCommonAbstrac
                 /** {@inheritDoc} */
                 @Override public Void apply(CacheConfiguration cfg) {
                     cfg.setEvictSynchronized(true);
-                    cfg.setEvictionPolicy(new FifoEvictionPolicy(100));
+
+                    FifoEvictionPolicy plc = new FifoEvictionPolicy();
+
+                    plc.setMaxSize(100);
+
+                    cfg.setEvictionPolicy(plc);
                     return null;
                 }
             },
@@ -450,7 +455,12 @@ public class GridCacheConfigurationConsistencySelfTest extends GridCommonAbstrac
                 /** {@inheritDoc} */
                 @Override public Void apply(CacheConfiguration cfg) {
                     cfg.setEvictSynchronized(false);
-                    cfg.setEvictionPolicy(new FifoEvictionPolicy(100));
+
+                    FifoEvictionPolicy plc = new FifoEvictionPolicy();
+
+                    plc.setMaxSize(100);
+
+                    cfg.setEvictionPolicy(plc);
                     return null;
                 }
             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/965a7fe5/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMemoryModeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMemoryModeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMemoryModeSelfTest.java
index 08e35ce..b4e523e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMemoryModeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMemoryModeSelfTest.java
@@ -28,6 +28,7 @@ import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
 import org.apache.ignite.spi.swapspace.file.*;
 import org.apache.ignite.testframework.junits.common.*;
+
 import org.junit.*;
 
 import java.util.*;
@@ -81,6 +82,11 @@ public class GridCacheMemoryModeSelfTest extends GridCommonAbstractTest {
         return cfg;
     }
 
+    /**
+     * Returns cache configuration.
+     *
+     * @return cache configuration.
+     */
     protected CacheConfiguration cacheConfiguration() {
         CacheConfiguration cacheCfg = defaultCacheConfiguration();
 
@@ -89,7 +95,16 @@ public class GridCacheMemoryModeSelfTest extends GridCommonAbstractTest {
         cacheCfg.setSwapEnabled(swapEnabled);
         cacheCfg.setCacheMode(mode);
         cacheCfg.setMemoryMode(memoryMode);
-        cacheCfg.setEvictionPolicy(maxOnheapSize == Integer.MAX_VALUE ? null : new LruEvictionPolicy(maxOnheapSize));
+
+        LruEvictionPolicy plc = null;
+
+        if (maxOnheapSize != Integer.MAX_VALUE) {
+            plc = new LruEvictionPolicy();
+            plc.setMaxSize(maxOnheapSize);
+        }
+
+        cacheCfg.setEvictionPolicy(plc);
+
         cacheCfg.setAtomicityMode(atomicity);
         cacheCfg.setOffHeapMaxMemory(offheapSize);
 
@@ -199,7 +214,8 @@ public class GridCacheMemoryModeSelfTest extends GridCommonAbstractTest {
      * @param swapEmpty Swap is empty.
      * @throws Exception If failed.
      */
-    private void doTestPutAndPutAll(int cache, int offheapSwap, boolean offheapEmpty, boolean swapEmpty) throws Exception {
+    private void doTestPutAndPutAll(int cache, int offheapSwap, boolean offheapEmpty, boolean swapEmpty)
+        throws Exception {
         final int all = cache + offheapSwap;
 
         // put
@@ -231,7 +247,8 @@ public class GridCacheMemoryModeSelfTest extends GridCommonAbstractTest {
      * @param x Cache modifier.
      * @throws IgniteCheckedException If failed.
      */
-    void doTest(int cache, int offheapSwap, boolean offheapEmpty, boolean swapEmpty, CIX1<IgniteCache<String, Integer>> x) throws Exception {
+    void doTest(int cache, int offheapSwap, boolean offheapEmpty, boolean swapEmpty,
+        CIX1<IgniteCache<String, Integer>> x) throws Exception {
         ipFinder = new TcpDiscoveryVmIpFinder(true);
 
         startGrid();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/965a7fe5/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapTest.java
index d69f7fa..b0f07f0 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapTest.java
@@ -76,7 +76,10 @@ public class GridCacheOffHeapTest extends GridCommonAbstractTest {
         cacheCfg.setStartSize(startSize);
 
         if (onheap > 0) {
-            cacheCfg.setEvictionPolicy(new FifoEvictionPolicy(onheap));
+            FifoEvictionPolicy plc = new FifoEvictionPolicy();
+            plc.setMaxSize(onheap);
+
+            cacheCfg.setEvictionPolicy(plc);
 
             cacheCfg.setOffHeapMaxMemory(80 * 1024L * 1024L * 1024L); // 10GB
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/965a7fe5/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheReloadSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheReloadSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheReloadSelfTest.java
index e2cdd08..b6bfbc7 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheReloadSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheReloadSelfTest.java
@@ -74,7 +74,11 @@ public class GridCacheReloadSelfTest extends GridCommonAbstractTest {
         CacheConfiguration cacheCfg = defaultCacheConfiguration();
         cacheCfg.setName(CACHE_NAME);
         cacheCfg.setCacheMode(cacheMode);
-        cacheCfg.setEvictionPolicy(new LruEvictionPolicy(MAX_CACHE_ENTRIES));
+
+        LruEvictionPolicy plc = new LruEvictionPolicy();
+        plc.setMaxSize(MAX_CACHE_ENTRIES);
+
+        cacheCfg.setEvictionPolicy(plc);
         cacheCfg.setNearConfiguration(nearEnabled ? new NearCacheConfiguration() : null);
 
         final CacheStore store = new CacheStoreAdapter<Integer, Integer>() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/965a7fe5/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingNearErrorTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingNearErrorTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingNearErrorTest.java
index 732d12d..30f7b92 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingNearErrorTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingNearErrorTest.java
@@ -36,7 +36,11 @@ public class IgniteCacheP2pUnmarshallingNearErrorTest extends IgniteCacheP2pUnma
         cfg.getCacheConfiguration()[0].setEvictMaxOverflowRatio(0);
         cfg.getCacheConfiguration()[0].setEvictSynchronized(true);
         cfg.getCacheConfiguration()[0].setEvictSynchronizedKeyBufferSize(1);
-        cfg.getCacheConfiguration()[0].setEvictionPolicy(new FifoEvictionPolicy(1));
+
+        FifoEvictionPolicy plc = new FifoEvictionPolicy();
+        plc.setMaxSize(1);
+
+        cfg.getCacheConfiguration()[0].setEvictionPolicy(plc);
 
         return cfg;
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/965a7fe5/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCachePeekModesAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCachePeekModesAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCachePeekModesAbstractTest.java
index 7cd8414..448f171 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCachePeekModesAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCachePeekModesAbstractTest.java
@@ -93,7 +93,10 @@ public abstract class IgniteCachePeekModesAbstractTest extends IgniteCacheAbstra
         if (hasNearCache())
             ccfg.setNearConfiguration(new NearCacheConfiguration());
 
-        ccfg.setEvictionPolicy(new FifoEvictionPolicy(HEAP_ENTRIES));
+        FifoEvictionPolicy plc = new FifoEvictionPolicy();
+        plc.setMaxSize(HEAP_ENTRIES);
+
+        ccfg.setEvictionPolicy(plc);
 
         return ccfg;
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/965a7fe5/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionNearReadersSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionNearReadersSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionNearReadersSelfTest.java
index aff5512..7adeba8 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionNearReadersSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionNearReadersSelfTest.java
@@ -77,11 +77,18 @@ public class GridCacheDhtEvictionNearReadersSelfTest extends GridCommonAbstractT
         // Set eviction queue size explicitly.
         cacheCfg.setEvictSynchronizedKeyBufferSize(1);
         cacheCfg.setEvictMaxOverflowRatio(0);
-        cacheCfg.setEvictionPolicy(new FifoEvictionPolicy(10));
+
+        FifoEvictionPolicy plc = new FifoEvictionPolicy();
+        plc.setMaxSize(10);
+
+        cacheCfg.setEvictionPolicy(plc);
 
         NearCacheConfiguration nearCfg = new NearCacheConfiguration();
 
-        nearCfg.setNearEvictionPolicy(new FifoEvictionPolicy(10));
+        FifoEvictionPolicy nearPlc = new FifoEvictionPolicy();
+        nearPlc.setMaxSize(10);
+
+        nearCfg.setNearEvictionPolicy(nearPlc);
 
         cacheCfg.setNearConfiguration(nearCfg);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/965a7fe5/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionSelfTest.java
index 97e8657..8eada9c 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionSelfTest.java
@@ -76,14 +76,21 @@ public class GridCacheDhtEvictionSelfTest extends GridCommonAbstractTest {
         cacheCfg.setBackups(1);
 
         NearCacheConfiguration nearCfg = new NearCacheConfiguration();
-        nearCfg.setNearEvictionPolicy(new FifoEvictionPolicy(10000));
+
+        FifoEvictionPolicy nearPlc = new FifoEvictionPolicy();
+        nearPlc.setMaxSize(10000);
+
+        nearCfg.setNearEvictionPolicy(nearPlc);
         cacheCfg.setNearConfiguration(nearCfg);
 
         // Set eviction queue size explicitly.
         cacheCfg.setEvictMaxOverflowRatio(0);
         cacheCfg.setEvictSynchronizedKeyBufferSize(1);
-        cacheCfg.setEvictionPolicy(new FifoEvictionPolicy(10000));
 
+        FifoEvictionPolicy plc = new FifoEvictionPolicy();
+        plc.setMaxSize(10000);
+
+        cacheCfg.setEvictionPolicy(plc);
 
         cfg.setCacheConfiguration(cacheCfg);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/965a7fe5/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheMultiTxLockSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheMultiTxLockSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheMultiTxLockSelfTest.java
index 5983c1b..77fe8cf 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheMultiTxLockSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheMultiTxLockSelfTest.java
@@ -71,7 +71,11 @@ public class IgniteCacheMultiTxLockSelfTest extends GridCommonAbstractTest {
         ccfg.setBackups(2);
         ccfg.setCacheMode(CacheMode.PARTITIONED);
         ccfg.setStartSize(100000);
-        ccfg.setEvictionPolicy(new LruEvictionPolicy(100000));
+
+        LruEvictionPolicy plc = new LruEvictionPolicy();
+        plc.setMaxSize(100000);
+
+        ccfg.setEvictionPolicy(plc);
         ccfg.setEvictSynchronized(true);
 
         c.setCacheConfiguration(ccfg);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/965a7fe5/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedEvictionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedEvictionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedEvictionSelfTest.java
index 28ad7a8..c0d36e2 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedEvictionSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedEvictionSelfTest.java
@@ -78,8 +78,15 @@ public class GridCachePartitionedEvictionSelfTest extends GridCacheAbstractSelfT
 
         cc.setCacheMode(PARTITIONED);
         cc.setWriteSynchronizationMode(FULL_SYNC);
-        cc.setEvictionPolicy(new FifoEvictionPolicy(EVICT_CACHE_SIZE));
-        cc.getNearConfiguration().setNearEvictionPolicy(new FifoEvictionPolicy(EVICT_CACHE_SIZE));
+
+        FifoEvictionPolicy plc = new FifoEvictionPolicy();
+        plc.setMaxSize(EVICT_CACHE_SIZE);
+        cc.setEvictionPolicy(plc);
+
+        FifoEvictionPolicy nearPlc = new FifoEvictionPolicy();
+        nearPlc.setMaxSize(EVICT_CACHE_SIZE);
+        cc.getNearConfiguration().setNearEvictionPolicy(nearPlc);
+
         cc.setSwapEnabled(false);
 
         // We set 1 backup explicitly.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/965a7fe5/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiThreadedPutGetSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiThreadedPutGetSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiThreadedPutGetSelfTest.java
index 8131d3e..1c45de8 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiThreadedPutGetSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiThreadedPutGetSelfTest.java
@@ -66,7 +66,11 @@ public class GridCachePartitionedMultiThreadedPutGetSelfTest extends GridCommonA
         cc.setCacheMode(PARTITIONED);
         cc.setBackups(1);
         cc.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
-        cc.setEvictionPolicy(new FifoEvictionPolicy<>(1000));
+
+        FifoEvictionPolicy plc = new FifoEvictionPolicy();
+        plc.setMaxMemSize(1000);
+
+        cc.setEvictionPolicy(plc);
         cc.setSwapEnabled(false);
         cc.setAtomicityMode(TRANSACTIONAL);
         cc.setEvictSynchronized(false);