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/29 20:21:58 UTC

[49/50] [abbrv] 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/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();
-    }
-}