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 2015/09/24 03:12:49 UTC

[2/3] ignite git commit: IGNITE-1016 - Exclude neighbors flag for fair affinity. Fixes #80.

http://git-wip-us.apache.org/repos/asf/ignite/blob/8f5a4c9d/modules/core/src/test/java/org/apache/ignite/cache/affinity/IgniteClientNodeAffinityTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/affinity/IgniteClientNodeAffinityTest.java b/modules/core/src/test/java/org/apache/ignite/cache/affinity/IgniteClientNodeAffinityTest.java
deleted file mode 100644
index 888904b..0000000
--- a/modules/core/src/test/java/org/apache/ignite/cache/affinity/IgniteClientNodeAffinityTest.java
+++ /dev/null
@@ -1,194 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.cache.affinity;
-
-import java.util.Collection;
-import org.apache.ignite.Ignite;
-import org.apache.ignite.IgniteCache;
-import org.apache.ignite.cache.affinity.fair.FairAffinityFunction;
-import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
-import org.apache.ignite.cluster.ClusterNode;
-import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.configuration.NearCacheConfiguration;
-import org.apache.ignite.internal.IgniteNodeAttributes;
-import org.apache.ignite.lang.IgnitePredicate;
-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 static org.apache.ignite.cache.CacheMode.REPLICATED;
-
-/**
- *
- */
-public class IgniteClientNodeAffinityTest extends GridCommonAbstractTest {
-    /** */
-    protected static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
-
-    /** */
-    private static final int NODE_CNT = 4;
-
-    /** */
-    private static final String CACHE1 = "cache1";
-
-    /** */
-    private static final String CACHE2 = "cache2";
-
-    /** */
-    private static final String CACHE3 = "cache3";
-
-    /** */
-    private static final String CACHE4 = "cache4";
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(gridName);
-
-        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
-
-        if (gridName.equals(getTestGridName(NODE_CNT - 1)))
-            cfg.setClientMode(true);
-
-        CacheConfiguration ccfg1 = new CacheConfiguration();
-
-        ccfg1.setBackups(1);
-        ccfg1.setName(CACHE1);
-        ccfg1.setAffinity(new RendezvousAffinityFunction());
-        ccfg1.setNodeFilter(new TestNodesFilter());
-
-        CacheConfiguration ccfg2 = new CacheConfiguration();
-
-        ccfg2.setBackups(1);
-        ccfg2.setName(CACHE2);
-        ccfg2.setAffinity(new RendezvousAffinityFunction());
-
-        CacheConfiguration ccfg3 = new CacheConfiguration();
-
-        ccfg3.setBackups(1);
-        ccfg3.setName(CACHE3);
-        ccfg3.setAffinity(new FairAffinityFunction());
-        ccfg3.setNodeFilter(new TestNodesFilter());
-
-        CacheConfiguration ccfg4 = new CacheConfiguration();
-
-        ccfg4.setCacheMode(REPLICATED);
-        ccfg4.setName(CACHE4);
-        ccfg4.setNodeFilter(new TestNodesFilter());
-
-        cfg.setCacheConfiguration(ccfg1, ccfg2, ccfg3, ccfg4);
-
-        return cfg;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        super.beforeTestsStarted();
-
-        startGrids(NODE_CNT);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTestsStopped() throws Exception {
-        stopAllGrids();
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testClientNodeNotInAffinity() throws Exception {
-        checkCache(CACHE1, 2);
-
-        checkCache(CACHE2, 2);
-
-        checkCache(CACHE3, 2);
-
-        checkCache(CACHE4, 3);
-
-        Ignite client = ignite(NODE_CNT - 1);
-
-        CacheConfiguration ccfg = new CacheConfiguration();
-
-        ccfg.setBackups(0);
-
-        ccfg.setNodeFilter(new TestNodesFilter());
-
-        IgniteCache<Integer, Integer> cache = client.createCache(ccfg);
-
-        try {
-            checkCache(null, 1);
-        }
-        finally {
-            cache.destroy();
-        }
-
-        cache = client.createCache(ccfg, new NearCacheConfiguration());
-
-        try {
-            checkCache(null, 1);
-        }
-        finally {
-            cache.destroy();
-        }
-    }
-
-    /**
-     * @param cacheName Cache name.
-     * @param expNodes Expected number of nodes per partition.
-     */
-    private void checkCache(String cacheName, int expNodes) {
-        log.info("Test cache: " + cacheName);
-
-        Ignite client = ignite(NODE_CNT - 1);
-
-        assertTrue(client.configuration().isClientMode());
-
-        ClusterNode clientNode = client.cluster().localNode();
-
-        for (int i = 0; i < NODE_CNT; i++) {
-            Ignite ignite = ignite(i);
-
-            Affinity<Integer> aff = ignite.affinity(cacheName);
-
-            for (int part = 0; part < aff.partitions(); part++) {
-                Collection<ClusterNode> nodes = aff.mapPartitionToPrimaryAndBackups(part);
-
-                assertEquals(expNodes, nodes.size());
-
-                assertFalse(nodes.contains(clientNode));
-            }
-        }
-    }
-
-    /**
-     *
-     */
-    private static class TestNodesFilter implements IgnitePredicate<ClusterNode> {
-        /** {@inheritDoc} */
-        @Override public boolean apply(ClusterNode clusterNode) {
-            Boolean attr = clusterNode.attribute(IgniteNodeAttributes.ATTR_CLIENT_MODE);
-
-            assertNotNull(attr);
-
-            assertFalse(attr);
-
-            return true;
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/8f5a4c9d/modules/core/src/test/java/org/apache/ignite/cache/affinity/fair/FairAffinityDynamicCacheSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/affinity/fair/FairAffinityDynamicCacheSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cache/affinity/fair/FairAffinityDynamicCacheSelfTest.java
new file mode 100644
index 0000000..0b32320
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/cache/affinity/fair/FairAffinityDynamicCacheSelfTest.java
@@ -0,0 +1,97 @@
+/*
+ * 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.affinity.fair;
+
+import java.util.concurrent.Callable;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteInternalFuture;
+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.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+/**
+ *
+ */
+public class FairAffinityDynamicCacheSelfTest extends GridCommonAbstractTest {
+    /** */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    public FairAffinityDynamicCacheSelfTest(){
+        super(false);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        TcpDiscoverySpi disco = new TcpDiscoverySpi();
+
+        disco.setIpFinder(IP_FINDER);
+
+        cfg.getTransactionConfiguration().setTxSerializableEnabled(true);
+
+        cfg.setDiscoverySpi(disco);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        startGridsMultiThreaded(3);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testStartStopCache() throws Exception {
+        CacheConfiguration<Integer, Integer> cacheCfg = new CacheConfiguration<>();
+
+        cacheCfg.setCacheMode(CacheMode.PARTITIONED);
+        cacheCfg.setAtomicityMode(CacheAtomicityMode.ATOMIC);
+        cacheCfg.setBackups(1);
+        cacheCfg.setName("test");
+        cacheCfg.setAffinity(new FairAffinityFunction());
+
+        final IgniteCache<Integer, Integer> cache = ignite(0).createCache(cacheCfg);
+
+        for (int i = 0; i < 10_000; i++)
+            cache.put(i, i);
+
+        IgniteInternalFuture<Object> destFut = GridTestUtils.runAsync(new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                ignite(0).destroyCache(cache.getName());
+
+                return null;
+            }
+        });
+
+        destFut.get(2000L);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/8f5a4c9d/modules/core/src/test/java/org/apache/ignite/cache/affinity/fair/FairAffinityFunctionBackupFilterSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/affinity/fair/FairAffinityFunctionBackupFilterSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cache/affinity/fair/FairAffinityFunctionBackupFilterSelfTest.java
new file mode 100644
index 0000000..eedc9e4
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/cache/affinity/fair/FairAffinityFunctionBackupFilterSelfTest.java
@@ -0,0 +1,35 @@
+/*
+ * 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.affinity.fair;
+
+import org.apache.ignite.cache.affinity.AffinityFunction;
+import org.apache.ignite.cache.affinity.AffinityFunctionBackupFilterAbstractSelfTest;
+
+/**
+ * Tests backup filter for {@link FairAffinityFunction}.
+ */
+public class FairAffinityFunctionBackupFilterSelfTest extends AffinityFunctionBackupFilterAbstractSelfTest {
+    /** {@inheritDoc} */
+    @Override protected AffinityFunction affinityFunction() {
+        FairAffinityFunction aff = new FairAffinityFunction(false);
+
+        aff.setBackupFilter(backupFilter);
+
+        return aff;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8f5a4c9d/modules/core/src/test/java/org/apache/ignite/cache/affinity/fair/FairAffinityFunctionExcludeNeighborsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/affinity/fair/FairAffinityFunctionExcludeNeighborsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cache/affinity/fair/FairAffinityFunctionExcludeNeighborsSelfTest.java
new file mode 100644
index 0000000..4182cd3
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/cache/affinity/fair/FairAffinityFunctionExcludeNeighborsSelfTest.java
@@ -0,0 +1,31 @@
+/*
+ * 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.affinity.fair;
+
+import org.apache.ignite.cache.affinity.AffinityFunction;
+import org.apache.ignite.cache.affinity.AffinityFunctionExcludeNeighborsAbstractSelfTest;
+
+/**
+ * Tests exclude neighbors flag for {@link FairAffinityFunction}.
+ */
+public class FairAffinityFunctionExcludeNeighborsSelfTest extends AffinityFunctionExcludeNeighborsAbstractSelfTest {
+    /** {@inheritDoc} */
+    @Override protected AffinityFunction affinityFunction() {
+        return new FairAffinityFunction(true);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8f5a4c9d/modules/core/src/test/java/org/apache/ignite/cache/affinity/fair/FairAffinityFunctionNodesSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/affinity/fair/FairAffinityFunctionNodesSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cache/affinity/fair/FairAffinityFunctionNodesSelfTest.java
new file mode 100644
index 0000000..7420a0d
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/cache/affinity/fair/FairAffinityFunctionNodesSelfTest.java
@@ -0,0 +1,245 @@
+/*
+ * 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.affinity.fair;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.TreeSet;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.IgniteKernal;
+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;
+
+/**
+ * Tests partition fair affinity in real grid.
+ */
+public class FairAffinityFunctionNodesSelfTest extends GridCommonAbstractTest {
+    /** IP finder. */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** Number of backups. */
+    private int backups;
+
+    /** Number of partitions. */
+    private int parts = 512;
+
+    /** Add nodes test. */
+    private static final boolean[] ADD_ONLY = new boolean[] {true, true, true, true, true, true};
+
+    /** Add nodes test. */
+    private static final boolean[] ADD_REMOVE = new boolean[]
+        {
+            true,  true,  true,  true,  true, true,
+            false, false, false, false, false
+        };
+
+    /** */
+    private static final boolean[] MIXED1 = new boolean[]
+        {
+            // 1     2     3      2     3     4      3     4     5      4      3      2
+            true, true, true, false, true, true, false, true, true, false, false, false
+        };
+
+    /** */
+    private static final boolean[] MIXED2 = new boolean[]
+        {
+            // 1     2     3      2      1     2      1     2     3      2      1     2
+            true, true, true, false, false, true, false, true, true, false, false, true
+        };
+
+    /** */
+    private static final boolean[] MIXED3 = new boolean[]
+        {
+            // 1     2     3     4     5     6      5     6     7     8     9      8      7     8     9
+            true, true, true, true, true, true, false, true, true, true, true, false, false, true, true,
+            //  8      7      6
+            false, false, false
+        };
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        CacheConfiguration ccfg = cacheConfiguration();
+
+        cfg.setCacheConfiguration(ccfg);
+
+        TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
+
+        discoSpi.setIpFinder(IP_FINDER);
+
+        cfg.setDiscoverySpi(discoSpi);
+
+        return cfg;
+    }
+
+    /**
+     * @return Cache configuration.
+     */
+    private CacheConfiguration cacheConfiguration() {
+        CacheConfiguration cfg = new CacheConfiguration();
+
+        cfg.setBackups(backups);
+
+        cfg.setCacheMode(CacheMode.PARTITIONED);
+
+        cfg.setNearConfiguration(null);
+
+        cfg.setAffinity(new FairAffinityFunction(parts));
+
+        return cfg;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAdd() throws Exception {
+        checkSequence(ADD_ONLY);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAddRemove() throws Exception {
+        checkSequence(ADD_REMOVE);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMixed1() throws Exception {
+        checkSequence(MIXED1);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMixed2() throws Exception {
+        checkSequence(MIXED2);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMixed3() throws Exception {
+        checkSequence(MIXED3);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void checkSequence(boolean[] seq) throws Exception {
+        for (int b = 0; b < 3; b++) {
+            backups = b;
+
+            info(">>>>>>>>>>>>>>>> Checking backups: " + backups);
+
+            checkSequence0(seq);
+
+            info(">>>>>>>>>>>>>>>> Finished check: " + backups);
+        }
+    }
+
+    /**
+     * @param seq Start/stop sequence.
+     * @throws Exception If failed.
+     */
+    private void checkSequence0(boolean[] seq) throws Exception {
+        try {
+            startGrid(0);
+
+            TreeSet<Integer> started = new TreeSet<>();
+
+            started.add(0);
+
+            int topVer = 1;
+
+            for (boolean start : seq) {
+                if (start) {
+                    int nextIdx = nextIndex(started);
+
+                    startGrid(nextIdx);
+
+                    started.add(nextIdx);
+                }
+                else {
+                    int idx = started.last();
+
+                    stopGrid(idx);
+
+                    started.remove(idx);
+                }
+
+                topVer++;
+
+                info("Grid 0: " + grid(0).localNode().id());
+
+                ((IgniteKernal)grid(0)).internalCache().context().affinity().affinityReadyFuture(topVer).get();
+
+                for (int i : started) {
+                    if (i != 0) {
+                        IgniteEx grid = grid(i);
+
+                        ((IgniteKernal)grid).internalCache().context().affinity().affinityReadyFuture(topVer).get();
+
+                        info("Grid " + i + ": " + grid.localNode().id());
+
+                        for (int part = 0; part < parts; part++) {
+                            List<ClusterNode> firstNodes = (List<ClusterNode>)grid(0).affinity(null)
+                                .mapPartitionToPrimaryAndBackups(part);
+
+                            List<ClusterNode> secondNodes = (List<ClusterNode>)grid.affinity(null)
+                                .mapPartitionToPrimaryAndBackups(part);
+
+                            assertEquals(firstNodes.size(), secondNodes.size());
+
+                            for (int n = 0; n < firstNodes.size(); n++)
+                                assertEquals(firstNodes.get(n), secondNodes.get(n));
+                        }
+                    }
+                }
+            }
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
+
+    /**
+     * First positive integer that is not present in started set.
+     *
+     * @param started Already started indices.
+     * @return First positive integer that is not present in started set.
+     */
+    private int nextIndex(Collection<Integer> started) {
+        assert started.contains(0);
+
+        for (int i = 1; i < 10000; i++) {
+            if (!started.contains(i))
+                return i;
+        }
+
+        throw new IllegalStateException();
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/8f5a4c9d/modules/core/src/test/java/org/apache/ignite/cache/affinity/fair/FairAffinityFunctionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/affinity/fair/FairAffinityFunctionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cache/affinity/fair/FairAffinityFunctionSelfTest.java
new file mode 100644
index 0000000..a79c9fc
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/cache/affinity/fair/FairAffinityFunctionSelfTest.java
@@ -0,0 +1,31 @@
+/*
+ * 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.affinity.fair;
+
+import org.apache.ignite.cache.affinity.AbstractAffinityFunctionSelfTest;
+import org.apache.ignite.cache.affinity.AffinityFunction;
+
+/**
+ * Tests for {@link FairAffinityFunction}.
+ */
+public class FairAffinityFunctionSelfTest extends AbstractAffinityFunctionSelfTest {
+    /** {@inheritDoc} */
+    @Override protected AffinityFunction affinityFunction() {
+        return new FairAffinityFunction();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8f5a4c9d/modules/core/src/test/java/org/apache/ignite/cache/affinity/fair/GridFairAffinityFunctionNodesSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/affinity/fair/GridFairAffinityFunctionNodesSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cache/affinity/fair/GridFairAffinityFunctionNodesSelfTest.java
deleted file mode 100644
index cf57b66..0000000
--- a/modules/core/src/test/java/org/apache/ignite/cache/affinity/fair/GridFairAffinityFunctionNodesSelfTest.java
+++ /dev/null
@@ -1,245 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.cache.affinity.fair;
-
-import java.util.Collection;
-import java.util.List;
-import java.util.TreeSet;
-import org.apache.ignite.cache.CacheMode;
-import org.apache.ignite.cluster.ClusterNode;
-import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.IgniteEx;
-import org.apache.ignite.internal.IgniteKernal;
-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;
-
-/**
- * Tests partition fair affinity in real grid.
- */
-public class GridFairAffinityFunctionNodesSelfTest extends GridCommonAbstractTest {
-    /** IP finder. */
-    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
-
-    /** Number of backups. */
-    private int backups;
-
-    /** Number of partitions. */
-    private int parts = 512;
-
-    /** Add nodes test. */
-    private static final boolean[] ADD_ONLY = new boolean[] {true, true, true, true, true, true};
-
-    /** Add nodes test. */
-    private static final boolean[] ADD_REMOVE = new boolean[]
-        {
-            true,  true,  true,  true,  true, true,
-            false, false, false, false, false
-        };
-
-    /** */
-    private static final boolean[] MIXED1 = new boolean[]
-        {
-            // 1     2     3      2     3     4      3     4     5      4      3      2
-            true, true, true, false, true, true, false, true, true, false, false, false
-        };
-
-    /** */
-    private static final boolean[] MIXED2 = new boolean[]
-        {
-            // 1     2     3      2      1     2      1     2     3      2      1     2
-            true, true, true, false, false, true, false, true, true, false, false, true
-        };
-
-    /** */
-    private static final boolean[] MIXED3 = new boolean[]
-        {
-            // 1     2     3     4     5     6      5     6     7     8     9      8      7     8     9
-            true, true, true, true, true, true, false, true, true, true, true, false, false, true, true,
-            //  8      7      6
-            false, false, false
-        };
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(gridName);
-
-        CacheConfiguration ccfg = cacheConfiguration();
-
-        cfg.setCacheConfiguration(ccfg);
-
-        TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
-
-        discoSpi.setIpFinder(IP_FINDER);
-
-        cfg.setDiscoverySpi(discoSpi);
-
-        return cfg;
-    }
-
-    /**
-     * @return Cache configuration.
-     */
-    private CacheConfiguration cacheConfiguration() {
-        CacheConfiguration cfg = new CacheConfiguration();
-
-        cfg.setBackups(backups);
-
-        cfg.setCacheMode(CacheMode.PARTITIONED);
-
-        cfg.setNearConfiguration(null);
-
-        cfg.setAffinity(new FairAffinityFunction(parts));
-
-        return cfg;
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testAdd() throws Exception {
-        checkSequence(ADD_ONLY);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testAddRemove() throws Exception {
-        checkSequence(ADD_REMOVE);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testMixed1() throws Exception {
-        checkSequence(MIXED1);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testMixed2() throws Exception {
-        checkSequence(MIXED2);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testMixed3() throws Exception {
-        checkSequence(MIXED3);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    private void checkSequence(boolean[] seq) throws Exception {
-        for (int b = 0; b < 3; b++) {
-            backups = b;
-
-            info(">>>>>>>>>>>>>>>> Checking backups: " + backups);
-
-            checkSequence0(seq);
-
-            info(">>>>>>>>>>>>>>>> Finished check: " + backups);
-        }
-    }
-
-    /**
-     * @param seq Start/stop sequence.
-     * @throws Exception If failed.
-     */
-    private void checkSequence0(boolean[] seq) throws Exception {
-        try {
-            startGrid(0);
-
-            TreeSet<Integer> started = new TreeSet<>();
-
-            started.add(0);
-
-            int topVer = 1;
-
-            for (boolean start : seq) {
-                if (start) {
-                    int nextIdx = nextIndex(started);
-
-                    startGrid(nextIdx);
-
-                    started.add(nextIdx);
-                }
-                else {
-                    int idx = started.last();
-
-                    stopGrid(idx);
-
-                    started.remove(idx);
-                }
-
-                topVer++;
-
-                info("Grid 0: " + grid(0).localNode().id());
-
-                ((IgniteKernal)grid(0)).internalCache().context().affinity().affinityReadyFuture(topVer).get();
-
-                for (int i : started) {
-                    if (i != 0) {
-                        IgniteEx grid = grid(i);
-
-                        ((IgniteKernal)grid).internalCache().context().affinity().affinityReadyFuture(topVer).get();
-
-                        info("Grid " + i + ": " + grid.localNode().id());
-
-                        for (int part = 0; part < parts; part++) {
-                            List<ClusterNode> firstNodes = (List<ClusterNode>)grid(0).affinity(null)
-                                .mapPartitionToPrimaryAndBackups(part);
-
-                            List<ClusterNode> secondNodes = (List<ClusterNode>)grid.affinity(null)
-                                .mapPartitionToPrimaryAndBackups(part);
-
-                            assertEquals(firstNodes.size(), secondNodes.size());
-
-                            for (int n = 0; n < firstNodes.size(); n++)
-                                assertEquals(firstNodes.get(n), secondNodes.get(n));
-                        }
-                    }
-                }
-            }
-        }
-        finally {
-            stopAllGrids();
-        }
-    }
-
-    /**
-     * First positive integer that is not present in started set.
-     *
-     * @param started Already started indices.
-     * @return First positive integer that is not present in started set.
-     */
-    private int nextIndex(Collection<Integer> started) {
-        assert started.contains(0);
-
-        for (int i = 1; i < 10000; i++) {
-            if (!started.contains(i))
-                return i;
-        }
-
-        throw new IllegalStateException();
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/8f5a4c9d/modules/core/src/test/java/org/apache/ignite/cache/affinity/fair/GridFairAffinityFunctionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/affinity/fair/GridFairAffinityFunctionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cache/affinity/fair/GridFairAffinityFunctionSelfTest.java
deleted file mode 100644
index e746b42..0000000
--- a/modules/core/src/test/java/org/apache/ignite/cache/affinity/fair/GridFairAffinityFunctionSelfTest.java
+++ /dev/null
@@ -1,270 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.cache.affinity.fair;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Random;
-import java.util.UUID;
-import org.apache.ignite.cache.affinity.AffinityFunction;
-import org.apache.ignite.cluster.ClusterNode;
-import org.apache.ignite.events.DiscoveryEvent;
-import org.apache.ignite.events.EventType;
-import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
-import org.apache.ignite.internal.processors.affinity.GridAffinityFunctionContextImpl;
-import org.apache.ignite.testframework.GridTestNode;
-import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-
-/**
- *
- */
-public class GridFairAffinityFunctionSelfTest extends GridCommonAbstractTest {
-    /**
-     * @throws Exception If failed.
-     */
-    public void testNodeRemovedNoBackups() throws Exception {
-        checkNodeRemoved(0);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testNodeRemovedOneBackup() throws Exception {
-        checkNodeRemoved(1);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testNodeRemovedTwoBackups() throws Exception {
-        checkNodeRemoved(2);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testNodeRemovedThreeBackups() throws Exception {
-        checkNodeRemoved(3);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testRandomReassignmentNoBackups() throws Exception {
-        checkRandomReassignment(0);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testRandomReassignmentOneBackup() throws Exception {
-        checkRandomReassignment(1);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testRandomReassignmentTwoBackups() throws Exception {
-        checkRandomReassignment(2);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testRandomReassignmentThreeBackups() throws Exception {
-        checkRandomReassignment(3);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    private void checkNodeRemoved(int backups) throws Exception {
-        int parts = 256;
-
-        AffinityFunction aff = new FairAffinityFunction(parts);
-
-        int nodesCnt = 50;
-
-        List<ClusterNode> nodes = new ArrayList<>(nodesCnt);
-
-        List<List<ClusterNode>> prev = null;
-
-        for (int i = 0; i < nodesCnt; i++) {
-            info("======================================");
-            info("Assigning partitions: " + i);
-            info("======================================");
-
-            ClusterNode node = new GridTestNode(UUID.randomUUID());
-
-            nodes.add(node);
-
-            DiscoveryEvent discoEvt = new DiscoveryEvent(node, "", EventType.EVT_NODE_JOINED,
-                node);
-
-            List<List<ClusterNode>> assignment = aff.assignPartitions(
-                new GridAffinityFunctionContextImpl(nodes, prev, discoEvt, new AffinityTopologyVersion(i),
-                    backups));
-
-            info("Assigned.");
-
-            verifyAssignment(assignment, backups, parts, nodes.size());
-
-            prev = assignment;
-        }
-
-        info("======================================");
-        info("Will remove nodes.");
-        info("======================================");
-
-        for (int i = 0; i < nodesCnt - 1; i++) {
-            info("======================================");
-            info("Assigning partitions: " + i);
-            info("======================================");
-
-            ClusterNode rmv = nodes.remove(nodes.size() - 1);
-
-            DiscoveryEvent discoEvt = new DiscoveryEvent(rmv, "", EventType.EVT_NODE_LEFT, rmv);
-
-            List<List<ClusterNode>> assignment = aff.assignPartitions(
-                new GridAffinityFunctionContextImpl(nodes, prev, discoEvt, new AffinityTopologyVersion(i),
-                    backups));
-
-            info("Assigned.");
-
-            verifyAssignment(assignment, backups, parts, nodes.size());
-
-            prev = assignment;
-        }
-    }
-
-    @SuppressWarnings("IfMayBeConditional")
-    private void checkRandomReassignment(int backups) {
-        int parts = 256;
-
-        AffinityFunction aff = new FairAffinityFunction(parts);
-
-        Random rnd = new Random();
-
-        int maxNodes = 50;
-
-        List<ClusterNode> nodes = new ArrayList<>(maxNodes);
-
-        List<List<ClusterNode>> prev = null;
-
-        int state = 0;
-
-        int i = 0;
-
-        while (true) {
-            boolean add;
-
-            if (nodes.size() < 2) {
-                // Returned back to one node?
-                if (state == 1)
-                    return;
-
-                add = true;
-            }
-            else if (nodes.size() == maxNodes) {
-                if (state == 0)
-                    state = 1;
-
-                add = false;
-            }
-            else {
-                // Nodes size in [2, maxNodes - 1].
-                if (state == 0)
-                    add = rnd.nextInt(3) != 0; // 66% to add, 33% to remove.
-                else
-                    add = rnd.nextInt(3) == 0; // 33% to add, 66% to remove.
-            }
-
-            DiscoveryEvent discoEvt;
-
-            if (add) {
-                ClusterNode addedNode = new GridTestNode(UUID.randomUUID());
-
-                nodes.add(addedNode);
-
-                discoEvt = new DiscoveryEvent(addedNode, "", EventType.EVT_NODE_JOINED, addedNode);
-            }
-            else {
-                ClusterNode rmvNode = nodes.remove(rnd.nextInt(nodes.size()));
-
-                discoEvt = new DiscoveryEvent(rmvNode, "", EventType.EVT_NODE_LEFT, rmvNode);
-            }
-
-            info("======================================");
-            info("Assigning partitions [iter=" + i + ", discoEvt=" + discoEvt + ", nodesSize=" + nodes.size() + ']');
-            info("======================================");
-
-            List<List<ClusterNode>> assignment = aff.assignPartitions(
-                new GridAffinityFunctionContextImpl(nodes, prev, discoEvt, new AffinityTopologyVersion(i),
-                    backups));
-
-            verifyAssignment(assignment, backups, parts, nodes.size());
-
-            prev = assignment;
-
-            i++;
-        }
-    }
-
-    /**
-     * @param assignment Assignment to verify.
-     */
-    private void verifyAssignment(List<List<ClusterNode>> assignment, int keyBackups, int partsCnt, int topSize) {
-        Map<UUID, Collection<Integer>> mapping = new HashMap<>();
-
-        int ideal = Math.round((float)partsCnt / topSize * Math.min(keyBackups + 1, topSize));
-
-        for (int part = 0; part < assignment.size(); part++) {
-            for (ClusterNode node : assignment.get(part)) {
-                assert node != null;
-
-                Collection<Integer> parts = mapping.get(node.id());
-
-                if (parts == null) {
-                    parts = new HashSet<>();
-
-                    mapping.put(node.id(), parts);
-                }
-
-                assertTrue(parts.add(part));
-            }
-        }
-
-        int max = -1, min = Integer.MAX_VALUE;
-
-        for (Collection<Integer> parts : mapping.values()) {
-            max = Math.max(max, parts.size());
-            min = Math.min(min, parts.size());
-        }
-
-        log().warning("max=" + max + ", min=" + min + ", ideal=" + ideal + ", minDev=" + deviation(min, ideal) + "%, " +
-            "maxDev=" + deviation(max, ideal) + "%");
-    }
-
-    private static int deviation(int val, int ideal) {
-        return Math.round(Math.abs(((float)val - ideal) / ideal * 100));
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/8f5a4c9d/modules/core/src/test/java/org/apache/ignite/cache/affinity/fair/IgniteFairAffinityDynamicCacheSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/affinity/fair/IgniteFairAffinityDynamicCacheSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cache/affinity/fair/IgniteFairAffinityDynamicCacheSelfTest.java
deleted file mode 100644
index eb1c455..0000000
--- a/modules/core/src/test/java/org/apache/ignite/cache/affinity/fair/IgniteFairAffinityDynamicCacheSelfTest.java
+++ /dev/null
@@ -1,97 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.cache.affinity.fair;
-
-import java.util.concurrent.Callable;
-import org.apache.ignite.IgniteCache;
-import org.apache.ignite.cache.CacheAtomicityMode;
-import org.apache.ignite.cache.CacheMode;
-import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.IgniteInternalFuture;
-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.GridTestUtils;
-import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-
-/**
- *
- */
-public class IgniteFairAffinityDynamicCacheSelfTest extends GridCommonAbstractTest {
-    /** */
-    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
-
-    /** */
-    public IgniteFairAffinityDynamicCacheSelfTest(){
-        super(false);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(gridName);
-
-        TcpDiscoverySpi disco = new TcpDiscoverySpi();
-
-        disco.setIpFinder(IP_FINDER);
-
-        cfg.getTransactionConfiguration().setTxSerializableEnabled(true);
-
-        cfg.setDiscoverySpi(disco);
-
-        return cfg;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        startGridsMultiThreaded(3);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTestsStopped() throws Exception {
-        stopAllGrids();
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testStartStopCache() throws Exception {
-        CacheConfiguration<Integer, Integer> cacheCfg = new CacheConfiguration<>();
-
-        cacheCfg.setCacheMode(CacheMode.PARTITIONED);
-        cacheCfg.setAtomicityMode(CacheAtomicityMode.ATOMIC);
-        cacheCfg.setBackups(1);
-        cacheCfg.setName("test");
-        cacheCfg.setAffinity(new FairAffinityFunction());
-
-        final IgniteCache<Integer, Integer> cache = ignite(0).createCache(cacheCfg);
-
-        for (int i = 0; i < 10_000; i++)
-            cache.put(i, i);
-
-        IgniteInternalFuture<Object> destFut = GridTestUtils.runAsync(new Callable<Object>() {
-            @Override public Object call() throws Exception {
-                ignite(0).destroyCache(cache.getName());
-
-                return null;
-            }
-        });
-
-        destFut.get(2000L);
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/8f5a4c9d/modules/core/src/test/java/org/apache/ignite/cache/affinity/rendezvous/RendezvousAffinityFunctionBackupFilterSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/affinity/rendezvous/RendezvousAffinityFunctionBackupFilterSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cache/affinity/rendezvous/RendezvousAffinityFunctionBackupFilterSelfTest.java
new file mode 100644
index 0000000..d5d8b8f
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/cache/affinity/rendezvous/RendezvousAffinityFunctionBackupFilterSelfTest.java
@@ -0,0 +1,35 @@
+/*
+ * 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.affinity.rendezvous;
+
+import org.apache.ignite.cache.affinity.AffinityFunction;
+import org.apache.ignite.cache.affinity.AffinityFunctionBackupFilterAbstractSelfTest;
+
+/**
+ * Partitioned affinity test.
+ */
+public class RendezvousAffinityFunctionBackupFilterSelfTest extends AffinityFunctionBackupFilterAbstractSelfTest {
+    /** {@inheritDoc} */
+    @Override protected AffinityFunction affinityFunction() {
+        RendezvousAffinityFunction aff = new RendezvousAffinityFunction(false);
+
+        aff.setBackupFilter(backupFilter);
+
+        return aff;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8f5a4c9d/modules/core/src/test/java/org/apache/ignite/cache/affinity/rendezvous/RendezvousAffinityFunctionExcludeNeighborsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/affinity/rendezvous/RendezvousAffinityFunctionExcludeNeighborsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cache/affinity/rendezvous/RendezvousAffinityFunctionExcludeNeighborsSelfTest.java
new file mode 100644
index 0000000..ea47c68
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/cache/affinity/rendezvous/RendezvousAffinityFunctionExcludeNeighborsSelfTest.java
@@ -0,0 +1,32 @@
+/*
+ * 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.affinity.rendezvous;
+
+import org.apache.ignite.cache.affinity.AffinityFunction;
+import org.apache.ignite.cache.affinity.AffinityFunctionExcludeNeighborsAbstractSelfTest;
+
+/**
+ * Tests exclude neighbors flag for rendezvous affinity function.
+ */
+public class RendezvousAffinityFunctionExcludeNeighborsSelfTest extends
+    AffinityFunctionExcludeNeighborsAbstractSelfTest {
+    /** {@inheritDoc} */
+    @Override protected AffinityFunction affinityFunction() {
+        return new RendezvousAffinityFunction(true);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/8f5a4c9d/modules/core/src/test/java/org/apache/ignite/cache/affinity/rendezvous/RendezvousAffinityFunctionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/affinity/rendezvous/RendezvousAffinityFunctionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cache/affinity/rendezvous/RendezvousAffinityFunctionSelfTest.java
new file mode 100644
index 0000000..d895315
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/cache/affinity/rendezvous/RendezvousAffinityFunctionSelfTest.java
@@ -0,0 +1,50 @@
+/*
+ * 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.affinity.rendezvous;
+
+import org.apache.ignite.Ignite;
+import org.apache.ignite.cache.affinity.AbstractAffinityFunctionSelfTest;
+import org.apache.ignite.cache.affinity.AffinityFunction;
+import org.apache.ignite.testframework.GridTestUtils;
+
+/**
+ * Tests for {@link RendezvousAffinityFunction}.
+ */
+public class RendezvousAffinityFunctionSelfTest extends AbstractAffinityFunctionSelfTest {
+    /** Ignite. */
+    private static Ignite ignite;
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        ignite = startGrid();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected AffinityFunction affinityFunction() {
+        AffinityFunction aff = new RendezvousAffinityFunction();
+
+        GridTestUtils.setFieldValue(aff, "ignite", ignite);
+
+        return aff;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8f5a4c9d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CrossCacheTxRandomOperationsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CrossCacheTxRandomOperationsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CrossCacheTxRandomOperationsTest.java
index 1495a2b..cedb693 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CrossCacheTxRandomOperationsTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CrossCacheTxRandomOperationsTest.java
@@ -124,7 +124,7 @@ public class CrossCacheTxRandomOperationsTest extends GridCommonAbstractTest {
     /**
      * @throws Exception If failed.
      */
-    public void _testCrossCacheTxOperationsFairAffinity() throws Exception {
+    public void testCrossCacheTxOperationsFairAffinity() throws Exception {
         txOperations(PARTITIONED, FULL_SYNC, true, true);
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/8f5a4c9d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractLocalStoreSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractLocalStoreSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractLocalStoreSelfTest.java
index a8d025c..3e12ebf 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractLocalStoreSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractLocalStoreSelfTest.java
@@ -37,6 +37,7 @@ import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheMemoryMode;
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.cache.CachePeekMode;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
 import org.apache.ignite.cache.store.CacheStore;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
@@ -96,8 +97,12 @@ public abstract class GridCacheAbstractLocalStoreSelfTest extends GridCommonAbst
 
         CacheConfiguration cacheCfg = cache(gridName, null, 0);
 
+        cacheCfg.setAffinity(new RendezvousAffinityFunction());
+
         CacheConfiguration cacheBackupCfg = cache(gridName, BACKUP_CACHE, 2);
 
+        cacheBackupCfg.setAffinity(new RendezvousAffinityFunction());
+
         cfg.setCacheConfiguration(cacheCfg, cacheBackupCfg);
 
         TcpDiscoverySpi spi = new TcpDiscoverySpi();

http://git-wip-us.apache.org/repos/asf/ignite/blob/8f5a4c9d/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 31e34bb..e28e89f 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
@@ -806,23 +806,6 @@ public class GridCacheConfigurationConsistencySelfTest extends GridCommonAbstrac
     public void testAffinityForReplicatedCache() throws Exception {
         cacheEnabled = true;
 
-        aff = new FairAffinityFunction(); // Check cannot use FairAffinityFunction.
-
-        GridTestUtils.assertThrows(log, new Callable<Object>() {
-            @Override public Object call() throws Exception {
-                return startGrid(1);
-            }
-        }, IgniteCheckedException.class, null);
-
-        aff = new RendezvousAffinityFunction(true); // Check cannot set 'excludeNeighbors' flag.
-        backups = Integer.MAX_VALUE;
-
-        GridTestUtils.assertThrows(log, new Callable<Object>() {
-            @Override public Object call() throws Exception {
-                return startGrid(1);
-            }
-        }, IgniteCheckedException.class, null);
-
         aff = new RendezvousAffinityFunction(false, 100);
 
         startGrid(1);

http://git-wip-us.apache.org/repos/asf/ignite/blob/8f5a4c9d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePartitionedAffinityFilterSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePartitionedAffinityFilterSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePartitionedAffinityFilterSelfTest.java
deleted file mode 100644
index cb8abec..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePartitionedAffinityFilterSelfTest.java
+++ /dev/null
@@ -1,143 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache.distributed;
-
-import java.util.Collection;
-import org.apache.ignite.IgniteCache;
-import org.apache.ignite.cache.CacheWriteSynchronizationMode;
-import org.apache.ignite.cache.affinity.AffinityFunction;
-import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
-import org.apache.ignite.cluster.ClusterNode;
-import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.util.typedef.F;
-import org.apache.ignite.lang.IgniteBiPredicate;
-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 static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
-import static org.apache.ignite.cache.CacheMode.PARTITIONED;
-import static org.apache.ignite.cache.CacheRebalanceMode.SYNC;
-
-/**
- * Partitioned affinity test.
- */
-@SuppressWarnings({"PointlessArithmeticExpression"})
-public class GridCachePartitionedAffinityFilterSelfTest extends GridCommonAbstractTest {
-    /** */
-    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
-
-    /** Backup count. */
-    private static final int BACKUPS = 1;
-
-    /** Split attribute name. */
-    private static final String SPLIT_ATTRIBUTE_NAME = "split-attribute";
-
-    /** Split attribute value. */
-    private String splitAttrVal;
-
-    /** Test backup filter. */
-    private static final IgniteBiPredicate<ClusterNode, ClusterNode> backupFilter =
-        new IgniteBiPredicate<ClusterNode, ClusterNode>() {
-            @Override public boolean apply(ClusterNode primary, ClusterNode backup) {
-                assert primary != null : "primary is null";
-                assert backup != null : "backup is null";
-
-                return !F.eq(primary.attribute(SPLIT_ATTRIBUTE_NAME), backup.attribute(SPLIT_ATTRIBUTE_NAME));
-            }
-        };
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        RendezvousAffinityFunction aff = new RendezvousAffinityFunction();
-
-        aff.setBackupFilter(backupFilter);
-
-        CacheConfiguration cacheCfg = defaultCacheConfiguration();
-
-        cacheCfg.setCacheMode(PARTITIONED);
-        cacheCfg.setBackups(BACKUPS);
-        cacheCfg.setAffinity(aff);
-        cacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
-        cacheCfg.setRebalanceMode(SYNC);
-        cacheCfg.setAtomicityMode(TRANSACTIONAL);
-
-        TcpDiscoverySpi spi = new TcpDiscoverySpi();
-
-        spi.setIpFinder(IP_FINDER);
-
-        IgniteConfiguration cfg = super.getConfiguration(gridName);
-
-        cfg.setCacheConfiguration(cacheCfg);
-        cfg.setDiscoverySpi(spi);
-
-        cfg.setUserAttributes(F.asMap(SPLIT_ATTRIBUTE_NAME, splitAttrVal));
-
-        return cfg;
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPartitionDistribution() throws Exception {
-        try {
-            for (int i = 0; i < 3; i++) {
-                splitAttrVal = "A";
-
-                startGrid(2 * i);
-
-                splitAttrVal = "B";
-
-                startGrid(2 * i + 1);
-
-                awaitPartitionMapExchange();
-
-                checkPartitions();
-            }
-        }
-        finally {
-            stopAllGrids();
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    private void checkPartitions() throws Exception {
-        int partCnt = RendezvousAffinityFunction.DFLT_PARTITION_COUNT;
-
-        AffinityFunction aff = cacheConfiguration(grid(0).configuration(), null).getAffinity();
-
-        IgniteCache<Object, Object> cache = grid(0).cache(null);
-
-        for (int i = 0; i < partCnt; i++) {
-            assertEquals(i, aff.partition(i));
-
-            Collection<ClusterNode> nodes = affinity(cache).mapKeyToPrimaryAndBackups(i);
-
-            assertEquals(2, nodes.size());
-
-            ClusterNode primary = F.first(nodes);
-            ClusterNode backup = F.last(nodes);
-
-            assertFalse(F.eq(primary.attribute(SPLIT_ATTRIBUTE_NAME), backup.attribute(SPLIT_ATTRIBUTE_NAME)));
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/8f5a4c9d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadPutGetSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadPutGetSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadPutGetSelfTest.java
index fa04b6b..c12e1ba 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadPutGetSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadPutGetSelfTest.java
@@ -24,6 +24,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.cache.CacheRebalanceMode;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.IgniteInternalFuture;
@@ -76,6 +77,8 @@ public class GridCacheDhtPreloadPutGetSelfTest extends GridCommonAbstractTest {
         cacheCfg.setRebalanceMode(preloadMode);
         cacheCfg.setBackups(backups);
 
+        cacheCfg.setAffinity(new RendezvousAffinityFunction());
+
         TcpDiscoverySpi disco = new TcpDiscoverySpi();
 
         disco.setIpFinder(ipFinder);

http://git-wip-us.apache.org/repos/asf/ignite/blob/8f5a4c9d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAffinityFunctionExcludeNeighborsAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAffinityFunctionExcludeNeighborsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAffinityFunctionExcludeNeighborsAbstractSelfTest.java
deleted file mode 100644
index 5b66174..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAffinityFunctionExcludeNeighborsAbstractSelfTest.java
+++ /dev/null
@@ -1,184 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache.distributed.near;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import org.apache.ignite.Ignite;
-import org.apache.ignite.cache.affinity.Affinity;
-import org.apache.ignite.cache.affinity.AffinityFunction;
-import org.apache.ignite.cluster.ClusterNode;
-import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.IgniteNodeAttributes;
-import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.lang.IgniteProductVersion;
-import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
-import org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoveryNode;
-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 static org.apache.ignite.cache.CacheMode.PARTITIONED;
-import static org.apache.ignite.cache.CacheRebalanceMode.NONE;
-
-/**
- * Partitioned affinity test.
- */
-@SuppressWarnings({"PointlessArithmeticExpression", "FieldCanBeLocal"})
-public abstract class GridCacheAffinityFunctionExcludeNeighborsAbstractSelfTest extends GridCommonAbstractTest {
-    /** Number of backups. */
-    private int backups = 2;
-
-    /** */
-    private int gridInstanceNum;
-
-    /** */
-    private TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(final String gridName) throws Exception {
-        IgniteConfiguration c = super.getConfiguration(gridName);
-
-        // Override node attributes in discovery spi.
-        TcpDiscoverySpi spi = new TcpDiscoverySpi() {
-            @Override public void setNodeAttributes(Map<String, Object> attrs,
-                IgniteProductVersion ver) {
-                super.setNodeAttributes(attrs, ver);
-
-                // Set unique mac addresses for every group of three nodes.
-                String macAddrs = "MOCK_MACS_" + (gridInstanceNum / 3);
-
-                attrs.put(IgniteNodeAttributes.ATTR_MACS, macAddrs);
-
-                gridInstanceNum++;
-            }
-        };
-
-        spi.setIpFinder(ipFinder);
-
-        c.setDiscoverySpi(spi);
-
-        CacheConfiguration cc = defaultCacheConfiguration();
-
-        cc.setCacheMode(PARTITIONED);
-
-        cc.setBackups(backups);
-
-        cc.setAffinity(affinityFunction());
-
-        cc.setRebalanceMode(NONE);
-
-        c.setCacheConfiguration(cc);
-
-        return c;
-    }
-
-    /**
-     * @return Affinity function for test.
-     */
-    protected abstract AffinityFunction affinityFunction();
-
-    /**
-     * @param aff Affinity.
-     * @param key Key.
-     * @return Nodes.
-     */
-    private static Collection<? extends ClusterNode> nodes(Affinity<Object> aff, Object key) {
-        return aff.mapKeyToPrimaryAndBackups(key);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testAffinityMultiNode() throws Exception {
-        int grids = 9;
-
-        startGrids(grids);
-
-        try {
-            Object key = 12345;
-
-            int copies = backups + 1;
-
-            for (int i = 0; i < grids; i++) {
-                final Ignite g = grid(i);
-
-                Affinity<Object> aff = g.affinity(null);
-
-                List<TcpDiscoveryNode> top = new ArrayList<>();
-
-                for (ClusterNode node : g.cluster().nodes())
-                    top.add((TcpDiscoveryNode) node);
-
-                Collections.sort(top);
-
-                assertEquals(grids, top.size());
-
-                int idx = 1;
-
-                for (ClusterNode n : top) {
-                    assertEquals(idx, n.order());
-
-                    idx++;
-                }
-
-                Collection<? extends ClusterNode> affNodes = nodes(aff, key);
-
-                info("Affinity picture for grid [i=" + i + ", aff=" + U.toShortString(affNodes));
-
-                assertEquals(copies, affNodes.size());
-
-                Set<String> macs = new HashSet<>();
-
-                for (ClusterNode node : affNodes)
-                    macs.add((String)node.attribute(IgniteNodeAttributes.ATTR_MACS));
-
-                assertEquals(copies, macs.size());
-            }
-        }
-        finally {
-            stopAllGrids();
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testAffinitySingleNode() throws Exception {
-        Ignite g = startGrid();
-
-        try {
-            Object key = 12345;
-
-            Collection<? extends ClusterNode> affNodes = nodes(g.affinity(null), key);
-
-            info("Affinity picture for grid: " + U.toShortString(affNodes));
-
-            assertEquals(1, affNodes.size());
-        }
-        finally {
-            stopAllGrids();
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/8f5a4c9d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheRendezvousAffinityFunctionExcludeNeighborsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheRendezvousAffinityFunctionExcludeNeighborsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheRendezvousAffinityFunctionExcludeNeighborsSelfTest.java
deleted file mode 100644
index f26a1ef..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheRendezvousAffinityFunctionExcludeNeighborsSelfTest.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache.distributed.near;
-
-import org.apache.ignite.cache.affinity.AffinityFunction;
-import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
-
-/**
- * Tests exclude neighbors flag for rendezvous affinity function.
- */
-public class GridCacheRendezvousAffinityFunctionExcludeNeighborsSelfTest extends
-    GridCacheAffinityFunctionExcludeNeighborsAbstractSelfTest {
-    /** {@inheritDoc} */
-    @Override protected AffinityFunction affinityFunction() {
-        return new RendezvousAffinityFunction(true);
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/8f5a4c9d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/CacheReplicatedFairAffinityExcludeNeighborsMultiNodeFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/CacheReplicatedFairAffinityExcludeNeighborsMultiNodeFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/CacheReplicatedFairAffinityExcludeNeighborsMultiNodeFullApiSelfTest.java
new file mode 100644
index 0000000..418449a
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/CacheReplicatedFairAffinityExcludeNeighborsMultiNodeFullApiSelfTest.java
@@ -0,0 +1,36 @@
+/*
+ * 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.distributed.replicated;
+
+import org.apache.ignite.cache.affinity.fair.FairAffinityFunction;
+import org.apache.ignite.configuration.CacheConfiguration;
+
+/**
+ * Multi-node tests for partitioned cache with {@link FairAffinityFunction}.
+ */
+public class CacheReplicatedFairAffinityExcludeNeighborsMultiNodeFullApiSelfTest
+    extends GridCacheReplicatedMultiNodeFullApiSelfTest {
+    /** {@inheritDoc} */
+    @Override protected CacheConfiguration cacheConfiguration(String gridName) throws Exception {
+        CacheConfiguration cfg = super.cacheConfiguration(gridName);
+
+        cfg.setAffinity(new FairAffinityFunction(true));
+
+        return cfg;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8f5a4c9d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/CacheReplicatedFairAffinityMultiNodeFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/CacheReplicatedFairAffinityMultiNodeFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/CacheReplicatedFairAffinityMultiNodeFullApiSelfTest.java
new file mode 100644
index 0000000..ea65913
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/CacheReplicatedFairAffinityMultiNodeFullApiSelfTest.java
@@ -0,0 +1,35 @@
+/*
+ * 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.distributed.replicated;
+
+import org.apache.ignite.cache.affinity.fair.FairAffinityFunction;
+import org.apache.ignite.configuration.CacheConfiguration;
+
+/**
+ * Multi-node tests for partitioned cache with {@link FairAffinityFunction}.
+ */
+public class CacheReplicatedFairAffinityMultiNodeFullApiSelfTest extends GridCacheReplicatedMultiNodeFullApiSelfTest {
+    /** {@inheritDoc} */
+    @Override protected CacheConfiguration cacheConfiguration(String gridName) throws Exception {
+        CacheConfiguration cfg = super.cacheConfiguration(gridName);
+
+        cfg.setAffinity(new FairAffinityFunction(false));
+
+        return cfg;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8f5a4c9d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/CacheReplicatedRendezvousAffinityExcludeNeighborsMultiNodeFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/CacheReplicatedRendezvousAffinityExcludeNeighborsMultiNodeFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/CacheReplicatedRendezvousAffinityExcludeNeighborsMultiNodeFullApiSelfTest.java
new file mode 100644
index 0000000..66aeefe
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/CacheReplicatedRendezvousAffinityExcludeNeighborsMultiNodeFullApiSelfTest.java
@@ -0,0 +1,36 @@
+/*
+ * 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.distributed.replicated;
+
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.configuration.CacheConfiguration;
+
+/**
+ * Multi-node tests for partitioned cache with {@link RendezvousAffinityFunction}.
+ */
+public class CacheReplicatedRendezvousAffinityExcludeNeighborsMultiNodeFullApiSelfTest
+    extends GridCacheReplicatedMultiNodeFullApiSelfTest {
+    /** {@inheritDoc} */
+    @Override protected CacheConfiguration cacheConfiguration(String gridName) throws Exception {
+        CacheConfiguration cfg = super.cacheConfiguration(gridName);
+
+        cfg.setAffinity(new RendezvousAffinityFunction(true));
+
+        return cfg;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8f5a4c9d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/CacheReplicatedRendezvousAffinityMultiNodeFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/CacheReplicatedRendezvousAffinityMultiNodeFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/CacheReplicatedRendezvousAffinityMultiNodeFullApiSelfTest.java
new file mode 100644
index 0000000..c6900a5
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/CacheReplicatedRendezvousAffinityMultiNodeFullApiSelfTest.java
@@ -0,0 +1,36 @@
+/*
+ * 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.distributed.replicated;
+
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.configuration.CacheConfiguration;
+
+/**
+ * Multi-node tests for partitioned cache with {@link RendezvousAffinityFunction}.
+ */
+public class CacheReplicatedRendezvousAffinityMultiNodeFullApiSelfTest
+    extends GridCacheReplicatedMultiNodeFullApiSelfTest {
+    /** {@inheritDoc} */
+    @Override protected CacheConfiguration cacheConfiguration(String gridName) throws Exception {
+        CacheConfiguration cfg = super.cacheConfiguration(gridName);
+
+        cfg.setAffinity(new RendezvousAffinityFunction());
+
+        return cfg;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8f5a4c9d/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFullApiSelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFullApiSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFullApiSelfTestSuite.java
index ff53250..78f82ad 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFullApiSelfTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFullApiSelfTestSuite.java
@@ -76,6 +76,10 @@ import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePar
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedOffHeapMultiNodeFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedOffHeapTieredFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedOffHeapTieredMultiNodeFullApiSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.replicated.CacheReplicatedFairAffinityExcludeNeighborsMultiNodeFullApiSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.replicated.CacheReplicatedFairAffinityMultiNodeFullApiSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.replicated.CacheReplicatedRendezvousAffinityExcludeNeighborsMultiNodeFullApiSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.replicated.CacheReplicatedRendezvousAffinityMultiNodeFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.replicated.GridCachePartitionedFairAffinityMultiNodeFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.replicated.GridCacheReplicatedAtomicFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.replicated.GridCacheReplicatedAtomicMultiNodeFullApiSelfTest;
@@ -174,6 +178,10 @@ public class IgniteCacheFullApiSelfTestSuite extends TestSuite {
         suite.addTestSuite(GridCacheAtomicNearOnlyMultiNodeFullApiSelfTest.class);
         suite.addTestSuite(GridCacheAtomicNearOnlyMultiNodeP2PDisabledFullApiSelfTest.class);
 
+        suite.addTestSuite(CacheReplicatedFairAffinityExcludeNeighborsMultiNodeFullApiSelfTest.class);
+        suite.addTestSuite(CacheReplicatedFairAffinityMultiNodeFullApiSelfTest.class);
+        suite.addTestSuite(CacheReplicatedRendezvousAffinityExcludeNeighborsMultiNodeFullApiSelfTest.class);
+        suite.addTestSuite(CacheReplicatedRendezvousAffinityMultiNodeFullApiSelfTest.class);
         suite.addTestSuite(GridCachePartitionedFairAffinityMultiNodeFullApiSelfTest.class);
         suite.addTestSuite(GridCachePartitionedNearDisabledFairAffinityMultiNodeFullApiSelfTest.class);
         suite.addTestSuite(GridCacheAtomicFairAffinityMultiNodeFullApiSelfTest.class);