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/06/03 00:54:02 UTC

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

Repository: incubator-ignite
Updated Branches:
  refs/heads/ignite-sprint-5 1bd6da1a1 -> ba6d05564


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e5d5d08d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/sorted/GridCacheSortedEvictionPolicyPerformanceTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/sorted/GridCacheSortedEvictionPolicyPerformanceTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/sorted/GridCacheSortedEvictionPolicyPerformanceTest.java
deleted file mode 100644
index 8d97a1e..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/sorted/GridCacheSortedEvictionPolicyPerformanceTest.java
+++ /dev/null
@@ -1,135 +0,0 @@
-/*
- * 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.eviction.sorted;
-
-import org.apache.ignite.*;
-import org.apache.ignite.cache.*;
-import org.apache.ignite.cache.eviction.fifo.*;
-import org.apache.ignite.cache.eviction.sorted.*;
-import org.apache.ignite.configuration.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-import org.apache.ignite.testframework.junits.common.*;
-
-import org.jsr166.*;
-
-import java.util.concurrent.*;
-import java.util.concurrent.atomic.*;
-
-/**
- * {@link SortedEvictionPolicy} performance test.
- */
-public class GridCacheSortedEvictionPolicyPerformanceTest extends GridCommonAbstractTest {
-    /** Threads. */
-    private static final int THREADS = 8;
-
-    /** Keys. */
-    private static final int KEYS = 100_000;
-
-    /** Max size. */
-    private static final int MAX_SIZE = 1000;
-
-    /** Put probability. */
-    private static final int P_PUT = 50;
-
-    /** Get probability. */
-    private static final int P_GET = 30;
-
-    /** Measurement count. */
-    private static final int MEASUREMENT_CNT = 100;
-
-    /** Rnd. */
-    private static final ThreadLocalRandom8 RND = ThreadLocalRandom8.current();
-
-    /** Ignite. */
-    private static Ignite ignite;
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        ignite = startGrid();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTestsStopped() throws Exception {
-        stopAllGrids();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(gridName);
-
-        CacheConfiguration ccfg = defaultCacheConfiguration();
-
-        ccfg.setCacheMode(CacheMode.PARTITIONED);
-        ccfg.setAtomicityMode(CacheAtomicityMode.ATOMIC);
-        ccfg.setNearConfiguration(null);
-        ccfg.setEvictionPolicy(new SortedEvictionPolicy(MAX_SIZE));
-//        ccfg.setEvictionPolicy(new FifoEvictionPolicy(MAX_SIZE));
-       ccfg.setEvictSynchronized(false);
-
-        cfg.setPeerClassLoadingEnabled(false);
-
-        cfg.setCacheConfiguration(ccfg);
-
-        return cfg;
-    }
-
-    /**
-     * Tests throughput.
-     */
-    public void testThroughput() throws Exception {
-        final LongAdder8 cnt = new LongAdder8();
-        final AtomicBoolean finished = new AtomicBoolean();
-
-        final int pPut = P_PUT;
-        final int pGet = P_PUT + P_GET;
-
-        final IgniteCache<Integer, Integer> cache = ignite.cache(null);
-
-        multithreadedAsync(new Callable<Object>() {
-            @Override public Object call() throws Exception {
-                for (;;) {
-                    U.sleep(1000);
-
-                    info("Ops/sec: " + cnt.sumThenReset());
-                }
-            }
-        }, 1);
-
-        multithreaded(
-            new Callable<Object>() {
-                @Override public Object call() throws Exception {
-                    while (!finished.get()) {
-                        int p = RND.nextInt(100);
-
-                        int key = RND.nextInt(KEYS);
-
-                        if (p >= 0 && p < pPut)
-                            cache.put(key, 0);
-                        else if (p >= pPut && p < pGet)
-                            cache.get(key);
-                        else
-                            cache.remove(key);
-
-                        cnt.increment();
-                    }
-
-                    return null;
-                }
-            }, THREADS);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e5d5d08d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/sorted/GridCacheSortedEvictionPolicySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/sorted/GridCacheSortedEvictionPolicySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/sorted/GridCacheSortedEvictionPolicySelfTest.java
deleted file mode 100644
index 041234e..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/sorted/GridCacheSortedEvictionPolicySelfTest.java
+++ /dev/null
@@ -1,373 +0,0 @@
-/*
- * 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.eviction.sorted;
-
-import org.apache.ignite.*;
-import org.apache.ignite.cache.eviction.*;
-import org.apache.ignite.cache.eviction.sorted.*;
-import org.apache.ignite.internal.processors.cache.eviction.*;
-
-import java.util.*;
-
-import static org.apache.ignite.cache.CacheMode.*;
-
-/**
- * Sorted eviction test.
- */
-public class GridCacheSortedEvictionPolicySelfTest extends
-    GridCacheEvictionAbstractTest<SortedEvictionPolicy<String, String>> {
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPolicy() throws Exception {
-        try {
-            startGrid();
-
-            MockEntry e1 = new MockEntry("1", "1");
-            MockEntry e2 = new MockEntry("2", "2");
-            MockEntry e3 = new MockEntry("3", "3");
-            MockEntry e4 = new MockEntry("4", "4");
-            MockEntry e5 = new MockEntry("5", "5");
-
-            SortedEvictionPolicy<String, String> p = policy();
-
-            p.setMaxSize(3);
-
-            p.onEntryAccessed(false, e1);
-
-            check(p.set(), e1);
-
-            p.onEntryAccessed(false, e2);
-
-            check(p.set(), e1, e2);
-
-            p.onEntryAccessed(false, e3);
-
-            check(p.set(), e1, e2, e3);
-
-            assertFalse(e1.isEvicted());
-            assertFalse(e2.isEvicted());
-            assertFalse(e3.isEvicted());
-
-            assertEquals(3, p.getCurrentSize());
-
-            p.onEntryAccessed(false, e4);
-
-            check(p.set(), e2, e3, e4);
-
-            assertEquals(3, p.getCurrentSize());
-
-            assertTrue(e1.isEvicted());
-            assertFalse(e2.isEvicted());
-            assertFalse(e3.isEvicted());
-            assertFalse(e4.isEvicted());
-
-            p.onEntryAccessed(false, e5);
-
-            check(p.set(), e3, e4, e5);
-
-            assertEquals(3, p.getCurrentSize());
-
-            assertTrue(e2.isEvicted());
-            assertFalse(e3.isEvicted());
-            assertFalse(e4.isEvicted());
-            assertFalse(e5.isEvicted());
-
-            p.onEntryAccessed(false, e1 = new MockEntry("1", "1"));
-
-            check(p.set(), e3, e4, e5);
-
-            assertEquals(3, p.getCurrentSize());
-
-            assertTrue(e1.isEvicted());
-            assertFalse(e3.isEvicted());
-            assertFalse(e4.isEvicted());
-            assertFalse(e5.isEvicted());
-
-            p.onEntryAccessed(false, e5);
-
-            check(p.set(), e3, e4, e5);
-
-            assertFalse(e3.isEvicted());
-            assertFalse(e4.isEvicted());
-            assertFalse(e5.isEvicted());
-
-            p.onEntryAccessed(false, e1);
-
-            assertEquals(3, p.getCurrentSize());
-
-            check(p.set(), e3, e4, e5);
-
-            assertTrue(e1.isEvicted());
-            assertFalse(e3.isEvicted());
-            assertFalse(e4.isEvicted());
-            assertFalse(e5.isEvicted());
-
-            p.onEntryAccessed(false, e5);
-
-            assertEquals(3, p.getCurrentSize());
-
-            check(p.set(), e3, e4, e5);
-
-            assertFalse(e3.isEvicted());
-            assertFalse(e4.isEvicted());
-            assertFalse(e5.isEvicted());
-
-            p.onEntryAccessed(true, e3);
-
-            assertEquals(2, p.getCurrentSize());
-
-            assertFalse(e3.isEvicted());
-            assertFalse(e4.isEvicted());
-            assertFalse(e5.isEvicted());
-
-            p.onEntryAccessed(true, e4);
-
-            assertEquals(1, p.getCurrentSize());
-
-            assertFalse(e4.isEvicted());
-            assertFalse(e5.isEvicted());
-
-            p.onEntryAccessed(true, e5);
-
-            assertEquals(0, p.getCurrentSize());
-
-            assertFalse(e5.isEvicted());
-
-            info(p);
-        }
-        finally {
-            stopAllGrids();
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testMemory() throws Exception {
-        try {
-            startGrid();
-
-            SortedEvictionPolicy<String, String> p = policy();
-
-            int max = 10;
-
-            p.setMaxSize(max);
-
-            int cnt = 11;
-
-            for (int i = 0; i < cnt; i++)
-                p.onEntryAccessed(false, new MockEntry(Integer.toString(i), Integer.toString(i)));
-
-            info(p);
-
-            assertEquals(max, p.getCurrentSize());
-        }
-        finally {
-            stopAllGrids();
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testRandom() throws Exception {
-        try {
-            startGrid();
-
-            SortedEvictionPolicy<String, String> p = policy();
-
-            int max = 10;
-
-            p.setMaxSize(max);
-
-            Random rand = new Random();
-
-            int keys = 31;
-
-            MockEntry[] fifos = new MockEntry[keys];
-
-            for (int i = 0; i < fifos.length; i++)
-                fifos[i] = new MockEntry(Integer.toString(i));
-
-            int runs = 5000000;
-
-            for (int i = 0; i < runs; i++) {
-                boolean rmv = rand.nextBoolean();
-
-                int j = rand.nextInt(fifos.length);
-
-                MockEntry e = entry(fifos, j);
-
-                if (rmv)
-                    fifos[j] = new MockEntry(Integer.toString(j));
-
-                p.onEntryAccessed(rmv, e);
-            }
-
-            info(p);
-
-            int curSize = p.getCurrentSize();
-
-            assertTrue("curSize <= max [curSize=" + curSize + ", max=" + max + ']', curSize <= max);
-        }
-        finally {
-            stopAllGrids();
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testAllowEmptyEntries() throws Exception {
-        try {
-            startGrid();
-
-            MockEntry e1 = new MockEntry("1");
-
-            MockEntry e2 = new MockEntry("2");
-
-            MockEntry e3 = new MockEntry("3");
-
-            MockEntry e4 = new MockEntry("4");
-
-            MockEntry e5 = new MockEntry("5");
-
-            SortedEvictionPolicy<String, String> p = policy();
-
-            p.setMaxSize(10);
-
-            p.onEntryAccessed(false, e1);
-
-            assertFalse(e1.isEvicted());
-
-            p.onEntryAccessed(false, e2);
-
-            assertFalse(e1.isEvicted());
-            assertFalse(e2.isEvicted());
-
-            p.onEntryAccessed(false, e3);
-
-            assertFalse(e1.isEvicted());
-            assertFalse(e3.isEvicted());
-
-            p.onEntryAccessed(false, e4);
-
-            assertFalse(e1.isEvicted());
-            assertFalse(e3.isEvicted());
-            assertFalse(e4.isEvicted());
-
-            p.onEntryAccessed(false, e5);
-
-            assertFalse(e1.isEvicted());
-            assertFalse(e3.isEvicted());
-            assertFalse(e5.isEvicted());
-        }
-        finally {
-            stopAllGrids();
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPut() throws Exception {
-        mode = LOCAL;
-        syncCommit = true;
-        plcMax = 100;
-
-        Ignite ignite = startGrid();
-
-        try {
-            IgniteCache<Object, Object> cache = ignite.cache(null);
-
-            int cnt = 500;
-
-            int min = Integer.MAX_VALUE;
-
-            int minIdx = 0;
-
-            for (int i = 0; i < cnt; i++) {
-                cache.put(i, i);
-
-                int cacheSize = cache.size();
-
-                if (i > plcMax && cacheSize < min) {
-                    min = cacheSize;
-                    minIdx = i;
-                }
-            }
-
-            assertTrue("Min cache size is too small: " + min, min >= plcMax);
-
-            info("Min cache size [min=" + min + ", idx=" + minIdx + ']');
-            info("Current cache size " + cache.size());
-            info("Current cache key size " + cache.size());
-
-            min = Integer.MAX_VALUE;
-
-            minIdx = 0;
-
-            // Touch.
-            for (int i = cnt; --i > cnt - plcMax;) {
-                cache.get(i);
-
-                int cacheSize = cache.size();
-
-                if (cacheSize < min) {
-                    min = cacheSize;
-                    minIdx = i;
-                }
-            }
-
-            info("----");
-            info("Min cache size [min=" + min + ", idx=" + minIdx + ']');
-            info("Current cache size " + cache.size());
-            info("Current cache key size " + cache.size());
-
-            assertTrue("Min cache size is too small: " + min, min >= plcMax);
-        }
-        finally {
-            stopAllGrids();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override protected SortedEvictionPolicy<String, String> createPolicy(int plcMax) {
-        return new SortedEvictionPolicy<>(plcMax);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected SortedEvictionPolicy<String, String> createNearPolicy(int nearMax) {
-        return new SortedEvictionPolicy<>(nearMax);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void checkNearPolicies(int endNearPlcSize) {
-        for (int i = 0; i < gridCnt; i++)
-            for (EvictableEntry<String, String> e : nearPolicy(i).set())
-                assert !e.isCached() : "Invalid near policy size: " + nearPolicy(i).set();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void checkPolicies(int plcMax) {
-        for (int i = 0; i < gridCnt; i++)
-            assert policy(i).set().size() <= plcMax;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e5d5d08d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/sorted/SortedEvictionPolicyPerformanceTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/sorted/SortedEvictionPolicyPerformanceTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/sorted/SortedEvictionPolicyPerformanceTest.java
new file mode 100644
index 0000000..a687da9
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/sorted/SortedEvictionPolicyPerformanceTest.java
@@ -0,0 +1,134 @@
+/*
+ * 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.eviction.sorted;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.*;
+import org.apache.ignite.cache.eviction.sorted.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.testframework.junits.common.*;
+
+import org.jsr166.*;
+
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.*;
+
+/**
+ * {@link SortedEvictionPolicy} performance test.
+ */
+public class SortedEvictionPolicyPerformanceTest extends GridCommonAbstractTest {
+    /** Threads. */
+    private static final int THREADS = 8;
+
+    /** Keys. */
+    private static final int KEYS = 100_000;
+
+    /** Max size. */
+    private static final int MAX_SIZE = 1000;
+
+    /** Put probability. */
+    private static final int P_PUT = 50;
+
+    /** Get probability. */
+    private static final int P_GET = 30;
+
+    /** Rnd. */
+    private static final ThreadLocalRandom8 RND = ThreadLocalRandom8.current();
+
+    /** Ignite. */
+    private static Ignite ignite;
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        ignite = startGrid();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        CacheConfiguration ccfg = defaultCacheConfiguration();
+
+        ccfg.setCacheMode(CacheMode.PARTITIONED);
+        ccfg.setAtomicityMode(CacheAtomicityMode.ATOMIC);
+        ccfg.setNearConfiguration(null);
+
+        SortedEvictionPolicy plc = new SortedEvictionPolicy();
+        plc.setMaxSize(MAX_SIZE);
+
+        ccfg.setEvictionPolicy(plc);
+        ccfg.setEvictSynchronized(false);
+
+        cfg.setPeerClassLoadingEnabled(false);
+
+        cfg.setCacheConfiguration(ccfg);
+
+        return cfg;
+    }
+
+    /**
+     * Tests throughput.
+     */
+    public void testThroughput() throws Exception {
+        final LongAdder8 cnt = new LongAdder8();
+        final AtomicBoolean finished = new AtomicBoolean();
+
+        final int pPut = P_PUT;
+        final int pGet = P_PUT + P_GET;
+
+        final IgniteCache<Integer, Integer> cache = ignite.cache(null);
+
+        multithreadedAsync(new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                for (;;) {
+                    U.sleep(1000);
+
+                    info("Ops/sec: " + cnt.sumThenReset());
+                }
+            }
+        }, 1);
+
+        multithreaded(
+            new Callable<Object>() {
+                @Override public Object call() throws Exception {
+                    while (!finished.get()) {
+                        int p = RND.nextInt(100);
+
+                        int key = RND.nextInt(KEYS);
+
+                        if (p >= 0 && p < pPut)
+                            cache.put(key, 0);
+                        else if (p >= pPut && p < pGet)
+                            cache.get(key);
+                        else
+                            cache.remove(key);
+
+                        cnt.increment();
+                    }
+
+                    return null;
+                }
+            }, THREADS);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e5d5d08d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/sorted/SortedEvictionPolicySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/sorted/SortedEvictionPolicySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/sorted/SortedEvictionPolicySelfTest.java
new file mode 100644
index 0000000..dbbbe5d
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/sorted/SortedEvictionPolicySelfTest.java
@@ -0,0 +1,266 @@
+/*
+ * 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.eviction.sorted;
+
+import org.apache.ignite.cache.eviction.sorted.*;
+import org.apache.ignite.internal.processors.cache.eviction.*;
+
+/**
+ * Sorted eviction policy tests.
+ */
+public class SortedEvictionPolicySelfTest extends
+    EvictionAbstractTest<SortedEvictionPolicy<String, String>> {
+    /** {@inheritDoc} */
+    @Override protected void doTestPolicy() throws Exception {
+        try {
+            startGrid();
+
+            MockEntry e1 = new MockEntry("1", "1");
+            MockEntry e2 = new MockEntry("2", "2");
+            MockEntry e3 = new MockEntry("3", "3");
+            MockEntry e4 = new MockEntry("4", "4");
+            MockEntry e5 = new MockEntry("5", "5");
+
+            SortedEvictionPolicy<String, String> p = policy();
+
+            p.onEntryAccessed(false, e1);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e1);
+
+            p.onEntryAccessed(false, e2);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e1, e2);
+
+            p.onEntryAccessed(false, e3);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e1, e2, e3);
+
+            assertFalse(e1.isEvicted());
+            assertFalse(e2.isEvicted());
+            assertFalse(e3.isEvicted());
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e1, e2, e3);
+
+            p.onEntryAccessed(false, e4);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e2, e3, e4);
+
+            assertTrue(e1.isEvicted());
+            assertFalse(e2.isEvicted());
+            assertFalse(e3.isEvicted());
+            assertFalse(e4.isEvicted());
+
+            p.onEntryAccessed(false, e5);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e3, e4, e5);
+
+            assertTrue(e2.isEvicted());
+            assertFalse(e3.isEvicted());
+            assertFalse(e4.isEvicted());
+            assertFalse(e5.isEvicted());
+
+            p.onEntryAccessed(false, e1 = new MockEntry("1", "1"));
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e3, e4, e5);
+
+            assertTrue(e1.isEvicted());
+            assertFalse(e3.isEvicted());
+            assertFalse(e4.isEvicted());
+            assertFalse(e5.isEvicted());
+
+            p.onEntryAccessed(false, e5);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e3, e4, e5);
+
+            assertFalse(e3.isEvicted());
+            assertFalse(e4.isEvicted());
+            assertFalse(e5.isEvicted());
+
+            p.onEntryAccessed(false, e1);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e3, e4, e5);
+
+            assertTrue(e1.isEvicted());
+            assertFalse(e3.isEvicted());
+            assertFalse(e4.isEvicted());
+            assertFalse(e5.isEvicted());
+
+            p.onEntryAccessed(false, e5);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e3, e4, e5);
+
+            assertFalse(e3.isEvicted());
+            assertFalse(e4.isEvicted());
+            assertFalse(e5.isEvicted());
+
+            p.onEntryAccessed(true, e3);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e4, e5);
+
+            assertFalse(e3.isEvicted());
+            assertFalse(e4.isEvicted());
+            assertFalse(e5.isEvicted());
+
+            p.onEntryAccessed(true, e4);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e5);
+
+            assertFalse(e4.isEvicted());
+            assertFalse(e5.isEvicted());
+
+            p.onEntryAccessed(true, e5);
+
+            check(MockEntry.ENTRY_SIZE, p.queue());
+
+            assertFalse(e5.isEvicted());
+
+            info(p);
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void doTestPolicyWithBatch() throws Exception {
+        try {
+            startGrid();
+
+            MockEntry e1 = new MockEntry("1", "1");
+            MockEntry e2 = new MockEntry("2", "2");
+            MockEntry e3 = new MockEntry("3", "3");
+            MockEntry e4 = new MockEntry("4", "4");
+            MockEntry e5 = new MockEntry("5", "5");
+
+            SortedEvictionPolicy<String, String> p = policy();
+
+            p.onEntryAccessed(false, e1);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e1);
+
+            p.onEntryAccessed(false, e2);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e1, e2);
+
+            p.onEntryAccessed(false, e3);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e1, e2, e3);
+
+            p.onEntryAccessed(false, e4);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e1, e2, e3, e4);
+
+            assertFalse(e1.isEvicted());
+            assertFalse(e2.isEvicted());
+            assertFalse(e3.isEvicted());
+            assertFalse(e4.isEvicted());
+
+            p.onEntryAccessed(false, e5);
+
+            // Batch evicted.
+            check(MockEntry.ENTRY_SIZE, p.queue(), e3, e4, e5);
+
+            assertTrue(e1.isEvicted());
+            assertTrue(e2.isEvicted());
+            assertFalse(e3.isEvicted());
+            assertFalse(e4.isEvicted());
+            assertFalse(e5.isEvicted());
+
+            p.onEntryAccessed(false, e1 = new MockEntry("1", "1"));
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e1, e3, e4, e5);
+
+            assertFalse(e1.isEvicted());
+            assertFalse(e3.isEvicted());
+            assertFalse(e4.isEvicted());
+            assertFalse(e5.isEvicted());
+
+            p.onEntryAccessed(false, e5);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e1, e3, e4, e5);
+
+            assertFalse(e1.isEvicted());
+            assertFalse(e3.isEvicted());
+            assertFalse(e4.isEvicted());
+            assertFalse(e5.isEvicted());
+
+            p.onEntryAccessed(false, e1);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e1, e3, e4, e5);
+
+            assertFalse(e1.isEvicted());
+            assertFalse(e3.isEvicted());
+            assertFalse(e4.isEvicted());
+            assertFalse(e5.isEvicted());
+
+            p.onEntryAccessed(true, e1);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e3, e4, e5);
+
+            assertFalse(e3.isEvicted());
+            assertFalse(e4.isEvicted());
+            assertFalse(e5.isEvicted());
+
+            p.onEntryAccessed(true, e4);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e3, e5);
+
+            assertFalse(e3.isEvicted());
+            assertFalse(e5.isEvicted());
+
+            p.onEntryAccessed(true, e5);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e3);
+
+            assertFalse(e3.isEvicted());
+
+            p.onEntryAccessed(true, e3);
+
+            check(MockEntry.ENTRY_SIZE, p.queue());
+
+            assertFalse(e3.isEvicted());
+
+            info(p);
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override protected SortedEvictionPolicy<String, String> createPolicy(int plcMax) {
+        SortedEvictionPolicy<String, String> plc = new SortedEvictionPolicy<>();
+
+        plc.setMaxSize(this.plcMax);
+        plc.setBatchSize(this.plcBatchSize);
+        plc.setMaxMemSize(this.plcMaxMemSize);
+
+        return plc;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected SortedEvictionPolicy<String, String> createNearPolicy(int nearMax) {
+        SortedEvictionPolicy<String, String> plc = new SortedEvictionPolicy<>();
+
+        plc.setMaxSize(nearMax);
+        plc.setBatchSize(plcBatchSize);
+
+        return plc;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e5d5d08d/modules/core/src/test/java/org/apache/ignite/loadtests/GridCacheMultiNodeLoadTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/loadtests/GridCacheMultiNodeLoadTest.java b/modules/core/src/test/java/org/apache/ignite/loadtests/GridCacheMultiNodeLoadTest.java
index d106f5b..0030ce1 100644
--- a/modules/core/src/test/java/org/apache/ignite/loadtests/GridCacheMultiNodeLoadTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/loadtests/GridCacheMultiNodeLoadTest.java
@@ -65,7 +65,10 @@ public class GridCacheMultiNodeLoadTest extends GridCommonAbstractTest {
         cacheCfg.setStartSize(10);
         cacheCfg.setWriteSynchronizationMode(FULL_SYNC);
 
-        cacheCfg.setEvictionPolicy(new LruEvictionPolicy(100000));
+        LruEvictionPolicy plc = new LruEvictionPolicy();
+        plc.setMaxSize(100000);
+
+        cacheCfg.setEvictionPolicy(plc);
         cacheCfg.setBackups(1);
 
         cacheCfg.setRebalanceMode(SYNC);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e5d5d08d/modules/core/src/test/java/org/apache/ignite/loadtests/datastructures/GridCachePartitionedAtomicLongLoadTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/loadtests/datastructures/GridCachePartitionedAtomicLongLoadTest.java b/modules/core/src/test/java/org/apache/ignite/loadtests/datastructures/GridCachePartitionedAtomicLongLoadTest.java
index 6960fa1..0d9ec8f 100644
--- a/modules/core/src/test/java/org/apache/ignite/loadtests/datastructures/GridCachePartitionedAtomicLongLoadTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/loadtests/datastructures/GridCachePartitionedAtomicLongLoadTest.java
@@ -70,7 +70,11 @@ public class GridCachePartitionedAtomicLongLoadTest extends GridCommonAbstractTe
         cc.setStartSize(200);
         cc.setRebalanceMode(CacheRebalanceMode.SYNC);
         cc.setWriteSynchronizationMode(FULL_SYNC);
-        cc.setEvictionPolicy(new LruEvictionPolicy<>(1000));
+
+        LruEvictionPolicy plc = new LruEvictionPolicy();
+        plc.setMaxSize(1000);
+
+        cc.setEvictionPolicy(plc);
         cc.setBackups(1);
         cc.setAffinity(new RendezvousAffinityFunction(true));
         cc.setEvictSynchronized(true);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e5d5d08d/modules/core/src/test/java/org/apache/ignite/loadtests/swap/GridSwapEvictAllBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/loadtests/swap/GridSwapEvictAllBenchmark.java b/modules/core/src/test/java/org/apache/ignite/loadtests/swap/GridSwapEvictAllBenchmark.java
index 1561b77..62066da 100644
--- a/modules/core/src/test/java/org/apache/ignite/loadtests/swap/GridSwapEvictAllBenchmark.java
+++ b/modules/core/src/test/java/org/apache/ignite/loadtests/swap/GridSwapEvictAllBenchmark.java
@@ -266,7 +266,11 @@ public class GridSwapEvictAllBenchmark {
 
         ccfg.setSwapEnabled(true);
         ccfg.setEvictSynchronized(false);
-        ccfg.setEvictionPolicy(new FifoEvictionPolicy(EVICT_PLC_SIZE));
+
+        FifoEvictionPolicy plc = new FifoEvictionPolicy();
+        plc.setMaxSize(EVICT_PLC_SIZE);
+
+        ccfg.setEvictionPolicy(plc);
 
         if (store != null) {
             ccfg.setCacheStoreFactory(new FactoryBuilder.SingletonFactory(store));

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e5d5d08d/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheEvictionSelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheEvictionSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheEvictionSelfTestSuite.java
index 0852496..917185e 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheEvictionSelfTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheEvictionSelfTestSuite.java
@@ -37,15 +37,14 @@ public class IgniteCacheEvictionSelfTestSuite extends TestSuite {
     public static TestSuite suite() throws Exception {
         TestSuite suite = new TestSuite("Ignite Cache Eviction Test Suite");
 
-        suite.addTest(new TestSuite(GridCacheFifoEvictionPolicySelfTest.class));
-        suite.addTest(new TestSuite(GridCacheFifoBatchEvictionPolicySelfTest.class));
-        suite.addTest(new TestSuite(GridCacheSortedEvictionPolicySelfTest.class));
-        suite.addTest(new TestSuite(GridCacheSortedBatchEvictionPolicySelfTest.class));
-        suite.addTest(new TestSuite(GridCacheLruEvictionPolicySelfTest.class));
-        suite.addTest(new TestSuite(GridCacheLruNearEvictionPolicySelfTest.class));
-        suite.addTest(new TestSuite(GridCacheNearOnlyLruNearEvictionPolicySelfTest.class));
         suite.addTest(new TestSuite(RandomEvictionPolicyCacheSizeSelfTest.class));
-        suite.addTest(new TestSuite(GridCacheRandomEvictionPolicySelfTest.class));
+        suite.addTest(new TestSuite(FifoEvictionPolicySelfTest.class));
+        suite.addTest(new TestSuite(SortedEvictionPolicySelfTest.class));
+        suite.addTest(new TestSuite(LruEvictionPolicySelfTest.class));
+        suite.addTest(new TestSuite(LruNearEvictionPolicySelfTest.class));
+        suite.addTest(new TestSuite(LruNearOnlyNearEvictionPolicySelfTest.class));
+        suite.addTest(new TestSuite(RandomEvictionPolicySelfTest.class));
+        suite.addTest(new TestSuite(RandomEvictionPolicyCacheSizeSelfTest.class));
         suite.addTest(new TestSuite(GridCacheNearEvictionSelfTest.class));
         suite.addTest(new TestSuite(GridCacheAtomicNearEvictionSelfTest.class));
         suite.addTest(new TestSuite(GridCacheEvictionFilterSelfTest.class));

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e5d5d08d/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridIndexingWithNoopSwapSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridIndexingWithNoopSwapSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridIndexingWithNoopSwapSelfTest.java
index 2c0962b..5b623da 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridIndexingWithNoopSwapSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridIndexingWithNoopSwapSelfTest.java
@@ -64,7 +64,11 @@ public class GridIndexingWithNoopSwapSelfTest extends GridCommonAbstractTest {
         cc.setRebalanceMode(SYNC);
         cc.setSwapEnabled(true);
         cc.setNearConfiguration(new NearCacheConfiguration());
-        cc.setEvictionPolicy(new FifoEvictionPolicy(1000));
+
+        FifoEvictionPolicy plc = new FifoEvictionPolicy();
+        plc.setMaxSize(1000);
+
+        cc.setEvictionPolicy(plc);
         cc.setBackups(1);
         cc.setAtomicityMode(TRANSACTIONAL);
         cc.setIndexedTypes(

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e5d5d08d/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryMultiThreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryMultiThreadedSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryMultiThreadedSelfTest.java
index 23a97c9..1d6bbc8 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryMultiThreadedSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryMultiThreadedSelfTest.java
@@ -104,7 +104,16 @@ public class IgniteCacheQueryMultiThreadedSelfTest extends GridCommonAbstractTes
         cacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
         cacheCfg.setSwapEnabled(true);
         cacheCfg.setBackups(1);
-        cacheCfg.setEvictionPolicy(evictsEnabled() ? new LruEvictionPolicy(100) : null);
+
+        LruEvictionPolicy plc = null;
+
+        if (evictsEnabled()) {
+            plc = new LruEvictionPolicy();
+            plc.setMaxSize(100);
+        }
+
+        cacheCfg.setEvictionPolicy(plc);
+
         cacheCfg.setSqlOnheapRowCacheSize(128);
         cacheCfg.setIndexedTypes(
             Integer.class, Integer.class,

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e5d5d08d/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/ttl/CacheTtlAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/ttl/CacheTtlAbstractSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/ttl/CacheTtlAbstractSelfTest.java
index 3833576..4553fec 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/ttl/CacheTtlAbstractSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/ttl/CacheTtlAbstractSelfTest.java
@@ -67,7 +67,11 @@ public abstract class CacheTtlAbstractSelfTest extends GridCommonAbstractTest {
         ccfg.setAtomicityMode(atomicityMode());
         ccfg.setMemoryMode(memoryMode());
         ccfg.setOffHeapMaxMemory(0);
-        ccfg.setEvictionPolicy(new LruEvictionPolicy(MAX_CACHE_SIZE));
+
+        LruEvictionPolicy plc = new LruEvictionPolicy();
+        plc.setMaxSize(MAX_CACHE_SIZE);
+
+        ccfg.setEvictionPolicy(plc);
         ccfg.setIndexedTypes(Integer.class, Integer.class);
         ccfg.setBackups(2);
         ccfg.setWriteSynchronizationMode(FULL_SYNC);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e5d5d08d/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteNode.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteNode.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteNode.java
index 8638623..2027b80 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteNode.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteNode.java
@@ -104,8 +104,11 @@ public class IgniteNode implements BenchmarkServer {
 
                 if (args.isOffheapValues())
                     cc.setMemoryMode(OFFHEAP_VALUES);
-                else
-                    cc.setEvictionPolicy(new LruEvictionPolicy(50000));
+                else {
+                    LruEvictionPolicy plc = new LruEvictionPolicy();
+                    plc.setMaxSize(50000);
+                    cc.setEvictionPolicy(plc);
+                }
             }
 
             cc.setReadThrough(args.isStoreEnabled());



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

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e5d5d08d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/EvictionAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/EvictionAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/EvictionAbstractTest.java
new file mode 100644
index 0000000..c2a599d
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/EvictionAbstractTest.java
@@ -0,0 +1,1057 @@
+/*
+ * 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.eviction;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.*;
+import org.apache.ignite.cache.eviction.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.processors.cache.distributed.dht.colocated.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+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 javax.cache.*;
+import java.lang.reflect.*;
+import java.util.*;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.*;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+import static org.apache.ignite.cache.CacheMode.*;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.*;
+import static org.apache.ignite.events.EventType.*;
+import static org.apache.ignite.internal.processors.cache.eviction.EvictionAbstractTest.EvictionPolicyProxy.*;
+import static org.apache.ignite.transactions.TransactionConcurrency.*;
+import static org.apache.ignite.transactions.TransactionIsolation.*;
+
+/**
+ * Base class for eviction tests.
+ */
+public abstract class EvictionAbstractTest<T extends EvictionPolicy<?, ?>>
+    extends GridCommonAbstractTest {
+    /** IP finder. */
+    protected static final TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** Put entry size. */
+    protected static final int PUT_ENTRY_SIZE = 10;
+
+    /** Replicated cache. */
+    protected CacheMode mode = REPLICATED;
+
+    /** Near enabled flag. */
+    protected boolean nearEnabled;
+
+    /** Evict backup sync. */
+    protected boolean evictSync;
+
+    /** Evict near sync. */
+    protected boolean evictNearSync = true;
+
+    /** Policy max. */
+    protected int plcMax = 10;
+
+    /** Policy batch size. */
+    protected int plcBatchSize = 1;
+
+    /** Policy max memory size. */
+    protected long plcMaxMemSize = 0;
+
+    /** Near policy max. */
+    protected int nearMax = 3;
+
+    /** Synchronous commit. */
+    protected boolean syncCommit;
+
+    /** */
+    protected int gridCnt = 2;
+
+    /** */
+    protected EvictionFilter<?, ?> filter;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration c = super.getConfiguration(gridName);
+
+        CacheConfiguration cc = defaultCacheConfiguration();
+
+        cc.setCacheMode(mode);
+        cc.setEvictionPolicy(createPolicy(plcMax));
+        cc.setEvictSynchronized(evictSync);
+        cc.setSwapEnabled(false);
+        cc.setWriteSynchronizationMode(syncCommit ? FULL_SYNC : FULL_ASYNC);
+        cc.setStartSize(plcMax);
+        cc.setAtomicityMode(TRANSACTIONAL);
+
+        if (nearEnabled) {
+            NearCacheConfiguration nearCfg = new NearCacheConfiguration();
+
+            nearCfg.setNearEvictionPolicy(createNearPolicy(nearMax));
+
+            cc.setNearConfiguration(nearCfg);
+        }
+        else
+            cc.setNearConfiguration(null);
+
+        if (mode == PARTITIONED)
+            cc.setBackups(1);
+
+        if (filter != null)
+            cc.setEvictionFilter(filter);
+
+        c.setCacheConfiguration(cc);
+
+        TcpDiscoverySpi disco = new TcpDiscoverySpi();
+
+        disco.setIpFinder(ipFinder);
+
+        c.setDiscoverySpi(disco);
+
+        c.setIncludeEventTypes(EVT_TASK_FAILED, EVT_TASK_FINISHED, EVT_JOB_MAPPED);
+
+        c.setIncludeProperties();
+
+        return c;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        filter = null;
+
+        super.afterTestsStopped();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMaxSizePolicy() throws Exception {
+        plcMax = 3;
+        plcMaxMemSize = 0;
+        plcBatchSize = 1;
+
+        doTestPolicy();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMaxSizePolicyWithBatch() throws Exception {
+        plcMax = 3;
+        plcMaxMemSize = 0;
+        plcBatchSize = 2;
+
+        doTestPolicyWithBatch();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMaxMemSizePolicy() throws Exception {
+        plcMax = 0;
+        plcMaxMemSize = 3 * MockEntry.ENTRY_SIZE;
+        plcBatchSize = 1;
+
+        doTestPolicy();
+    }
+
+    /**
+     * Batch ignored when {@code maxSize > 0} and {@code maxMemSize > 0}.
+     *
+     * @throws Exception If failed.
+     */
+    public void testMaxMemSizePolicyWithBatch() throws Exception {
+        plcMax = 3;
+        plcMaxMemSize = 10 * MockEntry.ENTRY_SIZE;
+        plcBatchSize = 2;
+
+        doTestPolicy();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMaxSizeMemory() throws Exception {
+        int max = 10;
+
+        plcMax = max;
+        plcMaxMemSize = 0;
+        plcBatchSize = 1;
+
+        doTestMemory(max);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMaxSizeMemoryWithBatch() throws Exception {
+        int max = 10;
+
+        plcMax = max;
+        plcMaxMemSize = 0;
+        plcBatchSize = 2;
+
+        doTestMemory(max);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMaxMemSizeMemory() throws Exception {
+        int max = 10;
+
+        plcMax = 0;
+        plcMaxMemSize = max * MockEntry.ENTRY_SIZE;
+        plcBatchSize = 1;
+
+        doTestMemory(max);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMaxSizeRandom() throws Exception {
+        plcMax = 10;
+        plcMaxMemSize = 0;
+        plcBatchSize = 1;
+
+        doTestRandom();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMaxSizeRandomWithBatch() throws Exception {
+        plcMax = 10;
+        plcMaxMemSize = 0;
+        plcBatchSize = 2;
+
+        doTestRandom();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMaxMemSizeRandom() throws Exception {
+        plcMax = 0;
+        plcMaxMemSize = 10 * MockEntry.KEY_SIZE;
+        plcBatchSize = 1;
+
+        doTestRandom();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMaxSizeAllowEmptyEntries() throws Exception {
+        plcMax = 10;
+        plcMaxMemSize = 0;
+        plcBatchSize = 1;
+
+        doTestAllowEmptyEntries();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMaxSizeAllowEmptyEntriesWithBatch() throws Exception {
+        plcMax = 10;
+        plcMaxMemSize = 0;
+        plcBatchSize = 2;
+
+        doTestAllowEmptyEntries();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMaxMemSizeAllowEmptyEntries() throws Exception {
+        plcMax = 0;
+        plcMaxMemSize = 10 * MockEntry.KEY_SIZE;
+        plcBatchSize = 1;
+
+        doTestAllowEmptyEntries();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMaxSizePut() throws Exception {
+        plcMax = 100;
+        plcBatchSize = 1;
+        plcMaxMemSize = 0;
+
+        doTestPut(plcMax);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMaxSizePutWithBatch() throws Exception {
+        plcMax = 100;
+        plcBatchSize = 2;
+        plcMaxMemSize = 0;
+
+        doTestPut(plcMax);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMaxMemSizePut() throws Exception {
+        int max = 100;
+
+        plcMax = 0;
+        plcBatchSize = 2;
+        plcMaxMemSize = max * PUT_ENTRY_SIZE;
+
+        doTestPut(max);
+    }
+
+    /**
+     * Tests policy behaviour.
+     *
+     * @throws Exception If failed.
+     */
+    protected abstract void doTestPolicy() throws Exception;
+
+    /**
+     * Tests policy behaviour with batch enabled.
+     *
+     * @throws Exception If failed.
+     */
+    protected abstract void doTestPolicyWithBatch() throws Exception;
+
+    /**
+     * @throws Exception If failed.
+     */
+    protected void doTestAllowEmptyEntries() throws Exception {
+        try {
+            startGrid();
+
+            MockEntry e1 = new MockEntry("1");
+            MockEntry e2 = new MockEntry("2");
+            MockEntry e3 = new MockEntry("3");
+            MockEntry e4 = new MockEntry("4");
+            MockEntry e5 = new MockEntry("5");
+
+            EvictionPolicyProxy p = proxy(policy());
+
+            p.onEntryAccessed(false, e1);
+
+            assertFalse(e1.isEvicted());
+
+            check(p.queue().size(), MockEntry.KEY_SIZE);
+
+            p.onEntryAccessed(false, e2);
+
+            assertFalse(e1.isEvicted());
+            assertFalse(e2.isEvicted());
+
+            check(p.queue().size(), MockEntry.KEY_SIZE);
+
+            p.onEntryAccessed(false, e3);
+
+            assertFalse(e1.isEvicted());
+            assertFalse(e3.isEvicted());
+
+            check(p.queue().size(), MockEntry.KEY_SIZE);
+
+            p.onEntryAccessed(false, e4);
+
+            assertFalse(e1.isEvicted());
+            assertFalse(e3.isEvicted());
+            assertFalse(e4.isEvicted());
+
+            check(p.queue().size(), MockEntry.KEY_SIZE);
+
+            p.onEntryAccessed(false, e5);
+
+            assertFalse(e1.isEvicted());
+            assertFalse(e3.isEvicted());
+            assertFalse(e5.isEvicted());
+
+            check(p.queue().size(), MockEntry.KEY_SIZE);
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    protected void doTestMemory(int max) throws Exception {
+        try {
+            startGrid();
+
+            EvictionPolicyProxy p = proxy(policy());
+
+            int cnt = max + plcBatchSize;
+
+            for (int i = 0; i < cnt; i++)
+                p.onEntryAccessed(false, new MockEntry(Integer.toString(i), Integer.toString(i)));
+
+            info(p);
+
+            check(max, MockEntry.ENTRY_SIZE);
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    protected void doTestRandom() throws Exception {
+        try {
+            startGrid();
+
+            EvictionPolicyProxy p = proxy(policy());
+
+            int max = 10;
+
+            Random rand = new Random();
+
+            int keys = 31;
+
+            MockEntry[] entries = new MockEntry[keys];
+
+            for (int i = 0; i < entries.length; i++)
+                entries[i] = new MockEntry(Integer.toString(i));
+
+            int runs = 5000000;
+
+            for (int i = 0; i < runs; i++) {
+                boolean rmv = rand.nextBoolean();
+
+                int j = rand.nextInt(entries.length);
+
+                MockEntry e = entry(entries, j);
+
+                if (rmv)
+                    entries[j] = new MockEntry(Integer.toString(j));
+
+                p.onEntryAccessed(rmv, e);
+            }
+
+            info(p);
+
+            assertTrue(p.getCurrentSize() <= (plcMaxMemSize > 0 ? max : max + plcBatchSize));
+            assertTrue(p.getCurrentMemSize() <= (plcMaxMemSize > 0 ? max : max + plcBatchSize) * MockEntry.KEY_SIZE);
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    protected void doTestPut(int max) throws Exception {
+        mode = LOCAL;
+        syncCommit = true;
+
+        try {
+            Ignite ignite = startGrid();
+
+            IgniteCache<Object, Object> cache = ignite.cache(null);
+
+            int cnt = 500;
+
+            int min = Integer.MAX_VALUE;
+
+            int minIdx = 0;
+
+            for (int i = 0; i < cnt; i++) {
+                cache.put(i, i);
+
+                int cacheSize = cache.size();
+
+                if (i > max && cacheSize < min) {
+                    min = cacheSize;
+                    minIdx = i;
+                }
+            }
+
+            assertTrue("Min cache size is too small: " + min, min >= max);
+
+            check(max, PUT_ENTRY_SIZE);
+
+            info("Min cache size [min=" + min + ", idx=" + minIdx + ']');
+            info("Current cache size " + cache.size());
+            info("Current cache key size " + cache.size());
+
+            min = Integer.MAX_VALUE;
+
+            minIdx = 0;
+
+            // Touch.
+            for (int i = cnt; --i > cnt - max;) {
+                cache.get(i);
+
+                int cacheSize = cache.size();
+
+                if (cacheSize < min) {
+                    min = cacheSize;
+                    minIdx = i;
+                }
+            }
+
+            info("----");
+            info("Min cache size [min=" + min + ", idx=" + minIdx + ']');
+            info("Current cache size " + cache.size());
+            info("Current cache key size " + cache.size());
+
+            check(max, PUT_ENTRY_SIZE);
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
+
+    /**
+     * @param arr Array.
+     * @param idx Index.
+     * @return Entry at the index.
+     */
+    protected MockEntry entry(MockEntry[] arr, int idx) {
+        MockEntry e = arr[idx];
+
+        if (e.isEvicted())
+            e = arr[idx] = new MockEntry(e.getKey());
+
+        return e;
+    }
+
+    /**
+     * @param prefix Prefix.
+     * @param p Policy.
+     */
+    protected void info(String prefix, EvictionPolicy<?, ?> p) {
+        info(prefix + ": " + p.toString());
+    }
+
+    /** @param p Policy. */
+    protected void info(EvictionPolicy<?, ?> p) {
+        info(p.toString());
+    }
+
+    /**
+     * @param c1 Policy collection.
+     * @param c2 Expected list.
+     */
+    protected static void check(Collection<EvictableEntry<String, String>> c1, MockEntry... c2) {
+        check(c1, F.asList(c2));
+    }
+
+    /**
+     * @param expSize Expected size.
+     * @param entrySize Entry size.
+     */
+    protected void check(int expSize, int entrySize) {
+        EvictionPolicyProxy proxy = proxy(policy());
+
+        assertEquals(expSize, proxy.getCurrentSize());
+        assertEquals(expSize * entrySize, proxy.getCurrentMemSize());
+    }
+
+    /**
+     * @param entrySize Entry size.
+     * @param c1 Closure 1.
+     * @param c2 Closure 2.
+     */
+    protected void check(int entrySize, Collection<EvictableEntry<String, String>> c1, MockEntry... c2) {
+        check(c2.length, entrySize);
+
+        check(c1, c2);
+    }
+
+    /** @return Policy. */
+    @SuppressWarnings({"unchecked"})
+    protected T policy() {
+        return (T)grid().cache(null).getConfiguration(CacheConfiguration.class).getEvictionPolicy();
+    }
+
+    /**
+     * @param i Grid index.
+     * @return Policy.
+     */
+    @SuppressWarnings({"unchecked"})
+    protected T policy(int i) {
+        return (T)grid(i).cache(null).getConfiguration(CacheConfiguration.class).getEvictionPolicy();
+    }
+
+    /**
+     * @param i Grid index.
+     * @return Policy.
+     */
+    @SuppressWarnings({"unchecked"})
+    protected T nearPolicy(int i) {
+        CacheConfiguration cfg = grid(i).cache(null).getConfiguration(CacheConfiguration.class);
+
+        NearCacheConfiguration nearCfg = cfg.getNearConfiguration();
+
+        return (T)(nearCfg == null ? null : nearCfg.getNearEvictionPolicy());
+    }
+
+    /**
+     * @param c1 Policy collection.
+     * @param c2 Expected list.
+     */
+    protected static void check(Collection<EvictableEntry<String, String>> c1, List<MockEntry> c2) {
+        assert c1.size() == c2.size() : "Mismatch [actual=" + string(c1) + ", expected=" + string(c2) + ']';
+
+        assert c1.containsAll(c2) : "Mismatch [actual=" + string(c1) + ", expected=" + string(c2) + ']';
+
+        int i = 0;
+
+        // Check order.
+        for (Cache.Entry<String, String> e : c1)
+            assertEquals(e, c2.get(i++));
+    }
+
+    /**
+     * @param c Collection.
+     * @return String.
+     */
+    @SuppressWarnings("unchecked")
+    protected static String string(Iterable<? extends Cache.Entry> c) {
+        return "[" +
+            F.fold(
+                c,
+                "",
+                new C2<Cache.Entry, String, String>() {
+                    @Override public String apply(Cache.Entry e, String b) {
+                        return b.isEmpty() ? e.getKey().toString() : b + ", " + e.getKey();
+                    }
+                }) +
+            "]]";
+    }
+
+    /** @throws Exception If failed. */
+    public void testMaxSizePartitionedNearDisabled() throws Exception {
+        mode = PARTITIONED;
+        nearEnabled = false;
+        plcMax = 10;
+        syncCommit = true;
+
+        gridCnt = 2;
+
+        checkPartitioned();
+    }
+
+    /** @throws Exception If failed. */
+    public void testMaxSizePartitionedNearDisabledWithBatch() throws Exception {
+        mode = PARTITIONED;
+        nearEnabled = false;
+        plcMax = 10;
+        plcBatchSize = 2;
+        syncCommit = true;
+
+        gridCnt = 2;
+
+        checkPartitioned();
+    }
+
+    /** @throws Exception If failed. */
+    public void testMaxMemSizePartitionedNearDisabled() throws Exception {
+        mode = PARTITIONED;
+        nearEnabled = false;
+        plcMax = 0;
+        plcMaxMemSize = 100;
+        syncCommit = true;
+
+        gridCnt = 2;
+
+        checkPartitioned();
+    }
+
+    /** @throws Exception If failed. */
+    public void testPartitionedNearEnabled() throws Exception {
+        mode = PARTITIONED;
+        nearEnabled = true;
+        nearMax = 3;
+        plcMax = 10;
+        evictNearSync = true;
+        syncCommit = true;
+
+        gridCnt = 2;
+
+        checkPartitioned(); // Near size is 0 because of backups present.
+    }
+
+    /** @throws Exception If failed. */
+    public void testPartitionedNearDisabledMultiThreaded() throws Exception {
+        mode = PARTITIONED;
+        nearEnabled = false;
+        plcMax = 100;
+        evictSync = false;
+
+        gridCnt = 2;
+
+        checkPartitionedMultiThreaded();
+    }
+
+    /** @throws Exception If failed. */
+    public void testPartitionedNearDisabledBackupSyncMultiThreaded() throws Exception {
+        mode = PARTITIONED;
+        nearEnabled = false;
+        plcMax = 100;
+        evictSync = true;
+
+        gridCnt = 2;
+
+        checkPartitionedMultiThreaded();
+    }
+
+    /** @throws Exception If failed. */
+    public void testPartitionedNearEnabledMultiThreaded() throws Exception {
+        mode = PARTITIONED;
+        nearEnabled = true;
+        plcMax = 10;
+        evictSync = false;
+
+        gridCnt = 2;
+
+        checkPartitionedMultiThreaded();
+    }
+
+    /** @throws Exception If failed. */
+    public void testPartitionedNearEnabledBackupSyncMultiThreaded() throws Exception {
+        mode = PARTITIONED;
+        nearEnabled = true;
+        plcMax = 10;
+        evictSync = true;
+
+        gridCnt = 2;
+
+        checkPartitionedMultiThreaded();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    protected void checkPartitioned() throws Exception {
+
+        int endSize = nearEnabled ? 0 : plcMax;
+
+        int endPlcSize = nearEnabled ? 0 : plcMax;
+
+        startGridsMultiThreaded(gridCnt);
+
+        try {
+            Random rand = new Random();
+
+            int cnt = 500;
+
+            for (int i = 0; i < cnt; i++) {
+                IgniteCache<Integer, String> cache = grid(rand.nextInt(2)).cache(null);
+
+                int key = rand.nextInt(100);
+                String val = Integer.toString(key);
+
+                cache.put(key, val);
+
+                if (i % 100 == 0)
+                    info("Stored cache object for key [key=" + key + ", idx=" + i + ']');
+            }
+
+            if (nearEnabled) {
+                for (int i = 0; i < gridCnt; i++)
+                    assertEquals(endSize, near(i).nearSize());
+
+                if (endPlcSize >= 0)
+                    checkNearPolicies(endPlcSize);
+            }
+            else {
+                if (plcMaxMemSize > 0) {
+                    for (int i = 0; i < gridCnt; i++) {
+                        GridDhtColocatedCache<Object, Object> cache = colocated(i);
+
+                        int memSize = 0;
+
+                        for (Cache.Entry<Object, Object> entry : cache.entrySet())
+                            memSize += entry.unwrap(EvictableEntry.class).size();
+
+                        EvictionPolicyProxy plc = proxy(policy(i));
+
+                        assertTrue(plc.getCurrentMemSize() <= memSize);
+                    }
+                }
+
+                if (plcMax > 0) {
+                    for (int i = 0; i < gridCnt; i++) {
+                        int actual = colocated(i).size();
+
+                        assertTrue("Cache size is greater then policy size [expected=" + endSize + ", actual=" + actual + ']',
+                            actual <= endSize + (plcMaxMemSize > 0 ? 1 : plcBatchSize));
+                    }
+                }
+
+                checkPolicies();
+            }
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    protected void checkPartitionedMultiThreaded() throws Exception {
+        try {
+            startGridsMultiThreaded(gridCnt);
+
+            final Random rand = new Random();
+
+            final AtomicInteger cntr = new AtomicInteger();
+
+            multithreaded(new Callable() {
+                @Nullable @Override public Object call() throws Exception {
+                    int cnt = 100;
+
+                    for (int i = 0; i < cnt && !Thread.currentThread().isInterrupted(); i++) {
+                        IgniteEx grid = grid(rand.nextInt(2));
+
+                        IgniteCache<Integer, String> cache = grid.cache(null);
+
+                        int key = rand.nextInt(1000);
+                        String val = Integer.toString(key);
+
+                        try (Transaction tx = grid.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+                            String v = cache.get(key);
+
+                            assert v == null || v.equals(Integer.toString(key)) : "Invalid value for key [key=" + key +
+                                ", val=" + v + ']';
+
+                            cache.put(key, val);
+
+                            tx.commit();
+                        }
+
+                        if (cntr.incrementAndGet() % 100 == 0)
+                            info("Stored cache object for key [key=" + key + ", idx=" + i + ']');
+                    }
+
+                    return null;
+                }
+            }, 10);
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
+
+    /**
+     * @param plcMax Policy max.
+     * @return Policy.
+     */
+    protected abstract T createPolicy(int plcMax);
+
+    /**
+     * @param nearMax Near max.
+     * @return Policy.
+     */
+    protected abstract T createNearPolicy(int nearMax);
+
+    /**
+     * Performs after-test near policy check.
+     *
+     * @param nearMax Near max.
+     */
+    protected void checkNearPolicies(int nearMax) {
+        for (int i = 0; i < gridCnt; i++) {
+
+            EvictionPolicyProxy proxy = proxy(nearPolicy(i));
+
+            for (EvictableEntry e : proxy.queue())
+                assert !e.isCached() : "Invalid near policy size: " + proxy.queue();
+        }
+    }
+
+    /**
+     * Performs after-test policy check.
+     */
+    protected void checkPolicies() {
+        for (int i = 0; i < gridCnt; i++) {
+            if (plcMaxMemSize > 0) {
+                int size = 0;
+
+                for (EvictableEntry entry : proxy(policy(i)).queue())
+                    size += entry.size();
+
+                assertEquals(size, proxy(policy(i)).getCurrentMemSize());
+            }
+            else
+                assertTrue(proxy(policy(i)).queue().size() <= plcMax + plcBatchSize);
+        }
+    }
+
+    /**
+     *
+     */
+    @SuppressWarnings({"PublicConstructorInNonPublicClass"})
+    protected static class MockEntry extends GridCacheMockEntry<String, String> {
+        /** Key size. */
+        public static final int KEY_SIZE = 1;
+
+        /** Value size. */
+        public static final int VALUE_SIZE = 1;
+
+        /** Entry size. */
+        public static final int ENTRY_SIZE = KEY_SIZE + VALUE_SIZE;
+
+        /** */
+        private IgniteCache<String, String> parent;
+
+        /** Entry value. */
+        private String val;
+
+        /** @param key Key. */
+        public MockEntry(String key) {
+            super(key);
+        }
+
+        /**
+         * @param key Key.
+         * @param val Value.
+         */
+        public MockEntry(String key, String val) {
+            super(key);
+
+            this.val = val;
+        }
+
+        /**
+         * @param key Key.
+         * @param parent Parent.
+         */
+        public MockEntry(String key, @Nullable IgniteCache<String, String> parent) {
+            super(key);
+
+            this.parent = parent;
+        }
+
+        /** {@inheritDoc} */
+        @SuppressWarnings("unchecked")
+        @Override public <T> T unwrap(Class<T> clazz) {
+            if (clazz.isAssignableFrom(IgniteCache.class))
+                return (T)parent;
+
+            return super.unwrap(clazz);
+        }
+
+        /** {@inheritDoc} */
+        @Override public String getValue() throws IllegalStateException {
+            return val;
+        }
+
+        /** {@inheritDoc} */
+        @Override public int size() {
+            return val == null ? KEY_SIZE : ENTRY_SIZE;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(MockEntry.class, this, super.toString());
+        }
+    }
+
+    /**
+     * Rvicition policy proxy.
+     */
+    public static class EvictionPolicyProxy implements EvictionPolicy {
+        /** Policy. */
+        private final EvictionPolicy plc;
+
+        /**
+         * @param plc Policy.
+         */
+        private EvictionPolicyProxy(EvictionPolicy plc) {
+            this.plc = plc;
+        }
+
+        /**
+         * @param plc Policy.
+         */
+        public static EvictionPolicyProxy proxy(EvictionPolicy plc) {
+            return new EvictionPolicyProxy(plc);
+        }
+
+        /**
+         * Get current size.
+         */
+        public int getCurrentSize() {
+            try {
+                return (Integer)plc.getClass().getDeclaredMethod("getCurrentSize").invoke(plc);
+            }
+            catch (NoSuchMethodException | InvocationTargetException | IllegalAccessException e) {
+                throw new RuntimeException(e);
+            }
+        }
+
+        /**
+         * Current memory size.
+         */
+        public long getCurrentMemSize() {
+            try {
+                return (Long)plc.getClass().getDeclaredMethod("getCurrentMemSize").invoke(plc);
+            }
+            catch (NoSuchMethodException | InvocationTargetException | IllegalAccessException e) {
+                throw new RuntimeException(e);
+            }
+        }
+
+        /**
+         * Current queue.
+         */
+        public Collection<EvictableEntry> queue() {
+            try {
+                return (Collection<EvictableEntry>)plc.getClass().getDeclaredMethod("queue").invoke(plc);
+            }
+            catch (NoSuchMethodException | InvocationTargetException | IllegalAccessException e) {
+                throw new RuntimeException(e);
+            }
+        }
+
+        /**
+         * @param rmv Remove.
+         * @param entry Entry.
+         */
+        @Override public void onEntryAccessed(boolean rmv, EvictableEntry entry) {
+            try {
+                plc.getClass()
+                    .getDeclaredMethod("onEntryAccessed", boolean.class, EvictableEntry.class)
+                    .invoke(plc, rmv, entry);
+            }
+            catch (NoSuchMethodException | InvocationTargetException | IllegalAccessException e) {
+                throw new RuntimeException(e);
+            }
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e5d5d08d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheBatchEvictUnswapSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheBatchEvictUnswapSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheBatchEvictUnswapSelfTest.java
index 36598fd..da5017a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheBatchEvictUnswapSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheBatchEvictUnswapSelfTest.java
@@ -90,7 +90,10 @@ public class GridCacheBatchEvictUnswapSelfTest extends GridCacheAbstractSelfTest
         cacheCfg.setWriteThrough(true);
         cacheCfg.setLoadPreviousValue(true);
 
-        cacheCfg.setEvictionPolicy(new FifoEvictionPolicy(EVICT_PLC_SIZE));
+        FifoEvictionPolicy plc = new FifoEvictionPolicy();
+        plc.setMaxSize(EVICT_PLC_SIZE);
+
+        cacheCfg.setEvictionPolicy(plc);
         cacheCfg.setSwapEnabled(true);
         cacheCfg.setEvictSynchronized(false);
         cacheCfg.setNearConfiguration(null);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e5d5d08d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheConcurrentEvictionConsistencySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheConcurrentEvictionConsistencySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheConcurrentEvictionConsistencySelfTest.java
index 4daa908..921422e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheConcurrentEvictionConsistencySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheConcurrentEvictionConsistencySelfTest.java
@@ -21,6 +21,7 @@ import org.apache.ignite.*;
 import 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.sorted.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
@@ -100,7 +101,10 @@ public class GridCacheConcurrentEvictionConsistencySelfTest extends GridCommonAb
      * @throws Exception If failed.
      */
     public void testPolicyConsistencyFifoLocalTwoKeys() throws Exception {
-        plc = new FifoEvictionPolicy<Object, Object>(1);
+        FifoEvictionPolicy<Object, Object> plc = new FifoEvictionPolicy<>();
+        plc.setMaxSize(1);
+
+        this.plc = plc;
 
         keyCnt = 2;
         threadCnt = Runtime.getRuntime().availableProcessors() / 2;
@@ -112,7 +116,25 @@ public class GridCacheConcurrentEvictionConsistencySelfTest extends GridCommonAb
      * @throws Exception If failed.
      */
     public void testPolicyConsistencyLruLocalTwoKeys() throws Exception {
-        plc = new LruEvictionPolicy<Object, Object>(1);
+        LruEvictionPolicy<Object, Object> plc = new LruEvictionPolicy<>();
+        plc.setMaxSize(1);
+
+        this.plc = plc;
+
+        keyCnt = 2;
+        threadCnt = Runtime.getRuntime().availableProcessors() / 2;
+
+        checkPolicyConsistency();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPolicyConsistencySortedLocalTwoKeys() throws Exception {
+        SortedEvictionPolicy<Object, Object> plc = new SortedEvictionPolicy<>();
+        plc.setMaxSize(1);
+
+        this.plc = plc;
 
         keyCnt = 2;
         threadCnt = Runtime.getRuntime().availableProcessors() / 2;
@@ -124,7 +146,10 @@ public class GridCacheConcurrentEvictionConsistencySelfTest extends GridCommonAb
      * @throws Exception If failed.
      */
     public void testPolicyConsistencyFifoLocalFewKeys() throws Exception {
-        plc = new FifoEvictionPolicy<Object, Object>(POLICY_QUEUE_SIZE);
+        FifoEvictionPolicy<Object, Object> plc = new FifoEvictionPolicy<>();
+        plc.setMaxSize(POLICY_QUEUE_SIZE);
+
+        this.plc = plc;
 
         keyCnt = POLICY_QUEUE_SIZE + 5;
 
@@ -135,7 +160,24 @@ public class GridCacheConcurrentEvictionConsistencySelfTest extends GridCommonAb
      * @throws Exception If failed.
      */
     public void testPolicyConsistencyLruLocalFewKeys() throws Exception {
-        plc = new LruEvictionPolicy<Object, Object>(POLICY_QUEUE_SIZE);
+        LruEvictionPolicy<Object, Object> plc = new LruEvictionPolicy<>();
+        plc.setMaxSize(POLICY_QUEUE_SIZE);
+
+        this.plc = plc;
+
+        keyCnt = POLICY_QUEUE_SIZE + 5;
+
+        checkPolicyConsistency();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPolicyConsistencySortedLocalFewKeys() throws Exception {
+        SortedEvictionPolicy<Object, Object> plc = new SortedEvictionPolicy<>();
+        plc.setMaxSize(POLICY_QUEUE_SIZE);
+
+        this.plc = plc;
 
         keyCnt = POLICY_QUEUE_SIZE + 5;
 
@@ -146,7 +188,10 @@ public class GridCacheConcurrentEvictionConsistencySelfTest extends GridCommonAb
      * @throws Exception If failed.
      */
     public void testPolicyConsistencyFifoLocal() throws Exception {
-        plc = new FifoEvictionPolicy<Object, Object>(POLICY_QUEUE_SIZE);
+        FifoEvictionPolicy<Object, Object> plc = new FifoEvictionPolicy<>();
+        plc.setMaxSize(POLICY_QUEUE_SIZE);
+
+        this.plc = plc;
 
         keyCnt = POLICY_QUEUE_SIZE * 10;
 
@@ -157,7 +202,24 @@ public class GridCacheConcurrentEvictionConsistencySelfTest extends GridCommonAb
      * @throws Exception If failed.
      */
     public void testPolicyConsistencyLruLocal() throws Exception {
-        plc = new LruEvictionPolicy<Object, Object>(POLICY_QUEUE_SIZE);
+        LruEvictionPolicy<Object, Object> plc = new LruEvictionPolicy<>();
+        plc.setMaxSize(POLICY_QUEUE_SIZE);
+
+        this.plc = plc;
+
+        keyCnt = POLICY_QUEUE_SIZE * 10;
+
+        checkPolicyConsistency();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPolicyConsistencySortedLocal() throws Exception {
+        SortedEvictionPolicy<Object, Object> plc = new SortedEvictionPolicy<>();
+        plc.setMaxSize(POLICY_QUEUE_SIZE);
+
+        this.plc = plc;
 
         keyCnt = POLICY_QUEUE_SIZE * 10;
 
@@ -177,8 +239,7 @@ public class GridCacheConcurrentEvictionConsistencySelfTest extends GridCommonAb
 
             IgniteInternalFuture<?> fut = multithreadedAsync(
                 new Callable<Object>() {
-                    @Override
-                    public Object call() throws Exception {
+                    @Override public Object call() throws Exception {
                         final Random rnd = new Random();
 
                         for (int i = 0; i < ITERATION_CNT; i++) {
@@ -258,6 +319,11 @@ public class GridCacheConcurrentEvictionConsistencySelfTest extends GridCommonAb
 
             return plc0.queue();
         }
+        else if (plc instanceof SortedEvictionPolicy) {
+            SortedEvictionPolicy<Integer, Integer> plc0 = (SortedEvictionPolicy<Integer, Integer>)plc;
+
+            return plc0.queue();
+        }
 
         assert false : "Unexpected policy type: " + plc.getClass().getName();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e5d5d08d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheConcurrentEvictionsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheConcurrentEvictionsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheConcurrentEvictionsSelfTest.java
index 3cadc45..270f991 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheConcurrentEvictionsSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheConcurrentEvictionsSelfTest.java
@@ -22,6 +22,7 @@ import org.apache.ignite.cache.*;
 import 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.sorted.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.spi.discovery.tcp.*;
@@ -98,7 +99,11 @@ public class GridCacheConcurrentEvictionsSelfTest extends GridCommonAbstractTest
      */
     public void testConcurrentPutsFifoLocal() throws Exception {
         mode = LOCAL;
-        plc = new FifoEvictionPolicy<Object, Object>(1000);
+
+        FifoEvictionPolicy plc = new FifoEvictionPolicy();
+        plc.setMaxSize(1000);
+
+        this.plc = plc;
         warmUpPutsCnt = 100000;
         iterCnt = 100000;
 
@@ -110,7 +115,27 @@ public class GridCacheConcurrentEvictionsSelfTest extends GridCommonAbstractTest
      */
     public void testConcurrentPutsLruLocal() throws Exception {
         mode = LOCAL;
-        plc = new LruEvictionPolicy<Object, Object>(1000);
+
+        LruEvictionPolicy plc = new LruEvictionPolicy();
+        plc.setMaxSize(1000);
+
+        this.plc = plc;
+        warmUpPutsCnt = 100000;
+        iterCnt = 100000;
+
+        checkConcurrentPuts();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testConcurrentPutsSortedLocal() throws Exception {
+        mode = LOCAL;
+
+        SortedEvictionPolicy plc = new SortedEvictionPolicy();
+        plc.setMaxSize(1000);
+
+        this.plc = plc;
         warmUpPutsCnt = 100000;
         iterCnt = 100000;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e5d5d08d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheDistributedEvictionsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheDistributedEvictionsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheDistributedEvictionsSelfTest.java
index 396f360..75c4dfe 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheDistributedEvictionsSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheDistributedEvictionsSelfTest.java
@@ -86,7 +86,10 @@ public class GridCacheDistributedEvictionsSelfTest extends GridCommonAbstractTes
         cc.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
 
         // Set only DHT policy, leave default near policy.
-        cc.setEvictionPolicy(new FifoEvictionPolicy<>(10));
+        FifoEvictionPolicy plc = new FifoEvictionPolicy();
+        plc.setMaxSize(10);
+
+        cc.setEvictionPolicy(plc);
         cc.setEvictSynchronized(evictSync);
         cc.setEvictSynchronizedKeyBufferSize(1);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e5d5d08d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheEmptyEntriesAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheEmptyEntriesAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheEmptyEntriesAbstractSelfTest.java
index 768bced..8e17eed 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheEmptyEntriesAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheEmptyEntriesAbstractSelfTest.java
@@ -120,14 +120,19 @@ public abstract class GridCacheEmptyEntriesAbstractSelfTest extends GridCommonAb
      * @throws Exception If failed.
      */
     public void testFifo() throws Exception {
-        plc = new FifoEvictionPolicy(50);
-        nearPlc = new FifoEvictionPolicy(50);
+        FifoEvictionPolicy plc = new FifoEvictionPolicy();
+        plc.setMaxSize(50);
+        this.plc = plc;
+
+        FifoEvictionPolicy nearPlc = new FifoEvictionPolicy();
+        nearPlc.setMaxSize(50);
+        this.nearPlc = nearPlc;
 
         checkPolicy();
     }
 
     /**
-     * Checks policy with and without store set.
+     * Checks policy with and without store queue.
      *
      * @throws Exception If failed.
      */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e5d5d08d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheEvictionAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheEvictionAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheEvictionAbstractTest.java
deleted file mode 100644
index e0dab7d..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheEvictionAbstractTest.java
+++ /dev/null
@@ -1,484 +0,0 @@
-/*
- * 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.eviction;
-
-import org.apache.ignite.*;
-import org.apache.ignite.cache.*;
-import org.apache.ignite.cache.eviction.*;
-import org.apache.ignite.configuration.*;
-import org.apache.ignite.internal.*;
-import org.apache.ignite.internal.util.typedef.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-import org.apache.ignite.spi.discovery.tcp.*;
-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 javax.cache.*;
-import java.util.*;
-import java.util.concurrent.*;
-import java.util.concurrent.atomic.*;
-
-import static org.apache.ignite.cache.CacheAtomicityMode.*;
-import static org.apache.ignite.cache.CacheMode.*;
-import static org.apache.ignite.cache.CacheWriteSynchronizationMode.*;
-import static org.apache.ignite.events.EventType.*;
-import static org.apache.ignite.transactions.TransactionConcurrency.*;
-import static org.apache.ignite.transactions.TransactionIsolation.*;
-
-/**
- * Base class for eviction tests.
- */
-public abstract class GridCacheEvictionAbstractTest<T extends EvictionPolicy<?, ?>>
-    extends GridCommonAbstractTest {
-    /** IP finder. */
-    protected static final TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
-
-    /** Replicated cache. */
-    protected CacheMode mode = REPLICATED;
-
-    /** Near enabled flag. */
-    protected boolean nearEnabled;
-
-    /** Evict backup sync. */
-    protected boolean evictSync;
-
-    /** Evict near sync. */
-    protected boolean evictNearSync = true;
-
-    /** Policy batch size. */
-    protected int plcBatchSize = 0;
-
-    /** Policy max. */
-    protected int plcMax = 10;
-
-    /** Near policy max. */
-    protected int nearMax = 3;
-
-    /** Synchronous commit. */
-    protected boolean syncCommit;
-
-    /** */
-    protected int gridCnt = 2;
-
-    /** */
-    protected EvictionFilter<?, ?> filter;
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        IgniteConfiguration c = super.getConfiguration(gridName);
-
-        CacheConfiguration cc = defaultCacheConfiguration();
-
-        cc.setCacheMode(mode);
-        cc.setEvictionPolicy(createPolicy(plcMax));
-        cc.setEvictSynchronized(evictSync);
-        cc.setSwapEnabled(false);
-        cc.setWriteSynchronizationMode(syncCommit ? FULL_SYNC : FULL_ASYNC);
-        cc.setStartSize(plcMax);
-        cc.setAtomicityMode(TRANSACTIONAL);
-
-        if (nearEnabled) {
-            NearCacheConfiguration nearCfg = new NearCacheConfiguration();
-
-            nearCfg.setNearEvictionPolicy(createNearPolicy(nearMax));
-
-            cc.setNearConfiguration(nearCfg);
-        }
-        else
-            cc.setNearConfiguration(null);
-
-        if (mode == PARTITIONED)
-            cc.setBackups(1);
-
-        if (filter != null)
-            cc.setEvictionFilter(filter);
-
-        c.setCacheConfiguration(cc);
-
-        TcpDiscoverySpi disco = new TcpDiscoverySpi();
-
-        disco.setIpFinder(ipFinder);
-
-        c.setDiscoverySpi(disco);
-
-        c.setIncludeEventTypes(EVT_TASK_FAILED, EVT_TASK_FINISHED, EVT_JOB_MAPPED);
-
-        c.setIncludeProperties();
-
-        return c;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTestsStopped() throws Exception {
-        filter = null;
-
-        super.afterTestsStopped();
-    }
-
-    /**
-     * @param arr Array.
-     * @param idx Index.
-     * @return Entry at the index.
-     */
-    protected MockEntry entry(MockEntry[] arr, int idx) {
-        MockEntry e = arr[idx];
-
-        if (e.isEvicted())
-            e = arr[idx] = new MockEntry(e.getKey());
-
-        return e;
-    }
-
-    /**
-     * @param prefix Prefix.
-     * @param p Policy.
-     */
-    protected void info(String prefix, EvictionPolicy<?, ?> p) {
-        info(prefix + ": " + p.toString());
-    }
-
-    /** @param p Policy. */
-    protected void info(EvictionPolicy<?, ?> p) {
-        info(p.toString());
-    }
-
-    /**
-     * @param c1 Policy collection.
-     * @param c2 Expected list.
-     */
-    protected void check(Collection<EvictableEntry<String, String>> c1, MockEntry... c2) {
-        check(c1, F.asList(c2));
-    }
-
-    /** @return Policy. */
-    @SuppressWarnings({"unchecked"})
-    protected T policy() {
-        return (T)grid().cache(null).getConfiguration(CacheConfiguration.class).getEvictionPolicy();
-    }
-
-    /**
-     * @param i Grid index.
-     * @return Policy.
-     */
-    @SuppressWarnings({"unchecked"})
-    protected T policy(int i) {
-        return (T)grid(i).cache(null).getConfiguration(CacheConfiguration.class).getEvictionPolicy();
-    }
-
-    /**
-     * @param i Grid index.
-     * @return Policy.
-     */
-    @SuppressWarnings({"unchecked"})
-    protected T nearPolicy(int i) {
-        CacheConfiguration cfg = grid(i).cache(null).getConfiguration(CacheConfiguration.class);
-
-        NearCacheConfiguration nearCfg = cfg.getNearConfiguration();
-
-        return (T)(nearCfg == null ? null : nearCfg.getNearEvictionPolicy());
-    }
-
-    /**
-     * @param c1 Policy collection.
-     * @param c2 Expected list.
-     */
-    protected void check(Collection<EvictableEntry<String, String>> c1, List<MockEntry> c2) {
-        assert c1.size() == c2.size() : "Mismatch [actual=" + string(c1) + ", expected=" + string(c2) + ']';
-
-        assert c1.containsAll(c2) : "Mismatch [actual=" + string(c1) + ", expected=" + string(c2) + ']';
-
-        int i = 0;
-
-        // Check order.
-        for (Cache.Entry<String, String> e : c1)
-            assertEquals(e, c2.get(i++));
-    }
-
-    /**
-     * @param c Collection.
-     * @return String.
-     */
-    @SuppressWarnings("unchecked")
-    protected String string(Iterable<? extends Cache.Entry> c) {
-        return "[" +
-            F.fold(
-                c,
-                "",
-                new C2<Cache.Entry, String, String>() {
-                    @Override public String apply(Cache.Entry e, String b) {
-                        return b.isEmpty() ? e.getKey().toString() : b + ", " + e.getKey();
-                    }
-                }) +
-            "]]";
-    }
-
-    /** @throws Exception If failed. */
-    public void testPartitionedNearDisabled() throws Exception {
-        mode = PARTITIONED;
-        nearEnabled = false;
-        plcMax = 10;
-        syncCommit = true;
-
-        gridCnt = 2;
-
-        checkPartitioned(plcMax, plcMax, false);
-    }
-
-    /** @throws Exception If failed. */
-    public void testPartitionedNearEnabled() throws Exception {
-        mode = PARTITIONED;
-        nearEnabled = true;
-        nearMax = 3;
-        plcMax = 10;
-        evictNearSync = true;
-        syncCommit = true;
-
-        gridCnt = 2;
-
-        checkPartitioned(0, 0, true); // Near size is 0 because of backups present.
-    }
-
-    /** @throws Exception If failed. */
-    public void testPartitionedNearDisabledMultiThreaded() throws Exception {
-        mode = PARTITIONED;
-        nearEnabled = false;
-        plcMax = 100;
-        evictSync = false;
-
-        gridCnt = 2;
-
-        checkPartitionedMultiThreaded(gridCnt);
-    }
-
-    /** @throws Exception If failed. */
-    public void testPartitionedNearDisabledBackupSyncMultiThreaded() throws Exception {
-        mode = PARTITIONED;
-        nearEnabled = false;
-        plcMax = 100;
-        evictSync = true;
-
-        gridCnt = 2;
-
-        checkPartitionedMultiThreaded(gridCnt);
-    }
-
-    /** @throws Exception If failed. */
-    public void testPartitionedNearEnabledMultiThreaded() throws Exception {
-        mode = PARTITIONED;
-        nearEnabled = true;
-        plcMax = 10;
-        evictSync = false;
-
-        gridCnt = 2;
-
-        checkPartitionedMultiThreaded(gridCnt);
-    }
-
-    /** @throws Exception If failed. */
-    public void testPartitionedNearEnabledBackupSyncMultiThreaded() throws Exception {
-        mode = PARTITIONED;
-        nearEnabled = true;
-        plcMax = 10;
-        evictSync = true;
-
-        gridCnt = 2;
-
-        checkPartitionedMultiThreaded(gridCnt);
-    }
-
-    /**
-     * @param endSize Final near size.
-     * @param endPlcSize Final near policy size.
-     * @throws Exception If failed.
-     */
-    private void checkPartitioned(int endSize, int endPlcSize, boolean near) throws Exception {
-        startGridsMultiThreaded(gridCnt);
-
-        try {
-            Random rand = new Random();
-
-            int cnt = 500;
-
-            for (int i = 0; i < cnt; i++) {
-                IgniteCache<Integer, String> cache = grid(rand.nextInt(2)).cache(null);
-
-                int key = rand.nextInt(100);
-                String val = Integer.toString(key);
-
-                cache.put(key, val);
-
-                if (i % 100 == 0)
-                    info("Stored cache object for key [key=" + key + ", idx=" + i + ']');
-            }
-
-            if (near) {
-                for (int i = 0; i < gridCnt; i++)
-                    assertEquals(endSize, near(i).nearSize());
-
-                if (endPlcSize >= 0)
-                    checkNearPolicies(endPlcSize);
-            }
-            else {
-                for (int i = 0; i < gridCnt; i++) {
-                    int actual = colocated(i).size();
-
-                    assertTrue("Cache size is greater then policy size [expected=" + endSize + ", actual=" + actual + ']',
-                        actual <= endSize + plcBatchSize);
-                }
-
-                checkPolicies(endPlcSize);
-            }
-        }
-        finally {
-            stopAllGrids();
-        }
-    }
-
-    /**
-     * @param gridCnt Grid count.
-     * @throws Exception If failed.
-     */
-    protected void checkPartitionedMultiThreaded(int gridCnt) throws Exception {
-        try {
-            startGridsMultiThreaded(gridCnt);
-
-            final Random rand = new Random();
-
-            final AtomicInteger cntr = new AtomicInteger();
-
-            multithreaded(new Callable() {
-                @Nullable @Override public Object call() throws Exception {
-                    int cnt = 100;
-
-                    for (int i = 0; i < cnt && !Thread.currentThread().isInterrupted(); i++) {
-                        IgniteEx grid = grid(rand.nextInt(2));
-
-                        IgniteCache<Integer, String> cache = grid.cache(null);
-
-                        int key = rand.nextInt(1000);
-                        String val = Integer.toString(key);
-
-                        try (Transaction tx = grid.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
-                            String v = cache.get(key);
-
-                            assert v == null || v.equals(Integer.toString(key)) : "Invalid value for key [key=" + key +
-                                ", val=" + v + ']';
-
-                            cache.put(key, val);
-
-                            tx.commit();
-                        }
-
-                        if (cntr.incrementAndGet() % 100 == 0)
-                            info("Stored cache object for key [key=" + key + ", idx=" + i + ']');
-                    }
-
-                    return null;
-                }
-            }, 10);
-        }
-        finally {
-            stopAllGrids();
-        }
-    }
-
-    /**
-     * @param plcMax Policy max.
-     * @return Policy.
-     */
-    protected abstract T createPolicy(int plcMax);
-
-    /**
-     * @param nearMax Near max.
-     * @return Policy.
-     */
-    protected abstract T createNearPolicy(int nearMax);
-
-    /**
-     * Performs after-test near policy check.
-     *
-     * @param nearMax Near max.
-     */
-    protected abstract void checkNearPolicies(int nearMax);
-
-    /**
-     * Performs after-test policy check.
-     *
-     * @param plcMax Maximum allowed size of ploicy.
-     */
-    protected abstract void checkPolicies(int plcMax);
-
-    /**
-     *
-     */
-    @SuppressWarnings({"PublicConstructorInNonPublicClass"})
-    protected static class MockEntry extends GridCacheMockEntry<String, String> {
-        /** */
-        private IgniteCache<String, String> parent;
-
-        /** Entry value. */
-        private String val;
-
-        /** @param key Key. */
-        public MockEntry(String key) {
-            super(key);
-        }
-
-        /**
-         * @param key Key.
-         * @param val Value.
-         */
-        public MockEntry(String key, String val) {
-            super(key);
-
-            this.val = val;
-        }
-
-        /**
-         * @param key Key.
-         * @param parent Parent.
-         */
-        public MockEntry(String key, @Nullable IgniteCache<String, String> parent) {
-            super(key);
-
-            this.parent = parent;
-        }
-
-        /** {@inheritDoc} */
-        @SuppressWarnings("unchecked")
-        @Override public <T> T unwrap(Class<T> clazz) {
-            if (clazz.isAssignableFrom(IgniteCache.class))
-                return (T)parent;
-
-            return super.unwrap(clazz);
-        }
-
-        /** {@inheritDoc} */
-        @Override public String getValue() throws IllegalStateException {
-            return val;
-        }
-
-        /** {@inheritDoc} */
-        @Override public String toString() {
-            return S.toString(MockEntry.class, this, super.toString());
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e5d5d08d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheEvictionTouchSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheEvictionTouchSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheEvictionTouchSelfTest.java
index 9873ce6..55b8d63 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheEvictionTouchSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheEvictionTouchSelfTest.java
@@ -110,7 +110,10 @@ public class GridCacheEvictionTouchSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testPolicyConsistency() throws Exception {
-        plc = new FifoEvictionPolicy<Object, Object>(500);
+        FifoEvictionPolicy<Object, Object> plc = new FifoEvictionPolicy<>();
+        plc.setMaxSize(500);
+
+        this.plc = plc;
 
         try {
             Ignite ignite = startGrid(1);
@@ -136,7 +139,7 @@ public class GridCacheEvictionTouchSelfTest extends GridCommonAbstractTest {
                         info("Stats [iterCnt=" + i + ", size=" + cache.size() + ']');
                 }
 
-                FifoEvictionPolicy<Integer, Integer> plc0 = (FifoEvictionPolicy<Integer, Integer>) plc;
+                FifoEvictionPolicy<Integer, Integer> plc0 = (FifoEvictionPolicy<Integer, Integer>)this.plc;
 
                 if (!plc0.queue().isEmpty()) {
                     for (Cache.Entry<Integer, Integer> e : plc0.queue())
@@ -162,7 +165,10 @@ public class GridCacheEvictionTouchSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testEvictSingle() throws Exception {
-        plc = new FifoEvictionPolicy<Object, Object>(500);
+        FifoEvictionPolicy<Object, Object> plc = new FifoEvictionPolicy<>();
+        plc.setMaxSize(500);
+
+        this.plc = plc;
 
         try {
             Ignite ignite = startGrid(1);
@@ -189,7 +195,10 @@ public class GridCacheEvictionTouchSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testEvictAll() throws Exception {
-        plc = new FifoEvictionPolicy<Object, Object>(500);
+        FifoEvictionPolicy<Object, Object> plc = new FifoEvictionPolicy<>();
+        plc.setMaxSize(500);
+
+        this.plc = plc;
 
         try {
             Ignite ignite = startGrid(1);
@@ -221,7 +230,10 @@ public class GridCacheEvictionTouchSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testReload() throws Exception {
-        plc = new FifoEvictionPolicy<Object, Object>(100);
+        FifoEvictionPolicy<Object, Object> plc = new FifoEvictionPolicy<>();
+        plc.setMaxSize(100);
+
+        this.plc = plc;
 
         try {
             Ignite ignite = startGrid(1);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e5d5d08d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheMockEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheMockEntry.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheMockEntry.java
index 5898ed9..5d183bf 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheMockEntry.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheMockEntry.java
@@ -83,6 +83,11 @@ public class GridCacheMockEntry<K, V> extends GridMetadataAwareAdapter implement
         return !evicted;
     }
 
+    /** {@inheritDoc} */
+    @Override public int size() {
+        return 0;
+    }
+
     /**
      *
      * @return Evicted or not.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e5d5d08d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/fifo/FifoEvictionPolicySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/fifo/FifoEvictionPolicySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/fifo/FifoEvictionPolicySelfTest.java
new file mode 100644
index 0000000..a908259
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/fifo/FifoEvictionPolicySelfTest.java
@@ -0,0 +1,262 @@
+/*
+ * 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.eviction.fifo;
+
+import org.apache.ignite.cache.eviction.fifo.*;
+import org.apache.ignite.internal.processors.cache.eviction.*;
+
+/**
+ * FIFO eviction policy tests.
+ */
+public class FifoEvictionPolicySelfTest extends
+    EvictionAbstractTest<FifoEvictionPolicy<String, String>> {
+    /** {@inheritDoc} */
+    @Override protected void doTestPolicy() throws Exception {
+        try {
+            startGrid();
+
+            MockEntry e1 = new MockEntry("1", "1");
+            MockEntry e2 = new MockEntry("2", "2");
+            MockEntry e3 = new MockEntry("3", "3");
+            MockEntry e4 = new MockEntry("4", "4");
+            MockEntry e5 = new MockEntry("5", "5");
+
+            FifoEvictionPolicy<String, String> p = policy();
+
+            p.onEntryAccessed(false, e1);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e1);
+
+            p.onEntryAccessed(false, e2);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e1, e2);
+
+            p.onEntryAccessed(false, e3);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e1, e2, e3);
+
+            assert !e1.isEvicted();
+            assert !e2.isEvicted();
+            assert !e3.isEvicted();
+
+            p.onEntryAccessed(false, e4);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e2, e3, e4);
+
+            assert e1.isEvicted();
+            assert !e2.isEvicted();
+            assert !e3.isEvicted();
+            assert !e4.isEvicted();
+
+            p.onEntryAccessed(false, e5);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e3, e4, e5);
+
+            assert e2.isEvicted();
+            assert !e3.isEvicted();
+            assert !e4.isEvicted();
+            assert !e5.isEvicted();
+
+            p.onEntryAccessed(false, e1 = new MockEntry("1", "1"));
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e4, e5, e1);
+
+            assert e3.isEvicted();
+            assert !e1.isEvicted();
+            assert !e4.isEvicted();
+            assert !e5.isEvicted();
+
+            p.onEntryAccessed(false, e5);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e4, e5, e1);
+
+            assert !e1.isEvicted();
+            assert !e4.isEvicted();
+            assert !e5.isEvicted();
+
+            p.onEntryAccessed(false, e1);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e4, e5, e1);
+
+            assert !e1.isEvicted();
+            assert !e4.isEvicted();
+            assert !e5.isEvicted();
+
+            p.onEntryAccessed(false, e5);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e4, e5, e1);
+
+            assert !e1.isEvicted();
+            assert !e4.isEvicted();
+            assert !e5.isEvicted();
+
+            p.onEntryAccessed(true, e1);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e4, e5);
+
+            assert !e1.isEvicted();
+            assert !e4.isEvicted();
+            assert !e5.isEvicted();
+
+            p.onEntryAccessed(true, e4);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e5);
+
+            assert !e4.isEvicted();
+            assert !e5.isEvicted();
+
+            p.onEntryAccessed(true, e5);
+
+            check(MockEntry.ENTRY_SIZE, p.queue());
+
+            assert !e5.isEvicted();
+
+            info(p);
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void doTestPolicyWithBatch() throws Exception {
+        try {
+            startGrid();
+
+            MockEntry e1 = new MockEntry("1", "1");
+            MockEntry e2 = new MockEntry("2", "2");
+            MockEntry e3 = new MockEntry("3", "3");
+            MockEntry e4 = new MockEntry("4", "4");
+            MockEntry e5 = new MockEntry("5", "5");
+
+            FifoEvictionPolicy<String, String> p = policy();
+
+            p.onEntryAccessed(false, e1);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e1);
+
+            p.onEntryAccessed(false, e2);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e1, e2);
+
+            p.onEntryAccessed(false, e3);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e1, e2, e3);
+
+            p.onEntryAccessed(false, e4);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e1, e2, e3, e4);
+
+            assertFalse(e1.isEvicted());
+            assertFalse(e2.isEvicted());
+            assertFalse(e3.isEvicted());
+            assertFalse(e4.isEvicted());
+
+            p.onEntryAccessed(false, e5);
+
+            // Batch evicted.
+            check(MockEntry.ENTRY_SIZE, p.queue(), e3, e4, e5);
+
+            assertTrue(e1.isEvicted());
+            assertTrue(e2.isEvicted());
+            assertFalse(e3.isEvicted());
+            assertFalse(e4.isEvicted());
+            assertFalse(e5.isEvicted());
+
+            p.onEntryAccessed(false, e1 = new MockEntry("1", "1"));
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e3, e4, e5, e1);
+
+            assertFalse(e3.isEvicted());
+            assertFalse(e4.isEvicted());
+            assertFalse(e5.isEvicted());
+            assertFalse(e1.isEvicted());
+
+            p.onEntryAccessed(false, e5);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e3, e4, e5, e1);
+
+            assertFalse(e3.isEvicted());
+            assertFalse(e4.isEvicted());
+            assertFalse(e5.isEvicted());
+            assertFalse(e1.isEvicted());
+
+            p.onEntryAccessed(false, e1);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e3, e4, e5, e1);
+
+            assertFalse(e3.isEvicted());
+            assertFalse(e4.isEvicted());
+            assertFalse(e5.isEvicted());
+            assertFalse(e1.isEvicted());
+
+            p.onEntryAccessed(true, e1);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e3, e4, e5);
+
+            assertFalse(e3.isEvicted());
+            assertFalse(e4.isEvicted());
+            assertFalse(e5.isEvicted());
+
+            p.onEntryAccessed(true, e4);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e3, e5);
+
+            assertFalse(e3.isEvicted());
+            assertFalse(e5.isEvicted());
+
+            p.onEntryAccessed(true, e5);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e3);
+
+            assertFalse(e3.isEvicted());
+
+            p.onEntryAccessed(true, e3);
+
+            check(MockEntry.ENTRY_SIZE, p.queue());
+
+            assertFalse(e3.isEvicted());
+
+            info(p);
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override protected FifoEvictionPolicy<String, String> createPolicy(int plcMax) {
+        FifoEvictionPolicy<String, String> plc = new FifoEvictionPolicy<>();
+
+        plc.setMaxSize(this.plcMax);
+        plc.setBatchSize(this.plcBatchSize);
+        plc.setMaxMemSize(this.plcMaxMemSize);
+
+        return plc;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected FifoEvictionPolicy<String, String> createNearPolicy(int nearMax) {
+        FifoEvictionPolicy<String, String> plc = new FifoEvictionPolicy<>();
+
+        plc.setMaxSize(nearMax);
+        plc.setBatchSize(plcBatchSize);
+
+        return plc;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e5d5d08d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/fifo/GridCacheFifoBatchEvictionPolicySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/fifo/GridCacheFifoBatchEvictionPolicySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/fifo/GridCacheFifoBatchEvictionPolicySelfTest.java
deleted file mode 100644
index f907c36..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/fifo/GridCacheFifoBatchEvictionPolicySelfTest.java
+++ /dev/null
@@ -1,384 +0,0 @@
-/*
- * 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.eviction.fifo;
-
-import org.apache.ignite.*;
-import org.apache.ignite.cache.eviction.*;
-import org.apache.ignite.cache.eviction.fifo.*;
-import org.apache.ignite.internal.processors.cache.eviction.*;
-
-import java.util.*;
-
-import static org.apache.ignite.cache.CacheMode.*;
-
-/**
- * FIFO batch eviction test.
- */
-public class GridCacheFifoBatchEvictionPolicySelfTest extends
-    GridCacheEvictionAbstractTest<FifoEvictionPolicy<String, String>> {
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPolicy() throws Exception {
-        try {
-            startGrid();
-
-            MockEntry e1 = new MockEntry("1", "1");
-            MockEntry e2 = new MockEntry("2", "2");
-            MockEntry e3 = new MockEntry("3", "3");
-            MockEntry e4 = new MockEntry("4", "4");
-            MockEntry e5 = new MockEntry("5", "5");
-
-            FifoEvictionPolicy<String, String> p = policy();
-
-            p.setMaxSize(3);
-
-            p.setBatchSize(2);
-
-            p.onEntryAccessed(false, e1);
-
-            check(p.queue(), e1);
-
-            p.onEntryAccessed(false, e2);
-
-            check(p.queue(), e1, e2);
-
-            p.onEntryAccessed(false, e3);
-
-            check(p.queue(), e1, e2, e3);
-
-            p.onEntryAccessed(false, e4);
-
-            check(p.queue(), e1, e2, e3, e4);
-
-            assertFalse(e1.isEvicted());
-            assertFalse(e2.isEvicted());
-            assertFalse(e3.isEvicted());
-            assertFalse(e4.isEvicted());
-
-            assertEquals(4, p.getCurrentSize());
-
-            p.onEntryAccessed(false, e5);
-
-            // Batch evicted.
-            check(p.queue(), e3, e4, e5);
-
-            assertEquals(3, p.getCurrentSize());
-
-            assertTrue(e1.isEvicted());
-            assertTrue(e2.isEvicted());
-            assertFalse(e3.isEvicted());
-            assertFalse(e4.isEvicted());
-            assertFalse(e5.isEvicted());
-
-            p.onEntryAccessed(false, e1 = new MockEntry("1", "1"));
-
-            check(p.queue(), e3, e4, e5, e1);
-
-            assertEquals(4, p.getCurrentSize());
-
-            assertFalse(e3.isEvicted());
-            assertFalse(e4.isEvicted());
-            assertFalse(e5.isEvicted());
-            assertFalse(e1.isEvicted());
-
-            p.onEntryAccessed(false, e5);
-
-            check(p.queue(), e3, e4, e5, e1);
-
-            assertFalse(e3.isEvicted());
-            assertFalse(e4.isEvicted());
-            assertFalse(e5.isEvicted());
-            assertFalse(e1.isEvicted());
-
-            p.onEntryAccessed(false, e1);
-
-            assertEquals(4, p.getCurrentSize());
-
-            check(p.queue(), e3, e4, e5, e1);
-
-            assertFalse(e3.isEvicted());
-            assertFalse(e4.isEvicted());
-            assertFalse(e5.isEvicted());
-            assertFalse(e1.isEvicted());
-
-            p.onEntryAccessed(true, e1);
-
-            assertEquals(3, p.getCurrentSize());
-
-            assertFalse(e3.isEvicted());
-            assertFalse(e4.isEvicted());
-            assertFalse(e5.isEvicted());
-
-            p.onEntryAccessed(true, e4);
-
-            assertEquals(2, p.getCurrentSize());
-
-            assertFalse(e3.isEvicted());
-            assertFalse(e5.isEvicted());
-
-            p.onEntryAccessed(true, e5);
-
-            assertEquals(1, p.getCurrentSize());
-
-            assertFalse(e3.isEvicted());
-
-            p.onEntryAccessed(true, e3);
-
-            assertEquals(0, p.getCurrentSize());
-
-            assertFalse(e3.isEvicted());
-
-            info(p);
-        }
-        finally {
-            stopAllGrids();
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testMemory() throws Exception {
-        try {
-            startGrid();
-
-            FifoEvictionPolicy<String, String> p = policy();
-
-            int max = 10;
-
-            int batchSize = 2;
-
-            p.setMaxSize(max);
-            p.setBatchSize(batchSize);
-
-            int cnt = max + batchSize;
-
-            for (int i = 0; i < cnt; i++)
-                p.onEntryAccessed(false, new MockEntry(Integer.toString(i), Integer.toString(i)));
-
-            info(p);
-
-            assertEquals(cnt - batchSize, p.getCurrentSize());
-        }
-        finally {
-            stopAllGrids();
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testRandom() throws Exception {
-        try {
-            startGrid();
-
-            FifoEvictionPolicy<String, String> p = policy();
-
-            int max = 10;
-
-            int batchSize = 2;
-
-            p.setMaxSize(max);
-
-            p.setBatchSize(batchSize);
-
-            Random rand = new Random();
-
-            int keys = 31;
-
-            MockEntry[] fifos = new MockEntry[keys];
-
-            for (int i = 0; i < fifos.length; i++)
-                fifos[i] = new MockEntry(Integer.toString(i));
-
-            int runs = 5000000;
-
-            for (int i = 0; i < runs; i++) {
-                boolean rmv = rand.nextBoolean();
-
-                int j = rand.nextInt(fifos.length);
-
-                MockEntry e = entry(fifos, j);
-
-                if (rmv)
-                    fifos[j] = new MockEntry(Integer.toString(j));
-
-                p.onEntryAccessed(rmv, e);
-            }
-
-            info(p);
-
-            int curSize = p.getCurrentSize();
-
-            assert curSize < max + batchSize :
-                "curSize < max + batchSize [curSize=" + curSize + ", max=" + max + ", batchSize=" + batchSize + ']';
-        }
-        finally {
-            stopAllGrids();
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testAllowEmptyEntries() throws Exception {
-        try {
-            startGrid();
-
-            MockEntry e1 = new MockEntry("1");
-
-            MockEntry e2 = new MockEntry("2");
-
-            MockEntry e3 = new MockEntry("3");
-
-            MockEntry e4 = new MockEntry("4");
-
-            MockEntry e5 = new MockEntry("5");
-
-            FifoEvictionPolicy<String, String> p = policy();
-
-            p.setBatchSize(2);
-
-            p.onEntryAccessed(false, e1);
-
-            assertFalse(e1.isEvicted());
-
-            p.onEntryAccessed(false, e2);
-
-            assertFalse(e1.isEvicted());
-            assertFalse(e2.isEvicted());
-
-            p.onEntryAccessed(false, e3);
-
-            assertFalse(e1.isEvicted());
-            assertFalse(e3.isEvicted());
-
-            p.onEntryAccessed(false, e4);
-
-            assertFalse(e1.isEvicted());
-            assertFalse(e3.isEvicted());
-            assertFalse(e4.isEvicted());
-
-            p.onEntryAccessed(false, e5);
-
-            assertFalse(e1.isEvicted());
-            assertFalse(e3.isEvicted());
-            assertFalse(e5.isEvicted());
-        }
-        finally {
-            stopAllGrids();
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPut() throws Exception {
-        mode = LOCAL;
-        syncCommit = true;
-        plcMax = 10;
-
-        Ignite ignite = startGrid();
-
-        try {
-            IgniteCache<Object, Object> cache = ignite.cache(null);
-
-            int cnt = 500;
-
-            int min = Integer.MAX_VALUE;
-
-            int minIdx = 0;
-
-            for (int i = 0; i < cnt; i++) {
-                cache.put(i, i);
-
-                int cacheSize = cache.size();
-
-                if (i > plcMax && cacheSize < min) {
-                    min = cacheSize;
-                    minIdx = i;
-                }
-            }
-
-            // Batch evicted.
-            assert min >= plcMax : "Min cache size is too small: " + min;
-
-            info("Min cache size [min=" + min + ", idx=" + minIdx + ']');
-            info("Current cache size " + cache.size());
-            info("Current cache key size " + cache.size());
-
-            min = Integer.MAX_VALUE;
-
-            minIdx = 0;
-
-            // Touch.
-            for (int i = cnt; --i > cnt - plcMax;) {
-                cache.get(i);
-
-                int cacheSize = cache.size();
-
-                if (cacheSize < min) {
-                    min = cacheSize;
-                    minIdx = i;
-                }
-            }
-
-            info("----");
-            info("Min cache size [min=" + min + ", idx=" + minIdx + ']');
-            info("Current cache size " + cache.size());
-            info("Current cache key size " + cache.size());
-
-            // Batch evicted.
-            assert min >= plcMax : "Min cache size is too small: " + min;
-        }
-        finally {
-            stopAllGrids();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void testPartitionedNearDisabled() throws Exception {
-        plcBatchSize = 2;
-
-        super.testPartitionedNearDisabled();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected FifoEvictionPolicy<String, String> createPolicy(int plcMax) {
-        return new FifoEvictionPolicy<>(10, 2);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected FifoEvictionPolicy<String, String> createNearPolicy(int nearMax) {
-        return new FifoEvictionPolicy<>(nearMax, 2);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void checkNearPolicies(int endNearPlcSize) {
-        for (int i = 0; i < gridCnt; i++)
-            for (EvictableEntry<String, String> e : nearPolicy(i).queue())
-                assert !e.isCached() : "Invalid near policy size: " + nearPolicy(i).queue();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void checkPolicies(int plcMax) {
-        for (int i = 0; i < gridCnt; i++)
-            assert policy(i).queue().size() <= plcMax + policy(i).getBatchSize();
-    }
-}


[6/8] incubator-ignite git commit: Merge branches 'ignite-916' and 'ignite-sprint-5' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-916

Posted by sb...@apache.org.
Merge branches 'ignite-916' and 'ignite-sprint-5' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-916

Conflicts:
	modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingNearErrorTest.java


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

Branch: refs/heads/ignite-sprint-5
Commit: adc7b234708cb53a026fcf488fa2aaa03bac02fc
Parents: 859f182
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Tue Jun 2 16:23:32 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Tue Jun 2 16:23:32 2015 +0300

----------------------------------------------------------------------
 .../src/main/java/org/apache/ignite/yardstick/IgniteNode.java | 7 ++-----
 1 file changed, 2 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/adc7b234/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteNode.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteNode.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteNode.java
index 2027b80..8638623 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteNode.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteNode.java
@@ -104,11 +104,8 @@ public class IgniteNode implements BenchmarkServer {
 
                 if (args.isOffheapValues())
                     cc.setMemoryMode(OFFHEAP_VALUES);
-                else {
-                    LruEvictionPolicy plc = new LruEvictionPolicy();
-                    plc.setMaxSize(50000);
-                    cc.setEvictionPolicy(plc);
-                }
+                else
+                    cc.setEvictionPolicy(new LruEvictionPolicy(50000));
             }
 
             cc.setReadThrough(args.isStoreEnabled());


[8/8] incubator-ignite git commit: Merge branch 'ignite-916' into ignite-sprint-5

Posted by sb...@apache.org.
Merge branch 'ignite-916' into ignite-sprint-5


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

Branch: refs/heads/ignite-sprint-5
Commit: ba6d055640917ba904277b872196fe269fdd74c8
Parents: 1bd6da1 fbb5763
Author: agura <ag...@gridgain.com>
Authored: Wed Jun 3 01:27:02 2015 +0300
Committer: agura <ag...@gridgain.com>
Committed: Wed Jun 3 01:27:02 2015 +0300

----------------------------------------------------------------------
 .../ignite/cache/eviction/EvictableEntry.java   |    7 +
 .../ignite/cache/eviction/EvictionPolicy.java   |    2 +
 .../cache/eviction/fifo/FifoEvictionPolicy.java |  117 +-
 .../eviction/fifo/FifoEvictionPolicyMBean.java  |   22 +
 .../cache/eviction/lru/LruEvictionPolicy.java   |  135 ++-
 .../eviction/lru/LruEvictionPolicyMBean.java    |   38 +
 .../eviction/random/RandomEvictionPolicy.java   |    7 +-
 .../eviction/sorted/SortedEvictionPolicy.java   |  141 ++-
 .../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 +-
 .../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    | 1056 ++++++++++++++++++
 .../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 |  168 ---
 .../eviction/lru/LruEvictionPolicySelfTest.java |  353 ++++++
 .../lru/LruNearEvictionPolicySelfTest.java      |  140 +++
 .../LruNearOnlyNearEvictionPolicySelfTest.java  |  172 +++
 .../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 +-
 53 files changed, 3465 insertions(+), 3228 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ba6d0556/modules/core/src/main/java/org/apache/ignite/cache/eviction/random/RandomEvictionPolicy.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/cache/eviction/random/RandomEvictionPolicy.java
index 0ac9197,0d840e3..00a912f
--- 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
@@@ -18,9 -18,7 +18,8 @@@
  package org.apache.ignite.cache.eviction.random;
  
  import org.apache.ignite.*;
 +import org.apache.ignite.cache.*;
  import org.apache.ignite.cache.eviction.*;
- import org.apache.ignite.configuration.*;
  import org.apache.ignite.internal.util.typedef.internal.*;
  
  import javax.cache.*;


[5/8] incubator-ignite git commit: Merge branches 'ignite-916' and 'ignite-sprint-5' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-916

Posted by sb...@apache.org.
Merge branches 'ignite-916' and 'ignite-sprint-5' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-916

Conflicts:
	modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingNearErrorTest.java


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

Branch: refs/heads/ignite-sprint-5
Commit: 859f1829c60d7f148090c79112968b5ea3ac5809
Parents: e5d5d08 82cfb47
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Tue Jun 2 16:13:17 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Tue Jun 2 16:13:17 2015 +0300

----------------------------------------------------------------------
 assembly/dependencies-visor-console.xml         |    3 +
 .../hibernate/CacheHibernatePersonStore.java    |  202 +-
 .../hibernate/CacheHibernateStoreExample.java   |   17 +
 .../store/jdbc/CacheJdbcPersonStore.java        |  180 +-
 .../store/jdbc/CacheJdbcStoreExample.java       |   13 +
 .../store/spring/CacheSpringPersonStore.java    |  128 +
 .../store/spring/CacheSpringStoreExample.java   |  143 +
 .../datagrid/store/spring/package-info.java     |   22 +
 .../main/java/org/apache/ignite/Ignition.java   |   18 +-
 .../apache/ignite/cache/store/CacheStore.java   |    2 +
 .../ignite/cache/store/CacheStoreSession.java   |   22 +
 .../cache/store/CacheStoreSessionListener.java  |  133 +
 .../jdbc/CacheJdbcStoreSessionListener.java     |  141 +
 .../org/apache/ignite/cluster/ClusterNode.java  |   24 +-
 .../configuration/CacheConfiguration.java       |   53 +-
 .../configuration/IgniteConfiguration.java      |   48 +-
 .../ignite/internal/ClusterMetricsSnapshot.java |   14 +
 .../internal/GridEventConsumeHandler.java       |  100 +-
 .../ignite/internal/GridKernalContext.java      |    5 +
 .../ignite/internal/GridKernalContextImpl.java  |    5 +
 .../apache/ignite/internal/IgniteKernal.java    |   23 +-
 .../org/apache/ignite/internal/IgnitionEx.java  |   22 +-
 .../ignite/internal/MarshallerContextImpl.java  |    2 +-
 .../internal/events/DiscoveryCustomEvent.java   |   18 +-
 .../managers/communication/GridIoManager.java   |   12 +-
 .../managers/discovery/CustomEventListener.java |   31 +
 .../discovery/CustomMessageWrapper.java         |   63 +
 .../discovery/DiscoveryCustomMessage.java       |   48 +
 .../discovery/GridDiscoveryManager.java         |  214 +-
 .../affinity/GridAffinityAssignmentCache.java   |   32 +
 .../cache/DynamicCacheChangeBatch.java          |   20 +-
 .../cache/DynamicCacheDescriptor.java           |    2 +
 .../processors/cache/GridCacheAdapter.java      |   17 +-
 .../cache/GridCacheAffinityManager.java         |   14 +
 .../cache/GridCacheConcurrentMap.java           |   21 +-
 .../processors/cache/GridCacheContext.java      |    6 +-
 .../processors/cache/GridCacheEntryEx.java      |    6 +
 .../processors/cache/GridCacheGateway.java      |    2 +-
 .../processors/cache/GridCacheIoManager.java    |    8 +-
 .../processors/cache/GridCacheMapEntry.java     |    5 +
 .../processors/cache/GridCacheMvccManager.java  |   32 +-
 .../GridCachePartitionExchangeManager.java      |   71 +-
 .../processors/cache/GridCachePreloader.java    |    6 +-
 .../cache/GridCachePreloaderAdapter.java        |   11 +-
 .../processors/cache/GridCacheProcessor.java    |   86 +-
 .../processors/cache/GridCacheProxyImpl.java    |   12 +
 .../cache/GridCacheSharedContext.java           |   48 +-
 .../processors/cache/GridCacheUtils.java        |  288 +-
 .../processors/cache/IgniteInternalCache.java   |    5 +
 .../cache/affinity/GridCacheAffinityImpl.java   |   10 +-
 .../CacheDataStructuresManager.java             |    2 +-
 .../distributed/GridDistributedCacheEntry.java  |    7 -
 .../distributed/GridDistributedTxMapping.java   |   17 +
 .../dht/GridClientPartitionTopology.java        |    8 +-
 .../dht/GridDhtAssignmentFetchFuture.java       |    4 +-
 .../distributed/dht/GridDhtCacheAdapter.java    |   27 +-
 .../distributed/dht/GridDhtCacheEntry.java      |    6 +-
 .../distributed/dht/GridDhtLockFuture.java      |   10 +-
 .../dht/GridDhtPartitionTopologyImpl.java       |   22 +-
 .../dht/GridDhtTransactionalCacheAdapter.java   |  224 +-
 .../distributed/dht/GridDhtTxLocalAdapter.java  |    8 +-
 .../distributed/dht/GridDhtTxPrepareFuture.java |    3 +-
 .../dht/atomic/GridDhtAtomicCache.java          |   18 +-
 .../dht/atomic/GridDhtAtomicUpdateFuture.java   |   10 +-
 .../dht/atomic/GridNearAtomicUpdateFuture.java  |   78 +-
 .../dht/atomic/GridNearAtomicUpdateRequest.java |  112 +-
 .../dht/colocated/GridDhtColocatedCache.java    |   12 +-
 .../colocated/GridDhtColocatedLockFuture.java   |  213 +-
 .../dht/preloader/GridDhtForceKeysFuture.java   |    4 +-
 .../preloader/GridDhtPartitionDemandPool.java   |   26 +-
 .../dht/preloader/GridDhtPartitionMap.java      |    2 +-
 .../preloader/GridDhtPartitionSupplyPool.java   |   29 +-
 .../GridDhtPartitionsExchangeFuture.java        |  313 +-
 .../preloader/GridDhtPartitionsFullMessage.java |    4 +-
 .../GridDhtPartitionsSingleMessage.java         |   33 +-
 .../dht/preloader/GridDhtPreloader.java         |   37 +-
 .../preloader/GridDhtPreloaderAssignments.java  |    3 +-
 .../distributed/near/GridNearAtomicCache.java   |    5 +
 .../distributed/near/GridNearCacheAdapter.java  |    2 +-
 .../distributed/near/GridNearGetFuture.java     |    2 +-
 .../distributed/near/GridNearLockFuture.java    |  271 +-
 .../distributed/near/GridNearLockRequest.java   |   68 +-
 .../distributed/near/GridNearLockResponse.java  |   48 +-
 .../near/GridNearOptimisticTxPrepareFuture.java |   83 +-
 .../GridNearPessimisticTxPrepareFuture.java     |    5 +-
 .../near/GridNearTransactionalCache.java        |    4 +-
 .../cache/distributed/near/GridNearTxLocal.java |   43 +-
 .../near/GridNearTxPrepareRequest.java          |   72 +-
 .../near/GridNearTxPrepareResponse.java         |   70 +-
 .../processors/cache/local/GridLocalCache.java  |    6 +-
 .../local/atomic/GridLocalAtomicCache.java      |    6 +-
 .../cache/query/GridCacheQueryAdapter.java      |   12 +-
 .../cache/query/GridCacheQueryErrorFuture.java  |    2 +
 .../cache/query/GridCacheQueryManager.java      |    2 +-
 .../continuous/CacheContinuousQueryManager.java |   28 +-
 .../cache/store/CacheOsStoreManager.java        |    1 -
 .../cache/store/CacheStoreManager.java          |    7 +-
 .../store/GridCacheStoreManagerAdapter.java     |  202 +-
 .../cache/transactions/IgniteInternalTx.java    |    5 +
 .../cache/transactions/IgniteTxAdapter.java     |   48 +-
 .../cache/transactions/IgniteTxHandler.java     |  148 +-
 .../transactions/IgniteTxLocalAdapter.java      |  142 +-
 .../cacheobject/IgniteCacheObjectProcessor.java |    5 +-
 .../IgniteCacheObjectProcessorImpl.java         |    2 +-
 .../continuous/AbstractContinuousMessage.java   |   54 +
 .../continuous/GridContinuousMessageType.java   |   12 -
 .../continuous/GridContinuousProcessor.java     |  836 +--
 .../processors/continuous/StartRequestData.java |  267 +
 .../StartRoutineAckDiscoveryMessage.java        |   63 +
 .../StartRoutineDiscoveryMessage.java           |   85 +
 .../StopRoutineAckDiscoveryMessage.java         |   49 +
 .../continuous/StopRoutineDiscoveryMessage.java |   49 +
 .../datastreamer/DataStreamerImpl.java          |    2 +
 .../service/GridServiceProcessor.java           |  121 +-
 .../internal/util/future/GridFutureAdapter.java |    4 +-
 .../apache/ignite/plugin/PluginProvider.java    |   26 +-
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |    2 +-
 .../communication/tcp/TcpCommunicationSpi.java  |    2 +-
 .../ignite/spi/discovery/DiscoverySpi.java      |   20 +-
 .../discovery/DiscoverySpiCustomMessage.java    |   40 +
 .../spi/discovery/DiscoverySpiListener.java     |    5 +-
 .../ignite/spi/discovery/tcp/ClientImpl.java    | 1478 +++++
 .../ignite/spi/discovery/tcp/ServerImpl.java    | 4766 ++++++++++++++
 .../discovery/tcp/TcpClientDiscoverySpi.java    | 1264 ----
 .../tcp/TcpClientDiscoverySpiMBean.java         |  164 -
 .../spi/discovery/tcp/TcpDiscoveryImpl.java     |  170 +
 .../spi/discovery/tcp/TcpDiscoverySpi.java      | 5799 ++++--------------
 .../discovery/tcp/TcpDiscoverySpiAdapter.java   | 1160 ----
 .../spi/discovery/tcp/TcpDiscoverySpiMBean.java |    9 +
 .../tcp/internal/TcpDiscoveryNode.java          |    7 +-
 .../tcp/internal/TcpDiscoveryNodesRing.java     |    2 +-
 .../TcpDiscoveryMulticastIpFinder.java          |   10 +-
 .../messages/TcpDiscoveryAbstractMessage.java   |   24 +-
 .../TcpDiscoveryClientHeartbeatMessage.java     |   67 +
 .../messages/TcpDiscoveryClientPingRequest.java |   56 +
 .../TcpDiscoveryClientPingResponse.java         |   67 +
 .../TcpDiscoveryCustomEventMessage.java         |   41 +-
 .../messages/TcpDiscoveryHeartbeatMessage.java  |   28 +-
 .../TcpDiscoveryNodeAddFinishedMessage.java     |   43 +
 .../messages/TcpDiscoveryNodeAddedMessage.java  |    2 +-
 .../tcp/messages/TcpDiscoveryPingRequest.java   |    6 +
 .../tcp/messages/TcpDiscoveryPingResponse.java  |   15 +-
 .../affinity/IgniteClientNodeAffinityTest.java  |  182 +
 ...cheStoreSessionListenerAbstractSelfTest.java |  315 +
 ...heStoreSessionListenerLifecycleSelfTest.java |  395 ++
 .../CacheJdbcStoreSessionListenerSelfTest.java  |  175 +
 .../ignite/internal/GridAffinitySelfTest.java   |    1 +
 .../internal/GridDiscoveryEventSelfTest.java    |    7 +-
 ...ridFailFastNodeFailureDetectionSelfTest.java |    7 +-
 .../internal/GridProjectionAbstractTest.java    |   16 +
 .../GridProjectionForCachesSelfTest.java        |   11 +-
 .../internal/GridReleaseTypeSelfTest.java       |   77 +-
 .../apache/ignite/internal/GridSelfTest.java    |    4 +-
 .../GridDiscoveryManagerAliveCacheSelfTest.java |   62 +-
 .../GridDiscoveryManagerAttributesSelfTest.java |  122 +-
 .../discovery/GridDiscoveryManagerSelfTest.java |   46 +-
 .../GridAffinityProcessorAbstractSelfTest.java  |    1 +
 .../cache/CacheRemoveAllSelfTest.java           |    2 +-
 .../GridCacheAbstractFailoverSelfTest.java      |    2 +-
 .../cache/GridCacheAbstractFullApiSelfTest.java |    3 +
 .../GridCacheAbstractRemoveFailureTest.java     |   23 +
 .../cache/GridCacheAbstractSelfTest.java        |    2 +-
 .../GridCacheAtomicMessageCountSelfTest.java    |    1 +
 ...GridCacheMixedPartitionExchangeSelfTest.java |    2 +-
 .../cache/GridCachePutAllFailoverSelfTest.java  |    1 +
 .../GridCacheReturnValueTransferSelfTest.java   |    3 +
 ...acheTcpClientDiscoveryMultiThreadedTest.java |  190 +
 .../processors/cache/GridCacheTestEntryEx.java  |    4 +
 .../GridCacheVariableTopologySelfTest.java      |   12 +-
 .../IgniteCacheAbstractStopBusySelfTest.java    |    6 +-
 .../cache/IgniteCacheAbstractTest.java          |    2 +-
 .../IgniteCacheConfigurationTemplateTest.java   |    2 +-
 .../cache/IgniteCacheNearLockValueSelfTest.java |    3 +
 .../IgniteCacheP2pUnmarshallingErrorTest.java   |   29 +-
 ...gniteCacheP2pUnmarshallingNearErrorTest.java |   17 +-
 .../IgniteCachePartitionMapUpdateTest.java      |  226 +
 .../IgniteDynamicClientCacheStartSelfTest.java  |  283 +
 .../cache/IgniteSystemCacheOnClientTest.java    |   97 +
 .../GridCacheQueueApiSelfAbstractTest.java      |    4 +-
 .../IgniteClientDataStructuresAbstractTest.java |  283 +
 .../IgniteClientDataStructuresTest.java         |   28 +
 ...IgniteClientDiscoveryDataStructuresTest.java |   28 +
 .../GridCacheClientModesAbstractSelfTest.java   |   94 +-
 ...ientModesTcpClientDiscoveryAbstractTest.java |  168 +
 .../distributed/GridCacheMixedModeSelfTest.java |    3 +
 ...niteCacheClientNodeChangingTopologyTest.java | 1803 ++++++
 .../IgniteCacheClientNodeConcurrentStart.java   |  105 +
 ...teCacheClientNodePartitionsExchangeTest.java |  632 ++
 .../IgniteCrossCacheTxStoreSelfTest.java        |  147 +-
 .../dht/GridCacheClientOnlySelfTest.java        |   60 +-
 .../GridCacheDhtClientRemoveFailureTest.java    |   28 +
 .../dht/IgniteCacheMultiTxLockSelfTest.java     |   47 +-
 ...cClientInvalidPartitionHandlingSelfTest.java |   29 +
 .../GridCacheAtomicClientRemoveFailureTest.java |   28 +
 ...eAtomicInvalidPartitionHandlingSelfTest.java |   23 +-
 ...unctionExcludeNeighborsAbstractSelfTest.java |    3 +-
 .../near/GridCacheAtomicNearOnlySelfTest.java   |   32 -
 ...idCacheNearOnlyMultiNodeFullApiSelfTest.java |    2 +
 .../near/GridCacheNearOnlySelfTest.java         |   63 +-
 .../near/GridCacheNearOnlyTopologySelfTest.java |    1 +
 ...ionedClientOnlyNoPrimaryFullApiSelfTest.java |    5 +-
 ...idCacheRendezvousAffinityClientSelfTest.java |    4 +
 .../GridCacheReplicatedClientOnlySelfTest.java  |   43 -
 .../GridCacheReplicatedNearOnlySelfTest.java    |   43 -
 .../GridCacheSyncReplicatedPreloadSelfTest.java |    1 -
 .../LruNearOnlyNearEvictionPolicySelfTest.java  |   25 +-
 ...ridCacheContinuousQueryAbstractSelfTest.java |    6 +-
 .../continuous/GridEventConsumeSelfTest.java    |   93 +-
 .../DataStreamProcessorSelfTest.java            |    1 +
 .../DataStreamerMultiThreadedSelfTest.java      |    2 -
 .../igfs/IgfsClientCacheSelfTest.java           |    3 +-
 .../processors/igfs/IgfsCommonAbstractTest.java |   10 -
 .../processors/igfs/IgfsOneClientNodeTest.java  |    8 +-
 .../service/ClosureServiceClientsNodesTest.java |   16 +-
 .../service/GridServiceClientNodeTest.java      |   81 +
 .../loadtests/hashmap/GridCacheTestContext.java |    3 +-
 .../OptimizedMarshallerNodeFailoverTest.java    |    4 +-
 ...GridMessagingNoPeerClassLoadingSelfTest.java |    7 +-
 .../ignite/messaging/GridMessagingSelfTest.java |   13 +-
 .../discovery/AbstractDiscoverySelfTest.java    |    8 +-
 ...pClientDiscoveryMarshallerCheckSelfTest.java |   76 +
 .../tcp/TcpClientDiscoverySelfTest.java         |  700 ---
 .../tcp/TcpClientDiscoverySpiSelfTest.java      | 1171 ++++
 .../tcp/TcpDiscoveryConcurrentStartTest.java    |   61 +-
 .../tcp/TcpDiscoveryMultiThreadedTest.java      |   18 +-
 .../spi/discovery/tcp/TcpDiscoverySelfTest.java |    2 +-
 .../ignite/testframework/GridTestUtils.java     |   15 +
 .../testframework/junits/GridAbstractTest.java  |   52 +-
 .../junits/cache/TestCacheSession.java          |   18 +
 .../cache/TestThreadLocalCacheSession.java      |   15 +
 .../junits/common/GridCommonAbstractTest.java   |   83 +-
 .../ignite/testsuites/IgniteBasicTestSuite.java |    2 +-
 .../IgniteCacheDataStructuresSelfTestSuite.java |    3 +
 .../IgniteCacheFailoverTestSuite.java           |    4 +-
 .../IgniteCacheNearOnlySelfTestSuite.java       |   16 +-
 ...gniteCacheP2pUnmarshallingErrorTestSuit.java |   41 -
 ...niteCacheP2pUnmarshallingErrorTestSuite.java |   41 +
 .../IgniteCacheTcpClientDiscoveryTestSuite.java |   47 +
 .../ignite/testsuites/IgniteCacheTestSuite.java |    4 +
 .../testsuites/IgniteCacheTestSuite2.java       |   11 +-
 .../testsuites/IgniteCacheTestSuite4.java       |    5 +
 .../testsuites/IgniteKernalSelfTestSuite.java   |    7 +-
 .../IgniteSpiDiscoverySelfTestSuite.java        |    3 +-
 .../igfs/IgfsNearOnlyMultiNodeSelfTest.java     |    5 +-
 .../CacheHibernateStoreSessionListener.java     |  216 +
 ...heHibernateStoreSessionListenerSelfTest.java |  228 +
 .../testsuites/IgniteHibernateTestSuite.java    |    2 +
 .../cache/IgniteCacheAbstractQuerySelfTest.java |    6 +-
 ...niteCacheP2pUnmarshallingQueryErrorTest.java |    3 +-
 .../IgniteCacheQuerySelfTestSuite.java          |    2 +-
 modules/spring/pom.xml                          |   14 +
 .../spring/CacheSpringStoreSessionListener.java |  207 +
 ...CacheSpringStoreSessionListenerSelfTest.java |  197 +
 .../testsuites/IgniteSpringTestSuite.java       |    3 +
 scripts/git-format-patch.sh                     |   14 +-
 255 files changed, 20612 insertions(+), 10742 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/859f1829/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheMultiTxLockSelfTest.java
----------------------------------------------------------------------
diff --cc modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheMultiTxLockSelfTest.java
index 77fe8cf,f33de76..9e54673
--- 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
@@@ -66,16 -72,12 +72,16 @@@ public class IgniteCacheMultiTxLockSelf
          CacheConfiguration ccfg = new CacheConfiguration();
  
          ccfg.setName(CACHE_NAME);
-         ccfg.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL);
-         ccfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.PRIMARY_SYNC);
+         ccfg.setAtomicityMode(TRANSACTIONAL);
+         ccfg.setWriteSynchronizationMode(PRIMARY_SYNC);
          ccfg.setBackups(2);
-         ccfg.setCacheMode(CacheMode.PARTITIONED);
+         ccfg.setCacheMode(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/859f1829/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/LruNearOnlyNearEvictionPolicySelfTest.java
----------------------------------------------------------------------
diff --cc modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/LruNearOnlyNearEvictionPolicySelfTest.java
index faca01d,0000000..0d3c692
mode 100644,000000..100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/LruNearOnlyNearEvictionPolicySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/LruNearOnlyNearEvictionPolicySelfTest.java
@@@ -1,175 -1,0 +1,172 @@@
 +/*
 + * 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.eviction.lru;
 +
 +import org.apache.ignite.*;
 +import org.apache.ignite.cache.*;
 +import org.apache.ignite.cache.eviction.lru.*;
 +import org.apache.ignite.configuration.*;
 +import org.apache.ignite.spi.discovery.tcp.*;
 +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 static org.apache.ignite.cache.CacheAtomicityMode.*;
 +import static org.apache.ignite.cache.CacheMode.*;
 +import static org.apache.ignite.cache.CacheRebalanceMode.*;
 +import static org.apache.ignite.cache.CacheWriteSynchronizationMode.*;
 +
 +/**
 + * LRU near eviction tests for NEAR_ONLY distribution mode (GG-8884).
 + */
 +public class LruNearOnlyNearEvictionPolicySelfTest extends GridCommonAbstractTest {
 +    /** */
 +    private static final TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
 +
 +    /** Grid count. */
 +    private static final int GRID_COUNT = 2;
 +
 +    /** Maximum size for near eviction policy. */
 +    private static final int EVICTION_MAX_SIZE = 10;
 +
 +    /** Node count. */
 +    private int cnt;
 +
 +    /** Caching mode specified by test. */
 +    private CacheMode cacheMode;
 +
 +    /** Cache atomicity mode specified by test. */
 +    private CacheAtomicityMode atomicityMode;
 +
 +    /** {@inheritDoc} */
 +    @Override protected void beforeTest() throws Exception {
 +        super.beforeTest();
 +
 +        cnt = 0;
 +    }
 +
 +    /** {@inheritDoc} */
 +    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
 +        IgniteConfiguration c = super.getConfiguration(gridName);
 +
 +        if (cnt == 0)
 +            c.setClientMode(true);
++        else {
++            CacheConfiguration cc = new CacheConfiguration();
 +
-         CacheConfiguration cc = new CacheConfiguration();
++            cc.setAtomicityMode(atomicityMode);
++            cc.setCacheMode(cacheMode);
++            cc.setWriteSynchronizationMode(PRIMARY_SYNC);
++            cc.setRebalanceMode(SYNC);
++            cc.setStartSize(100);
++            cc.setBackups(0);
 +
-         cc.setAtomicityMode(atomicityMode);
-         cc.setCacheMode(cacheMode);
-         cc.setWriteSynchronizationMode(PRIMARY_SYNC);
-         cc.setRebalanceMode(SYNC);
-         cc.setStartSize(100);
-         cc.setBackups(0);
++            c.setCacheConfiguration(cc);
++        }
 +
-         c.setCacheConfiguration(cc);
- 
-         TcpDiscoverySpi disco = new TcpDiscoverySpi();
- 
-         disco.setIpFinder(ipFinder);
- 
-         c.setDiscoverySpi(disco);
++        c.setDiscoverySpi(new TcpDiscoverySpi().setIpFinder(ipFinder).setForceServerMode(true));
 +
 +        cnt++;
 +
 +        return c;
 +    }
 +
 +    /**
 +     * @throws Exception If failed.
 +     */
 +    public void testPartitionedAtomicNearEvictionMaxSize() throws Exception {
 +        atomicityMode = ATOMIC;
 +        cacheMode = PARTITIONED;
 +
 +        checkNearEvictionMaxSize();
 +    }
 +
 +    /**
 +     * @throws Exception If failed.
 +     */
 +    public void testPartitionedTransactionalNearEvictionMaxSize() throws Exception {
 +        atomicityMode = TRANSACTIONAL;
 +        cacheMode = PARTITIONED;
 +
 +        checkNearEvictionMaxSize();
 +    }
 +
 +    /**
 +     * @throws Exception If failed.
 +     */
 +    public void testReplicatedAtomicNearEvictionMaxSize() throws Exception {
 +        atomicityMode = ATOMIC;
 +        cacheMode = REPLICATED;
 +
 +        checkNearEvictionMaxSize();
 +    }
 +
 +    /**
 +     * @throws Exception If failed.
 +     */
 +    public void testReplicatedTransactionalNearEvictionMaxSize() throws Exception {
 +        atomicityMode = TRANSACTIONAL;
 +        cacheMode = REPLICATED;
 +
 +        checkNearEvictionMaxSize();
 +    }
 +
 +    /**
 +     * @throws Exception If failed.
 +     */
 +    private void checkNearEvictionMaxSize() throws Exception {
 +        startGrids(GRID_COUNT);
 +
 +        try {
 +            NearCacheConfiguration nearCfg = new NearCacheConfiguration();
 +
 +            LruEvictionPolicy plc = new LruEvictionPolicy();
 +            plc.setMaxSize(EVICTION_MAX_SIZE);
 +
 +            nearCfg.setNearEvictionPolicy(plc);
 +
 +            grid(0).createNearCache(null, nearCfg);
 +
 +            int cnt = 1000;
 +
 +            info("Inserting " + cnt + " keys to cache.");
 +
 +            try (IgniteDataStreamer<Integer, String> ldr = grid(1).dataStreamer(null)) {
 +                for (int i = 0; i < cnt; i++)
 +                    ldr.addData(i, Integer.toString(i));
 +            }
 +
 +            assertTrue("Near cache size " + near(0).nearSize() + ", but eviction maximum size " + EVICTION_MAX_SIZE,
 +                near(0).nearSize() <= EVICTION_MAX_SIZE);
 +
 +            info("Getting " + cnt + " keys from cache.");
 +
 +            for (int i = 0; i < cnt; i++) {
 +                IgniteCache<Integer, String> cache = grid(0).cache(null);
 +
 +                assertTrue(cache.get(i).equals(Integer.toString(i)));
 +            }
 +
 +            assertTrue("Near cache size " + near(0).nearSize() + ", but eviction maximum size " + EVICTION_MAX_SIZE,
 +                near(0).nearSize() <= EVICTION_MAX_SIZE);
 +        }
 +        finally {
 +            stopAllGrids();
 +        }
 +    }
 +}


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

Posted by sb...@apache.org.
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/e5d5d08d
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/e5d5d08d
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/e5d5d08d

Branch: refs/heads/ignite-sprint-5
Commit: e5d5d08d91fc0564b655c6e45afe0760957bff0c
Parents: 5c30f9c
Author: agura <ag...@gridgain.com>
Authored: Mon May 18 21:01:36 2015 +0300
Committer: agura <ag...@gridgain.com>
Committed: Fri May 29 21:12:32 2015 +0300

----------------------------------------------------------------------
 .../ignite/cache/eviction/EvictableEntry.java   |    7 +
 .../ignite/cache/eviction/EvictionPolicy.java   |    2 +
 .../cache/eviction/fifo/FifoEvictionPolicy.java |  117 +-
 .../eviction/fifo/FifoEvictionPolicyMBean.java  |   22 +
 .../cache/eviction/lru/LruEvictionPolicy.java   |  135 ++-
 .../eviction/lru/LruEvictionPolicyMBean.java    |   38 +
 .../eviction/random/RandomEvictionPolicy.java   |    7 +-
 .../eviction/sorted/SortedEvictionPolicy.java   |  141 ++-
 .../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       |   15 +-
 .../cache/GridIndexingWithNoopSwapSelfTest.java |    6 +-
 .../IgniteCacheQueryMultiThreadedSelfTest.java  |   11 +-
 .../cache/ttl/CacheTtlAbstractSelfTest.java     |    6 +-
 .../org/apache/ignite/yardstick/IgniteNode.java |    7 +-
 55 files changed, 3480 insertions(+), 3234 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e5d5d08d/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/e5d5d08d/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/e5d5d08d/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..7222ae4 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,11 +51,17 @@ 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<>();
@@ -63,7 +79,7 @@ public class FifoEvictionPolicy<K, V> implements EvictionPolicy<K, V>, FifoEvict
      * @param max Maximum allowed size of cache before entry will start getting evicted.
      */
     public FifoEvictionPolicy(int max) {
-        A.ensure(max > 0, "max > 0");
+        A.ensure(max >= 0, "max >= 0");
 
         this.max = max;
     }
@@ -75,7 +91,7 @@ public class FifoEvictionPolicy<K, V> implements EvictionPolicy<K, V>, FifoEvict
      * @param batchSize Batch size.
      */
     public FifoEvictionPolicy(int max, int batchSize) {
-        A.ensure(max > 0, "max > 0");
+        A.ensure(max >= 0, "max >= 0");
         A.ensure(batchSize > 0, "batchSize > 0");
 
         this.max = max;
@@ -97,7 +113,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 +135,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 +174,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 +209,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 +232,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;
+        }
+    }
 
-        int startSize = queue.sizex();
+    /**
+     * 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();
 
-        // 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();
+        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())
-                    touch(entry);
-            }
+        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/e5d5d08d/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/e5d5d08d/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..d56f99a 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,26 +18,48 @@
 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;
+
+    /** Batch size. */
+    private volatile int batchSize = 1;
+
+    /** Max memory size. */
+    private volatile long maxMemSize;
+
+    /** Memory size. */
+    private final LongAdder8 memSize = new LongAdder8();
 
     /** Queue. */
     private final ConcurrentLinkedDeque8<EvictableEntry<K, V>> queue =
@@ -56,7 +78,7 @@ public class LruEvictionPolicy<K, V> implements EvictionPolicy<K, V>, LruEvictio
      * @param max Maximum allowed size of cache before entry will start getting evicted.
      */
     public LruEvictionPolicy(int max) {
-        A.ensure(max > 0, "max > 0");
+        A.ensure(max >= 0, "max >= 0");
 
         this.max = max;
     }
@@ -76,16 +98,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 +158,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 +193,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 +224,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();
 
-        for (int i = 0; i < startSize && queue.sizex() > max; i++) {
-            EvictableEntry<K, V> entry = queue.poll();
+            if (startSize >= max + (maxMem > 0 ? 1 : this.batchSize))
+                for (int i = max; i < startSize && queue.sizex() > max; i++)
+                    if (shrink0() == -1)
+                        break;
+        }
+    }
 
-            if (entry == null)
-                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();
 
-            Node<EvictableEntry<K, V>> meta = entry.removeMeta();
+        if (entry == null)
+            return -1;
 
-            if (meta != null && !entry.evict())
+        int size = 0;
+
+        Node<EvictableEntry<K, V>> meta = entry.removeMeta();
+
+        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/e5d5d08d/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/e5d5d08d/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..0d840e3 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,7 +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;
+    private volatile int max = DFLT_CACHE_SIZE;
 
     /**
      * Constructs random eviction policy with all defaults.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e5d5d08d/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..abfc1d0 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,18 +56,24 @@ 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;
 
     /**
@@ -96,7 +110,7 @@ public class SortedEvictionPolicy<K, V> implements EvictionPolicy<K, V>, SortedE
      * @param comp Entries comparator.
      */
     public SortedEvictionPolicy(int max, int batchSize, @Nullable Comparator<EvictableEntry<K, V>> comp) {
-        A.ensure(max > 0, "max > 0");
+        A.ensure(max >= 0, "max >= 0");
         A.ensure(batchSize > 0, "batchSize > 0");
 
         this.max = max;
@@ -106,6 +120,16 @@ public class SortedEvictionPolicy<K, V> implements EvictionPolicy<K, V>, SortedE
     }
 
     /**
+     * Constructs sorted eviction policy with given maximum size and given entry comparator.
+     *
+     * @param comp Entries comparator.
+     */
+    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);
+    }
+
+    /**
      * Gets maximum allowed size of cache before entry will start getting evicted.
      *
      * @return Maximum allowed size of cache before entry will start getting evicted.
@@ -120,7 +144,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 +166,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 +209,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 +232,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 +240,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 +266,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;
+
+        if (maxMem > 0) {
+            long startMemSize = memSize.longValue();
 
-        int batchSize = this.batchSize;
+            if (startMemSize >= maxMem)
+                for (long i = maxMem; i < startMemSize && memSize.longValue() > maxMem;) {
+                    int size = shrink0();
 
-        int startSize = set.sizex();
+                    if (size == -1)
+                        break;
 
-        if (startSize >= max + batchSize) {
-            for (int i = max; i < startSize && set.sizex() > max; i++) {
-                Holder<K, V> h = set.pollFirst();
+                    i += size;
+                }
+        }
 
-                if (h == null)
-                    break;
+        int max = this.max;
 
-                EvictableEntry<K, V> entry = h.entry;
+        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 +339,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/e5d5d08d/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/e5d5d08d/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/e5d5d08d/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/e5d5d08d/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/e5d5d08d/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/e5d5d08d/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/e5d5d08d/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/e5d5d08d/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/e5d5d08d/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/e5d5d08d/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/e5d5d08d/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/e5d5d08d/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/e5d5d08d/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/e5d5d08d/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);


[7/8] incubator-ignite git commit: ignite-916 Review fixes

Posted by sb...@apache.org.
ignite-916 Review fixes


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

Branch: refs/heads/ignite-sprint-5
Commit: fbb5763a6d548d348a0617f0d770a34534b675fb
Parents: adc7b23
Author: agura <ag...@gridgain.com>
Authored: Tue Jun 2 18:18:22 2015 +0300
Committer: agura <ag...@gridgain.com>
Committed: Tue Jun 2 18:18:22 2015 +0300

----------------------------------------------------------------------
 .../ignite/cache/eviction/fifo/FifoEvictionPolicy.java |  6 +++---
 .../cache/eviction/fifo/FifoEvictionPolicyMBean.java   |  6 +++---
 .../ignite/cache/eviction/lru/LruEvictionPolicy.java   |  6 +++---
 .../cache/eviction/lru/LruEvictionPolicyMBean.java     |  6 +++---
 .../cache/eviction/sorted/SortedEvictionPolicy.java    |  6 +++---
 .../eviction/sorted/SortedEvictionPolicyMBean.java     |  6 +++---
 ...ridCachePartitionedMultiThreadedPutGetSelfTest.java |  2 +-
 .../cache/eviction/EvictionAbstractTest.java           | 13 ++++++-------
 .../eviction/fifo/FifoEvictionPolicySelfTest.java      |  2 +-
 .../cache/eviction/lru/LruEvictionPolicySelfTest.java  |  4 ++--
 .../eviction/sorted/SortedEvictionPolicySelfTest.java  |  2 +-
 .../testsuites/IgniteCacheEvictionSelfTestSuite.java   |  1 -
 12 files changed, 29 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fbb5763a/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 7222ae4..221bc39 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
@@ -136,19 +136,19 @@ public class FifoEvictionPolicy<K, V> implements EvictionPolicy<K, V>, FifoEvict
     }
 
     /** {@inheritDoc} */
-    @Override public long getMaxMemSize() {
+    @Override public long getMaxMemorySize() {
         return maxMemSize;
     }
 
     /** {@inheritDoc} */
-    @Override public void setMaxMemSize(long maxMemSize) {
+    @Override public void setMaxMemorySize(long maxMemSize) {
         A.ensure(maxMemSize >= 0, "maxMemSize >= 0");
 
         this.maxMemSize = maxMemSize;
     }
 
     /** {@inheritDoc} */
-    @Override public long getCurrentMemSize() {
+    @Override public long getCurrentMemorySize() {
         return memSize.longValue();
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fbb5763a/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 c9a09ec..793aa66 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
@@ -70,13 +70,13 @@ public interface FifoEvictionPolicyMBean {
      * @return maximum allowed cache size in bytes.
      */
     @MXBeanDescription("Maximum allowed cache size in bytes.")
-    public long getMaxMemSize();
+    public long getMaxMemorySize();
 
     /**
      * Sets maximum allowed cache size in bytes.
      */
     @MXBeanDescription("Set maximum allowed cache size in bytes.")
-    public void setMaxMemSize(long maxMemSize);
+    public void setMaxMemorySize(long maxMemSize);
 
     /**
      * Gets current queue size in bytes.
@@ -84,5 +84,5 @@ public interface FifoEvictionPolicyMBean {
      * @return current queue size in bytes.
      */
     @MXBeanDescription("Current FIFO queue size in bytes.")
-    public long getCurrentMemSize();
+    public long getCurrentMemorySize();
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fbb5763a/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 d56f99a..0be26c8 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
@@ -121,19 +121,19 @@ public class LruEvictionPolicy<K, V> implements EvictionPolicy<K, V>, LruEvictio
     }
 
     /** {@inheritDoc} */
-    @Override public long getMaxMemSize() {
+    @Override public long getMaxMemorySize() {
         return maxMemSize;
     }
 
     /** {@inheritDoc} */
-    @Override public void setMaxMemSize(long maxMemSize) {
+    @Override public void setMaxMemorySize(long maxMemSize) {
         A.ensure(maxMemSize >= 0, "maxMemSize >= 0");
 
         this.maxMemSize = maxMemSize;
     }
 
     /** {@inheritDoc} */
-    @Override public long getCurrentMemSize() {
+    @Override public long getCurrentMemorySize() {
         return memSize.longValue();
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fbb5763a/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 69347ee..e17c057 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
@@ -70,13 +70,13 @@ public interface LruEvictionPolicyMBean {
      * @return maximum allowed cache size in bytes.
      */
     @MXBeanDescription("Maximum allowed cache size in bytes.")
-    public long getMaxMemSize();
+    public long getMaxMemorySize();
 
     /**
      * Sets maximum allowed cache size in bytes.
      */
     @MXBeanDescription("Set maximum allowed cache size in bytes.")
-    public void setMaxMemSize(long maxMemSize);
+    public void setMaxMemorySize(long maxMemSize);
 
     /**
      * Gets current queue size in bytes.
@@ -84,5 +84,5 @@ public interface LruEvictionPolicyMBean {
      * @return current queue size in bytes.
      */
     @MXBeanDescription("Current queue size in  bytes.")
-    public long getCurrentMemSize();
+    public long getCurrentMemorySize();
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fbb5763a/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 abfc1d0..b8b82fa 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
@@ -167,19 +167,19 @@ public class SortedEvictionPolicy<K, V> implements EvictionPolicy<K, V>, SortedE
     }
 
     /** {@inheritDoc} */
-    @Override public long getMaxMemSize() {
+    @Override public long getMaxMemorySize() {
         return maxMemSize;
     }
 
     /** {@inheritDoc} */
-    @Override public void setMaxMemSize(long maxMemSize) {
+    @Override public void setMaxMemorySize(long maxMemSize) {
         A.ensure(maxMemSize >= 0, "maxMemSize >= 0");
 
         this.maxMemSize = maxMemSize;
     }
 
     /** {@inheritDoc} */
-    @Override public long getCurrentMemSize() {
+    @Override public long getCurrentMemorySize() {
         return memSize.longValue();
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fbb5763a/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 aada1ce..7283453 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
@@ -70,13 +70,13 @@ public interface SortedEvictionPolicyMBean {
      * @return maximum allowed cache size in bytes.
      */
     @MXBeanDescription("Maximum allowed cache size in bytes.")
-    public long getMaxMemSize();
+    public long getMaxMemorySize();
 
     /**
      * Sets maximum allowed cache size in bytes.
      */
     @MXBeanDescription("Set maximum allowed cache size in bytes.")
-    public void setMaxMemSize(long maxMemSize);
+    public void setMaxMemorySize(long maxMemSize);
 
     /**
      * Gets current sorted entries queue size in bytes.
@@ -84,5 +84,5 @@ public interface SortedEvictionPolicyMBean {
      * @return current sorted entries queue size in bytes.
      */
     @MXBeanDescription("Current sorted entries set size in bytes.")
-    public long getCurrentMemSize();
+    public long getCurrentMemorySize();
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fbb5763a/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 1c45de8..feb94b2 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
@@ -68,7 +68,7 @@ public class GridCachePartitionedMultiThreadedPutGetSelfTest extends GridCommonA
         cc.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
 
         FifoEvictionPolicy plc = new FifoEvictionPolicy();
-        plc.setMaxMemSize(1000);
+        plc.setMaxMemorySize(1000);
 
         cc.setEvictionPolicy(plc);
         cc.setSwapEnabled(false);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fbb5763a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/EvictionAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/EvictionAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/EvictionAbstractTest.java
index c2a599d..a8da54b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/EvictionAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/EvictionAbstractTest.java
@@ -460,7 +460,7 @@ public abstract class EvictionAbstractTest<T extends EvictionPolicy<?, ?>>
             info(p);
 
             assertTrue(p.getCurrentSize() <= (plcMaxMemSize > 0 ? max : max + plcBatchSize));
-            assertTrue(p.getCurrentMemSize() <= (plcMaxMemSize > 0 ? max : max + plcBatchSize) * MockEntry.KEY_SIZE);
+            assertTrue(p.getCurrentMemorySize() <= (plcMaxMemSize > 0 ? max : max + plcBatchSize) * MockEntry.KEY_SIZE);
         }
         finally {
             stopAllGrids();
@@ -575,7 +575,7 @@ public abstract class EvictionAbstractTest<T extends EvictionPolicy<?, ?>>
         EvictionPolicyProxy proxy = proxy(policy());
 
         assertEquals(expSize, proxy.getCurrentSize());
-        assertEquals(expSize * entrySize, proxy.getCurrentMemSize());
+        assertEquals(expSize * entrySize, proxy.getCurrentMemorySize());
     }
 
     /**
@@ -798,7 +798,7 @@ public abstract class EvictionAbstractTest<T extends EvictionPolicy<?, ?>>
 
                         EvictionPolicyProxy plc = proxy(policy(i));
 
-                        assertTrue(plc.getCurrentMemSize() <= memSize);
+                        assertTrue(plc.getCurrentMemorySize() <= memSize);
                     }
                 }
 
@@ -904,7 +904,7 @@ public abstract class EvictionAbstractTest<T extends EvictionPolicy<?, ?>>
                 for (EvictableEntry entry : proxy(policy(i)).queue())
                     size += entry.size();
 
-                assertEquals(size, proxy(policy(i)).getCurrentMemSize());
+                assertEquals(size, proxy(policy(i)).getCurrentMemorySize());
             }
             else
                 assertTrue(proxy(policy(i)).queue().size() <= plcMax + plcBatchSize);
@@ -1017,9 +1017,9 @@ public abstract class EvictionAbstractTest<T extends EvictionPolicy<?, ?>>
         /**
          * Current memory size.
          */
-        public long getCurrentMemSize() {
+        public long getCurrentMemorySize() {
             try {
-                return (Long)plc.getClass().getDeclaredMethod("getCurrentMemSize").invoke(plc);
+                return (Long)plc.getClass().getDeclaredMethod("getCurrentMemorySize").invoke(plc);
             }
             catch (NoSuchMethodException | InvocationTargetException | IllegalAccessException e) {
                 throw new RuntimeException(e);
@@ -1053,5 +1053,4 @@ public abstract class EvictionAbstractTest<T extends EvictionPolicy<?, ?>>
             }
         }
     }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fbb5763a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/fifo/FifoEvictionPolicySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/fifo/FifoEvictionPolicySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/fifo/FifoEvictionPolicySelfTest.java
index a908259..6b6ba51 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/fifo/FifoEvictionPolicySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/fifo/FifoEvictionPolicySelfTest.java
@@ -245,7 +245,7 @@ public class FifoEvictionPolicySelfTest extends
 
         plc.setMaxSize(this.plcMax);
         plc.setBatchSize(this.plcBatchSize);
-        plc.setMaxMemSize(this.plcMaxMemSize);
+        plc.setMaxMemorySize(this.plcMaxMemSize);
 
         return plc;
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fbb5763a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/LruEvictionPolicySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/LruEvictionPolicySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/LruEvictionPolicySelfTest.java
index 8e1e8cf..3750e5a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/LruEvictionPolicySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/LruEvictionPolicySelfTest.java
@@ -312,7 +312,7 @@ public class LruEvictionPolicySelfTest extends
 
         plc.setMaxSize(this.plcMax);
         plc.setBatchSize(this.plcBatchSize);
-        plc.setMaxMemSize(this.plcMaxMemSize);
+        plc.setMaxMemorySize(this.plcMaxMemSize);
 
         return plc;
     }
@@ -343,7 +343,7 @@ public class LruEvictionPolicySelfTest extends
                 for (EvictableEntry<String, String> entry : policy(i).queue())
                     size += ((CacheEvictableEntryImpl)entry).size();
 
-                assertEquals(size, ((LruEvictionPolicy)policy(i)).getCurrentMemSize());
+                assertEquals(size, ((LruEvictionPolicy)policy(i)).getCurrentMemorySize());
             }
             else
                 assertTrue(policy(i).queue().size() <= plcMax + plcBatchSize);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fbb5763a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/sorted/SortedEvictionPolicySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/sorted/SortedEvictionPolicySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/sorted/SortedEvictionPolicySelfTest.java
index dbbbe5d..a283352 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/sorted/SortedEvictionPolicySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/sorted/SortedEvictionPolicySelfTest.java
@@ -248,7 +248,7 @@ public class SortedEvictionPolicySelfTest extends
 
         plc.setMaxSize(this.plcMax);
         plc.setBatchSize(this.plcBatchSize);
-        plc.setMaxMemSize(this.plcMaxMemSize);
+        plc.setMaxMemorySize(this.plcMaxMemSize);
 
         return plc;
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fbb5763a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheEvictionSelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheEvictionSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheEvictionSelfTestSuite.java
index 917185e..7f07d3d 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheEvictionSelfTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheEvictionSelfTestSuite.java
@@ -37,7 +37,6 @@ public class IgniteCacheEvictionSelfTestSuite extends TestSuite {
     public static TestSuite suite() throws Exception {
         TestSuite suite = new TestSuite("Ignite Cache Eviction Test Suite");
 
-        suite.addTest(new TestSuite(RandomEvictionPolicyCacheSizeSelfTest.class));
         suite.addTest(new TestSuite(FifoEvictionPolicySelfTest.class));
         suite.addTest(new TestSuite(SortedEvictionPolicySelfTest.class));
         suite.addTest(new TestSuite(LruEvictionPolicySelfTest.class));


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

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e5d5d08d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/fifo/GridCacheFifoEvictionPolicySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/fifo/GridCacheFifoEvictionPolicySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/fifo/GridCacheFifoEvictionPolicySelfTest.java
deleted file mode 100644
index 64d3831..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/fifo/GridCacheFifoEvictionPolicySelfTest.java
+++ /dev/null
@@ -1,372 +0,0 @@
-/*
- * 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.eviction.fifo;
-
-import org.apache.ignite.*;
-import org.apache.ignite.cache.eviction.*;
-import org.apache.ignite.cache.eviction.fifo.*;
-import org.apache.ignite.internal.processors.cache.eviction.*;
-
-import java.util.*;
-
-import static org.apache.ignite.cache.CacheMode.*;
-
-/**
- * FIFO Eviction test.
- */
-@SuppressWarnings({"TypeMayBeWeakened"})
-public class GridCacheFifoEvictionPolicySelfTest extends
-    GridCacheEvictionAbstractTest<FifoEvictionPolicy<String, String>> {
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPolicy() throws Exception {
-        try {
-            startGrid();
-
-            MockEntry e1 = new MockEntry("1", "1");
-            MockEntry e2 = new MockEntry("2", "2");
-            MockEntry e3 = new MockEntry("3", "3");
-            MockEntry e4 = new MockEntry("4", "4");
-            MockEntry e5 = new MockEntry("5", "5");
-
-            FifoEvictionPolicy<String, String> p = policy();
-
-            p.setMaxSize(3);
-
-            p.onEntryAccessed(false, e1);
-
-            check(p.queue(), e1);
-
-            p.onEntryAccessed(false, e2);
-
-            check(p.queue(), e1, e2);
-
-            p.onEntryAccessed(false, e3);
-
-            check(p.queue(), e1, e2, e3);
-
-            assert !e1.isEvicted();
-            assert !e2.isEvicted();
-            assert !e3.isEvicted();
-
-            assertEquals(3, p.getCurrentSize());
-
-            p.onEntryAccessed(false, e4);
-
-            check(p.queue(), e2, e3, e4);
-
-            assertEquals(3, p.getCurrentSize());
-
-            assert e1.isEvicted();
-            assert !e2.isEvicted();
-            assert !e3.isEvicted();
-            assert !e4.isEvicted();
-
-            p.onEntryAccessed(false, e5);
-
-            check(p.queue(), e3, e4, e5);
-
-            assertEquals(3, p.getCurrentSize());
-
-            assert e2.isEvicted();
-            assert !e3.isEvicted();
-            assert !e4.isEvicted();
-            assert !e5.isEvicted();
-
-            p.onEntryAccessed(false, e1 = new MockEntry("1", "1"));
-
-            check(p.queue(), e4, e5, e1);
-
-            assertEquals(3, p.getCurrentSize());
-
-            assert e3.isEvicted();
-            assert !e1.isEvicted();
-            assert !e4.isEvicted();
-            assert !e5.isEvicted();
-
-            p.onEntryAccessed(false, e5);
-
-            check(p.queue(), e4, e5, e1);
-
-            assert !e1.isEvicted();
-            assert !e4.isEvicted();
-            assert !e5.isEvicted();
-
-            p.onEntryAccessed(false, e1);
-
-            assertEquals(3, p.getCurrentSize());
-
-            check(p.queue(), e4, e5, e1);
-
-            assert !e1.isEvicted();
-            assert !e4.isEvicted();
-            assert !e5.isEvicted();
-
-            p.onEntryAccessed(false, e5);
-
-            assertEquals(3, p.getCurrentSize());
-
-            check(p.queue(), e4, e5, e1);
-
-            assert !e1.isEvicted();
-            assert !e4.isEvicted();
-            assert !e5.isEvicted();
-
-            p.onEntryAccessed(true, e1);
-
-            assertEquals(2, p.getCurrentSize());
-
-            assert !e1.isEvicted();
-            assert !e4.isEvicted();
-            assert !e5.isEvicted();
-
-            p.onEntryAccessed(true, e4);
-
-            assertEquals(1, p.getCurrentSize());
-
-            assert !e4.isEvicted();
-            assert !e5.isEvicted();
-
-            p.onEntryAccessed(true, e5);
-
-            assertEquals(0, p.getCurrentSize());
-
-            assert !e5.isEvicted();
-
-            info(p);
-        }
-        finally {
-            stopAllGrids();
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testMemory() throws Exception {
-        try {
-            startGrid();
-
-            FifoEvictionPolicy<String, String> p = policy();
-
-            int max = 10;
-
-            p.setMaxSize(max);
-
-            int cnt = 11;
-
-            for (int i = 0; i < cnt; i++)
-                p.onEntryAccessed(false, new MockEntry(Integer.toString(i), Integer.toString(i)));
-
-            info(p);
-
-            assertEquals(max, p.getCurrentSize());
-        }
-        finally {
-            stopAllGrids();
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testRandom() throws Exception {
-        try {
-            startGrid();
-
-            FifoEvictionPolicy<String, String> p = policy();
-
-            int max = 10;
-
-            p.setMaxSize(max);
-
-            Random rand = new Random();
-
-            int keys = 31;
-
-            MockEntry[] fifos = new MockEntry[keys];
-
-            for (int i = 0; i < fifos.length; i++)
-                fifos[i] = new MockEntry(Integer.toString(i));
-
-            int runs = 5000000;
-
-            for (int i = 0; i < runs; i++) {
-                boolean rmv = rand.nextBoolean();
-
-                int j = rand.nextInt(fifos.length);
-
-                MockEntry e = entry(fifos, j);
-
-                if (rmv)
-                    fifos[j] = new MockEntry(Integer.toString(j));
-
-                p.onEntryAccessed(rmv, e);
-            }
-
-            info(p);
-
-            int curSize = p.getCurrentSize();
-
-            assert curSize <= max : "curSize <= max [curSize=" + curSize + ", max=" + max + ']';
-        }
-        finally {
-            stopAllGrids();
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testAllowEmptyEntries() throws Exception {
-        try {
-            startGrid();
-
-            MockEntry e1 = new MockEntry("1");
-
-            MockEntry e2 = new MockEntry("2");
-
-            MockEntry e3 = new MockEntry("3");
-
-            MockEntry e4 = new MockEntry("4");
-
-            MockEntry e5 = new MockEntry("5");
-
-            FifoEvictionPolicy<String, String> p = policy();
-
-            p.setMaxSize(10);
-
-            p.onEntryAccessed(false, e1);
-
-            assertFalse(e1.isEvicted());
-
-            p.onEntryAccessed(false, e2);
-
-            assertFalse(e1.isEvicted());
-            assertFalse(e2.isEvicted());
-
-            p.onEntryAccessed(false, e3);
-
-            assertFalse(e1.isEvicted());
-            assertFalse(e3.isEvicted());
-
-            p.onEntryAccessed(false, e4);
-
-            assertFalse(e1.isEvicted());
-            assertFalse(e3.isEvicted());
-            assertFalse(e4.isEvicted());
-
-            p.onEntryAccessed(false, e5);
-
-            assertFalse(e1.isEvicted());
-            assertFalse(e3.isEvicted());
-            assertFalse(e5.isEvicted());
-        }
-        finally {
-            stopAllGrids();
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPut() throws Exception {
-        mode = LOCAL;
-        syncCommit = true;
-        plcMax = 100;
-
-        Ignite ignite = startGrid();
-
-        try {
-            IgniteCache<Object, Object> cache = ignite.cache(null);
-
-            int cnt = 500;
-
-            int min = Integer.MAX_VALUE;
-
-            int minIdx = 0;
-
-            for (int i = 0; i < cnt; i++) {
-                cache.put(i, i);
-
-                int cacheSize = cache.size();
-
-                if (i > plcMax && cacheSize < min) {
-                    min = cacheSize;
-                    minIdx = i;
-                }
-            }
-
-            assert min >= plcMax : "Min cache size is too small: " + min;
-
-            info("Min cache size [min=" + min + ", idx=" + minIdx + ']');
-            info("Current cache size " + cache.size());
-            info("Current cache key size " + cache.size());
-
-            min = Integer.MAX_VALUE;
-
-            minIdx = 0;
-
-            // Touch.
-            for (int i = cnt; --i > cnt - plcMax;) {
-                cache.get(i);
-
-                int cacheSize = cache.size();
-
-                if (cacheSize < min) {
-                    min = cacheSize;
-                    minIdx = i;
-                }
-            }
-
-            info("----");
-            info("Min cache size [min=" + min + ", idx=" + minIdx + ']');
-            info("Current cache size " + cache.size());
-            info("Current cache key size " + cache.size());
-
-            assert min >= plcMax : "Min cache size is too small: " + min;
-        }
-        finally {
-            stopAllGrids();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override protected FifoEvictionPolicy<String, String> createPolicy(int plcMax) {
-        return new FifoEvictionPolicy<>(plcMax);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected FifoEvictionPolicy<String, String> createNearPolicy(int nearMax) {
-        return new FifoEvictionPolicy<>(nearMax);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void checkNearPolicies(int endNearPlcSize) {
-        for (int i = 0; i < gridCnt; i++)
-            for (EvictableEntry<String, String> e : nearPolicy(i).queue())
-                assert !e.isCached() : "Invalid near policy size: " + nearPolicy(i).queue();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void checkPolicies(int plcMax) {
-        for (int i = 0; i < gridCnt; i++)
-            assert policy(i).queue().size() <= plcMax;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e5d5d08d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/GridCacheLruEvictionPolicySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/GridCacheLruEvictionPolicySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/GridCacheLruEvictionPolicySelfTest.java
deleted file mode 100644
index c623b38..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/GridCacheLruEvictionPolicySelfTest.java
+++ /dev/null
@@ -1,417 +0,0 @@
-/*
- * 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.eviction.lru;
-
-import org.apache.ignite.*;
-import org.apache.ignite.cache.*;
-import org.apache.ignite.cache.eviction.*;
-import org.apache.ignite.cache.eviction.lru.*;
-import org.apache.ignite.internal.processors.cache.eviction.*;
-
-import java.util.*;
-
-/**
- * LRU Eviction test.
- */
-@SuppressWarnings( {"TypeMayBeWeakened"})
-public class GridCacheLruEvictionPolicySelfTest extends
-    GridCacheEvictionAbstractTest<LruEvictionPolicy<String, String>> {
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPolicy() throws Exception {
-        startGrid();
-
-        try {
-            MockEntry e1 = new MockEntry("1", "1");
-            MockEntry e2 = new MockEntry("2", "2");
-            MockEntry e3 = new MockEntry("3", "3");
-            MockEntry e4 = new MockEntry("4", "4");
-            MockEntry e5 = new MockEntry("5", "5");
-
-            LruEvictionPolicy<String, String> p = policy();
-
-            p.setMaxSize(3);
-
-            p.onEntryAccessed(false, e1);
-
-            check(p.queue(), e1);
-
-            p.onEntryAccessed(false, e2);
-
-            check(p.queue(), e1, e2);
-
-            p.onEntryAccessed(false, e3);
-
-            check(p.queue(), e1, e2, e3);
-
-            assert !e1.isEvicted();
-            assert !e2.isEvicted();
-            assert !e3.isEvicted();
-
-            assertEquals(3, p.getCurrentSize());
-
-            p.onEntryAccessed(false, e4);
-
-            check(p.queue(), e2, e3, e4);
-
-            assertEquals(3, p.getCurrentSize());
-
-            assert e1.isEvicted();
-            assert !e2.isEvicted();
-            assert !e3.isEvicted();
-            assert !e4.isEvicted();
-
-            p.onEntryAccessed(false, e5);
-
-            check(p.queue(), e3, e4, e5);
-
-            assertEquals(3, p.getCurrentSize());
-
-            assert e2.isEvicted();
-            assert !e3.isEvicted();
-            assert !e4.isEvicted();
-            assert !e5.isEvicted();
-
-            p.onEntryAccessed(false, e1 = new MockEntry("1", "1"));
-
-            check(p.queue(), e4, e5, e1);
-
-            assertEquals(3, p.getCurrentSize());
-
-            assert e3.isEvicted();
-            assert !e1.isEvicted();
-            assert !e4.isEvicted();
-            assert !e5.isEvicted();
-
-            p.onEntryAccessed(false, e5);
-
-            assertEquals(3, p.getCurrentSize());
-
-            check(p.queue(), e4, e1, e5);
-
-            assert !e1.isEvicted();
-            assert !e4.isEvicted();
-            assert !e5.isEvicted();
-
-            p.onEntryAccessed(false, e1);
-
-            assertEquals(3, p.getCurrentSize());
-
-            check(p.queue(), e4, e5, e1);
-
-            assert !e1.isEvicted();
-            assert !e4.isEvicted();
-            assert !e5.isEvicted();
-
-            p.onEntryAccessed(false, e5);
-
-            assertEquals(3, p.getCurrentSize());
-
-            check(p.queue(), e4, e1, e5);
-
-            assert !e1.isEvicted();
-            assert !e4.isEvicted();
-            assert !e5.isEvicted();
-
-            p.onEntryAccessed(true, e1);
-
-            assertEquals(2, p.getCurrentSize());
-
-            assert !e1.isEvicted();
-            assert !e4.isEvicted();
-            assert !e5.isEvicted();
-
-            p.onEntryAccessed(true, e4);
-
-            assertEquals(1, p.getCurrentSize());
-
-            assert !e4.isEvicted();
-            assert !e5.isEvicted();
-
-            p.onEntryAccessed(true, e5);
-
-            assertEquals(0, p.getCurrentSize());
-
-            assert !e5.isEvicted();
-
-            info(p);
-        }
-        finally {
-            stopGrid();
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testMemory() throws Exception {
-        startGrid();
-
-        try {
-            LruEvictionPolicy<String, String> p = policy();
-
-            int max = 10;
-
-            p.setMaxSize(max);
-
-            int cnt = 11;
-
-            for (int i = 0; i < cnt; i++)
-                p.onEntryAccessed(false, new MockEntry(Integer.toString(i), Integer.toString(i)));
-
-            info(p);
-
-            assertEquals(max, p.getCurrentSize());
-        }
-        finally {
-            stopGrid();
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testMiddleAccess() throws Exception {
-        startGrid();
-
-        try {
-            LruEvictionPolicy<String, String> p = policy();
-
-            int max = 8;
-
-            p.setMaxSize(max);
-
-            MockEntry entry1 = new MockEntry("1", "1");
-            MockEntry entry2 = new MockEntry("2", "2");
-            MockEntry entry3 = new MockEntry("3", "3");
-
-            p.onEntryAccessed(false, entry1);
-            p.onEntryAccessed(false, entry2);
-            p.onEntryAccessed(false, entry3);
-
-            MockEntry[] freqUsed = new MockEntry[] {
-                new MockEntry("4", "4"),
-                new MockEntry("5", "5"),
-                new MockEntry("6", "6"),
-                new MockEntry("7", "7"),
-                new MockEntry("8", "7")
-            };
-
-            for (MockEntry e : freqUsed)
-                p.onEntryAccessed(false, e);
-
-            for (MockEntry e : freqUsed)
-                assert !e.isEvicted();
-
-            int cnt = 1001;
-
-            for (int i = 0; i < cnt; i++)
-                p.onEntryAccessed(false, entry(freqUsed, i % freqUsed.length));
-
-            info(p);
-
-            assertEquals(max, p.getCurrentSize());
-        }
-        finally {
-            stopGrid();
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testRandom() throws Exception {
-        startGrid();
-
-        try {
-            LruEvictionPolicy<String, String> p = policy();
-
-            int max = 10;
-
-            p.setMaxSize(max);
-
-            Random rand = new Random();
-
-            int keys = 31;
-
-            MockEntry[] lrus = new MockEntry[keys];
-
-            for (int i = 0; i < lrus.length; i++)
-                lrus[i] = new MockEntry(Integer.toString(i));
-
-            int runs = 500000;
-
-            for (int i = 0; i < runs; i++) {
-                boolean rmv = rand.nextBoolean();
-
-                int j = rand.nextInt(lrus.length);
-
-                MockEntry e = entry(lrus, j);
-
-                if (rmv)
-                    lrus[j] = new MockEntry(Integer.toString(j));
-
-                p.onEntryAccessed(rmv, e);
-            }
-
-            info(p);
-
-            assert p.getCurrentSize() <= max;
-        }
-        finally {
-            stopGrid();
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testAllowEmptyEntries() throws Exception {
-        try {
-            startGrid();
-
-            MockEntry e1 = new MockEntry("1");
-
-            MockEntry e2 = new MockEntry("2");
-
-            MockEntry e3 = new MockEntry("3");
-
-            MockEntry e4 = new MockEntry("4");
-
-            MockEntry e5 = new MockEntry("5");
-
-            LruEvictionPolicy<String, String> p = policy();
-
-            p.setMaxSize(10);
-
-            p.onEntryAccessed(false, e1);
-
-            assertFalse(e1.isEvicted());
-
-            p.onEntryAccessed(false, e2);
-
-            assertFalse(e1.isEvicted());
-            assertFalse(e2.isEvicted());
-
-            p.onEntryAccessed(false, e3);
-
-            assertFalse(e1.isEvicted());
-            assertFalse(e3.isEvicted());
-
-            p.onEntryAccessed(false, e4);
-
-            assertFalse(e1.isEvicted());
-            assertFalse(e3.isEvicted());
-            assertFalse(e4.isEvicted());
-
-            p.onEntryAccessed(false, e5);
-
-            assertFalse(e1.isEvicted());
-            assertFalse(e3.isEvicted());
-            assertFalse(e5.isEvicted());
-        }
-        finally {
-            stopAllGrids();
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPut() throws Exception {
-        mode = CacheMode.LOCAL;
-        syncCommit = true;
-        plcMax = 100;
-
-        Ignite ignite = startGrid();
-
-        try {
-            IgniteCache<Integer, Integer> cache = ignite.cache(null);
-
-            int cnt = 500;
-
-            int min = Integer.MAX_VALUE;
-
-            int minIdx = 0;
-
-            for (int i = 0; i < cnt; i++) {
-                cache.put(i, i);
-
-                int cacheSize = cache.size();
-
-                if (i > plcMax && cacheSize < min) {
-                    min = cacheSize;
-                    minIdx = i;
-                }
-            }
-
-            assert min >= plcMax : "Min cache size is too small: " + min;
-
-            info("Min cache size [min=" + min + ", idx=" + minIdx + ']');
-            info("Current cache size " + cache.size());
-
-            min = Integer.MAX_VALUE;
-
-            minIdx = 0;
-
-            // Touch.
-            for (int i = cnt; --i > cnt - plcMax;) {
-                cache.get(i);
-
-                int cacheSize = cache.size();
-
-                if (cacheSize < min) {
-                    min = cacheSize;
-                    minIdx = i;
-                }
-            }
-
-            info("----");
-            info("Min cache size [min=" + min + ", idx=" + minIdx + ']');
-            info("Current cache size " + cache.size());
-
-            assert min >= plcMax : "Min cache size is too small: " + min;
-        }
-        finally {
-            stopAllGrids();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override protected LruEvictionPolicy<String, String> createPolicy(int plcMax) {
-        return new LruEvictionPolicy<>(plcMax);
-    }
-
-    @Override protected LruEvictionPolicy<String, String> createNearPolicy(int nearMax) {
-        return new LruEvictionPolicy<>(nearMax);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void checkNearPolicies(int endNearPlcSize) {
-        for (int i = 0; i < gridCnt; i++)
-            for (EvictableEntry<String, String> e : nearPolicy(i).queue())
-                assert !e.isCached() : "Invalid near policy size: " + nearPolicy(i).queue();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void checkPolicies(int plcMax) {
-        for (int i = 0; i < gridCnt; i++)
-            assert policy(i).queue().size() <= plcMax;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e5d5d08d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/GridCacheLruNearEvictionPolicySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/GridCacheLruNearEvictionPolicySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/GridCacheLruNearEvictionPolicySelfTest.java
deleted file mode 100644
index e0606a7..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/GridCacheLruNearEvictionPolicySelfTest.java
+++ /dev/null
@@ -1,136 +0,0 @@
-/*
- * 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.eviction.lru;
-
-import org.apache.ignite.*;
-import org.apache.ignite.cache.*;
-import org.apache.ignite.cache.eviction.lru.*;
-import org.apache.ignite.configuration.*;
-import org.apache.ignite.spi.discovery.tcp.*;
-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 java.util.*;
-
-import static org.apache.ignite.cache.CacheAtomicityMode.*;
-import static org.apache.ignite.cache.CacheMode.*;
-import static org.apache.ignite.cache.CacheRebalanceMode.*;
-import static org.apache.ignite.cache.CacheWriteSynchronizationMode.*;
-
-/**
- * LRU near eviction tests (GG-8884).
- */
-public class GridCacheLruNearEvictionPolicySelfTest extends GridCommonAbstractTest {
-    /** */
-    private static final TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
-
-    /** Maximum size for near eviction policy. */
-    private static final int EVICTION_MAX_SIZE = 10;
-
-    /** Grid count. */
-    private static final int GRID_COUNT = 2;
-
-    /** Cache atomicity mode specified by test. */
-    private CacheAtomicityMode atomicityMode;
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        IgniteConfiguration c = super.getConfiguration(gridName);
-
-        CacheConfiguration cc = new CacheConfiguration();
-
-        cc.setAtomicityMode(atomicityMode);
-        cc.setCacheMode(PARTITIONED);
-        cc.setWriteSynchronizationMode(PRIMARY_SYNC);
-        cc.setRebalanceMode(SYNC);
-        cc.setStartSize(100);
-        cc.setBackups(0);
-
-        NearCacheConfiguration nearCfg = new NearCacheConfiguration();
-        nearCfg.setNearEvictionPolicy(new LruEvictionPolicy(EVICTION_MAX_SIZE));
-        cc.setNearConfiguration(nearCfg);
-
-        c.setCacheConfiguration(cc);
-
-        TcpDiscoverySpi disco = new TcpDiscoverySpi();
-
-        disco.setIpFinder(ipFinder);
-
-        c.setDiscoverySpi(disco);
-
-        return c;
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testAtomicNearEvictionMaxSize() throws Exception {
-        atomicityMode = ATOMIC;
-
-        checkNearEvictionMaxSize();
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testTransactionalNearEvictionMaxSize() throws Exception {
-        atomicityMode = TRANSACTIONAL;
-
-        checkNearEvictionMaxSize();
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    private void checkNearEvictionMaxSize() throws Exception {
-        startGridsMultiThreaded(GRID_COUNT);
-
-        try {
-            Random rand = new Random(0);
-
-            int cnt = 1000;
-
-            info("Inserting " + cnt + " keys to cache.");
-
-            try (IgniteDataStreamer<Integer, String> ldr = grid(0).dataStreamer(null)) {
-                for (int i = 0; i < cnt; i++)
-                    ldr.addData(i, Integer.toString(i));
-            }
-
-            for (int i = 0; i < GRID_COUNT; i++)
-                assertTrue("Near cache size " + near(i).nearSize() + ", but eviction maximum size " + EVICTION_MAX_SIZE,
-                    near(i).nearSize() <= EVICTION_MAX_SIZE);
-
-            info("Getting " + cnt + " keys from cache.");
-
-            for (int i = 0; i < cnt; i++) {
-                IgniteCache<Integer, String> cache = grid(rand.nextInt(GRID_COUNT)).cache(null);
-
-                assertTrue(cache.get(i).equals(Integer.toString(i)));
-            }
-
-            for (int i = 0; i < GRID_COUNT; i++)
-                assertTrue("Near cache size " + near(i).nearSize() + ", but eviction maximum size " + EVICTION_MAX_SIZE,
-                    near(i).nearSize() <= EVICTION_MAX_SIZE);
-        }
-        finally {
-            stopAllGrids();
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e5d5d08d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/GridCacheNearOnlyLruNearEvictionPolicySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/GridCacheNearOnlyLruNearEvictionPolicySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/GridCacheNearOnlyLruNearEvictionPolicySelfTest.java
deleted file mode 100644
index 5d4ff85..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/GridCacheNearOnlyLruNearEvictionPolicySelfTest.java
+++ /dev/null
@@ -1,171 +0,0 @@
-/*
- * 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.eviction.lru;
-
-import org.apache.ignite.*;
-import org.apache.ignite.cache.*;
-import org.apache.ignite.cache.eviction.lru.*;
-import org.apache.ignite.configuration.*;
-import org.apache.ignite.spi.discovery.tcp.*;
-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 static org.apache.ignite.cache.CacheAtomicityMode.*;
-import static org.apache.ignite.cache.CacheMode.*;
-import static org.apache.ignite.cache.CacheRebalanceMode.*;
-import static org.apache.ignite.cache.CacheWriteSynchronizationMode.*;
-
-/**
- * LRU near eviction tests for NEAR_ONLY distribution mode (GG-8884).
- */
-public class GridCacheNearOnlyLruNearEvictionPolicySelfTest extends GridCommonAbstractTest {
-    /** */
-    private static final TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
-
-    /** Grid count. */
-    private static final int GRID_COUNT = 2;
-
-    /** Maximum size for near eviction policy. */
-    private static final int EVICTION_MAX_SIZE = 10;
-
-    /** Node count. */
-    private int cnt;
-
-    /** Caching mode specified by test. */
-    private CacheMode cacheMode;
-
-    /** Cache atomicity mode specified by test. */
-    private CacheAtomicityMode atomicityMode;
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTest() throws Exception {
-        super.beforeTest();
-
-        cnt = 0;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        IgniteConfiguration c = super.getConfiguration(gridName);
-
-        if (cnt == 0)
-            c.setClientMode(true);
-
-        CacheConfiguration cc = new CacheConfiguration();
-
-        cc.setAtomicityMode(atomicityMode);
-        cc.setCacheMode(cacheMode);
-        cc.setWriteSynchronizationMode(PRIMARY_SYNC);
-        cc.setRebalanceMode(SYNC);
-        cc.setStartSize(100);
-        cc.setBackups(0);
-
-        c.setCacheConfiguration(cc);
-
-        TcpDiscoverySpi disco = new TcpDiscoverySpi();
-
-        disco.setIpFinder(ipFinder);
-
-        c.setDiscoverySpi(disco);
-
-        cnt++;
-
-        return c;
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPartitionedAtomicNearEvictionMaxSize() throws Exception {
-        atomicityMode = ATOMIC;
-        cacheMode = PARTITIONED;
-
-        checkNearEvictionMaxSize();
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPartitionedTransactionalNearEvictionMaxSize() throws Exception {
-        atomicityMode = TRANSACTIONAL;
-        cacheMode = PARTITIONED;
-
-        checkNearEvictionMaxSize();
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testReplicatedAtomicNearEvictionMaxSize() throws Exception {
-        atomicityMode = ATOMIC;
-        cacheMode = REPLICATED;
-
-        checkNearEvictionMaxSize();
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testReplicatedTransactionalNearEvictionMaxSize() throws Exception {
-        atomicityMode = TRANSACTIONAL;
-        cacheMode = REPLICATED;
-
-        checkNearEvictionMaxSize();
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    private void checkNearEvictionMaxSize() throws Exception {
-        startGrids(GRID_COUNT);
-
-        try {
-            NearCacheConfiguration nearCfg = new NearCacheConfiguration();
-            nearCfg.setNearEvictionPolicy(new LruEvictionPolicy(EVICTION_MAX_SIZE));
-
-            grid(0).createNearCache(null, nearCfg);
-
-            int cnt = 1000;
-
-            info("Inserting " + cnt + " keys to cache.");
-
-            try (IgniteDataStreamer<Integer, String> ldr = grid(1).dataStreamer(null)) {
-                for (int i = 0; i < cnt; i++)
-                    ldr.addData(i, Integer.toString(i));
-            }
-
-            assertTrue("Near cache size " + near(0).nearSize() + ", but eviction maximum size " + EVICTION_MAX_SIZE,
-                near(0).nearSize() <= EVICTION_MAX_SIZE);
-
-            info("Getting " + cnt + " keys from cache.");
-
-            for (int i = 0; i < cnt; i++) {
-                IgniteCache<Integer, String> cache = grid(0).cache(null);
-
-                assertTrue(cache.get(i).equals(Integer.toString(i)));
-            }
-
-            assertTrue("Near cache size " + near(0).nearSize() + ", but eviction maximum size " + EVICTION_MAX_SIZE,
-                near(0).nearSize() <= EVICTION_MAX_SIZE);
-        }
-        finally {
-            stopAllGrids();
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e5d5d08d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/LruEvictionPolicySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/LruEvictionPolicySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/LruEvictionPolicySelfTest.java
new file mode 100644
index 0000000..8e1e8cf
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/LruEvictionPolicySelfTest.java
@@ -0,0 +1,353 @@
+/*
+ * 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.eviction.lru;
+
+import org.apache.ignite.cache.eviction.*;
+import org.apache.ignite.cache.eviction.lru.*;
+import org.apache.ignite.internal.processors.cache.*;
+import org.apache.ignite.internal.processors.cache.eviction.*;
+
+/**
+ * LRU Eviction policy tests.
+ */
+public class LruEvictionPolicySelfTest extends
+    EvictionAbstractTest<LruEvictionPolicy<String, String>> {
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMiddleAccess() throws Exception {
+        startGrid();
+
+        try {
+            LruEvictionPolicy<String, String> p = policy();
+
+            int max = 8;
+
+            p.setMaxSize(max * MockEntry.ENTRY_SIZE);
+
+            MockEntry entry1 = new MockEntry("1", "1");
+            MockEntry entry2 = new MockEntry("2", "2");
+            MockEntry entry3 = new MockEntry("3", "3");
+
+            p.onEntryAccessed(false, entry1);
+            p.onEntryAccessed(false, entry2);
+            p.onEntryAccessed(false, entry3);
+
+            MockEntry[] freqUsed = new MockEntry[] {
+                new MockEntry("4", "4"),
+                new MockEntry("5", "5"),
+                new MockEntry("6", "6"),
+                new MockEntry("7", "7"),
+                new MockEntry("8", "7")
+            };
+
+            for (MockEntry e : freqUsed)
+                p.onEntryAccessed(false, e);
+
+            for (MockEntry e : freqUsed)
+                assert !e.isEvicted();
+
+            int cnt = 1001;
+
+            for (int i = 0; i < cnt; i++)
+                p.onEntryAccessed(false, entry(freqUsed, i % freqUsed.length));
+
+            info(p);
+
+            check(max, MockEntry.ENTRY_SIZE);
+        }
+        finally {
+            stopGrid();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void doTestPolicy() throws Exception {
+        startGrid();
+
+        try {
+            MockEntry e1 = new MockEntry("1", "1");
+            MockEntry e2 = new MockEntry("2", "2");
+            MockEntry e3 = new MockEntry("3", "3");
+            MockEntry e4 = new MockEntry("4", "4");
+            MockEntry e5 = new MockEntry("5", "5");
+
+            LruEvictionPolicy<String, String> p = policy();
+
+            p.onEntryAccessed(false, e1);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e1);
+
+            p.onEntryAccessed(false, e2);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e1, e2);
+
+            p.onEntryAccessed(false, e3);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e1, e2, e3);
+
+            assertFalse(e1.isEvicted());
+            assertFalse(e2.isEvicted());
+            assertFalse(e3.isEvicted());
+
+            p.onEntryAccessed(false, e4);
+
+            check(p.queue(), e2, e3, e4);
+            check(MockEntry.ENTRY_SIZE, p.queue(), e2, e3, e4);
+
+            assertTrue(e1.isEvicted());
+            assertFalse(e2.isEvicted());
+            assertFalse(e3.isEvicted());
+            assertFalse(e4.isEvicted());
+
+            p.onEntryAccessed(false, e5);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e3, e4, e5);
+
+            assertTrue(e2.isEvicted());
+            assertFalse(e3.isEvicted());
+            assertFalse(e4.isEvicted());
+            assertFalse(e5.isEvicted());
+
+            p.onEntryAccessed(false, e1 = new MockEntry("1", "1"));
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e4, e5, e1);
+
+            assertTrue(e3.isEvicted());
+            assertFalse(e1.isEvicted());
+            assertFalse(e4.isEvicted());
+            assertFalse(e5.isEvicted());
+
+            p.onEntryAccessed(false, e5);
+
+            assertEquals(3, p.getCurrentSize());
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e4, e1, e5);
+
+            assertFalse(e1.isEvicted());
+            assertFalse(e4.isEvicted());
+            assertFalse(e5.isEvicted());
+
+            p.onEntryAccessed(false, e1);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e4, e5, e1);
+
+            assertFalse(e1.isEvicted());
+            assertFalse(e4.isEvicted());
+            assertFalse(e5.isEvicted());
+
+            p.onEntryAccessed(false, e5);
+
+            assertEquals(3, p.getCurrentSize());
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e4, e1, e5);
+
+            assertFalse(e1.isEvicted());
+            assertFalse(e4.isEvicted());
+            assertFalse(e5.isEvicted());
+
+            p.onEntryAccessed(true, e1);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e4, e5);
+
+            assertFalse(e1.isEvicted());
+            assertFalse(e4.isEvicted());
+            assertFalse(e5.isEvicted());
+
+            p.onEntryAccessed(true, e4);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e5);
+
+            assertFalse(e4.isEvicted());
+            assertFalse(e5.isEvicted());
+
+            p.onEntryAccessed(true, e5);
+
+            check(MockEntry.ENTRY_SIZE, p.queue());
+
+            assertFalse(e5.isEvicted());
+
+            info(p);
+        }
+        finally {
+            stopGrid();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void doTestPolicyWithBatch() throws Exception {
+        startGrid();
+
+        try {
+            MockEntry e1 = new MockEntry("1", "1");
+            MockEntry e2 = new MockEntry("2", "2");
+            MockEntry e3 = new MockEntry("3", "3");
+            MockEntry e4 = new MockEntry("4", "4");
+            MockEntry e5 = new MockEntry("5", "5");
+
+            LruEvictionPolicy<String, String> p = policy();
+
+            p.onEntryAccessed(false, e1);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e1);
+
+            p.onEntryAccessed(false, e2);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e1, e2);
+
+            p.onEntryAccessed(false, e3);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e1, e2, e3);
+
+            assertFalse(e1.isEvicted());
+            assertFalse(e2.isEvicted());
+            assertFalse(e3.isEvicted());
+
+            p.onEntryAccessed(false, e4);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e1, e2, e3, e4);
+
+            assertFalse(e1.isEvicted());
+            assertFalse(e2.isEvicted());
+            assertFalse(e3.isEvicted());
+            assertFalse(e4.isEvicted());
+
+            p.onEntryAccessed(false, e5);
+
+            // Batch evicted
+            check(MockEntry.ENTRY_SIZE, p.queue(), e3, e4, e5);
+
+            assertTrue(e1.isEvicted());
+            assertTrue(e2.isEvicted());
+            assertFalse(e3.isEvicted());
+            assertFalse(e4.isEvicted());
+            assertFalse(e5.isEvicted());
+
+            p.onEntryAccessed(false, e1 = new MockEntry("1", "1"));
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e3, e4, e5, e1);
+
+            assertFalse(e3.isEvicted());
+            assertFalse(e4.isEvicted());
+            assertFalse(e5.isEvicted());
+            assertFalse(e1.isEvicted());
+
+            p.onEntryAccessed(false, e5);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e3, e4, e1, e5);
+
+            assertFalse(e3.isEvicted());
+            assertFalse(e4.isEvicted());
+            assertFalse(e1.isEvicted());
+            assertFalse(e5.isEvicted());
+
+            p.onEntryAccessed(false, e1);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e3, e4, e5, e1);
+
+            assertFalse(e3.isEvicted());
+            assertFalse(e4.isEvicted());
+            assertFalse(e5.isEvicted());
+            assertFalse(e1.isEvicted());
+
+            p.onEntryAccessed(false, e5);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e3, e4, e1, e5);
+
+            assertFalse(e3.isEvicted());
+            assertFalse(e4.isEvicted());
+            assertFalse(e1.isEvicted());
+            assertFalse(e5.isEvicted());
+
+            p.onEntryAccessed(true, e1);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e3, e4, e5);
+
+            assertFalse(e3.isEvicted());
+            assertFalse(e4.isEvicted());
+            assertFalse(e5.isEvicted());
+
+            p.onEntryAccessed(true, e4);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e3, e5);
+
+            assertFalse(e3.isEvicted());
+            assertFalse(e5.isEvicted());
+
+            p.onEntryAccessed(true, e5);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e3);
+
+            assertFalse(e3.isEvicted());
+
+            p.onEntryAccessed(true, e3);
+
+            check(MockEntry.ENTRY_SIZE, p.queue());
+
+            info(p);
+        }
+        finally {
+            stopGrid();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override protected LruEvictionPolicy<String, String> createPolicy(int plcMax) {
+        LruEvictionPolicy<String, String> plc = new LruEvictionPolicy<>();
+
+        plc.setMaxSize(this.plcMax);
+        plc.setBatchSize(this.plcBatchSize);
+        plc.setMaxMemSize(this.plcMaxMemSize);
+
+        return plc;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected LruEvictionPolicy<String, String> createNearPolicy(int nearMax) {
+        LruEvictionPolicy<String, String> plc = new LruEvictionPolicy<>();
+
+        plc.setMaxSize(nearMax);
+        plc.setBatchSize(plcBatchSize);
+
+        return plc;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void checkNearPolicies(int endNearPlcSize) {
+        for (int i = 0; i < gridCnt; i++)
+            for (EvictableEntry<String, String> e : nearPolicy(i).queue())
+                assert !e.isCached() : "Invalid near policy size: " + nearPolicy(i).queue();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void checkPolicies() {
+        for (int i = 0; i < gridCnt; i++) {
+            if (plcMaxMemSize > 0) {
+                int size = 0;
+
+                for (EvictableEntry<String, String> entry : policy(i).queue())
+                    size += ((CacheEvictableEntryImpl)entry).size();
+
+                assertEquals(size, ((LruEvictionPolicy)policy(i)).getCurrentMemSize());
+            }
+            else
+                assertTrue(policy(i).queue().size() <= plcMax + plcBatchSize);
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e5d5d08d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/LruNearEvictionPolicySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/LruNearEvictionPolicySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/LruNearEvictionPolicySelfTest.java
new file mode 100644
index 0000000..218b817
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/LruNearEvictionPolicySelfTest.java
@@ -0,0 +1,140 @@
+/*
+ * 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.eviction.lru;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.*;
+import org.apache.ignite.cache.eviction.lru.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+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 java.util.*;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+import static org.apache.ignite.cache.CacheMode.*;
+import static org.apache.ignite.cache.CacheRebalanceMode.*;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.*;
+
+/**
+ * LRU near eviction tests (GG-8884).
+ */
+public class LruNearEvictionPolicySelfTest extends GridCommonAbstractTest {
+    /** */
+    private static final TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** Maximum size for near eviction policy. */
+    private static final int EVICTION_MAX_SIZE = 10;
+
+    /** Grid count. */
+    private static final int GRID_COUNT = 2;
+
+    /** Cache atomicity mode specified by test. */
+    private CacheAtomicityMode atomicityMode;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration c = super.getConfiguration(gridName);
+
+        CacheConfiguration cc = new CacheConfiguration();
+
+        cc.setAtomicityMode(atomicityMode);
+        cc.setCacheMode(PARTITIONED);
+        cc.setWriteSynchronizationMode(PRIMARY_SYNC);
+        cc.setRebalanceMode(SYNC);
+        cc.setStartSize(100);
+        cc.setBackups(0);
+
+        NearCacheConfiguration nearCfg = new NearCacheConfiguration();
+
+        LruEvictionPolicy plc = new LruEvictionPolicy();
+        plc.setMaxSize(EVICTION_MAX_SIZE);
+
+        nearCfg.setNearEvictionPolicy(plc);
+        cc.setNearConfiguration(nearCfg);
+
+        c.setCacheConfiguration(cc);
+
+        TcpDiscoverySpi disco = new TcpDiscoverySpi();
+
+        disco.setIpFinder(ipFinder);
+
+        c.setDiscoverySpi(disco);
+
+        return c;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAtomicNearEvictionMaxSize() throws Exception {
+        atomicityMode = ATOMIC;
+
+        checkNearEvictionMaxSize();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTransactionalNearEvictionMaxSize() throws Exception {
+        atomicityMode = TRANSACTIONAL;
+
+        checkNearEvictionMaxSize();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void checkNearEvictionMaxSize() throws Exception {
+        startGridsMultiThreaded(GRID_COUNT);
+
+        try {
+            Random rand = new Random(0);
+
+            int cnt = 1000;
+
+            info("Inserting " + cnt + " keys to cache.");
+
+            try (IgniteDataStreamer<Integer, String> ldr = grid(0).dataStreamer(null)) {
+                for (int i = 0; i < cnt; i++)
+                    ldr.addData(i, Integer.toString(i));
+            }
+
+            for (int i = 0; i < GRID_COUNT; i++)
+                assertTrue("Near cache size " + near(i).nearSize() + ", but eviction maximum size " + EVICTION_MAX_SIZE,
+                    near(i).nearSize() <= EVICTION_MAX_SIZE);
+
+            info("Getting " + cnt + " keys from cache.");
+
+            for (int i = 0; i < cnt; i++) {
+                IgniteCache<Integer, String> cache = grid(rand.nextInt(GRID_COUNT)).cache(null);
+
+                assertTrue(cache.get(i).equals(Integer.toString(i)));
+            }
+
+            for (int i = 0; i < GRID_COUNT; i++)
+                assertTrue("Near cache size " + near(i).nearSize() + ", but eviction maximum size " + EVICTION_MAX_SIZE,
+                    near(i).nearSize() <= EVICTION_MAX_SIZE);
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e5d5d08d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/LruNearOnlyNearEvictionPolicySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/LruNearOnlyNearEvictionPolicySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/LruNearOnlyNearEvictionPolicySelfTest.java
new file mode 100644
index 0000000..faca01d
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/LruNearOnlyNearEvictionPolicySelfTest.java
@@ -0,0 +1,175 @@
+/*
+ * 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.eviction.lru;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.*;
+import org.apache.ignite.cache.eviction.lru.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+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 static org.apache.ignite.cache.CacheAtomicityMode.*;
+import static org.apache.ignite.cache.CacheMode.*;
+import static org.apache.ignite.cache.CacheRebalanceMode.*;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.*;
+
+/**
+ * LRU near eviction tests for NEAR_ONLY distribution mode (GG-8884).
+ */
+public class LruNearOnlyNearEvictionPolicySelfTest extends GridCommonAbstractTest {
+    /** */
+    private static final TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** Grid count. */
+    private static final int GRID_COUNT = 2;
+
+    /** Maximum size for near eviction policy. */
+    private static final int EVICTION_MAX_SIZE = 10;
+
+    /** Node count. */
+    private int cnt;
+
+    /** Caching mode specified by test. */
+    private CacheMode cacheMode;
+
+    /** Cache atomicity mode specified by test. */
+    private CacheAtomicityMode atomicityMode;
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        cnt = 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration c = super.getConfiguration(gridName);
+
+        if (cnt == 0)
+            c.setClientMode(true);
+
+        CacheConfiguration cc = new CacheConfiguration();
+
+        cc.setAtomicityMode(atomicityMode);
+        cc.setCacheMode(cacheMode);
+        cc.setWriteSynchronizationMode(PRIMARY_SYNC);
+        cc.setRebalanceMode(SYNC);
+        cc.setStartSize(100);
+        cc.setBackups(0);
+
+        c.setCacheConfiguration(cc);
+
+        TcpDiscoverySpi disco = new TcpDiscoverySpi();
+
+        disco.setIpFinder(ipFinder);
+
+        c.setDiscoverySpi(disco);
+
+        cnt++;
+
+        return c;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPartitionedAtomicNearEvictionMaxSize() throws Exception {
+        atomicityMode = ATOMIC;
+        cacheMode = PARTITIONED;
+
+        checkNearEvictionMaxSize();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPartitionedTransactionalNearEvictionMaxSize() throws Exception {
+        atomicityMode = TRANSACTIONAL;
+        cacheMode = PARTITIONED;
+
+        checkNearEvictionMaxSize();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testReplicatedAtomicNearEvictionMaxSize() throws Exception {
+        atomicityMode = ATOMIC;
+        cacheMode = REPLICATED;
+
+        checkNearEvictionMaxSize();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testReplicatedTransactionalNearEvictionMaxSize() throws Exception {
+        atomicityMode = TRANSACTIONAL;
+        cacheMode = REPLICATED;
+
+        checkNearEvictionMaxSize();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void checkNearEvictionMaxSize() throws Exception {
+        startGrids(GRID_COUNT);
+
+        try {
+            NearCacheConfiguration nearCfg = new NearCacheConfiguration();
+
+            LruEvictionPolicy plc = new LruEvictionPolicy();
+            plc.setMaxSize(EVICTION_MAX_SIZE);
+
+            nearCfg.setNearEvictionPolicy(plc);
+
+            grid(0).createNearCache(null, nearCfg);
+
+            int cnt = 1000;
+
+            info("Inserting " + cnt + " keys to cache.");
+
+            try (IgniteDataStreamer<Integer, String> ldr = grid(1).dataStreamer(null)) {
+                for (int i = 0; i < cnt; i++)
+                    ldr.addData(i, Integer.toString(i));
+            }
+
+            assertTrue("Near cache size " + near(0).nearSize() + ", but eviction maximum size " + EVICTION_MAX_SIZE,
+                near(0).nearSize() <= EVICTION_MAX_SIZE);
+
+            info("Getting " + cnt + " keys from cache.");
+
+            for (int i = 0; i < cnt; i++) {
+                IgniteCache<Integer, String> cache = grid(0).cache(null);
+
+                assertTrue(cache.get(i).equals(Integer.toString(i)));
+            }
+
+            assertTrue("Near cache size " + near(0).nearSize() + ", but eviction maximum size " + EVICTION_MAX_SIZE,
+                near(0).nearSize() <= EVICTION_MAX_SIZE);
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e5d5d08d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/random/GridCacheRandomEvictionPolicySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/random/GridCacheRandomEvictionPolicySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/random/GridCacheRandomEvictionPolicySelfTest.java
deleted file mode 100644
index 7088714..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/random/GridCacheRandomEvictionPolicySelfTest.java
+++ /dev/null
@@ -1,258 +0,0 @@
-/*
- * 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.eviction.random;
-
-import org.apache.ignite.*;
-import org.apache.ignite.cache.eviction.random.*;
-import org.apache.ignite.internal.processors.cache.eviction.*;
-import org.jetbrains.annotations.*;
-
-import java.util.*;
-import java.util.concurrent.*;
-
-/**
- * Random eviction policy test.
- */
-public class GridCacheRandomEvictionPolicySelfTest extends
-    GridCacheEvictionAbstractTest<RandomEvictionPolicy<String, String>> {
-    /**
-     * @throws Exception If failed.
-     */
-    public void testMemory() throws Exception {
-        try {
-            Ignite g = startGrid(0);
-
-            int max = 10;
-
-            policy(0).setMaxSize(max);
-
-            int keys = 31;
-
-            for (int i = 0; i < keys; i++) {
-                String s = Integer.toString(i);
-
-                g.cache(null).put(s, s);
-            }
-
-            assert g.cache(null).size() <= max;
-        }
-        finally {
-            stopAllGrids();
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testRandom() throws Exception {
-        try {
-            Ignite g = startGrid(0);
-
-            int max = 10;
-
-            policy(0).setMaxSize(max);
-
-            Random rand = new Random();
-
-            int keys = 31;
-
-            String[] t = new String[keys];
-
-            for (int i = 0; i < t.length; i++)
-                t[i] = Integer.toString(i);
-
-            int runs = 10000;
-
-            for (int i = 0; i < runs; i++) {
-                boolean rmv = rand.nextBoolean();
-
-                int j = rand.nextInt(t.length);
-
-                if (rmv)
-                    g.cache(null).remove(t[j]);
-                else
-                    g.cache(null).put(t[j], t[j]);
-
-                if (i % 1000 == 0)
-                    info("Stats [cntr=" + i + ", total=" + runs + ']');
-            }
-
-            assert g.cache(null).size() <= max;
-
-            info(policy(0));
-        }
-        finally {
-            stopAllGrids();
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testAllowEmptyEntries() throws Exception {
-        try {
-            startGrid();
-
-            IgniteCache<String, String> c = jcache();
-
-            MockEntry e1 = new MockEntry("1", c);
-
-            MockEntry e2 = new MockEntry("2", c);
-
-            MockEntry e3 = new MockEntry("3", c);
-
-            MockEntry e4 = new MockEntry("4", c);
-
-            MockEntry e5 = new MockEntry("5", c);
-
-            RandomEvictionPolicy<String, String> p = policy();
-
-            p.setMaxSize(10);
-
-            p.onEntryAccessed(false, e1);
-
-            assertFalse(e1.isEvicted());
-
-            p.onEntryAccessed(false, e2);
-
-            assertFalse(e1.isEvicted());
-            assertFalse(e2.isEvicted());
-
-            p.onEntryAccessed(false, e3);
-
-            assertFalse(e1.isEvicted());
-            assertFalse(e3.isEvicted());
-
-            p.onEntryAccessed(false, e4);
-
-            assertFalse(e1.isEvicted());
-            assertFalse(e3.isEvicted());
-            assertFalse(e4.isEvicted());
-
-            p.onEntryAccessed(false, e5);
-
-            assertFalse(e1.isEvicted());
-            assertFalse(e3.isEvicted());
-            assertFalse(e5.isEvicted());
-        }
-        finally {
-            stopAllGrids();
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testRandomMultiThreaded() throws Exception {
-        try {
-            final Ignite g = startGrid(0);
-
-            int max = 10;
-
-            policy(0).setMaxSize(max);
-
-            final Random rand = new Random();
-
-            int keys = 31;
-
-            final String[] t = new String[keys];
-
-            for (int i = 0; i < t.length; i++)
-                t[i] = Integer.toString(i);
-
-            multithreaded(new Callable() {
-                @Nullable @Override public Object call() {
-                    int runs = 3000;
-
-                    for (int i = 0; i < runs; i++) {
-                        boolean rmv = rand.nextBoolean();
-
-                        int j = rand.nextInt(t.length);
-
-                        if (rmv)
-                            g.cache(null).remove(t[j]);
-                        else
-                            g.cache(null).put(t[j], t[j]);
-
-                        if (i != 0 && i % 1000 == 0)
-                            info("Stats [cntr=" + i + ", total=" + runs + ']');
-                    }
-
-                    return null;
-                }
-            }, 10);
-
-            assert g.cache(null).size() <= max;
-
-            info(policy(0));
-        }
-        finally {
-            stopAllGrids();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void testPartitionedNearDisabled() throws Exception {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
-    @Override public void testPartitionedNearEnabled() throws Exception {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
-    @Override public void testPartitionedNearDisabledMultiThreaded() throws Exception {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
-    @Override public void testPartitionedNearDisabledBackupSyncMultiThreaded() throws Exception {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
-    @Override public void testPartitionedNearEnabledMultiThreaded() throws Exception {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
-    @Override public void testPartitionedNearEnabledBackupSyncMultiThreaded() throws Exception {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
-    @Override protected RandomEvictionPolicy<String, String> createPolicy(int plcMax) {
-        return new RandomEvictionPolicy<>(plcMax);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected RandomEvictionPolicy<String, String> createNearPolicy(int nearMax) {
-        return new RandomEvictionPolicy<>(plcMax);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void checkNearPolicies(int nearMax) {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void checkPolicies(int plcMax) {
-        // No-op.
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e5d5d08d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/random/RandomEvictionPolicySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/random/RandomEvictionPolicySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/random/RandomEvictionPolicySelfTest.java
new file mode 100644
index 0000000..ef34a13
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/random/RandomEvictionPolicySelfTest.java
@@ -0,0 +1,357 @@
+/*
+ * 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.eviction.random;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.eviction.random.*;
+import org.apache.ignite.internal.processors.cache.eviction.*;
+import org.jetbrains.annotations.*;
+
+import java.util.*;
+import java.util.concurrent.*;
+
+/**
+ * Random eviction policy test.
+ */
+public class RandomEvictionPolicySelfTest extends
+    EvictionAbstractTest<RandomEvictionPolicy<String, String>> {
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMemory() throws Exception {
+        try {
+            Ignite g = startGrid(0);
+
+            int max = 10;
+
+            policy(0).setMaxSize(max);
+
+            int keys = 31;
+
+            for (int i = 0; i < keys; i++) {
+                String s = Integer.toString(i);
+
+                g.cache(null).put(s, s);
+            }
+
+            assert g.cache(null).size() <= max;
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testRandom() throws Exception {
+        try {
+            Ignite g = startGrid(0);
+
+            int max = 10;
+
+            policy(0).setMaxSize(max);
+
+            Random rand = new Random();
+
+            int keys = 31;
+
+            String[] t = new String[keys];
+
+            for (int i = 0; i < t.length; i++)
+                t[i] = Integer.toString(i);
+
+            int runs = 10000;
+
+            for (int i = 0; i < runs; i++) {
+                boolean rmv = rand.nextBoolean();
+
+                int j = rand.nextInt(t.length);
+
+                if (rmv)
+                    g.cache(null).remove(t[j]);
+                else
+                    g.cache(null).put(t[j], t[j]);
+
+                if (i % 1000 == 0)
+                    info("Stats [cntr=" + i + ", total=" + runs + ']');
+            }
+
+            assert g.cache(null).size() <= max;
+
+            info(policy(0));
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAllowEmptyEntries() throws Exception {
+        try {
+            startGrid();
+
+            IgniteCache<String, String> c = jcache();
+
+            MockEntry e1 = new MockEntry("1", c);
+            MockEntry e2 = new MockEntry("2", c);
+            MockEntry e3 = new MockEntry("3", c);
+            MockEntry e4 = new MockEntry("4", c);
+            MockEntry e5 = new MockEntry("5", c);
+
+            RandomEvictionPolicy<String, String> p = policy();
+
+            p.setMaxSize(10);
+
+            p.onEntryAccessed(false, e1);
+
+            assertFalse(e1.isEvicted());
+
+            p.onEntryAccessed(false, e2);
+
+            assertFalse(e1.isEvicted());
+            assertFalse(e2.isEvicted());
+
+            p.onEntryAccessed(false, e3);
+
+            assertFalse(e1.isEvicted());
+            assertFalse(e3.isEvicted());
+
+            p.onEntryAccessed(false, e4);
+
+            assertFalse(e1.isEvicted());
+            assertFalse(e3.isEvicted());
+            assertFalse(e4.isEvicted());
+
+            p.onEntryAccessed(false, e5);
+
+            assertFalse(e1.isEvicted());
+            assertFalse(e3.isEvicted());
+            assertFalse(e5.isEvicted());
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testRandomMultiThreaded() throws Exception {
+        try {
+            final Ignite g = startGrid(0);
+
+            int max = 10;
+
+            policy(0).setMaxSize(max);
+
+            final Random rand = new Random();
+
+            int keys = 31;
+
+            final String[] t = new String[keys];
+
+            for (int i = 0; i < t.length; i++)
+                t[i] = Integer.toString(i);
+
+            multithreaded(new Callable() {
+                @Nullable @Override public Object call() {
+                    int runs = 3000;
+
+                    for (int i = 0; i < runs; i++) {
+                        boolean rmv = rand.nextBoolean();
+
+                        int j = rand.nextInt(t.length);
+
+                        if (rmv)
+                            g.cache(null).remove(t[j]);
+                        else
+                            g.cache(null).put(t[j], t[j]);
+
+                        if (i != 0 && i % 1000 == 0)
+                            info("Stats [cntr=" + i + ", total=" + runs + ']');
+                    }
+
+                    return null;
+                }
+            }, 10);
+
+            assert g.cache(null).size() <= max;
+
+            info(policy(0));
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void testMaxMemSizeAllowEmptyEntries() throws Exception {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void testMaxMemSizeMemory() throws Exception {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void testMaxMemSizePartitionedNearDisabled() throws Exception {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void testMaxMemSizePolicy() throws Exception {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void testMaxMemSizePolicyWithBatch() throws Exception {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void testMaxMemSizePut() throws Exception {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void testMaxMemSizeRandom() throws Exception {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void testMaxSizeAllowEmptyEntries() throws Exception {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void testMaxSizeAllowEmptyEntriesWithBatch() throws Exception {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void testMaxSizeMemory() throws Exception {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void testMaxSizeMemoryWithBatch() throws Exception {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void doTestPolicy() throws Exception {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void testMaxSizePut() throws Exception {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void testMaxSizePutWithBatch() throws Exception {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void testMaxSizeRandom() throws Exception {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void testMaxSizeRandomWithBatch() throws Exception {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void testMaxSizePolicyWithBatch() throws Exception {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void testMaxSizePartitionedNearDisabledWithBatch() throws Exception {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void doTestPolicyWithBatch() throws Exception {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void testMaxSizePartitionedNearDisabled() throws Exception {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void testPartitionedNearEnabled() throws Exception {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void testPartitionedNearDisabledMultiThreaded() throws Exception {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void testPartitionedNearDisabledBackupSyncMultiThreaded() throws Exception {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void testPartitionedNearEnabledMultiThreaded() throws Exception {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void testPartitionedNearEnabledBackupSyncMultiThreaded() throws Exception {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override protected RandomEvictionPolicy<String, String> createPolicy(int plcMax) {
+        RandomEvictionPolicy<String, String> plc = new RandomEvictionPolicy<>();
+
+        plc.setMaxSize(plcMax);
+
+        return plc;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected RandomEvictionPolicy<String, String> createNearPolicy(int nearMax) {
+        RandomEvictionPolicy<String, String> plc = new RandomEvictionPolicy<>();
+
+        plc.setMaxSize(plcMax);
+
+        return plc;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void checkNearPolicies(int nearMax) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void checkPolicies() {
+        // No-op.
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e5d5d08d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/sorted/GridCacheSortedBatchEvictionPolicySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/sorted/GridCacheSortedBatchEvictionPolicySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/sorted/GridCacheSortedBatchEvictionPolicySelfTest.java
deleted file mode 100644
index 3cec217..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/sorted/GridCacheSortedBatchEvictionPolicySelfTest.java
+++ /dev/null
@@ -1,385 +0,0 @@
-/*
- * 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.eviction.sorted;
-
-import org.apache.ignite.*;
-import org.apache.ignite.cache.eviction.*;
-import org.apache.ignite.cache.eviction.sorted.*;
-import org.apache.ignite.internal.processors.cache.eviction.*;
-
-import java.util.*;
-
-import static org.apache.ignite.cache.CacheMode.*;
-
-/**
- * Sorted batch eviction test.
- */
-public class GridCacheSortedBatchEvictionPolicySelfTest extends
-    GridCacheEvictionAbstractTest<SortedEvictionPolicy<String, String>>{
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPolicy() throws Exception {
-        try {
-            startGrid();
-
-            GridCacheEvictionAbstractTest.MockEntry e1 = new GridCacheEvictionAbstractTest.MockEntry("1", "1");
-            GridCacheEvictionAbstractTest.MockEntry e2 = new GridCacheEvictionAbstractTest.MockEntry("2", "2");
-            GridCacheEvictionAbstractTest.MockEntry e3 = new GridCacheEvictionAbstractTest.MockEntry("3", "3");
-            GridCacheEvictionAbstractTest.MockEntry e4 = new GridCacheEvictionAbstractTest.MockEntry("4", "4");
-            GridCacheEvictionAbstractTest.MockEntry e5 = new GridCacheEvictionAbstractTest.MockEntry("5", "5");
-
-            SortedEvictionPolicy<String, String> p = policy();
-
-            p.setMaxSize(3);
-
-            p.setBatchSize(2);
-
-            p.onEntryAccessed(false, e1);
-
-            check(p.set(), e1);
-
-            p.onEntryAccessed(false, e2);
-
-            check(p.set(), e1, e2);
-
-            p.onEntryAccessed(false, e3);
-
-            check(p.set(), e1, e2, e3);
-
-            p.onEntryAccessed(false, e4);
-
-            check(p.set(), e1, e2, e3, e4);
-
-            assertFalse(e1.isEvicted());
-            assertFalse(e2.isEvicted());
-            assertFalse(e3.isEvicted());
-            assertFalse(e4.isEvicted());
-
-            assertEquals(4, p.getCurrentSize());
-
-            p.onEntryAccessed(false, e5);
-
-            // Batch evicted.
-            check(p.set(), e3, e4, e5);
-
-            assertEquals(3, p.getCurrentSize());
-
-            assertTrue(e1.isEvicted());
-            assertTrue(e2.isEvicted());
-            assertFalse(e3.isEvicted());
-            assertFalse(e4.isEvicted());
-            assertFalse(e5.isEvicted());
-
-            p.onEntryAccessed(false, e1 = new GridCacheEvictionAbstractTest.MockEntry("1", "1"));
-
-            check(p.set(), e1, e3, e4, e5);
-
-            assertEquals(4, p.getCurrentSize());
-
-            assertFalse(e1.isEvicted());
-            assertFalse(e3.isEvicted());
-            assertFalse(e4.isEvicted());
-            assertFalse(e5.isEvicted());
-
-            p.onEntryAccessed(false, e5);
-
-            check(p.set(), e1, e3, e4, e5);
-
-            assertFalse(e1.isEvicted());
-            assertFalse(e3.isEvicted());
-            assertFalse(e4.isEvicted());
-            assertFalse(e5.isEvicted());
-
-            p.onEntryAccessed(false, e1);
-
-            assertEquals(4, p.getCurrentSize());
-
-            check(p.set(), e1, e3, e4, e5);
-
-            assertFalse(e1.isEvicted());
-            assertFalse(e3.isEvicted());
-            assertFalse(e4.isEvicted());
-            assertFalse(e5.isEvicted());
-
-            p.onEntryAccessed(true, e1);
-
-            assertEquals(3, p.getCurrentSize());
-
-            assertFalse(e3.isEvicted());
-            assertFalse(e4.isEvicted());
-            assertFalse(e5.isEvicted());
-
-            p.onEntryAccessed(true, e4);
-
-            assertEquals(2, p.getCurrentSize());
-
-            assertFalse(e3.isEvicted());
-            assertFalse(e5.isEvicted());
-
-            p.onEntryAccessed(true, e5);
-
-            assertEquals(1, p.getCurrentSize());
-
-            assertFalse(e3.isEvicted());
-
-            p.onEntryAccessed(true, e3);
-
-            assertEquals(0, p.getCurrentSize());
-
-            assertFalse(e3.isEvicted());
-
-            info(p);
-        }
-        finally {
-            stopAllGrids();
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testMemory() throws Exception {
-        try {
-            startGrid();
-
-            SortedEvictionPolicy<String, String> p = policy();
-
-            int max = 10;
-
-            int batchSize = 2;
-
-            p.setMaxSize(max);
-            p.setBatchSize(batchSize);
-
-            int cnt = max + batchSize;
-
-            for (int i = 0; i < cnt; i++)
-                p.onEntryAccessed(false, new GridCacheEvictionAbstractTest.MockEntry(Integer.toString(i), Integer.toString(i)));
-
-            info(p);
-
-            assertEquals(cnt - batchSize, p.getCurrentSize());
-        }
-        finally {
-            stopAllGrids();
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testRandom() throws Exception {
-        try {
-            startGrid();
-
-            SortedEvictionPolicy<String, String> p = policy();
-
-            int max = 10;
-
-            int batchSize = 2;
-
-            p.setMaxSize(max);
-
-            p.setBatchSize(batchSize);
-
-            Random rand = new Random();
-
-            int keys = 31;
-
-            GridCacheEvictionAbstractTest.MockEntry[] fifos = new GridCacheEvictionAbstractTest.MockEntry[keys];
-
-            for (int i = 0; i < fifos.length; i++)
-                fifos[i] = new GridCacheEvictionAbstractTest.MockEntry(Integer.toString(i));
-
-            int runs = 5000000;
-
-            for (int i = 0; i < runs; i++) {
-                boolean rmv = rand.nextBoolean();
-
-                int j = rand.nextInt(fifos.length);
-
-                GridCacheEvictionAbstractTest.MockEntry e = entry(fifos, j);
-
-                if (rmv)
-                    fifos[j] = new GridCacheEvictionAbstractTest.MockEntry(Integer.toString(j));
-
-                p.onEntryAccessed(rmv, e);
-            }
-
-            info(p);
-
-            int curSize = p.getCurrentSize();
-
-            assert curSize < max + batchSize :
-                "curSize < max + batchSize [curSize=" + curSize + ", max=" + max + ", batchSize=" + batchSize + ']';
-        }
-        finally {
-            stopAllGrids();
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testAllowEmptyEntries() throws Exception {
-        try {
-            startGrid();
-
-            GridCacheEvictionAbstractTest.MockEntry e1 = new GridCacheEvictionAbstractTest.MockEntry("1");
-
-            GridCacheEvictionAbstractTest.MockEntry e2 = new GridCacheEvictionAbstractTest.MockEntry("2");
-
-            GridCacheEvictionAbstractTest.MockEntry e3 = new GridCacheEvictionAbstractTest.MockEntry("3");
-
-            GridCacheEvictionAbstractTest.MockEntry e4 = new GridCacheEvictionAbstractTest.MockEntry("4");
-
-            GridCacheEvictionAbstractTest.MockEntry e5 = new GridCacheEvictionAbstractTest.MockEntry("5");
-
-            SortedEvictionPolicy<String, String> p = policy();
-
-            p.setBatchSize(2);
-
-            p.onEntryAccessed(false, e1);
-
-            assertFalse(e1.isEvicted());
-
-            p.onEntryAccessed(false, e2);
-
-            assertFalse(e1.isEvicted());
-            assertFalse(e2.isEvicted());
-
-            p.onEntryAccessed(false, e3);
-
-            assertFalse(e1.isEvicted());
-            assertFalse(e3.isEvicted());
-
-            p.onEntryAccessed(false, e4);
-
-            assertFalse(e1.isEvicted());
-            assertFalse(e3.isEvicted());
-            assertFalse(e4.isEvicted());
-
-            p.onEntryAccessed(false, e5);
-
-            assertFalse(e1.isEvicted());
-            assertFalse(e3.isEvicted());
-            assertFalse(e5.isEvicted());
-        }
-        finally {
-            stopAllGrids();
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPut() throws Exception {
-        mode = LOCAL;
-        syncCommit = true;
-        plcMax = 10;
-
-        Ignite ignite = startGrid();
-
-        try {
-            IgniteCache<Object, Object> cache = ignite.cache(null);
-
-            int cnt = 500;
-
-            int min = Integer.MAX_VALUE;
-
-            int minIdx = 0;
-
-            for (int i = 0; i < cnt; i++) {
-                cache.put(i, i);
-
-                int cacheSize = cache.size();
-
-                if (i > plcMax && cacheSize < min) {
-                    min = cacheSize;
-                    minIdx = i;
-                }
-            }
-
-            // Batch evicted.
-            assert min >= plcMax : "Min cache size is too small: " + min;
-
-            info("Min cache size [min=" + min + ", idx=" + minIdx + ']');
-            info("Current cache size " + cache.size());
-            info("Current cache key size " + cache.size());
-
-            min = Integer.MAX_VALUE;
-
-            minIdx = 0;
-
-            // Touch.
-            for (int i = cnt; --i > cnt - plcMax;) {
-                cache.get(i);
-
-                int cacheSize = cache.size();
-
-                if (cacheSize < min) {
-                    min = cacheSize;
-                    minIdx = i;
-                }
-            }
-
-            info("----");
-            info("Min cache size [min=" + min + ", idx=" + minIdx + ']');
-            info("Current cache size " + cache.size());
-            info("Current cache key size " + cache.size());
-
-            // Batch evicted.
-            assert min >= plcMax : "Min cache size is too small: " + min;
-        }
-        finally {
-            stopAllGrids();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void testPartitionedNearDisabled() throws Exception {
-        plcBatchSize = 2;
-
-        super.testPartitionedNearDisabled();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected SortedEvictionPolicy<String, String> createPolicy(int plcMax) {
-        return new SortedEvictionPolicy<>(10, 2, null);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected SortedEvictionPolicy<String, String> createNearPolicy(int nearMax) {
-        return new SortedEvictionPolicy<>(nearMax, 2, null);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void checkNearPolicies(int endNearPlcSize) {
-        for (int i = 0; i < gridCnt; i++)
-            for (EvictableEntry<String, String> e : nearPolicy(i).set())
-                assert !e.isCached() : "Invalid near policy size: " + nearPolicy(i).set();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void checkPolicies(int plcMax) {
-        for (int i = 0; i < gridCnt; i++)
-            assert policy(i).set().size() <= plcMax + policy(i).getBatchSize();
-    }
-
-}