You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ag...@apache.org on 2017/04/12 07:33:54 UTC

[23/57] [abbrv] ignite git commit: IGNITE-4534 - Added offheap evictions

http://git-wip-us.apache.org/repos/asf/ignite/blob/ff5b3e16/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPageIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPageIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPageIO.java
index 190349a..9546890 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPageIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPageIO.java
@@ -18,9 +18,11 @@
 package org.apache.ignite.internal.processors.cache.database.tree.io;
 
 import java.nio.ByteBuffer;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.HashSet;
+import java.util.List;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.pagemem.PageIdUtils;
 import org.apache.ignite.internal.pagemem.PageMemory;
@@ -254,6 +256,29 @@ public class DataPageIO extends PageIO {
 
     /**
      * @param pageAddr Page address.
+     * @param c Closure.
+     * @param <T> Closure return type.
+     * @return Collection of closure results for all items in page.
+     * @throws IgniteCheckedException In case of error in closure body.
+     */
+    public <T> List<T> forAllItems(long pageAddr, CC<T> c) throws IgniteCheckedException {
+        long pageId = getPageId(pageAddr);
+
+        int cnt = getDirectCount(pageAddr);
+
+        List<T> res = new ArrayList<>(cnt);
+
+        for (int i = 0; i < cnt; i++) {
+            long link = PageIdUtils.link(pageId, i);
+
+            res.add(c.apply(link));
+        }
+
+        return res;
+    }
+
+    /**
+     * @param pageAddr Page address.
      * @param cnt Indirect count.
      */
     private void setIndirectCount(long pageAddr, int cnt) {
@@ -1007,7 +1032,8 @@ public class DataPageIO extends PageIO {
         final int keySize = row.key().valueBytesLength(null);
         final int valSize = row.value().valueBytesLength(null);
 
-        int written = writeFragment(row, buf, rowOff, payloadSize, EntryPart.KEY, keySize, valSize);
+        int written = writeFragment(row, buf, rowOff, payloadSize, EntryPart.CACHE_ID, keySize, valSize);
+        written += writeFragment(row, buf, rowOff + written, payloadSize - written, EntryPart.KEY, keySize, valSize);
         written += writeFragment(row, buf, rowOff + written, payloadSize - written, EntryPart.EXPIRE_TIME, keySize, valSize);
         written += writeFragment(row, buf, rowOff + written, payloadSize - written, EntryPart.VALUE, keySize, valSize);
         written += writeFragment(row, buf, rowOff + written, payloadSize - written, EntryPart.VERSION, keySize, valSize);
@@ -1039,28 +1065,36 @@ public class DataPageIO extends PageIO {
         final int prevLen;
         final int curLen;
 
+        int cacheIdSize = row.cacheId() == 0 ? 0 : 4;
+
         switch (type) {
-            case KEY:
+            case CACHE_ID:
                 prevLen = 0;
-                curLen = keySize;
+                curLen = cacheIdSize;
+
+                break;
+
+            case KEY:
+                prevLen = cacheIdSize;
+                curLen = cacheIdSize + keySize;
 
                 break;
 
             case EXPIRE_TIME:
-                prevLen = keySize;
-                curLen = keySize + 8;
+                prevLen = cacheIdSize + keySize;
+                curLen = cacheIdSize + keySize + 8;
 
                 break;
 
             case VALUE:
-                prevLen = keySize + 8;
-                curLen = keySize + valSize + 8;
+                prevLen = cacheIdSize + keySize + 8;
+                curLen = cacheIdSize + keySize + valSize + 8;
 
                 break;
 
             case VERSION:
-                prevLen = keySize + valSize + 8;
-                curLen = keySize + valSize + CacheVersionIO.size(row.version(), false) + 8;
+                prevLen = cacheIdSize + keySize + valSize + 8;
+                curLen = cacheIdSize + keySize + valSize + CacheVersionIO.size(row.version(), false) + 8;
 
                 break;
 
@@ -1075,6 +1109,8 @@ public class DataPageIO extends PageIO {
 
         if (type == EntryPart.EXPIRE_TIME)
             writeExpireTimeFragment(buf, row.expireTime(), rowOff, len, prevLen);
+        else if (type == EntryPart.CACHE_ID)
+            writeCacheIdFragment(buf, row.cacheId(), rowOff, len, prevLen);
         else if (type != EntryPart.VERSION) {
             // Write key or value.
             final CacheObject co = type == EntryPart.KEY ? row.key() : row.value();
@@ -1139,6 +1175,32 @@ public class DataPageIO extends PageIO {
     }
 
     /**
+     * @param buf Buffer.
+     * @param cacheId Cache ID.
+     * @param rowOff Row offset.
+     * @param len Length.
+     * @param prevLen Prev length.
+     */
+    private void writeCacheIdFragment(ByteBuffer buf, int cacheId, int rowOff, int len, int prevLen) {
+        if (cacheId == 0)
+            return;
+
+        int size = 4;
+
+        if (size <= len)
+            buf.putInt(cacheId);
+        else {
+            ByteBuffer cacheIdBuf = ByteBuffer.allocate(size);
+
+            cacheIdBuf.order(buf.order());
+
+            cacheIdBuf.putInt(cacheId);
+
+            buf.put(cacheIdBuf.array(), rowOff - prevLen, len);
+        }
+    }
+
+    /**
      *
      */
     private enum EntryPart {
@@ -1152,7 +1214,10 @@ public class DataPageIO extends PageIO {
         VERSION,
 
         /** */
-        EXPIRE_TIME
+        EXPIRE_TIME,
+
+        /** */
+        CACHE_ID
     }
 
     /**
@@ -1326,14 +1391,21 @@ public class DataPageIO extends PageIO {
     ) throws IgniteCheckedException {
         long addr = pageAddr + dataOff;
 
+        int cacheIdSize = row.cacheId() != 0 ? 4 : 0;
+
         if (newRow) {
             PageUtils.putShort(addr, 0, (short)payloadSize);
             addr += 2;
 
+            if (cacheIdSize != 0)
+                PageUtils.putInt(addr, 0, row.cacheId());
+
+            addr += cacheIdSize;
+
             addr += row.key().putValue(addr);
         }
         else
-            addr += (2 + row.key().valueBytesLength(null));
+            addr += (2 + cacheIdSize + row.key().valueBytesLength(null));
 
         addr += row.value().putValue(addr);
 
@@ -1358,4 +1430,20 @@ public class DataPageIO extends PageIO {
 
         PageUtils.putBytes(pageAddr, dataOff, payload);
     }
+
+    /**
+     * Defines closure interface for applying computations to data page items.
+     *
+     * @param <T> Closure return type.
+     */
+    public interface CC<T> {
+        /**
+         * Closure body.
+         *
+         * @param link Link to item.
+         * @return Closure return value.
+         * @throws IgniteCheckedException In case of error in closure body.
+         */
+        public T apply(long link) throws IgniteCheckedException;
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/ff5b3e16/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
index d652767..5a8dfa0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
@@ -1795,6 +1795,8 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
                     // Do not check topology version if topology was locked on near node by
                     // external transaction or explicit lock.
                     if (req.topologyLocked() || !needRemap(req.topologyVersion(), top.topologyVersion())) {
+                        ctx.shared().database().ensureFreeSpace(ctx.memoryPolicy());
+
                         locked = lockEntries(req, req.topologyVersion());
 
                         boolean hasNear = ctx.discovery().cacheNearNode(node, name());

http://git-wip-us.apache.org/repos/asf/ignite/blob/ff5b3e16/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
index c257154..64f6187 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
@@ -2673,6 +2673,8 @@ public class GridNearTxLocal extends GridDhtTxLocalAdapter implements AutoClosea
                                 GridCacheEntryEx entry = cacheCtx.cache().entryEx(key, topVer);
 
                                 try {
+                                    cacheCtx.shared().database().ensureFreeSpace(cacheCtx.memoryPolicy());
+
                                     EntryGetResult verVal = entry.versionedValue(cacheVal,
                                         ver,
                                         null,

http://git-wip-us.apache.org/repos/asf/ignite/blob/ff5b3e16/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxRemote.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxRemote.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxRemote.java
index 767ce84..e5cd469 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxRemote.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxRemote.java
@@ -334,7 +334,7 @@ public class GridNearTxRemote extends GridDistributedTxRemoteAdapter {
 
                 CacheObject val = cached.peek(null);
 
-                if (val == null && cached.evictInternal(xidVer, null)) {
+                if (val == null && cached.evictInternal(xidVer, null, false)) {
                     evicted.add(entry.txKey());
 
                     return false;
@@ -395,7 +395,7 @@ public class GridNearTxRemote extends GridDistributedTxRemoteAdapter {
 
                 CacheObject peek = cached.peek(null);
 
-                if (peek == null && cached.evictInternal(xidVer, null)) {
+                if (peek == null && cached.evictInternal(xidVer, null, false)) {
                     cached.context().cache().removeIfObsolete(key.key());
 
                     evicted.add(key);

http://git-wip-us.apache.org/repos/asf/ignite/blob/ff5b3e16/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
index da92692..b8c0e36 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
@@ -791,6 +791,8 @@ public class GridLocalAtomicCache<K, V> extends GridLocalCache<K, V> {
 
         CacheEntryPredicate[] filters = CU.filterArray(filter);
 
+        ctx.shared().database().ensureFreeSpace(ctx.memoryPolicy());
+
         if (writeThrough && keys.size() > 1) {
             return updateWithBatch(op,
                 keys,
@@ -995,8 +997,8 @@ public class GridLocalAtomicCache<K, V> extends GridLocalCache<K, V> {
                             null,
                             null,
                             /*read-through*/true,
-                            /**update-metrics*/true,
-                            /**event*/true,
+                            /*update-metrics*/true,
+                            /*event*/true,
                             subjId,
                             entryProcessor,
                             taskName,

http://git-wip-us.apache.org/repos/asf/ignite/blob/ff5b3e16/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java
index 4ec89e1..9beb296 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java
@@ -180,13 +180,6 @@ public interface IgniteCacheObjectProcessor extends GridProcessor {
     public CacheObject toCacheObject(CacheObjectContext ctx, ByteBuffer buf);
 
     /**
-     * @param ctx Cache context.
-     * @param buf Buffer to read from.
-     * @return Cache object.
-     */
-    public KeyCacheObject toKeyCacheObject(CacheObjectContext ctx, ByteBuffer buf) throws IgniteCheckedException;
-
-    /**
      * @param ctx Cache object context.
      * @param buf Buffer.
      * @param incompleteObj Incomplete cache object or {@code null} if it's a first read.

http://git-wip-us.apache.org/repos/asf/ignite/blob/ff5b3e16/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
index 4726d86..a8595fb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
@@ -188,22 +188,6 @@ public class IgniteCacheObjectProcessorImpl extends GridProcessorAdapter impleme
     }
 
     /** {@inheritDoc} */
-    @Override public KeyCacheObject toKeyCacheObject(CacheObjectContext ctx, ByteBuffer buf) throws IgniteCheckedException {
-        int len = buf.getInt();
-
-        if (len == 0)
-            return null;
-
-        byte type = buf.get();
-
-        byte[] data = new byte[len];
-
-        buf.get(data);
-
-        return toKeyCacheObject(ctx, type, data);
-    }
-
-    /** {@inheritDoc} */
     @Override public IncompleteCacheObject toCacheObject(
         final CacheObjectContext ctx,
         final ByteBuffer buf,

http://git-wip-us.apache.org/repos/asf/ignite/blob/ff5b3e16/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
index 46dec44..adbd6f1 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
@@ -369,7 +369,7 @@ public class GridCacheTestEntryEx extends GridMetadataAwareAdapter implements Gr
 
     /** @inheritDoc */
     @Override public boolean evictInternal(GridCacheVersion obsoleteVer,
-        @Nullable CacheEntryPredicate[] filter) {
+        @Nullable CacheEntryPredicate[] filter, boolean evictOffheap) {
         assert false;
 
         return false;

http://git-wip-us.apache.org/repos/asf/ignite/blob/ff5b3e16/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEntrySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEntrySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEntrySelfTest.java
index bc7fffe..58c3841 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEntrySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEntrySelfTest.java
@@ -280,7 +280,7 @@ public class GridCacheDhtEntrySelfTest extends GridCommonAbstractTest {
         assert e0.readers().contains(other.id());
         assert e1 == null || e1.readers().isEmpty();
 
-        assert !e0.evictInternal(dht0.context().versions().next(), null);
+        assert !e0.evictInternal(dht0.context().versions().next(), null, false);
 
         assertEquals(1, near0.localSize(CachePeekMode.ALL));
         assertEquals(1, dht0.localSize(null));
@@ -288,7 +288,7 @@ public class GridCacheDhtEntrySelfTest extends GridCommonAbstractTest {
         assertEquals(1, near1.localSize(CachePeekMode.ALL));
         assertEquals(0, dht1.localSize(null));
 
-        assert !e0.evictInternal(dht0.context().versions().next(), null);
+        assert !e0.evictInternal(dht0.context().versions().next(), null, false);
 
         assertEquals(1, near0.localSize(CachePeekMode.ALL));
         assertEquals(1, dht0.localSize(null));

http://git-wip-us.apache.org/repos/asf/ignite/blob/ff5b3e16/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/PageEvictionAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/PageEvictionAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/PageEvictionAbstractTest.java
new file mode 100644
index 0000000..bf05146
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/PageEvictionAbstractTest.java
@@ -0,0 +1,124 @@
+/*
+* 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.paged;
+
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.CacheWriteSynchronizationMode;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.MemoryConfiguration;
+import org.apache.ignite.configuration.MemoryPolicyConfiguration;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+/**
+ *
+ */
+public class PageEvictionAbstractTest extends GridCommonAbstractTest {
+    /** */
+    protected static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** Offheap size for memory policy. */
+    private static final int SIZE = 256 * 1024 * 1024;
+
+    /** Page size. */
+    static final int PAGE_SIZE = 2048;
+
+    /** Number of entries. */
+    static final int ENTRIES = 400_000;
+
+    /** Empty pages pool size. */
+    private static final int EMPTY_PAGES_POOL_SIZE = 100;
+
+    /** Eviction threshold. */
+    private static final double EVICTION_THRESHOLD = 0.9;
+
+    /** Default policy name. */
+    private static final String DEFAULT_POLICY_NAME = "dfltPlc";
+
+    /**
+     * @param mode Eviction mode.
+     * @param configuration Configuration.
+     * @return Configuration with given eviction mode set.
+     */
+    static IgniteConfiguration setEvictionMode(DataPageEvictionMode mode, IgniteConfiguration configuration) {
+        MemoryPolicyConfiguration[] policies = configuration.getMemoryConfiguration().getMemoryPolicies();
+
+        for (MemoryPolicyConfiguration plcCfg : policies)
+            plcCfg.setPageEvictionMode(mode);
+
+        return configuration;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(IP_FINDER);
+
+        MemoryConfiguration dbCfg = new MemoryConfiguration();
+
+        MemoryPolicyConfiguration plc = new MemoryPolicyConfiguration();
+
+        plc.setSize(SIZE);
+        plc.setEmptyPagesPoolSize(EMPTY_PAGES_POOL_SIZE);
+        plc.setEvictionThreshold(EVICTION_THRESHOLD);
+        plc.setName(DEFAULT_POLICY_NAME);
+
+        dbCfg.setMemoryPolicies(plc);
+        dbCfg.setPageSize(PAGE_SIZE);
+        dbCfg.setDefaultMemoryPolicyName(DEFAULT_POLICY_NAME);
+
+        cfg.setMemoryConfiguration(dbCfg);
+
+        return cfg;
+    }
+
+    /**
+     * @param name Name.
+     * @param cacheMode Cache mode.
+     * @param atomicityMode Atomicity mode.
+     * @param writeSynchronizationMode Write synchronization mode.
+     * @param memoryPlcName Memory policy name.
+     * @return Cache configuration.
+     */
+    protected static CacheConfiguration<Object, Object> cacheConfig(
+        String name,
+        String memoryPlcName,
+        CacheMode cacheMode,
+        CacheAtomicityMode atomicityMode,
+        CacheWriteSynchronizationMode writeSynchronizationMode
+    ) {
+        CacheConfiguration<Object, Object> cacheConfiguration = new CacheConfiguration<>()
+            .setName(name)
+            .setAffinity(new RendezvousAffinityFunction(false, 32))
+            .setCacheMode(cacheMode)
+            .setAtomicityMode(atomicityMode)
+            .setMemoryPolicyName(memoryPlcName)
+            .setWriteSynchronizationMode(writeSynchronizationMode);
+
+        if (cacheMode == CacheMode.PARTITIONED)
+            cacheConfiguration.setBackups(1);
+
+        return cacheConfiguration;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/ff5b3e16/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/PageEvictionMultinodeTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/PageEvictionMultinodeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/PageEvictionMultinodeTest.java
new file mode 100644
index 0000000..2302de1
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/PageEvictionMultinodeTest.java
@@ -0,0 +1,110 @@
+/*
+* 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.paged;
+
+import java.util.concurrent.ThreadLocalRandom;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.CachePeekMode;
+import org.apache.ignite.cache.CacheWriteSynchronizationMode;
+import org.apache.ignite.configuration.CacheConfiguration;
+
+/**
+ *
+ */
+public abstract class PageEvictionMultinodeTest extends PageEvictionAbstractTest {
+    /** Cache modes. */
+    private static final CacheMode[] CACHE_MODES = {CacheMode.PARTITIONED, CacheMode.REPLICATED};
+
+    /** Atomicity modes. */
+    private static final CacheAtomicityMode[] ATOMICITY_MODES = {
+        CacheAtomicityMode.ATOMIC, CacheAtomicityMode.TRANSACTIONAL};
+
+    /** Write modes. */
+    private static final CacheWriteSynchronizationMode[] WRITE_MODES = {CacheWriteSynchronizationMode.PRIMARY_SYNC,
+        CacheWriteSynchronizationMode.FULL_SYNC, CacheWriteSynchronizationMode.FULL_ASYNC};
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        startGridsMultiThreaded(4, false);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected long getTestTimeout() {
+        return 10 * 60 * 1000;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPageEviction() throws Exception {
+        for (int i = 0; i < CACHE_MODES.length; i++) {
+            for (int j = 0; j < ATOMICITY_MODES.length; j++) {
+                for (int k = 0; k < WRITE_MODES.length; k++) {
+                    if (i + j + Math.min(k, 1) <= 1) {
+                        CacheConfiguration<Object, Object> cfg = cacheConfig(
+                            "evict" + i + j + k, null, CACHE_MODES[i], ATOMICITY_MODES[j], WRITE_MODES[k]);
+
+                        createCacheAndTestEvcition(cfg);
+                    }
+                }
+            }
+        }
+    }
+
+    /**
+     * @param cfg Config.
+     * @throws Exception If failed.
+     */
+    private void createCacheAndTestEvcition(CacheConfiguration<Object, Object> cfg) throws Exception {
+        IgniteCache<Object, Object> cache = ignite(0).getOrCreateCache(cfg);
+
+        for (int i = 1; i <= ENTRIES; i++) {
+            ThreadLocalRandom r = ThreadLocalRandom.current();
+
+            if (r.nextInt() % 5 == 0)
+                cache.put(i, new TestObject(PAGE_SIZE / 4 - 50 + r.nextInt(5000))); // Fragmented object.
+            else
+                cache.put(i, new TestObject(r.nextInt(PAGE_SIZE / 4 - 50))); // Fits in one page.
+
+            if (r.nextInt() % 7 == 0)
+                cache.get(r.nextInt(i)); // Touch.
+            else if (r.nextInt() % 11 == 0)
+                cache.remove(r.nextInt(i)); // Remove.
+            else if (r.nextInt() % 13 == 0)
+                cache.put(r.nextInt(i), new TestObject(r.nextInt(PAGE_SIZE / 2))); // Update.
+
+            if (i % (ENTRIES / 10) == 0)
+                System.out.println(">>> Entries put: " + i);
+        }
+
+        int resultingSize = cache.size(CachePeekMode.PRIMARY);
+
+        System.out.println(">>> Resulting size: " + resultingSize);
+
+        // More than half of entries evicted, no OutOfMemory occurred, success.
+        assertTrue(resultingSize < ENTRIES / 2);
+
+        ignite(0).destroyCache(cfg.getName());
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/ff5b3e16/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/PageEvictionReadThroughTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/PageEvictionReadThroughTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/PageEvictionReadThroughTest.java
new file mode 100644
index 0000000..c8cd7c9
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/PageEvictionReadThroughTest.java
@@ -0,0 +1,140 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*      http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+package org.apache.ignite.internal.processors.cache.eviction.paged;
+
+import java.util.concurrent.ThreadLocalRandom;
+import javax.cache.Cache;
+import javax.cache.configuration.Factory;
+import javax.cache.integration.CacheLoaderException;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.CachePeekMode;
+import org.apache.ignite.cache.CacheWriteSynchronizationMode;
+import org.apache.ignite.cache.store.CacheStoreAdapter;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.configuration.IgniteConfiguration;
+
+/**
+ *
+ */
+public class PageEvictionReadThroughTest extends PageEvictionAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        return setEvictionMode(DataPageEvictionMode.RANDOM_LRU, super.getConfiguration(gridName));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testEvictionWithReadThroughAtomicReplicated() throws Exception {
+        testEvictionWithReadThrough(CacheAtomicityMode.ATOMIC, CacheMode.REPLICATED);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testEvictionWithReadThroughAtomicLocal() throws Exception {
+        testEvictionWithReadThrough(CacheAtomicityMode.ATOMIC, CacheMode.LOCAL);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testEvictionWithReadThroughTxReplicated() throws Exception {
+        testEvictionWithReadThrough(CacheAtomicityMode.TRANSACTIONAL, CacheMode.REPLICATED);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testEvictionWithReadThroughTxLocal() throws Exception {
+        testEvictionWithReadThrough(CacheAtomicityMode.TRANSACTIONAL, CacheMode.LOCAL);
+    }
+
+    /**
+     * @param atomicityMode Atomicity mode.
+     * @param cacheMode Cache mode.
+     * @throws Exception If failed.
+     */
+    private void testEvictionWithReadThrough(CacheAtomicityMode atomicityMode, CacheMode cacheMode) throws Exception {
+        startGridsMultiThreaded(4);
+
+        CacheConfiguration<Object, Object> cfg = cacheConfig("evict-rebalance", null, cacheMode, atomicityMode,
+            CacheWriteSynchronizationMode.PRIMARY_SYNC);
+        cfg.setReadThrough(true);
+        cfg.setCacheStoreFactory(new TestStoreFactory());
+
+        IgniteCache<Object, Object> cache = ignite(0).getOrCreateCache(cfg);
+
+        for (int i = 1; i <= ENTRIES; i++) {
+            cache.get(i);
+
+            if (i % (ENTRIES / 10) == 0)
+                System.out.println(">>> Entries: " + i);
+        }
+
+        int size = cache.size(CachePeekMode.PRIMARY);
+
+        System.out.println(">>> Resulting size: " + size);
+
+        assertTrue(size > 0);
+
+        assertTrue(size < ENTRIES);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     *
+     */
+    private static class TestStoreFactory implements Factory<TestCacheStore> {
+        /** {@inheritDoc} */
+        @Override public TestCacheStore create() {
+            return new TestCacheStore();
+        }
+    }
+
+    /**
+     *
+     */
+    private static class TestCacheStore extends CacheStoreAdapter<Object, Object> {
+        /** {@inheritDoc} */
+        @Override public Object load(Object key) throws CacheLoaderException {
+            ThreadLocalRandom r = ThreadLocalRandom.current();
+
+            if (r.nextInt() % 5 == 0)
+                return new TestObject(PAGE_SIZE / 4 - 50 + r.nextInt(5000)); // Fragmented object.
+            else
+                return new TestObject(r.nextInt(PAGE_SIZE / 4 - 50)); // Fits in one page.
+        }
+
+        /** {@inheritDoc} */
+        @Override public void write(Cache.Entry<?, ?> entry) {
+            throw new UnsupportedOperationException();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void delete(Object key) {
+            // No-op.
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/ff5b3e16/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/PageEvictionTouchOrderTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/PageEvictionTouchOrderTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/PageEvictionTouchOrderTest.java
new file mode 100644
index 0000000..4335649
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/PageEvictionTouchOrderTest.java
@@ -0,0 +1,109 @@
+/*
+* 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.paged;
+
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.CacheWriteSynchronizationMode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.configuration.IgniteConfiguration;
+
+/**
+ *
+ */
+public class PageEvictionTouchOrderTest extends PageEvictionAbstractTest {
+    /** Test entries number. */
+    private static final int SAFE_ENTRIES = 1000;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        return setEvictionMode(/* Overriden by FairFifoPageEvictionTracker */DataPageEvictionMode.RANDOM_LRU,
+            super.getConfiguration(gridName));
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        System.setProperty("override.fair.fifo.page.eviction.tracker", "true");
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTouchOrderWithFairFifoEvictionAtomicReplicated() throws Exception {
+        testTouchOrderWithFairFifoEviction(CacheAtomicityMode.ATOMIC, CacheMode.REPLICATED);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTouchOrderWithFairFifoEvictionAtomicLocal() throws Exception {
+        testTouchOrderWithFairFifoEviction(CacheAtomicityMode.ATOMIC, CacheMode.LOCAL);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTouchOrderWithFairFifoEvictionTxReplicated() throws Exception {
+        testTouchOrderWithFairFifoEviction(CacheAtomicityMode.TRANSACTIONAL, CacheMode.REPLICATED);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTouchOrderWithFairFifoEvictionTxLocal() throws Exception {
+        testTouchOrderWithFairFifoEviction(CacheAtomicityMode.TRANSACTIONAL, CacheMode.LOCAL);
+    }
+
+    /**
+     * @param atomicityMode Atomicity mode.
+     * @param cacheMode Cache mode.
+     * @throws Exception If failed.
+     */
+    private void testTouchOrderWithFairFifoEviction(CacheAtomicityMode atomicityMode, CacheMode cacheMode)
+        throws Exception {
+        startGrid(0);
+
+        CacheConfiguration<Object, Object> cfg = cacheConfig("evict-fair", null, cacheMode, atomicityMode,
+            CacheWriteSynchronizationMode.PRIMARY_SYNC);
+
+        IgniteCache<Object, Object> cache = ignite(0).getOrCreateCache(cfg);
+
+        for (int i = 1; i <= ENTRIES; i++) {
+            cache.put(i, new TestObject(PAGE_SIZE / 6));
+            // Row size is between PAGE_SIZE / 2 and PAGE_SIZE. Enforces "one row - one page".
+
+            if (i % (ENTRIES / 10) == 0)
+                System.out.println(">>> Entries put: " + i);
+        }
+
+        for (int i = ENTRIES - SAFE_ENTRIES + 1; i <= ENTRIES; i++)
+            assertNotNull(cache.get(i));
+
+        for (int i = 1; i <= SAFE_ENTRIES; i++)
+            assertNull(cache.get(i));
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+
+        System.setProperty("override.fair.fifo.page.eviction.tracker", "false");
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/ff5b3e16/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/PageEvictionWithRebalanceTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/PageEvictionWithRebalanceTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/PageEvictionWithRebalanceTest.java
new file mode 100644
index 0000000..fd80201
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/PageEvictionWithRebalanceTest.java
@@ -0,0 +1,81 @@
+/*
+* 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.paged;
+
+import java.util.concurrent.ThreadLocalRandom;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.CachePeekMode;
+import org.apache.ignite.cache.CacheWriteSynchronizationMode;
+import org.apache.ignite.configuration.CacheConfiguration;
+
+/**
+ *
+ */
+public abstract class PageEvictionWithRebalanceTest extends PageEvictionAbstractTest {
+    /**
+     * @throws Exception If failed.
+     */
+    public void testEvictionWithRebalance() throws Exception {
+        startGridsMultiThreaded(4);
+
+        CacheConfiguration<Object, Object> cfg = cacheConfig("evict-rebalance", null, CacheMode.PARTITIONED,
+            CacheAtomicityMode.ATOMIC, CacheWriteSynchronizationMode.PRIMARY_SYNC);
+
+        IgniteCache<Object, Object> cache = ignite(0).getOrCreateCache(cfg);
+
+        for (int i = 1; i <= ENTRIES; i++) {
+            ThreadLocalRandom r = ThreadLocalRandom.current();
+
+            if (r.nextInt() % 5 == 0)
+                cache.put(i, new TestObject(PAGE_SIZE / 4 - 50 + r.nextInt(5000))); // Fragmented object.
+            else
+                cache.put(i, new TestObject(r.nextInt(PAGE_SIZE / 4 - 50))); // Fits in one page.
+
+            if (i % (ENTRIES / 10) == 0)
+                System.out.println(">>> Entries put: " + i);
+        }
+
+        int size = cache.size(CachePeekMode.PRIMARY);
+
+        System.out.println(">>> Resulting size: " + size);
+
+        assertTrue(size < ENTRIES);
+
+        for (int i = 3; i >= 1; i--) {
+            stopGrid(i);
+
+            cache.rebalance().get();
+
+            awaitPartitionMapExchange();
+
+            int rebalanceSize = cache.size(CachePeekMode.PRIMARY);
+
+            System.out.println(">>> Size after rebalance: " + rebalanceSize);
+
+            assertTrue(rebalanceSize < size);
+
+            size = rebalanceSize;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/ff5b3e16/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/Random2LruPageEvictionMultinodeTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/Random2LruPageEvictionMultinodeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/Random2LruPageEvictionMultinodeTest.java
new file mode 100644
index 0000000..b05ec43
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/Random2LruPageEvictionMultinodeTest.java
@@ -0,0 +1,30 @@
+/*
+* 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.paged;
+
+import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.configuration.IgniteConfiguration;
+
+/**
+ *
+ */
+public class Random2LruPageEvictionMultinodeTest extends PageEvictionMultinodeTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        return setEvictionMode(DataPageEvictionMode.RANDOM_2_LRU, super.getConfiguration(gridName));
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/ff5b3e16/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/Random2LruPageEvictionWithRebalanceTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/Random2LruPageEvictionWithRebalanceTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/Random2LruPageEvictionWithRebalanceTest.java
new file mode 100644
index 0000000..56698fb
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/Random2LruPageEvictionWithRebalanceTest.java
@@ -0,0 +1,30 @@
+/*
+* 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.paged;
+
+import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.configuration.IgniteConfiguration;
+
+/**
+ *
+ */
+public class Random2LruPageEvictionWithRebalanceTest extends PageEvictionWithRebalanceTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        return setEvictionMode(DataPageEvictionMode.RANDOM_2_LRU, super.getConfiguration(gridName));
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/ff5b3e16/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/RandomLruPageEvictionMultinodeTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/RandomLruPageEvictionMultinodeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/RandomLruPageEvictionMultinodeTest.java
new file mode 100644
index 0000000..38ca2af
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/RandomLruPageEvictionMultinodeTest.java
@@ -0,0 +1,30 @@
+/*
+* 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.paged;
+
+import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.configuration.IgniteConfiguration;
+
+/**
+ *
+ */
+public class RandomLruPageEvictionMultinodeTest extends PageEvictionMultinodeTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        return setEvictionMode(DataPageEvictionMode.RANDOM_LRU, super.getConfiguration(gridName));
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/ff5b3e16/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/RandomLruPageEvictionWithRebalanceTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/RandomLruPageEvictionWithRebalanceTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/RandomLruPageEvictionWithRebalanceTest.java
new file mode 100644
index 0000000..d961360
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/RandomLruPageEvictionWithRebalanceTest.java
@@ -0,0 +1,30 @@
+/*
+* 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.paged;
+
+import org.apache.ignite.configuration.DataPageEvictionMode;
+import org.apache.ignite.configuration.IgniteConfiguration;
+
+/**
+ *
+ */
+public class RandomLruPageEvictionWithRebalanceTest extends PageEvictionWithRebalanceTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        return setEvictionMode(DataPageEvictionMode.RANDOM_LRU, super.getConfiguration(gridName));
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/ff5b3e16/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/TestObject.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/TestObject.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/TestObject.java
new file mode 100644
index 0000000..baf2414
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/paged/TestObject.java
@@ -0,0 +1,78 @@
+/*
+* 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.paged;
+
+import java.util.Arrays;
+import java.util.concurrent.ThreadLocalRandom;
+
+/**
+ *
+ */
+class TestObject {
+    /** */
+    private int b;
+
+    /** */
+    private String c;
+
+    /** */
+    private int[] arr;
+
+    /**
+     * @param intArrSize Int array size.
+     */
+    public TestObject(int intArrSize) {
+        this.b = intArrSize;
+
+        this.c = String.valueOf(2 * intArrSize);
+
+        arr = new int[intArrSize];
+
+        for (int i = 0; i < intArrSize; i++)
+            arr[i] = ThreadLocalRandom.current().nextInt();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object o) {
+        if (this == o)
+            return true;
+
+        if (o == null || getClass() != o.getClass())
+            return false;
+
+        TestObject testObj = (TestObject)o;
+
+        if (b != testObj.b)
+            return false;
+
+        if (c != null ? !c.equals(testObj.c) : testObj.c != null)
+            return false;
+
+        return Arrays.equals(arr, testObj.arr);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        int res = b;
+
+        res = 31 * res + (c != null ? c.hashCode() : 0);
+
+        res = 31 * res + Arrays.hashCode(arr);
+
+        return res;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/ff5b3e16/modules/core/src/test/java/org/apache/ignite/internal/processors/database/FreeListImplSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/FreeListImplSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/FreeListImplSelfTest.java
index 7fccef1..d5011a8 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/FreeListImplSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/FreeListImplSelfTest.java
@@ -39,6 +39,8 @@ import org.apache.ignite.internal.processors.cache.CacheObjectContext;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 import org.apache.ignite.internal.processors.cache.database.CacheDataRow;
 import org.apache.ignite.internal.processors.cache.database.MemoryMetricsImpl;
+import org.apache.ignite.internal.processors.cache.database.MemoryPolicy;
+import org.apache.ignite.internal.processors.cache.database.evict.NoOpPageEvictionTracker;
 import org.apache.ignite.internal.processors.cache.database.freelist.FreeList;
 import org.apache.ignite.internal.processors.cache.database.freelist.FreeListImpl;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
@@ -334,7 +336,11 @@ public class FreeListImplSelfTest extends GridCommonAbstractTest {
 
         long metaPageId = pageMem.allocatePage(1, 1, PageIdAllocator.FLAG_DATA);
 
-        return new FreeListImpl(1, "freelist", pageMem, new MemoryMetricsImpl(null), null, null, metaPageId, true);
+        MemoryMetricsImpl metrics = new MemoryMetricsImpl(null);
+
+        MemoryPolicy memPlc = new MemoryPolicy(pageMem, null, metrics, new NoOpPageEvictionTracker());
+
+        return new FreeListImpl(1, "freelist", metrics, memPlc, null, null, metaPageId, true);
     }
 
     /**
@@ -407,6 +413,11 @@ public class FreeListImplSelfTest extends GridCommonAbstractTest {
         @Override public int hash() {
             throw new UnsupportedOperationException();
         }
+
+        /** {@inheritDoc} */
+        @Override public int cacheId() {
+            return 0;
+        }
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/ff5b3e16/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 94e1447..1bdfdd1 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
@@ -33,6 +33,12 @@ import org.apache.ignite.internal.processors.cache.eviction.fifo.FifoEvictionPol
 import org.apache.ignite.internal.processors.cache.eviction.lru.LruEvictionPolicySelfTest;
 import org.apache.ignite.internal.processors.cache.eviction.lru.LruNearEvictionPolicySelfTest;
 import org.apache.ignite.internal.processors.cache.eviction.lru.LruNearOnlyNearEvictionPolicySelfTest;
+import org.apache.ignite.internal.processors.cache.eviction.paged.PageEvictionReadThroughTest;
+import org.apache.ignite.internal.processors.cache.eviction.paged.PageEvictionTouchOrderTest;
+import org.apache.ignite.internal.processors.cache.eviction.paged.Random2LruPageEvictionMultinodeTest;
+import org.apache.ignite.internal.processors.cache.eviction.paged.Random2LruPageEvictionWithRebalanceTest;
+import org.apache.ignite.internal.processors.cache.eviction.paged.RandomLruPageEvictionMultinodeTest;
+import org.apache.ignite.internal.processors.cache.eviction.paged.RandomLruPageEvictionWithRebalanceTest;
 import org.apache.ignite.internal.processors.cache.eviction.sorted.SortedEvictionPolicySelfTest;
 
 /**
@@ -63,6 +69,13 @@ public class IgniteCacheEvictionSelfTestSuite extends TestSuite {
         suite.addTest(new TestSuite(GridCacheEmptyEntriesLocalSelfTest.class));
         suite.addTest(new TestSuite(GridCacheEvictableEntryEqualsSelfTest.class));
 
+        suite.addTest(new TestSuite(RandomLruPageEvictionMultinodeTest.class));
+        suite.addTest(new TestSuite(Random2LruPageEvictionMultinodeTest.class));
+        suite.addTest(new TestSuite(RandomLruPageEvictionWithRebalanceTest.class));
+        suite.addTest(new TestSuite(Random2LruPageEvictionWithRebalanceTest.class));
+        suite.addTest(new TestSuite(PageEvictionTouchOrderTest.class));
+        suite.addTest(new TestSuite(PageEvictionReadThroughTest.class));
+
         return suite;
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/ff5b3e16/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Row.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Row.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Row.java
index ce10cdb..042e163 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Row.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Row.java
@@ -174,4 +174,9 @@ public abstract class GridH2Row extends Row implements GridSearchRowPointer, Cac
     @Override public int hash() {
         throw new UnsupportedOperationException();
     }
+
+    /** {@inheritDoc} */
+    @Override public int cacheId() {
+        return 0;
+    }
 }
\ No newline at end of file