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

[28/41] ignite git commit: ignite-5446 Alway use late affinity assignment mode

ignite-5446 Alway use late affinity assignment mode


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

Branch: refs/heads/ignite-5578-1
Commit: a53544410dd15a3a5112d6de88648db21bd3fcf3
Parents: 17904cb
Author: sboikov <sb...@gridgain.com>
Authored: Mon Jul 10 11:38:04 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Jul 10 11:38:04 2017 +0300

----------------------------------------------------------------------
 .../configuration/IgniteConfiguration.java      |  15 +-
 .../affinity/AffinityHistoryCleanupTest.java    | 182 -------------------
 ...idAbstractCacheInterceptorRebalanceTest.java |   2 -
 .../cache/GridCacheDeploymentSelfTest.java      |   7 +-
 .../IgniteCacheP2pUnmarshallingTxErrorTest.java |  22 +--
 ...ffinityAssignmentNodeJoinValidationTest.java |  46 +----
 .../CacheLateAffinityAssignmentTest.java        |   2 -
 ...CacheLoadingConcurrentGridStartSelfTest.java |   2 -
 ...idCachePartitionedPreloadEventsSelfTest.java | 143 ---------------
 ...LateAffDisabledMultiNodeFullApiSelfTest.java |  35 ----
 ...LateAffDisabledMultiNodeFullApiSelfTest.java |  34 ----
 .../db/IgnitePdsWholeClusterRestartTest.java    |   1 -
 .../GridActivationPartitionedCacheSuit.java     |   2 -
 ...ContinuousQueryFailoverAbstractSelfTest.java |   2 -
 .../processors/igfs/IgfsStreamsSelfTest.java    |   1 -
 .../IgniteCacheFullApiSelfTestSuite.java        |   6 -
 .../testsuites/IgniteCacheTestSuite2.java       |   2 -
 17 files changed, 17 insertions(+), 487 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/a5354441/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
index ed05fa4..cafa675 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
@@ -191,6 +191,7 @@ public class IgniteConfiguration {
     public static final boolean DFLT_CACHE_SANITY_CHECK_ENABLED = true;
 
     /** Default value for late affinity assignment flag. */
+    @Deprecated
     public static final boolean DFLT_LATE_AFF_ASSIGNMENT = true;
 
     /** Default value for active on start flag. */
@@ -452,9 +453,6 @@ public class IgniteConfiguration {
     /** Custom executor configurations. */
     private ExecutorConfiguration[] execCfgs;
 
-    /** */
-    private boolean lateAffAssignment = DFLT_LATE_AFF_ASSIGNMENT;
-
     /** Page memory configuration. */
     private MemoryConfiguration memCfg;
 
@@ -530,7 +528,6 @@ public class IgniteConfiguration {
         igniteWorkDir = cfg.getWorkDirectory();
         inclEvtTypes = cfg.getIncludeEventTypes();
         includeProps = cfg.getIncludeProperties();
-        lateAffAssignment = cfg.isLateAffinityAssignment();
         lifecycleBeans = cfg.getLifecycleBeans();
         locHost = cfg.getLocalHost();
         log = cfg.getGridLogger();
@@ -2721,14 +2718,14 @@ public class IgniteConfiguration {
      * from assignment calculated by {@link AffinityFunction#assignPartitions}.
      * <p>
      * This property should have the same value for all nodes in cluster.
-     * <p>
-     * If not provided, default value is {@link #DFLT_LATE_AFF_ASSIGNMENT}.
      *
      * @return Late affinity assignment flag.
      * @see AffinityFunction
+     * @deprecated Starting from Ignite 2.1 late affinity assignment is always enabled.
      */
+    @Deprecated
     public boolean isLateAffinityAssignment() {
-        return lateAffAssignment;
+        return true;
     }
 
     /**
@@ -2736,10 +2733,10 @@ public class IgniteConfiguration {
      *
      * @param lateAffAssignment Late affinity assignment flag.
      * @return {@code this} for chaining.
+     * @deprecated Starting from Ignite 2.1 late affinity assignment is always enabled.
      */
+    @Deprecated
     public IgniteConfiguration setLateAffinityAssignment(boolean lateAffAssignment) {
-        this.lateAffAssignment = lateAffAssignment;
-
         return this;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a5354441/modules/core/src/test/java/org/apache/ignite/cache/affinity/AffinityHistoryCleanupTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/affinity/AffinityHistoryCleanupTest.java b/modules/core/src/test/java/org/apache/ignite/cache/affinity/AffinityHistoryCleanupTest.java
index 87c2050..605cc5f 100644
--- a/modules/core/src/test/java/org/apache/ignite/cache/affinity/AffinityHistoryCleanupTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/cache/affinity/AffinityHistoryCleanupTest.java
@@ -47,9 +47,6 @@ public class AffinityHistoryCleanupTest extends GridCommonAbstractTest {
     /** */
     private boolean client;
 
-    /** */
-    private boolean lateAffAssignment;
-
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
@@ -71,8 +68,6 @@ public class AffinityHistoryCleanupTest extends GridCommonAbstractTest {
 
         cfg.setClientMode(client);
 
-        cfg.setLateAffinityAssignment(lateAffAssignment);
-
         return cfg;
     }
 
@@ -96,183 +91,6 @@ public class AffinityHistoryCleanupTest extends GridCommonAbstractTest {
 
             checkHistory(ignite, F.asList(topVer(1, 0)), 1);
 
-            for (int i = 0; i < 3; i++) {
-                startGrid(1);
-
-                stopGrid(1);
-            }
-
-            checkHistory(ignite, F.asList(
-                topVer(3, 0),
-                topVer(4, 0),
-                topVer(5, 0),
-                topVer(6, 0),
-                topVer(7, 0)),
-                5);
-
-            client = true;
-
-            startGrid(1);
-
-            stopGrid(1);
-
-            checkHistory(ignite, F.asList(
-                topVer(3, 0),
-                topVer(4, 0),
-                topVer(5, 0),
-                topVer(6, 0),
-                topVer(7, 0),
-                topVer(8, 0),
-                topVer(9, 0)),
-                5);
-
-            startGrid(1);
-
-            stopGrid(1);
-
-            checkHistory(ignite, F.asList(
-                topVer(3, 0),
-                topVer(4, 0),
-                topVer(5, 0),
-                topVer(6, 0),
-                topVer(7, 0),
-                topVer(8, 0),
-                topVer(9, 0),
-                topVer(10, 0),
-                topVer(11, 0)),
-                5);
-
-            startGrid(1);
-
-            checkHistory(ignite, F.asList(
-                topVer(3, 0),
-                topVer(4, 0),
-                topVer(5, 0),
-                topVer(6, 0),
-                topVer(7, 0),
-                topVer(8, 0),
-                topVer(9, 0),
-                topVer(10, 0),
-                topVer(11, 0),
-                topVer(12, 0)),
-                5);
-
-            stopGrid(1);
-
-            checkHistory(ignite, F.asList(
-                topVer(8, 0),
-                topVer(9, 0),
-                topVer(10, 0),
-                topVer(11, 0),
-                topVer(12, 0),
-                topVer(13, 0)),
-                0);
-
-            client = false;
-
-            startGrid(1);
-
-            stopGrid(1);
-
-            checkHistory(ignite, F.asList(
-                topVer(8, 0),
-                topVer(9, 0),
-                topVer(10, 0),
-                topVer(11, 0),
-                topVer(12, 0),
-                topVer(13, 0),
-                topVer(14, 0),
-                topVer(15, 0)),
-                2);
-
-            startGrid(1);
-
-            stopGrid(1);
-
-            checkHistory(ignite, F.asList(
-                topVer(8, 0),
-                topVer(9, 0),
-                topVer(10, 0),
-                topVer(11, 0),
-                topVer(12, 0),
-                topVer(13, 0),
-                topVer(14, 0),
-                topVer(15, 0),
-                topVer(16, 0),
-                topVer(17, 0)),
-                4);
-
-            startGrid(1);
-
-            checkHistory(ignite, F.asList(
-                topVer(13, 0),
-                topVer(14, 0),
-                topVer(15, 0),
-                topVer(16, 0),
-                topVer(17, 0),
-                topVer(18, 0)),
-                5);
-
-            stopGrid(1);
-
-            checkHistory(ignite, F.asList(
-                topVer(14, 0),
-                topVer(15, 0),
-                topVer(16, 0),
-                topVer(17, 0),
-                topVer(18, 0),
-                topVer(19, 0)),
-                6);
-
-            startGrid(1);
-
-            checkHistory(ignite, F.asList(
-                topVer(16, 0),
-                topVer(17, 0),
-                topVer(18, 0),
-                topVer(19, 0),
-                topVer(20, 0)),
-                5);
-
-            client = true;
-
-            startGrid(2);
-
-            stopGrid(2);
-
-            checkHistory(ignite, F.asList(
-                topVer(16, 0),
-                topVer(17, 0),
-                topVer(18, 0),
-                topVer(19, 0),
-                topVer(20, 0),
-                topVer(21, 0),
-                topVer(22, 0)),
-                5);
-        }
-        finally {
-            if (histProp != null)
-                System.setProperty(IgniteSystemProperties.IGNITE_AFFINITY_HISTORY_SIZE, histProp);
-            else
-                System.clearProperty(IgniteSystemProperties.IGNITE_AFFINITY_HISTORY_SIZE);
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testAffinityHistoryCleanupLateAffinityAssignment() throws Exception {
-        lateAffAssignment = true;
-
-        String histProp = System.getProperty(IgniteSystemProperties.IGNITE_AFFINITY_HISTORY_SIZE);
-
-        try {
-            System.setProperty(IgniteSystemProperties.IGNITE_AFFINITY_HISTORY_SIZE, "5");
-
-            Ignite ignite = startGrid(0);
-
-            checkHistory(ignite, F.asList(topVer(1, 0)), 1);
-
             startGrid(1);
 
             checkHistory(ignite, F.asList(

http://git-wip-us.apache.org/repos/asf/ignite/blob/a5354441/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridAbstractCacheInterceptorRebalanceTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridAbstractCacheInterceptorRebalanceTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridAbstractCacheInterceptorRebalanceTest.java
index adfe085..99cf1f1 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridAbstractCacheInterceptorRebalanceTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridAbstractCacheInterceptorRebalanceTest.java
@@ -75,8 +75,6 @@ public abstract class GridAbstractCacheInterceptorRebalanceTest extends GridComm
     @Override protected IgniteConfiguration getConfiguration(final String igniteInstanceName) throws Exception {
         final IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
 
-        cfg.setLateAffinityAssignment(true);
-
         final CacheConfiguration<Integer, Integer> ccfg = new CacheConfiguration<>(CACHE_NAME);
 
         assertNotNull(interceptor);

http://git-wip-us.apache.org/repos/asf/ignite/blob/a5354441/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentSelfTest.java
index c88d0cc..ff3ab36 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentSelfTest.java
@@ -92,8 +92,6 @@ public class GridCacheDeploymentSelfTest extends GridCommonAbstractTest {
 
         cfg.setConnectorConfiguration(null);
 
-        cfg.setLateAffinityAssignment(false);
-
         return cfg;
     }
 
@@ -510,7 +508,10 @@ public class GridCacheDeploymentSelfTest extends GridCommonAbstractTest {
      * @return Key with described properties.
      * @throws IllegalStateException if such a key could not be found after 10000 iterations.
      */
-    private int getNextKey(int start, Ignite g, ClusterNode primary, ClusterNode backup, ClusterNode near) {
+    private int getNextKey(int start, Ignite g, ClusterNode primary, ClusterNode backup, ClusterNode near)
+        throws Exception {
+        awaitPartitionMapExchange();
+
         info("Primary: " + primary);
         info("Backup: " + backup);
         info("Near: " + near);

http://git-wip-us.apache.org/repos/asf/ignite/blob/a5354441/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingTxErrorTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingTxErrorTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingTxErrorTest.java
index 1a88d80..ba77c70 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingTxErrorTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingTxErrorTest.java
@@ -22,7 +22,6 @@ import javax.cache.CacheException;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.cache.CacheAtomicityMode;
-import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.transactions.Transaction;
 
@@ -39,26 +38,13 @@ public class IgniteCacheP2pUnmarshallingTxErrorTest extends IgniteCacheP2pUnmars
         return CacheAtomicityMode.TRANSACTIONAL;
     }
 
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
-
-        cfg.setLateAffinityAssignment(false);
-
-        if (!igniteInstanceName.endsWith("0"))
-            cfg.getCacheConfiguration()[0].setRebalanceDelay(-1); // Allows to check GridDhtLockRequest fail.
-
-        return cfg;
-    }
-
     /**
      * Sends put with optimistic lock and handles fail.
      */
-    protected void failOptimistic() {
+    private void failOptimistic() {
         IgniteCache<Object, Object> cache = jcache(0);
 
         try (Transaction tx = grid(0).transactions().txStart(OPTIMISTIC, REPEATABLE_READ)) {
-
             cache.put(new TestKey(String.valueOf(++key)), "");
 
             tx.commit();
@@ -75,12 +61,10 @@ public class IgniteCacheP2pUnmarshallingTxErrorTest extends IgniteCacheP2pUnmars
     /**
      * Sends put with pessimistic lock and handles fail.
      */
-    protected void failPessimictic() {
+    private void failPessimictic() {
         IgniteCache<Object, Object> cache = jcache(0);
 
-        try (Transaction tx = grid(0).transactions().txStart(PESSIMISTIC,
-            REPEATABLE_READ)) {
-
+        try (Transaction tx = grid(0).transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
             cache.put(new TestKey(String.valueOf(++key)), "");
 
             assert false : "p2p marshalling failed, but error response was not sent";

http://git-wip-us.apache.org/repos/asf/ignite/blob/a5354441/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLateAffinityAssignmentNodeJoinValidationTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLateAffinityAssignmentNodeJoinValidationTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLateAffinityAssignmentNodeJoinValidationTest.java
index 11ac063..48b33b6 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLateAffinityAssignmentNodeJoinValidationTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLateAffinityAssignmentNodeJoinValidationTest.java
@@ -19,8 +19,6 @@ package org.apache.ignite.internal.processors.cache.distributed;
 
 import org.apache.ignite.Ignite;
 import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.util.typedef.X;
-import org.apache.ignite.spi.IgniteSpiException;
 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;
@@ -77,7 +75,8 @@ public class CacheLateAffinityAssignmentNodeJoinValidationTest extends GridCommo
      * @param firstEnabled Flag value for first started node.
      * @throws Exception If failed.
      */
-    public void checkNodeJoinValidation(boolean firstEnabled) throws Exception {
+    private void checkNodeJoinValidation(boolean firstEnabled) throws Exception {
+        // LateAffinity should be always enabled, setLateAffinityAssignment should be ignored.
         lateAff = firstEnabled;
 
         Ignite ignite = startGrid(0);
@@ -86,49 +85,12 @@ public class CacheLateAffinityAssignmentNodeJoinValidationTest extends GridCommo
 
         lateAff = !firstEnabled;
 
-        try {
-            startGrid(1);
-
-            fail();
-        }
-        catch (Exception e) {
-            checkError(e);
-        }
-
-        client = true;
-
-        try {
-            startGrid(1);
-
-            fail();
-        }
-        catch (Exception e) {
-            checkError(e);
-        }
-
-        assertEquals(1, ignite.cluster().nodes().size());
-
-        lateAff = firstEnabled;
-
-        client = false;
-
         startGrid(1);
 
         client = true;
 
-        Ignite client = startGrid(2);
-
-        assertTrue(client.configuration().isClientMode());
-    }
-
-    /**
-     * @param e Error.
-     */
-    private void checkError(Exception e) {
-        IgniteSpiException err = X.cause(e, IgniteSpiException.class);
+        startGrid(2);
 
-        assertNotNull(err);
-        assertTrue(err.getMessage().contains("Local node's cache affinity assignment mode differs " +
-            "from the same property on remote node"));
+        assertEquals(3, ignite.cluster().nodes().size());
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/a5354441/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLateAffinityAssignmentTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLateAffinityAssignmentTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLateAffinityAssignmentTest.java
index 46520ca..6174209 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLateAffinityAssignmentTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLateAffinityAssignmentTest.java
@@ -142,8 +142,6 @@ public class CacheLateAffinityAssignmentTest extends GridCommonAbstractTest {
     @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
 
-        cfg.setLateAffinityAssignment(true);
-
         TestRecordingCommunicationSpi commSpi;
 
         if (spiC != null)

http://git-wip-us.apache.org/repos/asf/ignite/blob/a5354441/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLoadingConcurrentGridStartSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLoadingConcurrentGridStartSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLoadingConcurrentGridStartSelfTest.java
index 4f1b090..68e88ce 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLoadingConcurrentGridStartSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLoadingConcurrentGridStartSelfTest.java
@@ -79,8 +79,6 @@ public class CacheLoadingConcurrentGridStartSelfTest extends GridCommonAbstractT
     @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
 
-        cfg.setLateAffinityAssignment(true);
-
         ((TcpCommunicationSpi)cfg.getCommunicationSpi()).setSharedMemoryPort(-1);
 
         CacheConfiguration ccfg = new CacheConfiguration(DEFAULT_CACHE_NAME);

http://git-wip-us.apache.org/repos/asf/ignite/blob/a5354441/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedPreloadEventsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedPreloadEventsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedPreloadEventsSelfTest.java
deleted file mode 100644
index bc62a72..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedPreloadEventsSelfTest.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.dht;
-
-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.UUID;
-import org.apache.ignite.Ignite;
-import org.apache.ignite.IgniteCache;
-import org.apache.ignite.cache.CacheMode;
-import org.apache.ignite.cache.affinity.AffinityFunction;
-import org.apache.ignite.cache.affinity.AffinityFunctionContext;
-import org.apache.ignite.cluster.ClusterNode;
-import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.events.Event;
-import org.apache.ignite.internal.processors.cache.distributed.GridCachePreloadEventsAbstractSelfTest;
-import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtForceKeysFuture;
-import org.apache.ignite.internal.util.typedef.F;
-
-import static org.apache.ignite.cache.CacheMode.PARTITIONED;
-import static org.apache.ignite.events.EventType.EVT_CACHE_REBALANCE_OBJECT_LOADED;
-
-/**
- *
- */
-public class GridCachePartitionedPreloadEventsSelfTest extends GridCachePreloadEventsAbstractSelfTest {
-    /** */
-    private boolean replicatedAffinity = true;
-
-    /** */
-    private long rebalanceDelay;
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
-
-        // 'testForcePreload' is not valid with late assignment.
-        cfg.setLateAffinityAssignment(false);
-
-        return cfg;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected CacheConfiguration cacheConfiguration() {
-        CacheConfiguration cacheCfg = super.cacheConfiguration();
-
-        if (replicatedAffinity)
-            // replicate entries to all nodes
-            cacheCfg.setAffinity(notSerializableProxy(new AffinityFunction() {
-                /** {@inheritDoc} */
-                @Override public void reset() {
-                }
-
-                /** {@inheritDoc} */
-                @Override public int partitions() {
-                    return 1;
-                }
-
-                /** {@inheritDoc} */
-                @Override public int partition(Object key) {
-                    return 0;
-                }
-
-                /** {@inheritDoc} */
-                @Override public List<List<ClusterNode>> assignPartitions(AffinityFunctionContext affCtx) {
-                    List<ClusterNode> nodes = new ArrayList<>(affCtx.currentTopologySnapshot());
-
-                    return Collections.singletonList(nodes);
-                }
-
-                /** {@inheritDoc} */
-                @Override public void removeNode(UUID nodeId) {
-                }
-            }, AffinityFunction.class));
-
-        cacheCfg.setRebalanceDelay(rebalanceDelay);
-
-        return cacheCfg;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected CacheMode getCacheMode() {
-        return PARTITIONED;
-    }
-
-    /**
-     * Test events fired from
-     * {@link GridDhtForceKeysFuture}
-     *
-     * @throws Exception if failed.
-     */
-    public void testForcePreload() throws Exception {
-        replicatedAffinity = false;
-        rebalanceDelay = -1;
-
-        Ignite g1 = startGrid("g1");
-
-        Collection<Integer> keys = new HashSet<>();
-
-        IgniteCache<Integer, String> cache = g1.cache(DEFAULT_CACHE_NAME);
-
-        for (int i = 0; i < 100; i++) {
-            keys.add(i);
-            cache.put(i, "val");
-        }
-
-        Ignite g2 = startGrid("g2");
-
-        Map<ClusterNode, Collection<Object>> keysMap = g1.affinity(DEFAULT_CACHE_NAME).mapKeysToNodes(keys);
-        Collection<Object> g2Keys = keysMap.get(g2.cluster().localNode());
-
-        assertNotNull(g2Keys);
-        assertFalse("There are no keys assigned to g2", g2Keys.isEmpty());
-
-        for (Object key : g2Keys)
-            // Need to force keys loading.
-            assertEquals("val", g2.cache(DEFAULT_CACHE_NAME).getAndPut(key, "changed val"));
-
-        Collection<Event> evts = g2.events().localQuery(F.<Event>alwaysTrue(), EVT_CACHE_REBALANCE_OBJECT_LOADED);
-
-        checkPreloadEvents(evts, g2, g2Keys);
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/a5354441/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicLateAffDisabledMultiNodeFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicLateAffDisabledMultiNodeFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicLateAffDisabledMultiNodeFullApiSelfTest.java
deleted file mode 100644
index 267de66..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicLateAffDisabledMultiNodeFullApiSelfTest.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.configuration.IgniteConfiguration;
-
-/**
- *
- */
-public class GridCacheAtomicLateAffDisabledMultiNodeFullApiSelfTest extends
-    GridCacheAtomicMultiNodeFullApiSelfTest {
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
-
-        cfg.setLateAffinityAssignment(false);
-
-        return cfg;
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a5354441/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedLateAffDisabledMultiNodeFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedLateAffDisabledMultiNodeFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedLateAffDisabledMultiNodeFullApiSelfTest.java
deleted file mode 100644
index 025b68b..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedLateAffDisabledMultiNodeFullApiSelfTest.java
+++ /dev/null
@@ -1,34 +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.configuration.IgniteConfiguration;
-
-/**
- *
- */
-public class GridCachePartitionedLateAffDisabledMultiNodeFullApiSelfTest extends GridCachePartitionedMultiNodeFullApiSelfTest {
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
-
-        cfg.setLateAffinityAssignment(false);
-
-        return cfg;
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a5354441/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsWholeClusterRestartTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsWholeClusterRestartTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsWholeClusterRestartTest.java
index b512a64..c8ec304 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsWholeClusterRestartTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/IgnitePdsWholeClusterRestartTest.java
@@ -76,7 +76,6 @@ public class IgnitePdsWholeClusterRestartTest extends GridCommonAbstractTest {
         ccfg1.setAffinity(new RendezvousAffinityFunction(false, 32));
         ccfg1.setBackups(2);
 
-        cfg.setLateAffinityAssignment(false);
         cfg.setActiveOnStart(false);
 
         // To avoid hostname lookup on start.

http://git-wip-us.apache.org/repos/asf/ignite/blob/a5354441/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/extended/GridActivationPartitionedCacheSuit.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/extended/GridActivationPartitionedCacheSuit.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/extended/GridActivationPartitionedCacheSuit.java
index c74aada..303725f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/extended/GridActivationPartitionedCacheSuit.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/extended/GridActivationPartitionedCacheSuit.java
@@ -28,7 +28,6 @@ import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheNea
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedClientOnlyNoPrimaryFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedCopyOnReadDisabledMultiNodeFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedFullApiSelfTest;
-import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedLateAffDisabledMultiNodeFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedMultiNodeFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedMultiNodeP2PDisabledFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedNearOnlyNoPrimaryFullApiSelfTest;
@@ -45,7 +44,6 @@ public class GridActivationPartitionedCacheSuit extends GridActivationCacheAbstr
         addTest(GridCachePartitionedClientOnlyNoPrimaryFullApiSelfTest.class);
         addTest(GridCachePartitionedCopyOnReadDisabledMultiNodeFullApiSelfTest.class);
         addTest(GridCachePartitionedFullApiSelfTest.class);
-        addTest(GridCachePartitionedLateAffDisabledMultiNodeFullApiSelfTest.class);
         addTest(GridCachePartitionedMultiNodeFullApiSelfTest.class);
         addTest(GridCachePartitionedMultiNodeP2PDisabledFullApiSelfTest.class);
         addTest(GridCachePartitionedNearDisabledFullApiSelfTest.class);

http://git-wip-us.apache.org/repos/asf/ignite/blob/a5354441/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverAbstractSelfTest.java
index 937a059..43069cd 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryFailoverAbstractSelfTest.java
@@ -134,8 +134,6 @@ public abstract class CacheContinuousQueryFailoverAbstractSelfTest extends GridC
     @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
 
-        cfg.setLateAffinityAssignment(true);
-
         ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setForceServerMode(true);
         ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a5354441/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsStreamsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsStreamsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsStreamsSelfTest.java
index d77296a..e811604 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsStreamsSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsStreamsSelfTest.java
@@ -116,7 +116,6 @@ public class IgfsStreamsSelfTest extends IgfsCommonAbstractTest {
     @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
 
-        cfg.setLateAffinityAssignment(false);
         cfg.setCacheConfiguration();
 
         TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();

http://git-wip-us.apache.org/repos/asf/ignite/blob/a5354441/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 11a4a10..164ff6a 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
@@ -36,7 +36,6 @@ import org.apache.ignite.internal.processors.cache.distributed.near.CachePartiti
 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.GridCacheAtomicLateAffDisabledMultiNodeFullApiSelfTest;
 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.GridCacheAtomicMultiNodeWithGroupFullApiSelfTest;
@@ -57,7 +56,6 @@ import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePar
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedFilteredPutSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedFullApiMultithreadedSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedFullApiSelfTest;
-import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedLateAffDisabledMultiNodeFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedMultiNodeCounterSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedMultiNodeFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedMultiNodeP2PDisabledFullApiSelfTest;
@@ -155,10 +153,6 @@ public class IgniteCacheFullApiSelfTestSuite extends TestSuite {
         suite.addTestSuite(GridCachePartitionedNearDisabledAtomicOnheapMultiNodeFullApiSelfTest.class);
         suite.addTestSuite(GridCacheAtomicOnheapMultiNodeFullApiSelfTest.class);
 
-        // Old affinity assignment mode.
-        suite.addTestSuite(GridCachePartitionedLateAffDisabledMultiNodeFullApiSelfTest.class);
-        suite.addTestSuite(GridCacheAtomicLateAffDisabledMultiNodeFullApiSelfTest.class);
-
         // Multithreaded.
         suite.addTestSuite(GridCacheLocalFullApiMultithreadedSelfTest.class);
         suite.addTestSuite(GridCacheReplicatedFullApiMultithreadedSelfTest.class);

http://git-wip-us.apache.org/repos/asf/ignite/blob/a5354441/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 4c9accf..9ed7ee3 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
@@ -78,7 +78,6 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.GridCacheDhtP
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridCacheDhtPreloadStartStopSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridCacheDhtPreloadUnloadSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridCachePartitionedNearDisabledLockSelfTest;
-import org.apache.ignite.internal.processors.cache.distributed.dht.GridCachePartitionedPreloadEventsSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridCachePartitionedTopologyChangeSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridCachePartitionedUnloadEventsSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.IgniteCacheClearDuringRebalanceTest;
@@ -233,7 +232,6 @@ public class IgniteCacheTestSuite2 extends TestSuite {
         suite.addTest(new TestSuite(GridCacheAtomicNearEvictionEventSelfTest.class));
 
         suite.addTest(new TestSuite(GridCachePartitionedTopologyChangeSelfTest.class));
-        suite.addTest(new TestSuite(GridCachePartitionedPreloadEventsSelfTest.class));
         suite.addTest(new TestSuite(GridCachePartitionedUnloadEventsSelfTest.class));
         suite.addTest(new TestSuite(GridCacheColocatedOptimisticTransactionSelfTest.class));
         suite.addTestSuite(GridCacheAtomicMessageCountSelfTest.class);