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 2017/11/13 09:52:10 UTC

[16/28] ignite git commit: ignite-6669 Added eviction policy factory to cache configuration.

ignite-6669 Added eviction policy factory to cache configuration.

Signed-off-by: Andrey Gura <ag...@apache.org>


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

Branch: refs/heads/ignite-zk
Commit: 6579e69f20197567af43b14f72bb3a8852849353
Parents: 954e47b
Author: Andrey V. Mashenkov <an...@gmail.com>
Authored: Fri Nov 10 17:33:52 2017 +0300
Committer: Andrey Gura <ag...@apache.org>
Committed: Fri Nov 10 17:33:52 2017 +0300

----------------------------------------------------------------------
 .../java/org/apache/ignite/cache/CacheMode.java |    2 +-
 .../eviction/AbstractEvictionPolicyFactory.java |  104 ++
 .../fifo/FifoEvictionPolicyFactory.java         |   72 ++
 .../eviction/lru/LruEvictionPolicyFactory.java  |   72 ++
 .../sorted/SortedEvictionPolicyFactory.java     |   98 ++
 .../configuration/CacheConfiguration.java       |   42 +-
 .../configuration/NearCacheConfiguration.java   |   37 +
 .../processors/cache/ClusterCachesInfo.java     |    7 +
 .../processors/cache/GridCacheAttributes.java   |   19 +
 .../cache/GridCacheEvictionManager.java         |   15 +-
 .../processors/cache/GridCacheProcessor.java    |   19 +-
 .../processors/cache/GridCacheUtils.java        |    1 +
 .../processors/igfs/IgfsHelperImpl.java         |    8 +-
 .../internal/processors/igfs/IgfsImpl.java      |    4 +-
 .../cache/VisorCacheNearConfiguration.java      |    1 +
 ...idCacheConfigurationConsistencySelfTest.java |   52 +
 .../GridCacheNearEvictionEventSelfTest.java     |    5 -
 .../EvictionPolicyFactoryAbstractTest.java      | 1073 ++++++++++++++++++
 .../fifo/FifoEvictionPolicyFactorySelfTest.java |  261 +++++
 .../lru/LruEvictionPolicyFactorySelfTest.java   |  352 ++++++
 .../SortedEvictionPolicyFactorySelfTest.java    |  264 +++++
 .../IgniteCacheEvictionSelfTestSuite.java       |    6 +
 .../ApiParity/CacheConfigurationParityTest.cs   |    3 +-
 .../org/apache/ignite/yardstick/IgniteNode.java |    6 +-
 24 files changed, 2502 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/6579e69f/modules/core/src/main/java/org/apache/ignite/cache/CacheMode.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/CacheMode.java b/modules/core/src/main/java/org/apache/ignite/cache/CacheMode.java
index e75fa0c..4171b1a 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/CacheMode.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/CacheMode.java
@@ -55,7 +55,7 @@ public enum CacheMode {
      * <p>
      * Note that partitioned cache is always fronted by local
      * {@code 'near'} cache which stores most recent data. You
-     * can configure the size of near cache via {@link NearCacheConfiguration#getNearEvictionPolicy()}
+     * can configure the size of near cache via {@link NearCacheConfiguration#getNearEvictionPolicyFactory()}
      * configuration property.
      */
     PARTITIONED;

http://git-wip-us.apache.org/repos/asf/ignite/blob/6579e69f/modules/core/src/main/java/org/apache/ignite/cache/eviction/AbstractEvictionPolicyFactory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/eviction/AbstractEvictionPolicyFactory.java b/modules/core/src/main/java/org/apache/ignite/cache/eviction/AbstractEvictionPolicyFactory.java
new file mode 100644
index 0000000..012c7ee
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/cache/eviction/AbstractEvictionPolicyFactory.java
@@ -0,0 +1,104 @@
+/*
+ * 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.cache.eviction;
+
+import javax.cache.configuration.Factory;
+import org.apache.ignite.internal.util.typedef.internal.A;
+
+/**
+ * Common functionality implementation for eviction policies factories.
+ */
+public abstract class AbstractEvictionPolicyFactory<T> implements Factory<T> {
+    /** */
+    private int maxSize;
+
+    /** */
+    private int batchSize = 1;
+
+    /** */
+    private long maxMemSize;
+
+    /**
+     * Sets maximum allowed size of cache before entry will start getting evicted.
+     *
+     * @param max Maximum allowed size of cache before entry will start getting evicted.
+     * @return {@code this} for chaining.
+     */
+    public AbstractEvictionPolicyFactory setMaxSize(int max) {
+        A.ensure(max >= 0, "max >= 0");
+
+        this.maxSize = max;
+
+        return this;
+    }
+
+    /**
+     * Gets maximum allowed size of cache before entry will start getting evicted.
+     *
+     * @return Maximum allowed size of cache before entry will start getting evicted.
+     */
+    public int getMaxSize() {
+        return maxSize;
+    }
+
+    /**
+     * Sets batch size.
+     *
+     * @param batchSize Batch size.
+     * @return {@code this} for chaining.
+     */
+    public AbstractEvictionPolicyFactory setBatchSize(int batchSize) {
+        A.ensure(batchSize > 0, "batchSize > 0");
+
+        this.batchSize = batchSize;
+
+        return this;
+    }
+
+    /**
+     * Gets batch size.
+     *
+     * @return batch size.
+     */
+    public int getBatchSize() {
+        return batchSize;
+    }
+
+    /**
+     * Sets maximum allowed cache size in bytes.
+     *
+     * @return {@code this} for chaining.
+     */
+    public AbstractEvictionPolicyFactory setMaxMemorySize(long maxMemSize) {
+        A.ensure(maxMemSize >= 0, "maxMemSize >= 0");
+
+        this.maxMemSize = maxMemSize;
+
+        return this;
+    }
+
+    /**
+     * Gets maximum allowed cache size in bytes.
+     *
+     * @return maximum allowed cache size in bytes.
+     */
+    public long getMaxMemorySize() {
+        return maxMemSize;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/6579e69f/modules/core/src/main/java/org/apache/ignite/cache/eviction/fifo/FifoEvictionPolicyFactory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/eviction/fifo/FifoEvictionPolicyFactory.java b/modules/core/src/main/java/org/apache/ignite/cache/eviction/fifo/FifoEvictionPolicyFactory.java
new file mode 100644
index 0000000..856865a
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/cache/eviction/fifo/FifoEvictionPolicyFactory.java
@@ -0,0 +1,72 @@
+/*
+ * 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.cache.eviction.fifo;
+
+import org.apache.ignite.cache.eviction.AbstractEvictionPolicyFactory;
+
+/**
+ * Factory class for {@link FifoEvictionPolicy}.
+ *
+ * Creates cache Eviction policy based on {@code First In First Out (FIFO)} algorithm and supports batch eviction.
+ * <p>
+ * The eviction starts in the following cases:
+ * <ul>
+ *     <li>The cache size becomes {@code batchSize} elements greater than the maximum size.</li>
+ *     <li>
+ *         The size of cache entries in bytes becomes greater than the maximum memory size.
+ *         The size of cache entry calculates as sum of key size and value size.
+ *     </li>
+ * </ul>
+ * <b>Note:</b>Batch eviction is enabled only if maximum memory limit isn't set ({@code maxMemSize == 0}).
+ * {@code batchSize} elements will be evicted in this case. The default {@code batchSize} value is {@code 1}.
+ * <p>
+ * {@link FifoEvictionPolicy} implementation is very efficient since it does not create any additional
+ * table-like data structures. The {@code FIFO} ordering information is
+ * maintained by attaching ordering metadata to cache entries.
+ */
+public class FifoEvictionPolicyFactory<K, V> extends AbstractEvictionPolicyFactory<FifoEvictionPolicy<K, V>> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Constructor. */
+    public FifoEvictionPolicyFactory() {
+    }
+
+    /** Constructor. */
+    public FifoEvictionPolicyFactory(int maxSize) {
+        setMaxSize(maxSize);
+    }
+
+    /** */
+    public FifoEvictionPolicyFactory(int maxSize, int batchSize, long maxMemSize) {
+        setMaxSize(maxSize);
+        setBatchSize(batchSize);
+        setMaxMemorySize(maxMemSize);
+    }
+
+    /** {@inheritDoc} */
+    @Override public FifoEvictionPolicy<K, V> create() {
+        FifoEvictionPolicy<K, V> policy = new FifoEvictionPolicy<>();
+
+        policy.setBatchSize(getBatchSize());
+        policy.setMaxMemorySize(getMaxMemorySize());
+        policy.setMaxSize(getMaxSize());
+
+        return policy;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/6579e69f/modules/core/src/main/java/org/apache/ignite/cache/eviction/lru/LruEvictionPolicyFactory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/eviction/lru/LruEvictionPolicyFactory.java b/modules/core/src/main/java/org/apache/ignite/cache/eviction/lru/LruEvictionPolicyFactory.java
new file mode 100644
index 0000000..8f7fbc5
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/cache/eviction/lru/LruEvictionPolicyFactory.java
@@ -0,0 +1,72 @@
+/*
+ * 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.cache.eviction.lru;
+
+import org.apache.ignite.cache.eviction.AbstractEvictionPolicyFactory;
+
+/**
+ * Factory class for {@link LruEvictionPolicy}.
+ *
+ * Creates cache Eviction policy based on {@code Least Recently Used (LRU)} algorithm and supports batch eviction.
+ * <p>
+ * The eviction starts in the following cases:
+ * <ul>
+ *     <li>The cache size becomes {@code batchSize} elements greater than the maximum size.</li>
+ *     <li>
+ *         The size of cache entries in bytes becomes greater than the maximum memory size.
+ *         The size of cache entry calculates as sum of key size and value size.
+ *     </li>
+ * </ul>
+ * <b>Note:</b>Batch eviction is enabled only if maximum memory limit isn't set ({@code maxMemSize == 0}).
+ * {@code batchSize} elements will be evicted in this case. The default {@code batchSize} value is {@code 1}.
+
+ *  {@link LruEvictionPolicy} implementation is very efficient since it is lock-free and does not create any additional table-like
+ * data structures. The {@code LRU} ordering information is maintained by attaching ordering metadata to cache entries.
+ */
+public class LruEvictionPolicyFactory<K, V> extends AbstractEvictionPolicyFactory<LruEvictionPolicy<K, V>> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    public LruEvictionPolicyFactory() {
+    }
+
+    /** */
+    public LruEvictionPolicyFactory(int maxSize) {
+        setMaxSize(maxSize);
+    }
+
+    /** */
+    public LruEvictionPolicyFactory(int maxSize, int batchSize, long maxMemSize) {
+        setMaxSize(maxSize);
+        setBatchSize(batchSize);
+        setMaxMemorySize(maxMemSize);
+    }
+
+    /** {@inheritDoc} */
+    @Override public LruEvictionPolicy<K, V> create() {
+        LruEvictionPolicy<K, V> policy = new LruEvictionPolicy<>();
+
+        policy.setBatchSize(getBatchSize());
+        policy.setMaxMemorySize(getMaxMemorySize());
+        policy.setMaxSize(getMaxSize());
+
+        return policy;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/6579e69f/modules/core/src/main/java/org/apache/ignite/cache/eviction/sorted/SortedEvictionPolicyFactory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/eviction/sorted/SortedEvictionPolicyFactory.java b/modules/core/src/main/java/org/apache/ignite/cache/eviction/sorted/SortedEvictionPolicyFactory.java
new file mode 100644
index 0000000..a88c277
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/cache/eviction/sorted/SortedEvictionPolicyFactory.java
@@ -0,0 +1,98 @@
+/*
+ * 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.cache.eviction.sorted;
+
+import java.io.Serializable;
+import java.util.Comparator;
+import org.apache.ignite.cache.eviction.AbstractEvictionPolicyFactory;
+import org.apache.ignite.cache.eviction.EvictableEntry;
+
+/**
+ * Factory class for {@link SortedEvictionPolicy}.
+ *
+ * Creates cache Eviction policy which will select the minimum cache entry for eviction.
+ * <p>
+ * The eviction starts in the following cases:
+ * <ul>
+ *     <li>The cache size becomes {@code batchSize} elements greater than the maximum size.</li>
+ *     <li>
+ *         The size of cache entries in bytes becomes greater than the maximum memory size.
+ *         The size of cache entry calculates as sum of key size and value size.
+ *     </li>
+ * </ul>
+ * <b>Note:</b>Batch eviction is enabled only if maximum memory limit isn't set ({@code maxMemSize == 0}).
+ * {@code batchSize} elements will be evicted in this case. The default {@code batchSize} value is {@code 1}.
+ * <p>
+ * Entries comparison based on {@link Comparator} instance if provided.
+ * Default {@code Comparator} behaviour is use cache entries keys for comparison that imposes a requirement for keys
+ * to implement {@link Comparable} interface.
+ * <p>
+ * User defined comparator should implement {@link Serializable} interface.
+ */
+public class SortedEvictionPolicyFactory<K,V> extends AbstractEvictionPolicyFactory<SortedEvictionPolicy<K, V>> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Comparator. */
+    private Comparator<EvictableEntry<K, V>> comp;
+
+    /** */
+    public SortedEvictionPolicyFactory() {
+    }
+
+    /** */
+    public SortedEvictionPolicyFactory(int maxSize) {
+        setMaxSize(maxSize);
+    }
+
+    /** */
+    public SortedEvictionPolicyFactory(int maxSize, int batchSize, long maxMemSize) {
+        setMaxSize(maxSize);
+        setBatchSize(batchSize);
+        setMaxMemorySize(maxMemSize);
+    }
+
+    /**
+     * Gets entries comparator.
+     * @return entry comparator.
+     */
+    public Comparator<EvictableEntry<K, V>> getComp() {
+        return comp;
+    }
+
+    /**
+     * Sets entries comparator.
+     *
+     * @param comp entry comparator.
+     */
+    public void setComp(Comparator<EvictableEntry<K, V>> comp) {
+        this.comp = comp;
+    }
+
+    /** {@inheritDoc} */
+    @Override public SortedEvictionPolicy<K, V> create() {
+        SortedEvictionPolicy<K, V> policy = new SortedEvictionPolicy<>(comp);
+
+        policy.setBatchSize(getBatchSize());
+        policy.setMaxMemorySize(getMaxMemorySize());
+        policy.setMaxSize(getMaxSize());
+
+        return policy;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/6579e69f/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
index 807c817..aeb1b2f 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
@@ -200,9 +200,13 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
     /** Rebalance timeout. */
     private long rebalanceTimeout = DFLT_REBALANCE_TIMEOUT;
 
-    /** Cache expiration policy. */
+    /** Cache eviction policy. */
+    @Deprecated
     private EvictionPolicy evictPlc;
 
+    /** Cache eviction policy factory. */
+    private Factory evictPlcFactory;
+
     /** */
     private boolean onheapCache;
 
@@ -395,6 +399,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
         eagerTtl = cc.isEagerTtl();
         evictFilter = cc.getEvictionFilter();
         evictPlc = cc.getEvictionPolicy();
+        evictPlcFactory = cc.getEvictionPolicyFactory();
         expiryPolicyFactory = cc.getExpiryPolicyFactory();
         grpName = cc.getGroupName();
         indexedTypes = cc.getIndexedTypes();
@@ -553,7 +558,10 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * which means that evictions are disabled for cache.
      *
      * @return Cache eviction policy or {@code null} if evictions should be disabled.
+     *
+     * @deprecated Use {@link #getEvictionPolicyFactory()} instead.
      */
+    @Deprecated
     @SuppressWarnings({"unchecked"})
     @Nullable public EvictionPolicy<K, V> getEvictionPolicy() {
         return evictPlc;
@@ -562,9 +570,12 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
     /**
      * Sets cache eviction policy.
      *
-     * @param evictPlc Cache expiration policy.
+     * @param evictPlc Cache eviction policy.
      * @return {@code this} for chaining.
+     *
+     * @deprecated Use {@link #setEvictionPolicyFactory(Factory)} instead.
      */
+    @Deprecated
     public CacheConfiguration<K, V> setEvictionPolicy(@Nullable EvictionPolicy evictPlc) {
         this.evictPlc = evictPlc;
 
@@ -572,6 +583,31 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
     }
 
     /**
+     * Gets cache eviction policy factory. By default, returns {@code null}
+     * which means that evictions are disabled for cache.
+     *
+     * @return Cache eviction policy factory or {@code null} if evictions should be disabled
+     * or if {@link #getEvictionPolicy()} should be used instead.
+     */
+    @Nullable public Factory<EvictionPolicy<? super K, ? super V>> getEvictionPolicyFactory() {
+        return evictPlcFactory;
+    }
+
+    /**
+     * Sets cache eviction policy factory.
+     * Note: Eviction policy factory should be {@link Serializable}.
+     *
+     * @param evictPlcFactory Cache eviction policy factory.
+     * @return {@code this} for chaining.
+     */
+    public CacheConfiguration<K, V> setEvictionPolicyFactory(
+        @Nullable Factory<? extends EvictionPolicy<? super K, ? super V>> evictPlcFactory) {
+        this.evictPlcFactory = evictPlcFactory;
+
+        return this;
+    }
+
+    /**
      * Checks if the on-heap cache is enabled for the off-heap based page memory.
      *
      * @return On-heap cache enabled flag.
@@ -664,7 +700,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * never be evicted.
      * <p>
      * If not provided, any entry may be evicted depending on
-     * {@link #getEvictionPolicy() eviction policy} configuration.
+     * {@link #getEvictionPolicyFactory()} eviction policy} configuration.
      *
      * @return Eviction filter or {@code null}.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/6579e69f/modules/core/src/main/java/org/apache/ignite/configuration/NearCacheConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/NearCacheConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/NearCacheConfiguration.java
index 7b3022c..ae6e98b 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/NearCacheConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/NearCacheConfiguration.java
@@ -18,9 +18,11 @@
 package org.apache.ignite.configuration;
 
 import java.io.Serializable;
+import javax.cache.configuration.Factory;
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.cache.eviction.EvictionPolicy;
 import org.apache.ignite.internal.util.typedef.internal.S;
+import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.configuration.CacheConfiguration.DFLT_NEAR_START_SIZE;
 
@@ -37,8 +39,12 @@ public class NearCacheConfiguration<K, V> implements Serializable {
     private static final long serialVersionUID = 0L;
 
     /** Near cache eviction policy. */
+    @Deprecated
     private EvictionPolicy<K, V> nearEvictPlc;
 
+    /** Near cache eviction policy factory. */
+    private Factory nearEvictPlcFactory;
+
     /** Default near cache start size. */
     private int nearStartSize = DFLT_NEAR_START_SIZE;
 
@@ -55,6 +61,7 @@ public class NearCacheConfiguration<K, V> implements Serializable {
      * @param ccfg Configuration to copy.
      */
     public NearCacheConfiguration(NearCacheConfiguration<K, V> ccfg) {
+        nearEvictPlcFactory = ccfg.getNearEvictionPolicyFactory();
         nearEvictPlc = ccfg.getNearEvictionPolicy();
         nearStartSize = ccfg.getNearStartSize();
     }
@@ -65,7 +72,10 @@ public class NearCacheConfiguration<K, V> implements Serializable {
      *
      * @return Near eviction policy.
      * @see CacheConfiguration#getEvictionPolicy()
+     *
+     * @deprecated Use {@link #getNearEvictionPolicyFactory()}  instead.
      */
+    @Deprecated
     public EvictionPolicy<K, V> getNearEvictionPolicy() {
         return nearEvictPlc;
     }
@@ -75,7 +85,10 @@ public class NearCacheConfiguration<K, V> implements Serializable {
      *
      * @param nearEvictPlc Near eviction policy.
      * @return {@code this} for chaining.
+     *
+     * @deprecated Use {@link #setNearEvictionPolicyFactory(Factory)} instead.
      */
+    @Deprecated
     public NearCacheConfiguration<K, V> setNearEvictionPolicy(EvictionPolicy<K, V> nearEvictPlc) {
         this.nearEvictPlc = nearEvictPlc;
 
@@ -83,6 +96,30 @@ public class NearCacheConfiguration<K, V> implements Serializable {
     }
 
     /**
+     * Gets cache eviction policy factory. By default, returns {@code null}
+     * which means that evictions are disabled for cache.
+     *
+     * @return Cache eviction policy or {@code null} if evictions should be disabled.
+     */
+    @Nullable public Factory<EvictionPolicy<? super K, ? super V>> getNearEvictionPolicyFactory() {
+        return nearEvictPlcFactory;
+    }
+
+    /**
+     * Sets cache eviction policy factory.
+     * Note: Eviction policy factory should be {@link Serializable}.
+     *
+     * @param nearEvictPlcFactory Cache expiration policy.
+     * @return {@code this} for chaining.
+     */
+    public NearCacheConfiguration<K, V> setNearEvictionPolicyFactory(
+        @Nullable Factory<? extends EvictionPolicy<? super K, ? super V>> nearEvictPlcFactory) {
+        this.nearEvictPlcFactory = nearEvictPlcFactory;
+
+        return this;
+    }
+
+    /**
      * Gets initial cache size for near cache which will be used to pre-create internal
      * hash table after start. Default value is defined by {@link CacheConfiguration#DFLT_NEAR_START_SIZE}.
      *

http://git-wip-us.apache.org/repos/asf/ignite/blob/6579e69f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
index 8382821..69f1a27 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
@@ -275,6 +275,9 @@ class ClusterCachesInfo {
             CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "evictionPolicy", "Eviction policy",
                 locAttr.evictionPolicyClassName(), rmtAttr.evictionPolicyClassName(), true);
 
+            CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "evictionPolicyFactory", "Eviction policy factory",
+                locAttr.evictionPolicyFactoryClassName(), rmtAttr.evictionPolicyFactoryClassName(), true);
+
             CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "transactionManagerLookup",
                 "Transaction manager lookup", locAttr.transactionManagerLookupClassName(),
                 rmtAttr.transactionManagerLookupClassName(), false);
@@ -333,6 +336,10 @@ class ClusterCachesInfo {
                     "Near eviction policy", locAttr.nearEvictionPolicyClassName(),
                     rmtAttr.nearEvictionPolicyClassName(), false);
 
+                CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "nearEvictionPolicyFactory",
+                    "Near eviction policy factory", locAttr.nearEvictionPolicyFactoryClassName(),
+                    rmtAttr.nearEvictionPolicyFactoryClassName(), false);
+
                 CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "affinityIncludeNeighbors",
                     "Affinity include neighbors", locAttr.affinityIncludeNeighbors(),
                     rmtAttr.affinityIncludeNeighbors(), true);

http://git-wip-us.apache.org/repos/asf/ignite/blob/6579e69f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java
index d64ee8b..faad1ec 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAttributes.java
@@ -154,13 +154,25 @@ public class GridCacheAttributes implements Serializable {
 
     /**
      * @return Eviction policy class name.
+     *
+     * @deprecated Use evictionPolicyFactoryClassName() instead.
      */
+    @Deprecated
     public String evictionPolicyClassName() {
         return className(ccfg.getEvictionPolicy());
     }
 
     /**
+     * @return Eviction policy factory class name.
+     */
+    public String evictionPolicyFactoryClassName() {
+        return className(ccfg.getEvictionPolicyFactory());
+    }
+
+    /**
      * @return Near eviction policy class name.
+     *
+     * @deprecated Use nearEvictionPolicyFactoryClassName() instead.
      */
     public String nearEvictionPolicyClassName() {
         NearCacheConfiguration nearCfg = ccfg.getNearConfiguration();
@@ -172,6 +184,13 @@ public class GridCacheAttributes implements Serializable {
     }
 
     /**
+     * @return Near eviction policy factory class name.
+     */
+    public String nearEvictionPolicyFactoryClassName() {
+        return className(ccfg.getEvictionPolicyFactory());
+    }
+
+    /**
      * @return Store class name.
      */
     public String storeFactoryClassName() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/6579e69f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionManager.java
index 7735f74..084b235 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionManager.java
@@ -60,7 +60,15 @@ public class GridCacheEvictionManager extends GridCacheManagerAdapter implements
     @Override public void start0() throws IgniteCheckedException {
         CacheConfiguration cfg = cctx.config();
 
-        plc = cctx.isNear() ? cfg.getNearConfiguration().getNearEvictionPolicy() : cfg.getEvictionPolicy();
+        if (cctx.isNear()) {
+            plc = (cfg.getNearConfiguration().getNearEvictionPolicyFactory() != null) ?
+                (EvictionPolicy)cfg.getNearConfiguration().getNearEvictionPolicyFactory().create() :
+                cfg.getNearConfiguration().getNearEvictionPolicy();
+        }
+        else if (cfg.getEvictionPolicyFactory() != null)
+            plc = (EvictionPolicy)cfg.getEvictionPolicyFactory().create();
+        else
+            plc = cfg.getEvictionPolicy();
 
         plcEnabled = plc != null;
 
@@ -298,4 +306,9 @@ public class GridCacheEvictionManager extends GridCacheManagerAdapter implements
         X.println(">>> Eviction manager memory stats [igniteInstanceName=" + cctx.igniteInstanceName() +
             ", cache=" + cctx.name() + ']');
     }
+
+    /** For test purposes. */
+    public EvictionPolicy getEvictionPolicy() {
+        return plc;
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/6579e69f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index dc599c7..59d170c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -255,7 +255,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         String msg = "Disable eviction policy (remove from configuration)";
 
-        if (cfg.getEvictionPolicy() != null)
+        if (cfg.getEvictionPolicyFactory() != null || cfg.getEvictionPolicy() != null)
             perf.add(msg, false);
         else
             perf.add(msg, true);
@@ -470,7 +470,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             assertParameter(cc.getTransactionManagerLookupClassName() == null,
                 "transaction manager can not be used with ATOMIC cache");
 
-        if (cc.getEvictionPolicy() != null && !cc.isOnheapCacheEnabled())
+        if ((cc.getEvictionPolicyFactory() != null || cc.getEvictionPolicy() != null)&& !cc.isOnheapCacheEnabled())
             throw new IgniteCheckedException("Onheap cache must be enabled if eviction policy is configured [cacheName="
                 + U.maskName(cc.getName()) + "]");
 
@@ -511,6 +511,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
      * @throws IgniteCheckedException If failed to inject.
      */
     private void prepare(CacheConfiguration cfg, Collection<Object> objs) throws IgniteCheckedException {
+        prepare(cfg, cfg.getEvictionPolicyFactory(), false);
         prepare(cfg, cfg.getEvictionPolicy(), false);
         prepare(cfg, cfg.getAffinity(), false);
         prepare(cfg, cfg.getAffinityMapper(), false);
@@ -519,8 +520,10 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         NearCacheConfiguration nearCfg = cfg.getNearConfiguration();
 
-        if (nearCfg != null)
+        if (nearCfg != null) {
+            prepare(cfg, nearCfg.getNearEvictionPolicyFactory(), true);
             prepare(cfg, nearCfg.getNearEvictionPolicy(), true);
+        }
 
         for (Object obj : objs)
             prepare(cfg, obj, false);
@@ -548,6 +551,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     private void cleanup(GridCacheContext cctx) {
         CacheConfiguration cfg = cctx.config();
 
+        cleanup(cfg, cfg.getEvictionPolicyFactory(), false);
         cleanup(cfg, cfg.getEvictionPolicy(), false);
         cleanup(cfg, cfg.getAffinity(), false);
         cleanup(cfg, cfg.getAffinityMapper(), false);
@@ -562,8 +566,10 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         NearCacheConfiguration nearCfg = cfg.getNearConfiguration();
 
-        if (nearCfg != null)
+        if (nearCfg != null) {
+            cleanup(cfg, nearCfg.getNearEvictionPolicyFactory(), true);
             cleanup(cfg, nearCfg.getNearEvictionPolicy(), true);
+        }
 
         cctx.cleanup();
     }
@@ -3721,13 +3727,16 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         ret.add(ccfg.getAffinityMapper());
         ret.add(ccfg.getEvictionFilter());
+        ret.add(ccfg.getEvictionPolicyFactory());
         ret.add(ccfg.getEvictionPolicy());
         ret.add(ccfg.getInterceptor());
 
         NearCacheConfiguration nearCfg = ccfg.getNearConfiguration();
 
-        if (nearCfg != null)
+        if (nearCfg != null) {
+            ret.add(nearCfg.getNearEvictionPolicyFactory());
             ret.add(nearCfg.getNearEvictionPolicy());
+        }
 
         Collections.addAll(ret, objs);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/6579e69f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
index 53fb4d3..248f2aa 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
@@ -1046,6 +1046,7 @@ public class GridCacheUtils {
         cache.setAtomicityMode(TRANSACTIONAL);
         cache.setWriteSynchronizationMode(FULL_SYNC);
 
+        cache.setEvictionPolicyFactory(null);
         cache.setEvictionPolicy(null);
         cache.setCacheStoreFactory(null);
         cache.setNodeFilter(CacheConfiguration.ALL_NODES);

http://git-wip-us.apache.org/repos/asf/ignite/blob/6579e69f/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsHelperImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsHelperImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsHelperImpl.java
index 29e75a5..f20b787 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsHelperImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsHelperImpl.java
@@ -31,7 +31,9 @@ import org.apache.ignite.internal.util.typedef.internal.U;
 public class IgfsHelperImpl implements IgfsHelper {
     /** {@inheritDoc} */
     @Override public void preProcessCacheConfiguration(CacheConfiguration cfg) {
-        EvictionPolicy evictPlc = cfg.getEvictionPolicy();
+        EvictionPolicy evictPlc = cfg.getEvictionPolicyFactory() != null ?
+            (EvictionPolicy)cfg.getEvictionPolicyFactory().create()
+            : cfg.getEvictionPolicy();
 
         if (evictPlc instanceof IgfsPerBlockLruEvictionPolicy && cfg.getEvictionFilter() == null)
             cfg.setEvictionFilter(new IgfsEvictionFilter());
@@ -39,7 +41,9 @@ public class IgfsHelperImpl implements IgfsHelper {
 
     /** {@inheritDoc} */
     @Override public void validateCacheConfiguration(CacheConfiguration cfg) throws IgniteCheckedException {
-        EvictionPolicy evictPlc =  cfg.getEvictionPolicy();
+        EvictionPolicy evictPlc = cfg.getEvictionPolicyFactory() != null ?
+            (EvictionPolicy)cfg.getEvictionPolicyFactory().create()
+            : cfg.getEvictionPolicy();
 
         if (evictPlc != null && evictPlc instanceof IgfsPerBlockLruEvictionPolicy) {
             EvictionFilter evictFilter = cfg.getEvictionFilter();

http://git-wip-us.apache.org/repos/asf/ignite/blob/6579e69f/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
index 5808e7c..5a9e10a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
@@ -237,7 +237,9 @@ public final class IgfsImpl implements IgfsEx {
 
         for (CacheConfiguration cacheCfg : igfsCtx.kernalContext().config().getCacheConfiguration()) {
             if (F.eq(dataCacheName, cacheCfg.getName())) {
-                EvictionPolicy evictPlc = cacheCfg.getEvictionPolicy();
+                EvictionPolicy evictPlc = cacheCfg.getEvictionPolicyFactory() != null ?
+                    (EvictionPolicy)cacheCfg.getEvictionPolicyFactory().create()
+                    : cacheCfg.getEvictionPolicy();
 
                 if (evictPlc != null & evictPlc instanceof IgfsPerBlockLruEvictionPolicy)
                     this.evictPlc = (IgfsPerBlockLruEvictionPolicy)evictPlc;

http://git-wip-us.apache.org/repos/asf/ignite/blob/6579e69f/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheNearConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheNearConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheNearConfiguration.java
index ae55f83..a4cd3d9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheNearConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheNearConfiguration.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.visor.cache;
 import java.io.IOException;
 import java.io.ObjectInput;
 import java.io.ObjectOutput;
+import org.apache.ignite.cache.eviction.EvictionPolicy;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.NearCacheConfiguration;
 import org.apache.ignite.internal.processors.cache.GridCacheUtils;

http://git-wip-us.apache.org/repos/asf/ignite/blob/6579e69f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConfigurationConsistencySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConfigurationConsistencySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConfigurationConsistencySelfTest.java
index 2865627..3f4efc2 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConfigurationConsistencySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConfigurationConsistencySelfTest.java
@@ -29,8 +29,10 @@ import org.apache.ignite.cache.affinity.AffinityFunction;
 import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
 import org.apache.ignite.cache.eviction.EvictionFilter;
 import org.apache.ignite.cache.eviction.fifo.FifoEvictionPolicy;
+import org.apache.ignite.cache.eviction.fifo.FifoEvictionPolicyFactory;
 import org.apache.ignite.cache.eviction.lru.LruEvictionPolicy;
 import org.apache.ignite.cache.eviction.sorted.SortedEvictionPolicy;
+import org.apache.ignite.cache.eviction.sorted.SortedEvictionPolicyFactory;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.DeploymentMode;
@@ -377,6 +379,28 @@ public class GridCacheConfigurationConsistencySelfTest extends GridCommonAbstrac
     /**
      * @throws Exception If failed.
      */
+    public void testDifferentEvictionPolicyEnabled() throws Exception {
+        checkSecondGridStartFails(
+            new C1<CacheConfiguration, Void>() {
+                /** {@inheritDoc} */
+                @Override public Void apply(CacheConfiguration cfg) {
+                    cfg.setEvictionPolicyFactory(new FifoEvictionPolicyFactory<>());
+                    cfg.setOnheapCacheEnabled(true);
+                    return null;
+                }
+            },
+            new C1<CacheConfiguration, Void>() {
+                /** {@inheritDoc} */
+                @Override public Void apply(CacheConfiguration cfg) {
+                    return null;
+                }
+            }
+        );
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     public void testDifferentEvictionPolicies() throws Exception {
         checkSecondGridStartFails(
             new C1<CacheConfiguration, Void>() {
@@ -401,6 +425,30 @@ public class GridCacheConfigurationConsistencySelfTest extends GridCommonAbstrac
     /**
      * @throws Exception If failed.
      */
+    public void testDifferentEvictionPolicyFactories() throws Exception {
+        checkSecondGridStartFails(
+            new C1<CacheConfiguration, Void>() {
+                /** {@inheritDoc} */
+                @Override public Void apply(CacheConfiguration cfg) {
+                    cfg.setEvictionPolicyFactory(new SortedEvictionPolicyFactory());
+                    cfg.setOnheapCacheEnabled(true);
+                    return null;
+                }
+            },
+            new C1<CacheConfiguration, Void>() {
+                /** {@inheritDoc} */
+                @Override public Void apply(CacheConfiguration cfg) {
+                    cfg.setEvictionPolicyFactory(new FifoEvictionPolicyFactory<>());
+                    cfg.setOnheapCacheEnabled(true);
+                    return null;
+                }
+            }
+        );
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     public void testDifferentEvictionFilters() throws Exception {
         checkSecondGridStartFails(
             new C1<CacheConfiguration, Void>() {
@@ -585,6 +633,7 @@ public class GridCacheConfigurationConsistencySelfTest extends GridCommonAbstrac
             @Override public Void apply(CacheConfiguration cfg) {
                 NearCacheConfiguration nearCfg = new NearCacheConfiguration();
 
+                nearCfg.setNearEvictionPolicyFactory(new FifoEvictionPolicyFactory<>());
                 nearCfg.setNearEvictionPolicy(new LruEvictionPolicy());
 
                 cfg.setNearConfiguration(nearCfg);
@@ -599,6 +648,7 @@ public class GridCacheConfigurationConsistencySelfTest extends GridCommonAbstrac
             @Override public Void apply(CacheConfiguration cfg) {
                 NearCacheConfiguration nearCfg = new NearCacheConfiguration();
 
+                nearCfg.setNearEvictionPolicyFactory(new FifoEvictionPolicyFactory<>());
                 nearCfg.setNearEvictionPolicy(new FifoEvictionPolicy());
 
                 cfg.setNearConfiguration(nearCfg);
@@ -624,6 +674,7 @@ public class GridCacheConfigurationConsistencySelfTest extends GridCommonAbstrac
             @Override public Void apply(CacheConfiguration cfg) {
                 cfg.setAffinity(new TestRendezvousAffinityFunction());
 
+                cfg.setEvictionPolicyFactory(new FifoEvictionPolicyFactory<>());
                 cfg.setEvictionPolicy(new FifoEvictionPolicy());
                 cfg.setOnheapCacheEnabled(true);
 
@@ -643,6 +694,7 @@ public class GridCacheConfigurationConsistencySelfTest extends GridCommonAbstrac
             @Override public Void apply(CacheConfiguration cfg) {
                 cfg.setAffinity(new RendezvousAffinityFunction());
 
+                cfg.setEvictionPolicyFactory(new FifoEvictionPolicyFactory<>());
                 cfg.setEvictionPolicy(new LruEvictionPolicy());
                 cfg.setOnheapCacheEnabled(true);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/6579e69f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearEvictionEventSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearEvictionEventSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearEvictionEventSelfTest.java
index 7088ad7..0d36a5a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearEvictionEventSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearEvictionEventSelfTest.java
@@ -36,9 +36,4 @@ public class GridCacheNearEvictionEventSelfTest extends GridCacheEvictionEventAb
     @Override protected CacheAtomicityMode atomicityMode() {
         return TRANSACTIONAL;
     }
-
-    /** {@inheritDoc} */
-    @Override public void testEvictionEvent() throws Exception {
-        super.testEvictionEvent();
-    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/6579e69f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/EvictionPolicyFactoryAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/EvictionPolicyFactoryAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/EvictionPolicyFactoryAbstractTest.java
new file mode 100644
index 0000000..0aa2d7f
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/EvictionPolicyFactoryAbstractTest.java
@@ -0,0 +1,1073 @@
+/*
+ * 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 java.lang.reflect.InvocationTargetException;
+import java.util.Collection;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.Callable;
+import java.util.concurrent.atomic.AtomicInteger;
+import javax.cache.Cache;
+import javax.cache.configuration.Factory;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.eviction.EvictableEntry;
+import org.apache.ignite.cache.eviction.EvictionFilter;
+import org.apache.ignite.cache.eviction.EvictionPolicy;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.NearCacheConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.processors.cache.CacheEvictionManager;
+import org.apache.ignite.internal.processors.cache.GridCacheEvictionManager;
+import org.apache.ignite.internal.processors.cache.distributed.dht.colocated.GridDhtColocatedCache;
+import org.apache.ignite.internal.util.typedef.C2;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.S;
+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;
+import org.apache.ignite.transactions.Transaction;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheMode.LOCAL;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.cache.CacheMode.REPLICATED;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_ASYNC;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
+import static org.apache.ignite.events.EventType.EVT_JOB_MAPPED;
+import static org.apache.ignite.events.EventType.EVT_TASK_FAILED;
+import static org.apache.ignite.events.EventType.EVT_TASK_FINISHED;
+import static org.apache.ignite.internal.processors.cache.eviction.EvictionPolicyFactoryAbstractTest.EvictionPolicyProxy.proxy;
+import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
+import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
+
+/**
+ * Base class for eviction tests.
+ */
+public abstract class EvictionPolicyFactoryAbstractTest<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;
+
+    /** Policy max. */
+    protected int plcMax = 10;
+
+    /** Policy batch size. */
+    protected int plcBatchSize = 1;
+
+    /** Policy max memory size. */
+    protected long plcMaxMemSize = 0;
+
+    protected Factory<T> policyFactory;
+
+    /** Near policy max. */
+    protected int nearMax = 3;
+
+    /** Synchronous commit. */
+    protected boolean syncCommit;
+
+    /** */
+    protected int gridCnt = 2;
+
+    /** */
+    protected EvictionFilter<?, ?> filter;
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        policyFactory = null;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        assert policyFactory != null;
+
+        IgniteConfiguration c = super.getConfiguration(igniteInstanceName);
+
+        CacheConfiguration cc = defaultCacheConfiguration();
+
+        cc.setCacheMode(mode);
+        cc.setOnheapCacheEnabled(true);
+        cc.setEvictionPolicyFactory(policyFactory);
+        cc.setWriteSynchronizationMode(syncCommit ? FULL_SYNC : FULL_ASYNC);
+        cc.setAtomicityMode(TRANSACTIONAL);
+
+        if (nearEnabled) {
+            NearCacheConfiguration nearCfg = new NearCacheConfiguration();
+
+            nearCfg.setNearEvictionPolicyFactory(createNearPolicyFactory(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 {
+        policyFactory = createPolicyFactory();
+
+        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 {
+        policyFactory = createPolicyFactory();
+
+        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 {
+        policyFactory = createPolicyFactory();
+
+        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.getCurrentMemorySize() <= (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;
+
+        policyFactory = createPolicyFactory();
+
+        try {
+            Ignite ignite = startGrid();
+
+            IgniteCache<Object, Object> cache = ignite.cache(DEFAULT_CACHE_NAME);
+
+            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.getCurrentMemorySize());
+    }
+
+    /**
+     * @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. */
+    protected T policy() {
+        CacheEvictionManager evictMgr = grid().cachex(DEFAULT_CACHE_NAME).context().evicts();
+
+        assert evictMgr instanceof GridCacheEvictionManager : evictMgr;
+
+        return (T)((GridCacheEvictionManager)evictMgr).getEvictionPolicy();
+    }
+
+    /**
+     * @param i Grid index.
+     * @return Policy.
+     */
+    @SuppressWarnings({"unchecked"})
+    protected T policy(int i) {
+        CacheEvictionManager evictMgr = grid(i).cachex(DEFAULT_CACHE_NAME).context().evicts();
+
+        assert evictMgr instanceof GridCacheEvictionManager : evictMgr;
+
+        return (T)((GridCacheEvictionManager)evictMgr).getEvictionPolicy();
+    }
+
+    /**
+     * @param i Grid index.
+     * @return Policy.
+     */
+    @SuppressWarnings({"unchecked"})
+    protected T nearPolicy(int i) {
+        CacheEvictionManager evictMgr = grid(i).cachex(DEFAULT_CACHE_NAME).context().near().context().evicts();
+
+        assert evictMgr instanceof GridCacheEvictionManager : evictMgr;
+
+        return (T)((GridCacheEvictionManager)evictMgr).getEvictionPolicy();
+    }
+    /**
+     * @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;
+        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;
+
+        gridCnt = 2;
+
+        checkPartitionedMultiThreaded();
+    }
+
+    /** @throws Exception If failed. */
+    public void testPartitionedNearEnabledMultiThreaded() throws Exception {
+        mode = PARTITIONED;
+        nearEnabled = true;
+        plcMax = 10;
+
+        gridCnt = 2;
+
+        checkPartitionedMultiThreaded();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    protected void checkPartitioned() throws Exception {
+        int endSize = nearEnabled ? 0 : plcMax;
+
+        int endPlcSize = nearEnabled ? 0 : plcMax;
+
+        policyFactory = createPolicyFactory();
+
+        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(DEFAULT_CACHE_NAME);
+
+                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.getCurrentMemorySize() <= memSize);
+                    }
+                }
+
+                if (plcMax > 0) {
+                    for (int i = 0; i < gridCnt; i++) {
+                        int actual = colocated(i).map().internalSize();
+
+                        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 {
+        policyFactory = createPolicyFactory();
+
+        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(DEFAULT_CACHE_NAME);
+
+                        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();
+        }
+    }
+
+    /**
+     * @return Policy.
+     *
+     * @deprecated replace with getPolicyFactory();
+     */
+    @Deprecated
+    protected T createPolicy() {
+        return null;
+    };
+
+    /**
+     * @return Policy.
+     */
+    protected abstract Factory<T> createPolicyFactory();
+
+    /**
+     * @param nearMax Near max.
+     * @return Policy.
+     */
+    protected abstract Factory<T> createNearPolicyFactory(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)).getCurrentMemorySize());
+            }
+            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.
+         * @return Policy proxy.
+         */
+        public static EvictionPolicyProxy proxy(EvictionPolicy plc) {
+            return new EvictionPolicyProxy(plc);
+        }
+
+        /**
+         * @return Get current size.
+         */
+        int getCurrentSize() {
+            try {
+                return (Integer)plc.getClass().getDeclaredMethod("getCurrentSize").invoke(plc);
+            }
+            catch (NoSuchMethodException | InvocationTargetException | IllegalAccessException e) {
+                throw new RuntimeException(e);
+            }
+        }
+
+        /**
+         * @return Current memory size.
+         */
+        long getCurrentMemorySize() {
+            try {
+                return (Long)plc.getClass().getMethod("getCurrentMemorySize").invoke(plc);
+            }
+            catch (NoSuchMethodException | InvocationTargetException | IllegalAccessException e) {
+                throw new RuntimeException(e);
+            }
+        }
+
+        /**
+         * @return 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()
+                    .getMethod("onEntryAccessed", boolean.class, EvictableEntry.class)
+                    .invoke(plc, rmv, entry);
+            }
+            catch (NoSuchMethodException | InvocationTargetException | IllegalAccessException e) {
+                throw new RuntimeException(e);
+            }
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/6579e69f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/fifo/FifoEvictionPolicyFactorySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/fifo/FifoEvictionPolicyFactorySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/fifo/FifoEvictionPolicyFactorySelfTest.java
new file mode 100644
index 0000000..472bf41
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/fifo/FifoEvictionPolicyFactorySelfTest.java
@@ -0,0 +1,261 @@
+/*
+ * 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 javax.cache.configuration.Factory;
+import org.apache.ignite.cache.eviction.fifo.FifoEvictionPolicy;
+import org.apache.ignite.cache.eviction.fifo.FifoEvictionPolicyFactory;
+import org.apache.ignite.internal.processors.cache.eviction.EvictionPolicyFactoryAbstractTest;
+
+/**
+ * FIFO eviction policy tests.
+ */
+public class FifoEvictionPolicyFactorySelfTest extends EvictionPolicyFactoryAbstractTest<FifoEvictionPolicy<String, String>> {
+       /** {@inheritDoc} */
+    @Override protected Factory<FifoEvictionPolicy<String, String>> createPolicyFactory() {
+        return new FifoEvictionPolicyFactory<>(plcMax, plcBatchSize, plcMaxMemSize);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected Factory<FifoEvictionPolicy<String, String>> createNearPolicyFactory(int nearMax) {
+        FifoEvictionPolicyFactory<String, String> plc = new FifoEvictionPolicyFactory<>();
+
+        plc.setMaxSize(nearMax);
+        plc.setBatchSize(plcBatchSize);
+
+        return plc;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void doTestPolicy() throws Exception {
+        policyFactory = createPolicyFactory();
+
+        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 {
+        policyFactory = createPolicyFactory();
+
+        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();
+        }
+    }
+}
\ No newline at end of file