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

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

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b4aa05d8/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/b4aa05d8/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/b4aa05d8/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/b4aa05d8/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/b4aa05d8/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/b4aa05d8/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/b4aa05d8/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/b4aa05d8/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 8918a29..55d42ca 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,14 +37,12 @@ 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(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(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/b4aa05d8/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/b4aa05d8/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 54bc814..171f1c6 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/b4aa05d8/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/b4aa05d8/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());