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

[32/57] [abbrv] ignite git commit: ignite-3477-master - drop FairAffinityFunction

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9b64246/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheExchangeMessageDuplicatedStateTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheExchangeMessageDuplicatedStateTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheExchangeMessageDuplicatedStateTest.java
index 294951f..ef7c7a7 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheExchangeMessageDuplicatedStateTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheExchangeMessageDuplicatedStateTest.java
@@ -22,7 +22,6 @@ import java.util.List;
 import java.util.Map;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.Ignition;
-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;
@@ -54,12 +53,6 @@ public class CacheExchangeMessageDuplicatedStateTest extends GridCommonAbstractT
     private static final String AFF1_CACHE2 = "a1c2";
 
     /** */
-    private static final String AFF2_CACHE1 = "a2c1";
-
-    /** */
-    private static final String AFF2_CACHE2 = "a2c2";
-
-    /** */
     private static final String AFF3_CACHE1 = "a3c1";
 
     /** */
@@ -69,9 +62,6 @@ public class CacheExchangeMessageDuplicatedStateTest extends GridCommonAbstractT
     private static final String AFF4_FILTER_CACHE2 = "a4c2";
 
     /** */
-    private static final String AFF5_FILTER_CACHE1 = "a5c1";
-
-    /** */
     private boolean client;
 
     /** {@inheritDoc} */
@@ -104,18 +94,6 @@ public class CacheExchangeMessageDuplicatedStateTest extends GridCommonAbstractT
         }
         {
             CacheConfiguration ccfg = new CacheConfiguration();
-            ccfg.setName(AFF2_CACHE1);
-            ccfg.setAffinity(new FairAffinityFunction(false,512));
-            ccfgs.add(ccfg);
-        }
-        {
-            CacheConfiguration ccfg = new CacheConfiguration();
-            ccfg.setName(AFF2_CACHE2);
-            ccfg.setAffinity(new FairAffinityFunction(false,512));
-            ccfgs.add(ccfg);
-        }
-        {
-            CacheConfiguration ccfg = new CacheConfiguration();
             ccfg.setName(AFF3_CACHE1);
             ccfg.setBackups(3);
 
@@ -138,13 +116,6 @@ public class CacheExchangeMessageDuplicatedStateTest extends GridCommonAbstractT
             ccfg.setAffinity(new RendezvousAffinityFunction());
             ccfgs.add(ccfg);
         }
-        {
-            CacheConfiguration ccfg = new CacheConfiguration();
-            ccfg.setName(AFF5_FILTER_CACHE1);
-            ccfg.setNodeFilter(new TestNodeFilter());
-            ccfg.setAffinity(new FairAffinityFunction());
-            ccfgs.add(ccfg);
-        }
 
         cfg.setCacheConfiguration(ccfgs.toArray(new CacheConfiguration[ccfgs.size()]));
 
@@ -265,11 +236,9 @@ public class CacheExchangeMessageDuplicatedStateTest extends GridCommonAbstractT
         assertNotNull(dupPartsData);
 
         checkFullMessage(AFF1_CACHE1, AFF1_CACHE2, dupPartsData, msg);
-        checkFullMessage(AFF2_CACHE1, AFF2_CACHE2, dupPartsData, msg);
         checkFullMessage(AFF4_FILTER_CACHE1, AFF4_FILTER_CACHE2, dupPartsData, msg);
 
         assertFalse(dupPartsData.containsKey(CU.cacheId(AFF3_CACHE1)));
-        assertFalse(dupPartsData.containsKey(CU.cacheId(AFF5_FILTER_CACHE1)));
 
         Map<Integer, Map<Integer, Long>> partCntrs = GridTestUtils.getFieldValue(msg, "partCntrs");
 
@@ -288,11 +257,9 @@ public class CacheExchangeMessageDuplicatedStateTest extends GridCommonAbstractT
         assertNotNull(dupPartsData);
 
         checkSingleMessage(AFF1_CACHE1, AFF1_CACHE2, dupPartsData, msg);
-        checkSingleMessage(AFF2_CACHE1, AFF2_CACHE2, dupPartsData, msg);
         checkSingleMessage(AFF4_FILTER_CACHE1, AFF4_FILTER_CACHE2, dupPartsData, msg);
 
         assertFalse(dupPartsData.containsKey(CU.cacheId(AFF3_CACHE1)));
-        assertFalse(dupPartsData.containsKey(CU.cacheId(AFF5_FILTER_CACHE1)));
 
         Map<Integer, Map<Integer, Long>> partCntrs = GridTestUtils.getFieldValue(msg, "partCntrs");
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9b64246/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 db112a1..1194a27 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
@@ -30,7 +30,6 @@ import org.apache.ignite.IgniteTransactions;
 import org.apache.ignite.cache.CacheEntryProcessor;
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.cache.CacheWriteSynchronizationMode;
-import org.apache.ignite.cache.affinity.fair.FairAffinityFunction;
 import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
@@ -110,55 +109,46 @@ public class CrossCacheTxRandomOperationsTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testTxOperations() throws Exception {
-        txOperations(PARTITIONED, FULL_SYNC, false, false);
+        txOperations(PARTITIONED, FULL_SYNC, false);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testCrossCacheTxOperations() throws Exception {
-        txOperations(PARTITIONED, FULL_SYNC, true, false);
+        txOperations(PARTITIONED, FULL_SYNC, true);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testCrossCacheTxOperationsPrimarySync() throws Exception {
-        txOperations(PARTITIONED, PRIMARY_SYNC, true, false);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testCrossCacheTxOperationsFairAffinity() throws Exception {
-        txOperations(PARTITIONED, FULL_SYNC, true, true);
+        txOperations(PARTITIONED, PRIMARY_SYNC, true);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testCrossCacheTxOperationsReplicated() throws Exception {
-        txOperations(REPLICATED, FULL_SYNC, true, false);
+        txOperations(REPLICATED, FULL_SYNC, true);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testCrossCacheTxOperationsReplicatedPrimarySync() throws Exception {
-        txOperations(REPLICATED, PRIMARY_SYNC, true, false);
+        txOperations(REPLICATED, PRIMARY_SYNC, true);
     }
 
     /**
      * @param name Cache name.
      * @param cacheMode Cache mode.
      * @param writeSync Write synchronization mode.
-     * @param fairAff If {@code true} uses {@link FairAffinityFunction}, otherwise {@link RendezvousAffinityFunction}.
      * @return Cache configuration.
      */
     protected CacheConfiguration cacheConfiguration(String name,
         CacheMode cacheMode,
-        CacheWriteSynchronizationMode writeSync,
-        boolean fairAff) {
+        CacheWriteSynchronizationMode writeSync) {
         CacheConfiguration ccfg = new CacheConfiguration();
 
         ccfg.setName(name);
@@ -169,7 +159,7 @@ public class CrossCacheTxRandomOperationsTest extends GridCommonAbstractTest {
         if (cacheMode == PARTITIONED)
             ccfg.setBackups(1);
 
-        ccfg.setAffinity(fairAff ? new FairAffinityFunction() : new RendezvousAffinityFunction());
+        ccfg.setAffinity(new RendezvousAffinityFunction());
 
         return ccfg;
     }
@@ -177,34 +167,30 @@ public class CrossCacheTxRandomOperationsTest extends GridCommonAbstractTest {
     /**
      * @param cacheMode Cache mode.
      * @param writeSync Write synchronization mode.
-     * @param fairAff Fair affinity flag.
      * @param ignite Node to use.
      * @param name Cache name.
      */
     protected void createCache(CacheMode cacheMode,
         CacheWriteSynchronizationMode writeSync,
-        boolean fairAff,
         Ignite ignite,
         String name) {
-        ignite.createCache(cacheConfiguration(name, cacheMode, writeSync, fairAff));
+        ignite.createCache(cacheConfiguration(name, cacheMode, writeSync));
     }
 
     /**
      * @param cacheMode Cache mode.
      * @param writeSync Write synchronization mode.
      * @param crossCacheTx If {@code true} uses cross cache transaction.
-     * @param fairAff If {@code true} uses {@link FairAffinityFunction}, otherwise {@link RendezvousAffinityFunction}.
      * @throws Exception If failed.
      */
     private void txOperations(CacheMode cacheMode,
         CacheWriteSynchronizationMode writeSync,
-        boolean crossCacheTx,
-        boolean fairAff) throws Exception {
+        boolean crossCacheTx) throws Exception {
         Ignite ignite = ignite(0);
 
         try {
-            createCache(cacheMode, writeSync, fairAff, ignite, CACHE1);
-            createCache(cacheMode, writeSync, fairAff, ignite, CACHE2);
+            createCache(cacheMode, writeSync, ignite, CACHE1);
+            createCache(cacheMode, writeSync, ignite, CACHE2);
 
             txOperations(PESSIMISTIC, REPEATABLE_READ, crossCacheTx, false);
             txOperations(PESSIMISTIC, REPEATABLE_READ, crossCacheTx, true);

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9b64246/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheVersionTopologyChangeTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheVersionTopologyChangeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheVersionTopologyChangeTest.java
index bd9ad54..491e4dc 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheVersionTopologyChangeTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheVersionTopologyChangeTest.java
@@ -29,7 +29,6 @@ import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheEntry;
 import org.apache.ignite.cache.affinity.Affinity;
 import org.apache.ignite.cache.affinity.AffinityFunction;
-import org.apache.ignite.cache.affinity.fair.FairAffinityFunction;
 import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
@@ -214,10 +213,6 @@ public class GridCacheVersionTopologyChangeTest extends GridCommonAbstractTest {
         ccfgs.add(cacheConfiguration("c2", atomicityMode, new RendezvousAffinityFunction(), 1));
         ccfgs.add(cacheConfiguration("c3", atomicityMode, new RendezvousAffinityFunction(false, 10), 0));
 
-        ccfgs.add(cacheConfiguration("c4", atomicityMode, new FairAffinityFunction(), 0));
-        ccfgs.add(cacheConfiguration("c5", atomicityMode, new FairAffinityFunction(), 1));
-        ccfgs.add(cacheConfiguration("c6", atomicityMode, new FairAffinityFunction(false, 10), 0));
-
         return ccfgs;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9b64246/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientAffinityAssignmentSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientAffinityAssignmentSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientAffinityAssignmentSelfTest.java
index aea9167..09d1099 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientAffinityAssignmentSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientAffinityAssignmentSelfTest.java
@@ -23,7 +23,6 @@ import org.apache.ignite.Ignition;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.cache.affinity.Affinity;
-import org.apache.ignite.cache.affinity.fair.FairAffinityFunction;
 import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
@@ -48,9 +47,6 @@ public class IgniteClientAffinityAssignmentSelfTest extends GridCommonAbstractTe
     /** */
     private boolean cache;
 
-    /** */
-    private int aff;
-
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
@@ -66,10 +62,7 @@ public class IgniteClientAffinityAssignmentSelfTest extends GridCommonAbstractTe
 
             ccfg.setNearConfiguration(null);
 
-            if (aff == 0)
-                ccfg.setAffinity(new RendezvousAffinityFunction(false, PARTS));
-            else
-                ccfg.setAffinity(new FairAffinityFunction(PARTS));
+            ccfg.setAffinity(new RendezvousAffinityFunction(false, PARTS));
 
             cfg.setCacheConfiguration(ccfg);
         }
@@ -83,17 +76,6 @@ public class IgniteClientAffinityAssignmentSelfTest extends GridCommonAbstractTe
      * @throws Exception If failed.
      */
     public void testRendezvousAssignment() throws Exception {
-        aff = 0;
-
-        checkAffinityFunction();
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testFairAssignment() throws Exception {
-        aff = 1;
-
         checkAffinityFunction();
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9b64246/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartNoExchangeTimeoutTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartNoExchangeTimeoutTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartNoExchangeTimeoutTest.java
index 78ab065..0588139 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartNoExchangeTimeoutTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartNoExchangeTimeoutTest.java
@@ -24,7 +24,6 @@ import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteException;
-import org.apache.ignite.cache.affinity.fair.FairAffinityFunction;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
@@ -371,7 +370,6 @@ public class IgniteDynamicCacheStartNoExchangeTimeoutTest extends GridCommonAbst
             ccfg.setName("cache-3");
             ccfg.setAtomicityMode(ATOMIC);
             ccfg.setBackups(1);
-            ccfg.setAffinity(new FairAffinityFunction());
             ccfg.setWriteSynchronizationMode(FULL_SYNC);
 
             res.add(ccfg);
@@ -405,7 +403,6 @@ public class IgniteDynamicCacheStartNoExchangeTimeoutTest extends GridCommonAbst
             ccfg.setName("cache-4");
             ccfg.setAtomicityMode(TRANSACTIONAL);
             ccfg.setBackups(1);
-            ccfg.setAffinity(new FairAffinityFunction());
             ccfg.setWriteSynchronizationMode(FULL_SYNC);
 
             res.add(ccfg);

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9b64246/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLateAffinityAssignmentFairAffinityTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLateAffinityAssignmentFairAffinityTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLateAffinityAssignmentFairAffinityTest.java
deleted file mode 100644
index 0ab2314..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLateAffinityAssignmentFairAffinityTest.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;
-
-import org.apache.ignite.cache.affinity.AffinityFunction;
-import org.apache.ignite.cache.affinity.fair.FairAffinityFunction;
-import org.jetbrains.annotations.Nullable;
-
-/**
- *
- */
-public class CacheLateAffinityAssignmentFairAffinityTest extends CacheLateAffinityAssignmentTest {
-    /** {@inheritDoc} */
-    @Override protected AffinityFunction affinityFunction(@Nullable Integer parts) {
-        return new FairAffinityFunction(false, parts == null ? FairAffinityFunction.DFLT_PART_CNT : parts);
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9b64246/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodePartitionsExchangeTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodePartitionsExchangeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodePartitionsExchangeTest.java
index 3ed3ec7..d07398d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodePartitionsExchangeTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodePartitionsExchangeTest.java
@@ -29,7 +29,6 @@ import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.Ignition;
 import org.apache.ignite.cache.CacheServerNotFoundException;
 import org.apache.ignite.cache.affinity.Affinity;
-import org.apache.ignite.cache.affinity.fair.FairAffinityFunction;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
@@ -69,9 +68,6 @@ public class IgniteCacheClientNodePartitionsExchangeTest extends GridCommonAbstr
     /** */
     private boolean client;
 
-    /** */
-    private boolean fairAffinity;
-
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
@@ -82,9 +78,6 @@ public class IgniteCacheClientNodePartitionsExchangeTest extends GridCommonAbstr
 
         CacheConfiguration ccfg = new CacheConfiguration();
 
-        if (fairAffinity)
-            ccfg.setAffinity(new FairAffinityFunction());
-
         cfg.setCacheConfiguration(ccfg);
 
         cfg.setCommunicationSpi(new TestCommunicationSpi());
@@ -210,15 +203,6 @@ public class IgniteCacheClientNodePartitionsExchangeTest extends GridCommonAbstr
     /**
      * @throws Exception If failed.
      */
-    public void testPartitionsExchangeFairAffinity() throws Exception {
-        fairAffinity = true;
-
-        partitionsExchange();
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
     private void partitionsExchange() throws Exception {
         Ignite ignite0 = startGrid(0);
 
@@ -327,7 +311,7 @@ public class IgniteCacheClientNodePartitionsExchangeTest extends GridCommonAbstr
 
         if (lateAff) {
             // With FairAffinityFunction affinity calculation is different, this causes one more topology change.
-            boolean exchangeAfterRebalance = fairAffinity;
+            boolean exchangeAfterRebalance = false;
 
             waitForTopologyUpdate(4,
                 exchangeAfterRebalance ? new AffinityTopologyVersion(6, 1) : new AffinityTopologyVersion(6, 0));

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9b64246/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheTxFairAffinityNodeJoinTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheTxFairAffinityNodeJoinTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheTxFairAffinityNodeJoinTest.java
deleted file mode 100644
index 53079f9..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheTxFairAffinityNodeJoinTest.java
+++ /dev/null
@@ -1,35 +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 org.apache.ignite.cache.affinity.fair.FairAffinityFunction;
-import org.apache.ignite.configuration.CacheConfiguration;
-
-/**
- *
- */
-public class IgniteCacheTxFairAffinityNodeJoinTest extends IgniteCacheTxNodeJoinTest {
-    /** {@inheritDoc} */
-    @Override protected CacheConfiguration cacheConfiguration(String igniteInstanceName) throws Exception {
-        CacheConfiguration ccfg = super.cacheConfiguration(igniteInstanceName);
-
-        ccfg.setAffinity(new FairAffinityFunction(128));
-
-        return ccfg;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9b64246/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheTxNearDisabledFairAffinityPutGetRestartTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheTxNearDisabledFairAffinityPutGetRestartTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheTxNearDisabledFairAffinityPutGetRestartTest.java
deleted file mode 100644
index e8d59e3..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheTxNearDisabledFairAffinityPutGetRestartTest.java
+++ /dev/null
@@ -1,35 +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 org.apache.ignite.cache.affinity.fair.FairAffinityFunction;
-import org.apache.ignite.configuration.CacheConfiguration;
-
-/**
- *
- */
-public class IgniteCacheTxNearDisabledFairAffinityPutGetRestartTest extends IgniteCacheTxNearDisabledPutGetRestartTest {
-    /** {@inheritDoc} */
-    @Override protected CacheConfiguration cacheConfiguration(String igniteInstanceName) throws Exception {
-        CacheConfiguration cfg = super.cacheConfiguration(igniteInstanceName);
-
-        cfg.setAffinity(new FairAffinityFunction());
-
-        return cfg;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9b64246/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedNearDisabledFairAffinityMultiNodeFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedNearDisabledFairAffinityMultiNodeFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedNearDisabledFairAffinityMultiNodeFullApiSelfTest.java
deleted file mode 100644
index db30a89..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedNearDisabledFairAffinityMultiNodeFullApiSelfTest.java
+++ /dev/null
@@ -1,36 +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.dht;
-
-import org.apache.ignite.cache.affinity.fair.FairAffinityFunction;
-import org.apache.ignite.configuration.CacheConfiguration;
-
-/**
- *
- */
-public class GridCachePartitionedNearDisabledFairAffinityMultiNodeFullApiSelfTest
-    extends GridCachePartitionedNearDisabledMultiNodeFullApiSelfTest {
-    /** {@inheritDoc} */
-    @Override protected CacheConfiguration cacheConfiguration(String igniteInstanceName) throws Exception {
-        CacheConfiguration cfg = super.cacheConfiguration(igniteInstanceName);
-
-        cfg.setAffinity(new FairAffinityFunction());
-
-        return cfg;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9b64246/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/AtomicPutAllChangingTopologyTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/AtomicPutAllChangingTopologyTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/AtomicPutAllChangingTopologyTest.java
index 25b3901..1eb8347 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/AtomicPutAllChangingTopologyTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/AtomicPutAllChangingTopologyTest.java
@@ -26,7 +26,7 @@ import java.util.concurrent.CountDownLatch;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.cache.affinity.fair.FairAffinityFunction;
+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;
@@ -68,7 +68,7 @@ public class AtomicPutAllChangingTopologyTest extends GridCommonAbstractTest {
         return new CacheConfiguration<Integer, Integer>()
             .setAtomicityMode(ATOMIC)
             .setCacheMode(REPLICATED)
-            .setAffinity(new FairAffinityFunction(false, 1))
+            .setAffinity(new RendezvousAffinityFunction(false, 1))
             .setWriteSynchronizationMode(FULL_SYNC)
             .setRebalanceMode(SYNC)
             .setName(CACHE_NAME);

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9b64246/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicClientOnlyFairAffinityMultiNodeFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicClientOnlyFairAffinityMultiNodeFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicClientOnlyFairAffinityMultiNodeFullApiSelfTest.java
deleted file mode 100644
index 36e5054..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicClientOnlyFairAffinityMultiNodeFullApiSelfTest.java
+++ /dev/null
@@ -1,36 +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.fair.FairAffinityFunction;
-import org.apache.ignite.configuration.CacheConfiguration;
-
-/**
- *
- */
-public class GridCacheAtomicClientOnlyFairAffinityMultiNodeFullApiSelfTest
-    extends GridCacheAtomicClientOnlyMultiNodeFullApiSelfTest {
-    /** {@inheritDoc} */
-    @Override protected CacheConfiguration cacheConfiguration(String igniteInstanceName) throws Exception {
-        CacheConfiguration cfg = super.cacheConfiguration(igniteInstanceName);
-
-        cfg.setAffinity(new FairAffinityFunction());
-
-        return cfg;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9b64246/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicFairAffinityMultiNodeFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicFairAffinityMultiNodeFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicFairAffinityMultiNodeFullApiSelfTest.java
deleted file mode 100644
index eaaf808..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicFairAffinityMultiNodeFullApiSelfTest.java
+++ /dev/null
@@ -1,35 +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.fair.FairAffinityFunction;
-import org.apache.ignite.configuration.CacheConfiguration;
-
-/**
- * Multi-node tests for partitioned cache.
- */
-public class GridCacheAtomicFairAffinityMultiNodeFullApiSelfTest extends GridCacheAtomicMultiNodeFullApiSelfTest {
-    /** {@inheritDoc} */
-    @Override protected CacheConfiguration cacheConfiguration(String igniteInstanceName) throws Exception {
-        CacheConfiguration cfg = super.cacheConfiguration(igniteInstanceName);
-
-        cfg.setAffinity(new FairAffinityFunction());
-
-        return cfg;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9b64246/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicNearEnabledFairAffinityMultiNodeFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicNearEnabledFairAffinityMultiNodeFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicNearEnabledFairAffinityMultiNodeFullApiSelfTest.java
deleted file mode 100644
index 5563dd3..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicNearEnabledFairAffinityMultiNodeFullApiSelfTest.java
+++ /dev/null
@@ -1,36 +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.fair.FairAffinityFunction;
-import org.apache.ignite.configuration.CacheConfiguration;
-
-/**
- *
- */
-public class GridCacheAtomicNearEnabledFairAffinityMultiNodeFullApiSelfTest
-    extends GridCacheAtomicNearEnabledMultiNodeFullApiSelfTest {
-    /** {@inheritDoc} */
-    @Override protected CacheConfiguration cacheConfiguration(String igniteInstanceName) throws Exception {
-        CacheConfiguration cfg = super.cacheConfiguration(igniteInstanceName);
-
-        cfg.setAffinity(new FairAffinityFunction());
-
-        return cfg;
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9b64246/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicPrimaryWriteOrderFairAffinityMultiNodeFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicPrimaryWriteOrderFairAffinityMultiNodeFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicPrimaryWriteOrderFairAffinityMultiNodeFullApiSelfTest.java
deleted file mode 100644
index 28080b7..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicPrimaryWriteOrderFairAffinityMultiNodeFullApiSelfTest.java
+++ /dev/null
@@ -1,36 +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.fair.FairAffinityFunction;
-import org.apache.ignite.configuration.CacheConfiguration;
-
-/**
- *
- */
-public class GridCacheAtomicPrimaryWriteOrderFairAffinityMultiNodeFullApiSelfTest
-    extends GridCacheAtomicPrimaryWriteOrderMultiNodeFullApiSelfTest {
-    /** {@inheritDoc} */
-    @Override protected CacheConfiguration cacheConfiguration(String igniteInstanceName) throws Exception {
-        CacheConfiguration cfg = super.cacheConfiguration(igniteInstanceName);
-
-        cfg.setAffinity(new FairAffinityFunction());
-
-        return cfg;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9b64246/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOnlyFairAffinityMultiNodeFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOnlyFairAffinityMultiNodeFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOnlyFairAffinityMultiNodeFullApiSelfTest.java
deleted file mode 100644
index 860cdf4..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOnlyFairAffinityMultiNodeFullApiSelfTest.java
+++ /dev/null
@@ -1,35 +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.fair.FairAffinityFunction;
-import org.apache.ignite.configuration.CacheConfiguration;
-
-/**
- *
- */
-public class GridCacheNearOnlyFairAffinityMultiNodeFullApiSelfTest extends GridCacheNearOnlyMultiNodeFullApiSelfTest {
-    /** {@inheritDoc} */
-    @Override protected CacheConfiguration cacheConfiguration(String igniteInstanceName) throws Exception {
-        CacheConfiguration cfg = super.cacheConfiguration(igniteInstanceName);
-
-        cfg.setAffinity(new FairAffinityFunction());
-
-        return cfg;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9b64246/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
deleted file mode 100644
index 5fab7d5..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/CacheReplicatedFairAffinityExcludeNeighborsMultiNodeFullApiSelfTest.java
+++ /dev/null
@@ -1,36 +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.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 igniteInstanceName) throws Exception {
-        CacheConfiguration cfg = super.cacheConfiguration(igniteInstanceName);
-
-        cfg.setAffinity(new FairAffinityFunction(true));
-
-        return cfg;
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9b64246/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
deleted file mode 100644
index d06bf9b..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/CacheReplicatedFairAffinityMultiNodeFullApiSelfTest.java
+++ /dev/null
@@ -1,35 +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.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 igniteInstanceName) throws Exception {
-        CacheConfiguration cfg = super.cacheConfiguration(igniteInstanceName);
-
-        cfg.setAffinity(new FairAffinityFunction(false));
-
-        return cfg;
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9b64246/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCachePartitionedFairAffinityMultiNodeFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCachePartitionedFairAffinityMultiNodeFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCachePartitionedFairAffinityMultiNodeFullApiSelfTest.java
deleted file mode 100644
index 9dfaafb..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCachePartitionedFairAffinityMultiNodeFullApiSelfTest.java
+++ /dev/null
@@ -1,37 +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.replicated;
-
-import org.apache.ignite.cache.affinity.fair.FairAffinityFunction;
-import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedMultiNodeFullApiSelfTest;
-
-/**
- * Multi-node tests for partitioned cache with {@link FairAffinityFunction}.
- */
-public class GridCachePartitionedFairAffinityMultiNodeFullApiSelfTest
-    extends GridCachePartitionedMultiNodeFullApiSelfTest {
-    /** {@inheritDoc} */
-    @Override protected CacheConfiguration cacheConfiguration(String igniteInstanceName) throws Exception {
-        CacheConfiguration cfg = super.cacheConfiguration(igniteInstanceName);
-
-        cfg.setAffinity(new FairAffinityFunction());
-
-        return cfg;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9b64246/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicClientOnlyFairAffinityMultiJvmFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicClientOnlyFairAffinityMultiJvmFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicClientOnlyFairAffinityMultiJvmFullApiSelfTest.java
deleted file mode 100644
index fdc800f..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicClientOnlyFairAffinityMultiJvmFullApiSelfTest.java
+++ /dev/null
@@ -1,31 +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.multijvm;
-
-import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicClientOnlyFairAffinityMultiNodeFullApiSelfTest;
-
-/**
- * Multi-JVM tests.
- */
-public class GridCacheAtomicClientOnlyFairAffinityMultiJvmFullApiSelfTest
-    extends GridCacheAtomicClientOnlyFairAffinityMultiNodeFullApiSelfTest {
-    /** {@inheritDoc} */
-    protected boolean isMultiJvm() {
-        return true;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9b64246/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicFairAffinityMultiJvmFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicFairAffinityMultiJvmFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicFairAffinityMultiJvmFullApiSelfTest.java
deleted file mode 100644
index 1ccee61..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicFairAffinityMultiJvmFullApiSelfTest.java
+++ /dev/null
@@ -1,31 +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.multijvm;
-
-import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicFairAffinityMultiNodeFullApiSelfTest;
-
-/**
- * Multi-JVM tests.
- */
-public class GridCacheAtomicFairAffinityMultiJvmFullApiSelfTest extends
-    GridCacheAtomicFairAffinityMultiNodeFullApiSelfTest {
-    /** {@inheritDoc} */
-    protected boolean isMultiJvm() {
-        return true;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9b64246/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicNearEnabledFairAffinityMultiJvmFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicNearEnabledFairAffinityMultiJvmFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicNearEnabledFairAffinityMultiJvmFullApiSelfTest.java
deleted file mode 100644
index 0948a1d..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicNearEnabledFairAffinityMultiJvmFullApiSelfTest.java
+++ /dev/null
@@ -1,36 +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.multijvm;
-
-import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicNearEnabledFairAffinityMultiNodeFullApiSelfTest;
-
-/**
- * Multi-JVM tests.
- */
-public class GridCacheAtomicNearEnabledFairAffinityMultiJvmFullApiSelfTest extends
-    GridCacheAtomicNearEnabledFairAffinityMultiNodeFullApiSelfTest {
-    /** {@inheritDoc} */
-    protected boolean isMultiJvm() {
-        return true;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void testPutAllPutAll() throws Exception {
-        fail("https://issues.apache.org/jira/browse/IGNITE-1112");
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9b64246/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicPrimaryWriteOrderFairAffinityMultiJvmFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicPrimaryWriteOrderFairAffinityMultiJvmFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicPrimaryWriteOrderFairAffinityMultiJvmFullApiSelfTest.java
deleted file mode 100644
index c55f06f6..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheAtomicPrimaryWriteOrderFairAffinityMultiJvmFullApiSelfTest.java
+++ /dev/null
@@ -1,31 +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.multijvm;
-
-import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicPrimaryWriteOrderFairAffinityMultiNodeFullApiSelfTest;
-
-/**
- * Multi-JVM tests.
- */
-public class GridCacheAtomicPrimaryWriteOrderFairAffinityMultiJvmFullApiSelfTest extends
-    GridCacheAtomicPrimaryWriteOrderFairAffinityMultiNodeFullApiSelfTest {
-    /** {@inheritDoc} */
-    protected boolean isMultiJvm() {
-        return true;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9b64246/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheNearOnlyFairAffinityMultiJvmFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheNearOnlyFairAffinityMultiJvmFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheNearOnlyFairAffinityMultiJvmFullApiSelfTest.java
deleted file mode 100644
index 7db196a..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCacheNearOnlyFairAffinityMultiJvmFullApiSelfTest.java
+++ /dev/null
@@ -1,31 +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.multijvm;
-
-import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheNearOnlyFairAffinityMultiNodeFullApiSelfTest;
-
-/**
- * Multi-JVM tests.
- */
-public class GridCacheNearOnlyFairAffinityMultiJvmFullApiSelfTest extends
-    GridCacheNearOnlyFairAffinityMultiNodeFullApiSelfTest {
-    /** {@inheritDoc} */
-    protected boolean isMultiJvm() {
-        return true;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9b64246/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCachePartitionedFairAffinityMultiJvmFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCachePartitionedFairAffinityMultiJvmFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCachePartitionedFairAffinityMultiJvmFullApiSelfTest.java
deleted file mode 100644
index 9e4abd5..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCachePartitionedFairAffinityMultiJvmFullApiSelfTest.java
+++ /dev/null
@@ -1,31 +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.multijvm;
-
-import org.apache.ignite.internal.processors.cache.distributed.replicated.GridCachePartitionedFairAffinityMultiNodeFullApiSelfTest;
-
-/**
- * Multi-JVM tests.
- */
-public class GridCachePartitionedFairAffinityMultiJvmFullApiSelfTest extends
-    GridCachePartitionedFairAffinityMultiNodeFullApiSelfTest {
-    /** {@inheritDoc} */
-    protected boolean isMultiJvm() {
-        return true;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9b64246/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCachePartitionedNearDisabledFairAffinityMultiJvmFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCachePartitionedNearDisabledFairAffinityMultiJvmFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCachePartitionedNearDisabledFairAffinityMultiJvmFullApiSelfTest.java
deleted file mode 100644
index 2b9bf56..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/multijvm/GridCachePartitionedNearDisabledFairAffinityMultiJvmFullApiSelfTest.java
+++ /dev/null
@@ -1,31 +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.multijvm;
-
-import org.apache.ignite.internal.processors.cache.distributed.dht.GridCachePartitionedNearDisabledFairAffinityMultiNodeFullApiSelfTest;
-
-/**
- * Multi-JVM tests.
- */
-public class GridCachePartitionedNearDisabledFairAffinityMultiJvmFullApiSelfTest extends
-    GridCachePartitionedNearDisabledFairAffinityMultiNodeFullApiSelfTest {
-    /** {@inheritDoc} */
-    protected boolean isMultiJvm() {
-        return true;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9b64246/modules/core/src/test/java/org/apache/ignite/testframework/configvariations/ConfigVariations.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/configvariations/ConfigVariations.java b/modules/core/src/test/java/org/apache/ignite/testframework/configvariations/ConfigVariations.java
index 14cdc2a..26e7a90 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/configvariations/ConfigVariations.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/configvariations/ConfigVariations.java
@@ -32,7 +32,6 @@ import org.apache.ignite.cache.CacheInterceptorAdapter;
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.cache.CacheRebalanceMode;
 import org.apache.ignite.cache.CacheWriteSynchronizationMode;
-import org.apache.ignite.cache.affinity.fair.FairAffinityFunction;
 import org.apache.ignite.cache.eviction.EvictionFilter;
 import org.apache.ignite.cache.eviction.fifo.FifoEvictionPolicy;
 import org.apache.ignite.cache.store.CacheStoreSession;
@@ -120,7 +119,6 @@ public class ConfigVariations {
                 EVICTION_PARAM,
                 CACHE_STORE_PARAM,
                 REBALANCING_PARAM,
-                Parameters.parameter("setAffinity", Parameters.factory(FairAffinityFunction.class)),
                 Parameters.parameter("setInterceptor", Parameters.factory(NoopInterceptor.class)),
                 Parameters.parameter("setTopologyValidator", Parameters.factory(NoopTopologyValidator.class)),
                 Parameters.parameter("addCacheEntryListenerConfiguration", Parameters.factory(EmptyCacheEntryListenerConfiguration.class))

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9b64246/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFailoverTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFailoverTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFailoverTestSuite.java
index 986b8d4..3b42f16 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFailoverTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFailoverTestSuite.java
@@ -23,8 +23,6 @@ import org.apache.ignite.internal.processors.cache.GridCacheIncrementTransformTe
 import org.apache.ignite.internal.processors.cache.IgniteCacheTopologySafeGetSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.IgniteCacheAtomicNodeJoinTest;
 import org.apache.ignite.internal.processors.cache.distributed.IgniteCacheSizeFailoverTest;
-import org.apache.ignite.internal.processors.cache.distributed.IgniteCacheTxFairAffinityNodeJoinTest;
-import org.apache.ignite.internal.processors.cache.distributed.IgniteCacheTxNearDisabledFairAffinityPutGetRestartTest;
 import org.apache.ignite.internal.processors.cache.distributed.IgniteCacheTxNearDisabledPutGetRestartTest;
 import org.apache.ignite.internal.processors.cache.distributed.IgniteCacheTxNodeJoinTest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridCacheDhtAtomicRemoveFailureTest;
@@ -85,10 +83,8 @@ public class IgniteCacheFailoverTestSuite extends TestSuite {
 
         suite.addTestSuite(IgniteCacheAtomicNodeJoinTest.class);
         suite.addTestSuite(IgniteCacheTxNodeJoinTest.class);
-        suite.addTestSuite(IgniteCacheTxFairAffinityNodeJoinTest.class);
 
         suite.addTestSuite(IgniteCacheTxNearDisabledPutGetRestartTest.class);
-        suite.addTestSuite(IgniteCacheTxNearDisabledFairAffinityPutGetRestartTest.class);
 
         suite.addTestSuite(IgniteCacheSizeFailoverTest.class);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9b64246/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFullApiMultiJvmSelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFullApiMultiJvmSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFullApiMultiJvmSelfTestSuite.java
index 896bb0e..e9813dd 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFullApiMultiJvmSelfTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFullApiMultiJvmSelfTestSuite.java
@@ -18,32 +18,25 @@
 package org.apache.ignite.testsuites;
 
 import junit.framework.TestSuite;
-import org.apache.ignite.internal.processors.cache.multijvm.GridCacheAtomicClientOnlyFairAffinityMultiJvmFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.multijvm.GridCacheAtomicClientOnlyMultiJvmFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.multijvm.GridCacheAtomicClientOnlyMultiJvmP2PDisabledFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.multijvm.GridCacheAtomicCopyOnReadDisabledMultiJvmFullApiSelfTest;
-import org.apache.ignite.internal.processors.cache.multijvm.GridCacheAtomicFairAffinityMultiJvmFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.multijvm.GridCacheAtomicMultiJvmFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.multijvm.GridCacheAtomicMultiJvmP2PDisabledFullApiSelfTest;
-import org.apache.ignite.internal.processors.cache.multijvm.GridCacheAtomicNearEnabledFairAffinityMultiJvmFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.multijvm.GridCacheAtomicNearEnabledMultiJvmFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.multijvm.GridCacheAtomicNearEnabledPrimaryWriteOrderMultiJvmFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.multijvm.GridCacheAtomicNearOnlyMultiJvmFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.multijvm.GridCacheAtomicNearOnlyMultiJvmP2PDisabledFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.multijvm.GridCacheAtomicOnheapMultiJvmFullApiSelfTest;
-import org.apache.ignite.internal.processors.cache.multijvm.GridCacheAtomicPrimaryWriteOrderFairAffinityMultiJvmFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.multijvm.GridCacheAtomicPrimaryWriteOrderMultiJvmFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.multijvm.GridCacheAtomicPrimaryWriteOrderMultiJvmP2PDisabledFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.multijvm.GridCacheAtomicPrimaryWriteOrderOnheapMultiJvmFullApiSelfTest;
-import org.apache.ignite.internal.processors.cache.multijvm.GridCacheNearOnlyFairAffinityMultiJvmFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.multijvm.GridCacheNearOnlyMultiJvmFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.multijvm.GridCacheNearOnlyMultiJvmP2PDisabledFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.multijvm.GridCachePartitionedCopyOnReadDisabledMultiJvmFullApiSelfTest;
-import org.apache.ignite.internal.processors.cache.multijvm.GridCachePartitionedFairAffinityMultiJvmFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.multijvm.GridCachePartitionedMultiJvmFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.multijvm.GridCachePartitionedMultiJvmP2PDisabledFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.multijvm.GridCachePartitionedNearDisabledAtomicOnheapMultiJvmFullApiSelfTest;
-import org.apache.ignite.internal.processors.cache.multijvm.GridCachePartitionedNearDisabledFairAffinityMultiJvmFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.multijvm.GridCachePartitionedNearDisabledMultiJvmFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.multijvm.GridCachePartitionedNearDisabledMultiJvmP2PDisabledFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.multijvm.GridCachePartitionedNearDisabledOnheapMultiJvmFullApiSelfTest;
@@ -98,14 +91,6 @@ public class IgniteCacheFullApiMultiJvmSelfTestSuite extends TestSuite {
         suite.addTestSuite(GridCacheAtomicNearOnlyMultiJvmFullApiSelfTest.class);
         suite.addTestSuite(GridCacheAtomicNearOnlyMultiJvmP2PDisabledFullApiSelfTest.class);
 
-        suite.addTestSuite(GridCachePartitionedFairAffinityMultiJvmFullApiSelfTest.class);
-        suite.addTestSuite(GridCachePartitionedNearDisabledFairAffinityMultiJvmFullApiSelfTest.class);
-        suite.addTestSuite(GridCacheAtomicFairAffinityMultiJvmFullApiSelfTest.class);
-        suite.addTestSuite(GridCacheAtomicNearEnabledFairAffinityMultiJvmFullApiSelfTest.class);
-        suite.addTestSuite(GridCacheAtomicPrimaryWriteOrderFairAffinityMultiJvmFullApiSelfTest.class);
-        suite.addTestSuite(GridCacheNearOnlyFairAffinityMultiJvmFullApiSelfTest.class);
-        suite.addTestSuite(GridCacheAtomicClientOnlyFairAffinityMultiJvmFullApiSelfTest.class);
-
         suite.addTestSuite(GridCacheAtomicOnheapMultiJvmFullApiSelfTest.class);
         suite.addTestSuite(GridCachePartitionedNearDisabledAtomicOnheapMultiJvmFullApiSelfTest.class);
         suite.addTestSuite(GridCachePartitionedNearDisabledOnheapMultiJvmFullApiSelfTest.class);

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9b64246/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 fa11f24..1a1bfe7 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
@@ -22,15 +22,12 @@ import org.apache.ignite.internal.processors.cache.GridCacheClearSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridCacheAtomicFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridCacheAtomicNearEnabledFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridCacheAtomicNearEnabledPrimaryWriteOrderFullApiSelfTest;
-import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicOnheapFullApiSelfTest;
-import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicOnheapMultiNodeFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridCacheAtomicPrimaryWriteOrderFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridCacheAtomicPrimaryWriteOrderReloadAllSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridCacheAtomicReloadAllSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridCacheColocatedReloadAllSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridCachePartitionedNearDisabledAtomicOnheapFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridCachePartitionedNearDisabledAtomicOnheapMultiNodeFullApiSelfTest;
-import org.apache.ignite.internal.processors.cache.distributed.dht.GridCachePartitionedNearDisabledFairAffinityMultiNodeFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridCachePartitionedNearDisabledFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridCachePartitionedNearDisabledMultiNodeFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridCachePartitionedNearDisabledMultiNodeP2PDisabledFullApiSelfTest;
@@ -38,25 +35,22 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.GridCachePart
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridCachePartitionedNearDisabledOnheapMultiNodeFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.CachePartitionedMultiNodeLongTxTimeoutFullApiTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.CachePartitionedNearEnabledMultiNodeLongTxTimeoutFullApiTest;
-import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicClientOnlyFairAffinityMultiNodeFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicClientOnlyMultiNodeFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicClientOnlyMultiNodeP2PDisabledFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicCopyOnReadDisabledMultiNodeFullApiSelfTest;
-import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicFairAffinityMultiNodeFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicLateAffDisabledPrimaryWriteOrderMultiNodeFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicMultiNodeFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicMultiNodeP2PDisabledFullApiSelfTest;
-import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicNearEnabledFairAffinityMultiNodeFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicNearEnabledMultiNodeFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicNearEnabledPrimaryWriteOrderMultiNodeFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicNearOnlyMultiNodeFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicNearOnlyMultiNodeP2PDisabledFullApiSelfTest;
-import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicPrimaryWriteOrderFairAffinityMultiNodeFullApiSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicOnheapFullApiSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicOnheapMultiNodeFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicPrimaryWriteOrderMultiNodeFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicPrimaryWriteOrderMultiNodeP2PDisabledFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicPrimaryWriteOrderOnheapFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicPrimaryWriteOrderOnheapMultiNodeFullApiSelfTest;
-import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheNearOnlyFairAffinityMultiNodeFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheNearOnlyMultiNodeFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheNearOnlyMultiNodeP2PDisabledFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheNearReloadAllSelfTest;
@@ -75,11 +69,8 @@ import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePar
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedNearOnlyNoPrimaryFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedOnheapFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedOnheapMultiNodeFullApiSelfTest;
-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;
 import org.apache.ignite.internal.processors.cache.distributed.replicated.GridCacheReplicatedAtomicPrimaryWriteOrderMultiNodeFullApiSelfTest;
@@ -159,17 +150,8 @@ 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);
-        suite.addTestSuite(GridCacheAtomicNearEnabledFairAffinityMultiNodeFullApiSelfTest.class);
-        suite.addTestSuite(GridCacheAtomicPrimaryWriteOrderFairAffinityMultiNodeFullApiSelfTest.class);
-        suite.addTestSuite(GridCacheNearOnlyFairAffinityMultiNodeFullApiSelfTest.class);
-        suite.addTestSuite(GridCacheAtomicClientOnlyFairAffinityMultiNodeFullApiSelfTest.class);
 
         suite.addTestSuite(GridCacheNearReloadAllSelfTest.class);
         suite.addTestSuite(GridCacheColocatedReloadAllSelfTest.class);
@@ -182,7 +164,7 @@ public class IgniteCacheFullApiSelfTestSuite extends TestSuite {
         suite.addTestSuite(GridCachePartitionedAtomicOnheapMultiNodeFullApiSelfTest.class);
         suite.addTestSuite(GridCachePartitionedNearDisabledOnheapMultiNodeFullApiSelfTest.class);
         suite.addTestSuite(GridCachePartitionedNearDisabledAtomicOnheapMultiNodeFullApiSelfTest.class);
-        suite.addTestSuite(GridCacheAtomicOnheapMultiNodeFullApiSelfTest.class);;
+        suite.addTestSuite(GridCacheAtomicOnheapMultiNodeFullApiSelfTest.class);
         suite.addTestSuite(GridCacheAtomicPrimaryWriteOrderOnheapMultiNodeFullApiSelfTest.class);;
 
         // Old affinity assignment mode.

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9b64246/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
index 5c1a879..0c310a7 100755
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
@@ -20,13 +20,6 @@ package org.apache.ignite.testsuites;
 import java.util.Set;
 import junit.framework.TestSuite;
 import org.apache.ignite.cache.IgniteWarmupClosureSelfTest;
-import org.apache.ignite.cache.affinity.AffinityClientNodeSelfTest;
-import org.apache.ignite.cache.affinity.AffinityHistoryCleanupTest;
-import org.apache.ignite.cache.affinity.fair.FairAffinityDynamicCacheSelfTest;
-import org.apache.ignite.cache.affinity.fair.FairAffinityFunctionNodesSelfTest;
-import org.apache.ignite.cache.affinity.fair.FairAffinityFunctionSelfTest;
-import org.apache.ignite.cache.affinity.fair.FairAffinityNodesRestart;
-import org.apache.ignite.cache.affinity.local.LocalAffinityFunctionTest;
 import org.apache.ignite.cache.store.GridCacheBalancingStoreSelfTest;
 import org.apache.ignite.cache.store.GridCacheLoadOnlyStoreAdapterSelfTest;
 import org.apache.ignite.cache.store.StoreResourceInjectionSelfTest;
@@ -38,12 +31,12 @@ import org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreOptimizedMarshallerW
 import org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreTest;
 import org.apache.ignite.cache.store.jdbc.GridCacheJdbcBlobStoreMultithreadedSelfTest;
 import org.apache.ignite.cache.store.jdbc.GridCacheJdbcBlobStoreSelfTest;
+import org.apache.ignite.cache.store.jdbc.JdbcTypesDefaultTransformerTest;
 import org.apache.ignite.internal.managers.communication.IgniteCommunicationBalanceMultipleConnectionsTest;
 import org.apache.ignite.internal.managers.communication.IgniteCommunicationBalancePairedConnectionsTest;
 import org.apache.ignite.internal.managers.communication.IgniteCommunicationBalanceTest;
 import org.apache.ignite.internal.managers.communication.IgniteIoTestMessagesTest;
 import org.apache.ignite.internal.managers.communication.IgniteVariousConnectionNumberTest;
-import org.apache.ignite.cache.store.jdbc.JdbcTypesDefaultTransformerTest;
 import org.apache.ignite.internal.processors.cache.CacheAffinityCallSelfTest;
 import org.apache.ignite.internal.processors.cache.CacheDeferredDeleteQueueTest;
 import org.apache.ignite.internal.processors.cache.CacheDeferredDeleteSanitySelfTest;

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9b64246/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
index 1c50908..f902242 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
@@ -18,8 +18,6 @@
 package org.apache.ignite.testsuites;
 
 import junit.framework.TestSuite;
-import org.apache.ignite.cache.affinity.fair.FairAffinityFunctionBackupFilterSelfTest;
-import org.apache.ignite.cache.affinity.fair.FairAffinityFunctionExcludeNeighborsSelfTest;
 import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunctionBackupFilterSelfTest;
 import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunctionExcludeNeighborsSelfTest;
 import org.apache.ignite.internal.processors.cache.CacheConcurrentReadThroughTest;
@@ -173,7 +171,6 @@ public class IgniteCacheTestSuite2 extends TestSuite {
         suite.addTest(new TestSuite(GridCacheAtomicNearReadersSelfTest.class));
         suite.addTest(new TestSuite(GridCachePartitionedAffinitySelfTest.class));
         suite.addTest(new TestSuite(RendezvousAffinityFunctionExcludeNeighborsSelfTest.class));
-        suite.addTest(new TestSuite(FairAffinityFunctionExcludeNeighborsSelfTest.class));
         suite.addTest(new TestSuite(GridCacheRendezvousAffinityClientSelfTest.class));
         suite.addTest(new TestSuite(GridCachePartitionedProjectionAffinitySelfTest.class));
         suite.addTest(new TestSuite(GridCachePartitionedBasicOpSelfTest.class));
@@ -210,7 +207,6 @@ public class IgniteCacheTestSuite2 extends TestSuite {
         suite.addTest(new TestSuite(GridCacheDhtPreloadStartStopSelfTest.class));
         suite.addTest(new TestSuite(GridCacheDhtPreloadUnloadSelfTest.class));
         suite.addTest(new TestSuite(RendezvousAffinityFunctionBackupFilterSelfTest.class));
-        suite.addTest(new TestSuite(FairAffinityFunctionBackupFilterSelfTest.class));
         suite.addTest(new TestSuite(GridCachePartitionedPreloadLifecycleSelfTest.class));
         suite.addTest(new TestSuite(CacheLoadingConcurrentGridStartSelfTest.class));
         suite.addTest(new TestSuite(CacheLoadingConcurrentGridStartSelfTestAllowOverwrite.class));