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 2016/02/29 12:02:51 UTC

[1/4] ignite git commit: ignite-2521: Configuration variations tests framework + IgniteCacheBasicConfigVariationsFullApiTestSuite + ignite-2554: Fixed Affinity.mapKeyToNode() for dynamically started LOCAL cache

Repository: ignite
Updated Branches:
  refs/heads/master 49dcd6b96 -> 953b575f0


http://git-wip-us.apache.org/repos/asf/ignite/blob/953b575f/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteCacheConfigVariationsAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteCacheConfigVariationsAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteCacheConfigVariationsAbstractTest.java
new file mode 100644
index 0000000..28c6f55
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteCacheConfigVariationsAbstractTest.java
@@ -0,0 +1,583 @@
+/*
+ * 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.testframework.junits;
+
+import java.util.Map;
+import javax.cache.Cache;
+import javax.cache.configuration.Factory;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteTransactions;
+import org.apache.ignite.Ignition;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMemoryMode;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.CachePeekMode;
+import org.apache.ignite.cache.store.CacheStore;
+import org.apache.ignite.cache.store.CacheStoreAdapter;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.NearCacheConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.IgniteKernal;
+import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.util.lang.GridAbsPredicateX;
+import org.apache.ignite.internal.util.typedef.X;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteBiInClosure;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.configvariations.CacheStartMode;
+import org.apache.ignite.transactions.Transaction;
+import org.jetbrains.annotations.Nullable;
+import org.jsr166.ConcurrentHashMap8;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheMemoryMode.OFFHEAP_TIERED;
+import static org.apache.ignite.cache.CacheMemoryMode.ONHEAP_TIERED;
+
+/**
+ * Abstract class for cache configuration variations tests.
+ */
+public abstract class IgniteCacheConfigVariationsAbstractTest extends IgniteConfigVariationsAbstractTest {
+    /** */
+    protected static final int CLIENT_NEAR_ONLY_IDX = 2;
+
+    /** Test timeout. */
+    private static final long TEST_TIMEOUT = 30 * 1000;
+
+    /** Store map. */
+    protected static final Map<Object, Object> map = new ConcurrentHashMap8<>();
+
+    /** {@inheritDoc} */
+    @Override protected long getTestTimeout() {
+        return TEST_TIMEOUT;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected final void beforeTestsStarted() throws Exception {
+        assert testsCfg != null;
+        assert !testsCfg.withClients() || testsCfg.gridCount() >= 3;
+
+        assert testsCfg.testedNodeIndex() >= 0 : "testedNodeIdx: " + testedNodeIdx;
+
+        testedNodeIdx = testsCfg.testedNodeIndex();
+
+        if (testsCfg.isStartCache()) {
+            final CacheStartMode cacheStartMode = testsCfg.cacheStartMode();
+            final int cnt = testsCfg.gridCount();
+
+            if (cacheStartMode == CacheStartMode.STATIC) {
+                info("All nodes will be stopped, new " + cnt + " nodes will be started.");
+
+                Ignition.stopAll(true);
+
+                for (int i = 0; i < cnt; i++) {
+                    String gridName = getTestGridName(i);
+
+                    IgniteConfiguration cfg = optimize(getConfiguration(gridName));
+
+                    if (i != CLIENT_NODE_IDX && i != CLIENT_NEAR_ONLY_IDX) {
+                        CacheConfiguration cc = testsCfg.configurationFactory().cacheConfiguration(gridName);
+
+                        cc.setName(cacheName());
+
+                        cfg.setCacheConfiguration(cc);
+                    }
+
+                    startGrid(gridName, cfg, null);
+                }
+
+                if (testsCfg.withClients() && testsCfg.gridCount() > CLIENT_NEAR_ONLY_IDX)
+                    grid(CLIENT_NEAR_ONLY_IDX).createNearCache(cacheName(), new NearCacheConfiguration());
+            }
+            else if (cacheStartMode == null || cacheStartMode == CacheStartMode.DYNAMIC) {
+                super.beforeTestsStarted();
+
+                startCachesDinamically();
+            }
+            else
+                throw new IllegalArgumentException("Unknown cache start mode: " + cacheStartMode);
+        }
+
+        if (testsCfg.gridCount() > 1)
+            checkTopology(testsCfg.gridCount());
+
+        awaitPartitionMapExchange();
+
+        for (int i = 0; i < gridCount(); i++)
+            info("Grid " + i + ": " + grid(i).localNode().id());
+
+        if (testsCfg.withClients()) {
+            boolean testedNodeNearEnabled = grid(testedNodeIdx).cachex(cacheName()).context().isNear();
+
+            if (testedNodeIdx != SERVER_NODE_IDX)
+                assertEquals(testedNodeIdx == CLIENT_NEAR_ONLY_IDX, testedNodeNearEnabled);
+
+            info(">>> Starting set of tests [testedNodeIdx=" + testedNodeIdx
+                + ", id=" + grid(testedNodeIdx).localNode().id()
+                + ", isClient=" + grid(testedNodeIdx).configuration().isClientMode()
+                + ", nearEnabled=" + testedNodeNearEnabled + "]");
+        }
+    }
+
+    /**
+     * Starts caches dinamically.
+     */
+    private void startCachesDinamically() throws Exception {
+        for (int i = 0; i < gridCount(); i++) {
+            info("Starting cache dinamically on grid: " + i);
+
+            IgniteEx grid = grid(i);
+
+            if (i != CLIENT_NODE_IDX && i != CLIENT_NEAR_ONLY_IDX) {
+                CacheConfiguration cc = testsCfg.configurationFactory().cacheConfiguration(grid.name());
+
+                cc.setName(cacheName());
+
+                grid.getOrCreateCache(cc);
+            }
+
+            if (testsCfg.withClients() && i == CLIENT_NEAR_ONLY_IDX)
+                grid(CLIENT_NEAR_ONLY_IDX).createNearCache(cacheName(), new NearCacheConfiguration());
+        }
+
+        awaitPartitionMapExchange();
+
+        for (int i = 0; i < gridCount(); i++)
+            assertNotNull(jcache(i));
+
+        for (int i = 0; i < gridCount(); i++)
+            assertEquals("Cache is not empty [idx=" + i + ", entrySet=" + jcache(i).localEntries() + ']',
+                0, jcache(i).localSize(CachePeekMode.ALL));
+    }
+
+    /** {@inheritDoc} */
+    @Override protected boolean expectedClient(String testGridName) {
+        return getTestGridName(CLIENT_NODE_IDX).equals(testGridName)
+            || getTestGridName(CLIENT_NEAR_ONLY_IDX).equals(testGridName);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        if (testsCfg.isStopCache()) {
+            for (int i = 0; i < gridCount(); i++) {
+                info("Destroing cache on grid: " + i);
+
+                IgniteCache<String, Integer> cache = jcache(i);
+
+                assert i != 0 || cache != null;
+
+                if (cache != null)
+                    cache.destroy();
+            }
+        }
+
+        map.clear();
+
+        super.afterTestsStopped();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        awaitPartitionMapExchange();
+
+        assert jcache().unwrap(Ignite.class).transactions().tx() == null;
+
+        assertEquals(0, jcache().localSize());
+        assertEquals(0, jcache().size());
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        Transaction tx = jcache().unwrap(Ignite.class).transactions().tx();
+
+        if (tx != null) {
+            tx.close();
+
+            fail("Cache transaction remained after test completion: " + tx);
+        }
+
+        String cacheIsNotEmptyMsg = null;
+
+        for (int i = 0; i < gridCount(); i++) {
+            info("Checking grid: " + i);
+
+            while (true) {
+                try {
+                    final int fi = i;
+
+                    boolean cacheIsEmpty = GridTestUtils.waitForCondition(
+                        // Preloading may happen as nodes leave, so we need to wait.
+                        new GridAbsPredicateX() {
+                            @Override public boolean applyx() throws IgniteCheckedException {
+                                jcache(fi).removeAll();
+
+                                if (jcache(fi).size(CachePeekMode.ALL) > 0) {
+                                    for (Cache.Entry<?, ?> k : jcache(fi).localEntries())
+                                        jcache(fi).remove(k.getKey());
+                                }
+
+                                int locSize = jcache(fi).localSize(CachePeekMode.ALL);
+
+                                if (locSize != 0) {
+                                    info(">>>>> Debug localSize for grid: " + fi + " is " + locSize);
+                                    info(">>>>> Debug ONHEAP  localSize for grid: " + fi + " is "
+                                        + jcache(fi).localSize(CachePeekMode.ONHEAP));
+                                    info(">>>>> Debug OFFHEAP localSize for grid: " + fi + " is "
+                                        + jcache(fi).localSize(CachePeekMode.OFFHEAP));
+                                    info(">>>>> Debug PRIMARY localSize for grid: " + fi + " is "
+                                        + jcache(fi).localSize(CachePeekMode.PRIMARY));
+                                    info(">>>>> Debug BACKUP  localSize for grid: " + fi + " is "
+                                        + jcache(fi).localSize(CachePeekMode.BACKUP));
+                                    info(">>>>> Debug NEAR    localSize for grid: " + fi + " is "
+                                        + jcache(fi).localSize(CachePeekMode.NEAR));
+                                    info(">>>>> Debug SWAP    localSize for grid: " + fi + " is "
+                                        + jcache(fi).localSize(CachePeekMode.SWAP));
+                                }
+
+                                return locSize == 0;
+                            }
+                        }, 10_000);
+
+                    if (cacheIsEmpty)
+                        assertTrue("Cache is not empty: " + " localSize = " + jcache(fi).localSize(CachePeekMode.ALL)
+                            + ", local entries " + entrySet(jcache(fi).localEntries()), cacheIsEmpty);
+
+                    int primaryKeySize = jcache(i).localSize(CachePeekMode.PRIMARY);
+                    int keySize = jcache(i).localSize();
+                    int size = jcache(i).localSize();
+                    int globalSize = jcache(i).size();
+                    int globalPrimarySize = jcache(i).size(CachePeekMode.PRIMARY);
+
+                    info("Size after [idx=" + i +
+                        ", size=" + size +
+                        ", keySize=" + keySize +
+                        ", primarySize=" + primaryKeySize +
+                        ", globalSize=" + globalSize +
+                        ", globalPrimarySize=" + globalPrimarySize +
+                        ", entrySet=" + jcache(i).localEntries() + ']');
+
+                    if (!cacheIsEmpty) {
+                        cacheIsNotEmptyMsg = "Cache is not empty: localSize = "
+                            + jcache(fi).localSize(CachePeekMode.ALL) + ", local entries "
+                            + entrySet(jcache(fi).localEntries());
+
+                        break;
+                    }
+
+                    assertEquals("Cache is not empty [idx=" + i + ", entrySet=" + jcache(i).localEntries() + ']',
+                        0, jcache(i).localSize(CachePeekMode.ALL));
+
+                    break;
+                }
+                catch (Exception e) {
+                    if (X.hasCause(e, ClusterTopologyCheckedException.class)) {
+                        info("Got topology exception while tear down (will retry in 1000ms).");
+
+                        U.sleep(1000);
+                    }
+                    else
+                        throw e;
+                }
+            }
+
+            if (cacheIsNotEmptyMsg != null)
+                break;
+
+            for (Cache.Entry entry : jcache(i).localEntries(CachePeekMode.SWAP))
+                jcache(i).remove(entry.getKey());
+        }
+
+        assert jcache().unwrap(Ignite.class).transactions().tx() == null;
+
+        if (cacheIsNotEmptyMsg == null)
+            assertEquals("Cache is not empty", 0, jcache().localSize(CachePeekMode.ALL));
+
+        resetStore();
+
+        // Restore cache if current cache has garbage.
+        if (cacheIsNotEmptyMsg != null) {
+            for (int i = 0; i < gridCount(); i++) {
+                info("Destroing cache on grid: " + i);
+
+                IgniteCache<String, Integer> cache = jcache(i);
+
+                assert i != 0 || cache != null;
+
+                if (cache != null)
+                    cache.destroy();
+            }
+
+            assertTrue(GridTestUtils.waitForCondition(new GridAbsPredicateX() {
+                @Override public boolean applyx() {
+                    for (int i = 0; i < gridCount(); i++) {
+                        if (jcache(i) != null)
+                            return false;
+                    }
+
+                    return true;
+                }
+            }, 10_000));
+
+            startCachesDinamically();
+
+            log.warning(cacheIsNotEmptyMsg);
+
+            throw new IllegalStateException(cacheIsNotEmptyMsg);
+        }
+
+        assertEquals(0, jcache().localSize());
+        assertEquals(0, jcache().size());
+    }
+
+    /**
+     * Cleans up cache store.
+     */
+    protected void resetStore() {
+        map.clear();
+    }
+
+    /**
+     * Put entry to cache store.
+     *
+     * @param key Key.
+     * @param val Value.
+     */
+    protected void putToStore(Object key, Object val) {
+        if (!storeEnabled())
+            throw new IllegalStateException("Failed to put to store because store is disabled.");
+
+        map.put(key, val);
+    }
+
+    /**
+     * @return Default cache mode.
+     */
+    protected CacheMode cacheMode() {
+        CacheMode mode = cacheConfiguration().getCacheMode();
+
+        return mode == null ? CacheConfiguration.DFLT_CACHE_MODE : mode;
+    }
+
+    /**
+     * @return Load previous value flag.
+     */
+    protected boolean isLoadPreviousValue() {
+        return cacheConfiguration().isLoadPreviousValue();
+    }
+
+    /**
+     * @return Cache atomicity mode.
+     */
+    protected CacheAtomicityMode atomicityMode() {
+        return cacheConfiguration().getAtomicityMode();
+    }
+
+    /**
+     * @return {@code True} if values should be stored off-heap.
+     */
+    protected CacheMemoryMode memoryMode() {
+        return cacheConfiguration().getMemoryMode();
+    }
+
+    /**
+     * @return {@code True} if swap should happend after localEvict() call.
+     */
+    protected boolean swapAfterLocalEvict() {
+        if (memoryMode() == OFFHEAP_TIERED)
+            return false;
+
+        return memoryMode() == ONHEAP_TIERED ? (!offheapEnabled() && swapEnabled()) : swapEnabled();
+    }
+
+    /**
+     * @return {@code True} if store is enabled.
+     */
+    protected boolean storeEnabled() {
+        return cacheConfiguration().getCacheStoreFactory() != null;
+    }
+
+    /**
+     * @return {@code True} if offheap memory is enabled.
+     */
+    protected boolean offheapEnabled() {
+        return cacheConfiguration().getOffHeapMaxMemory() >= 0;
+    }
+
+    /**
+     * @return {@code True} if swap is enabled.
+     */
+    protected boolean swapEnabled() {
+        return cacheConfiguration().isSwapEnabled();
+    }
+
+    /**
+     * @return Write through storage emulator.
+     */
+    public static CacheStore<?, ?> cacheStore() {
+        return new CacheStoreAdapter<Object, Object>() {
+            @Override public void loadCache(IgniteBiInClosure<Object, Object> clo,
+                Object... args) {
+                for (Map.Entry<Object, Object> e : map.entrySet())
+                    clo.apply(e.getKey(), e.getValue());
+            }
+
+            @Override public Object load(Object key) {
+                return map.get(key);
+            }
+
+            @Override public void write(Cache.Entry<? extends Object, ? extends Object> e) {
+                map.put(e.getKey(), e.getValue());
+            }
+
+            @Override public void delete(Object key) {
+                map.remove(key);
+            }
+        };
+    }
+
+    /**
+     * @return {@code true} if near cache should be enabled.
+     */
+    protected boolean nearEnabled() {
+        return grid(testedNodeIdx).cachex(cacheName()).context().isNear();
+    }
+
+    /**
+     * @return {@code True} if transactions are enabled.
+     * @see #txShouldBeUsed()
+     */
+    protected boolean txEnabled() {
+        return atomicityMode() == TRANSACTIONAL;
+    }
+
+    /**
+     * @return Cache configuration.
+     */
+    protected CacheConfiguration cacheConfiguration() {
+        return testsCfg.configurationFactory().cacheConfiguration(getTestGridName(testedNodeIdx));
+    }
+
+    /**
+     * @return {@code True} if transactions should be used.
+     */
+    protected boolean txShouldBeUsed() {
+        return txEnabled() && !isMultiJvm();
+    }
+
+    /**
+     * @return {@code True} if locking is enabled.
+     */
+    protected boolean lockingEnabled() {
+        return txEnabled();
+    }
+
+    /**
+     * @return Default cache instance.
+     */
+    @SuppressWarnings({"unchecked"})
+    @Override protected <K, V> IgniteCache<K, V> jcache() {
+        return jcache(testedNodeIdx);
+    }
+
+    /**
+     * @return A not near-only cache.
+     */
+    protected IgniteCache<String, Integer> serverNodeCache() {
+        return jcache(SERVER_NODE_IDX);
+    }
+
+    /**
+     * @return Cache name.
+     */
+    protected String cacheName() {
+        return "testcache-" + testsCfg.description().hashCode();
+    }
+
+    /**
+     * @return Transactions instance.
+     */
+    protected IgniteTransactions transactions() {
+        return grid(0).transactions();
+    }
+
+    /**
+     * @param idx Index of grid.
+     * @return Default cache.
+     */
+    @SuppressWarnings({"unchecked"})
+    @Override protected <K, V> IgniteCache<K, V> jcache(int idx) {
+        return ignite(idx).cache(cacheName());
+    }
+
+    /**
+     * @param idx Index of grid.
+     * @return Cache context.
+     */
+    protected GridCacheContext<String, Integer> context(final int idx) {
+        if (isRemoteJvm(idx) && !isRemoteJvm())
+            throw new UnsupportedOperationException("Operation can't be done automatically via proxy. " +
+                "Send task with this logic on remote jvm instead.");
+
+        return ((IgniteKernal)grid(idx)).<String, Integer>internalCache(cacheName()).context();
+    }
+
+    /**
+     * @param cache Cache.
+     * @return {@code True} if cache has OFFHEAP_TIERED memory mode.
+     */
+    protected static <K, V> boolean offheapTiered(IgniteCache<K, V> cache) {
+        return cache.getConfiguration(CacheConfiguration.class).getMemoryMode() == OFFHEAP_TIERED;
+    }
+
+    /**
+     * Executes regular peek or peek from swap.
+     *
+     * @param cache Cache projection.
+     * @param key Key.
+     * @return Value.
+     */
+    @Nullable protected static <K, V> V peek(IgniteCache<K, V> cache, K key) {
+        return offheapTiered(cache) ? cache.localPeek(key, CachePeekMode.SWAP, CachePeekMode.OFFHEAP) :
+            cache.localPeek(key, CachePeekMode.ONHEAP);
+    }
+
+    /**
+     * @param cache Cache.
+     * @param key Key.
+     * @return {@code True} if cache contains given key.
+     * @throws Exception If failed.
+     */
+    @SuppressWarnings("unchecked")
+    protected static boolean containsKey(IgniteCache cache, Object key) throws Exception {
+        return offheapTiered(cache) ? cache.localPeek(key, CachePeekMode.OFFHEAP) != null : cache.containsKey(key);
+    }
+
+    /**
+     * Serializable factory.
+     */
+    public static class TestStoreFactory implements Factory<CacheStore> {
+        @Override public CacheStore create() {
+            return cacheStore();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/953b575f/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteConfigVariationsAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteConfigVariationsAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteConfigVariationsAbstractTest.java
new file mode 100644
index 0000000..b22f289
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteConfigVariationsAbstractTest.java
@@ -0,0 +1,420 @@
+/*
+ * 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.testframework.junits;
+
+import java.io.Externalizable;
+import java.io.File;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.io.Serializable;
+import org.apache.commons.io.FileUtils;
+import org.apache.ignite.Ignition;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.testframework.configvariations.VariationsTestsConfig;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+/**
+ * Common abstract test for Ignite tests based on configurations variations.
+ */
+public abstract class IgniteConfigVariationsAbstractTest extends GridCommonAbstractTest {
+    /** */
+    protected static final int SERVER_NODE_IDX = 0;
+
+    /** */
+    protected static final int CLIENT_NODE_IDX = 1;
+
+    /** */
+    protected int testedNodeIdx;
+
+    /** */
+    private static final File workDir = new File(U.getIgniteHome() + File.separator + "workOfConfigVariationsTests");
+
+    /** */
+    protected VariationsTestsConfig testsCfg;
+
+    /** */
+    protected volatile DataMode dataMode;
+
+    /**
+     * @param testsCfg Tests configuration.
+     */
+    public void setTestsConfiguration(VariationsTestsConfig testsCfg) {
+        assert this.testsCfg == null : "Test config must be set only once [oldTestCfg=" + this.testsCfg
+            + ", newTestCfg=" + testsCfg + "]";
+
+        this.testsCfg = testsCfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        assert testsCfg != null;
+
+        FileUtils.deleteDirectory(workDir);
+
+        info("Ignite's 'work' directory has been cleaned.");
+
+        if (Ignition.allGrids().size() != testsCfg.gridCount()) {
+            info("All nodes will be stopped, new " + testsCfg.gridCount() + " nodes will be started.");
+
+            Ignition.stopAll(true);
+
+            startGrids(testsCfg.gridCount());
+
+            for (int i = 0; i < testsCfg.gridCount(); i++)
+                info("Grid " + i + ": " + grid(i).localNode().id());
+        }
+
+        assert testsCfg.testedNodeIndex() >= 0 : "testedNodeIdx: " + testedNodeIdx;
+
+        testedNodeIdx = testsCfg.testedNodeIndex();
+
+        if (testsCfg.withClients()) {
+            for (int i = 0; i < gridCount(); i++)
+                assertEquals("i: " + i, expectedClient(getTestGridName(i)),
+                    (boolean)grid(i).configuration().isClientMode());
+        }
+    }
+
+    /**
+     * @param testGridName Name.
+     * @return {@code True} if node is client should be client.
+     */
+    protected boolean expectedClient(String testGridName) {
+        return getTestGridName(CLIENT_NODE_IDX).equals(testGridName);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        if (testsCfg.isStopNodes()) {
+            info("Stopping all grids...");
+
+            stopAllGrids();
+
+            FileUtils.deleteDirectory(workDir);
+
+            info("Ignite's 'work' directory has been cleaned.");
+
+            memoryUsage();
+
+            System.gc();
+
+            memoryUsage();
+        }
+    }
+
+    /**
+     * Prints memory usage.
+     */
+    private void memoryUsage() {
+        int mb = 1024 * 1024;
+
+        Runtime runtime = Runtime.getRuntime();
+
+        info("##### Heap utilization statistics [MB] #####");
+        info("Used Memory  (mb): " + (runtime.totalMemory() - runtime.freeMemory()) / mb);
+        info("Free Memory  (mb): " + runtime.freeMemory() / mb);
+        info("Total Memory (mb): " + runtime.totalMemory() / mb);
+        info("Max Memory   (mb): " + runtime.maxMemory() / mb);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected String testClassDescription() {
+        return super.testClassDescription() + '-' + testsCfg.description() + '-' + testsCfg.gridCount() + "-node(s)";
+    }
+
+    /** {@inheritDoc} */
+    @Override protected String testDescription() {
+        return super.testDescription() + '-' + testsCfg.description() + '-' + testsCfg.gridCount() + "-node(s)";
+    }
+
+    /** {@inheritDoc} */
+    @Override protected final IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        IgniteConfiguration resCfg = testsCfg.configurationFactory().getConfiguration(gridName, cfg);
+
+        resCfg.setWorkDirectory(workDir.getAbsolutePath());
+
+        if (testsCfg.withClients())
+            resCfg.setClientMode(expectedClient(gridName));
+
+        return resCfg;
+    }
+
+    /** {@inheritDoc} */
+    protected final int gridCount() {
+        return testsCfg.gridCount();
+    }
+
+    /**
+     * @return Count of clients.
+     */
+    protected int clientsCount() {
+        int cnt = 0;
+
+        for (int i = 0; i < gridCount(); i++) {
+            if (grid(i).configuration().isClientMode())
+                cnt++;
+        }
+
+        return cnt;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteEx grid() {
+        throw new UnsupportedOperationException("Not supported, grid(int idx) or testedGrid() should be used instead.");
+    }
+
+    /**
+     * @return Grid which should be tested.
+     */
+    protected IgniteEx testedGrid() {
+        return grid(testedNodeIdx);
+    }
+
+    /**
+     * Runs in all data modes.
+     */
+    protected void runInAllDataModes(TestRunnable call) throws Exception {
+        for (int i = 0; i < DataMode.values().length; i++) {
+            dataMode = DataMode.values()[i];
+
+            info("Running test in data mode: " + dataMode);
+
+            if (i != 0)
+                beforeTest();
+
+            try {
+                call.run();
+            }
+            finally {
+                if (i + 1 != DataMode.values().length)
+                    afterTest();
+            }
+        }
+    }
+
+    /**
+     * @param keyId Key Id.
+     * @return Key.
+     */
+    public Object key(int keyId) {
+        return key(keyId, dataMode);
+    }
+
+    /**
+     * @param valId Key Id.
+     * @return Value.
+     */
+    public Object value(int valId) {
+        return value(valId, dataMode);
+    }
+
+    /**
+     * @param keyId Key Id.
+     * @param mode Mode.
+     * @return Key.
+     */
+    public static Object key(int keyId, DataMode mode) {
+        switch (mode) {
+            case SERIALIZABLE:
+                return new SerializableObject(keyId);
+            case EXTERNALIZABLE:
+                return new ExternalizableObject(keyId);
+            case PLANE_OBJECT:
+                return new TestObject(keyId);
+            default:
+                throw new IllegalArgumentException("mode: " + mode);
+        }
+    }
+
+    /**
+     * @param obj Key or value object
+     * @return Value.
+     */
+    public static int valueOf(Object obj) {
+        if (obj instanceof TestObject)
+            return ((TestObject)obj).value();
+        else
+            throw new IllegalStateException();
+    }
+
+    /**
+     * @param idx Index.
+     * @param mode Mode.
+     * @return Value.
+     */
+    public static Object value(int idx, DataMode mode) {
+        switch (mode) {
+            case SERIALIZABLE:
+                return new SerializableObject(idx);
+            case EXTERNALIZABLE:
+                return new ExternalizableObject(idx);
+            case PLANE_OBJECT:
+                return new TestObject(idx);
+            default:
+                throw new IllegalArgumentException("mode: " + mode);
+        }
+    }
+
+    /**
+     *
+     */
+    public static class TestObject {
+        /** */
+        protected int val;
+
+        /** */
+        protected String strVal;
+
+        /** */
+        protected TestEnum enumVal;
+
+        /**
+         * @param val Value.
+         */
+        TestObject(int val) {
+            this.val = val;
+            strVal = "val" + val;
+
+            TestEnum[] values = TestEnum.values();
+            enumVal = values[Math.abs(val) % values.length];
+        }
+
+        /**
+         * @return Value.
+         */
+        public int value() {
+            return val;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (this == o)
+                return true;
+
+            if (!(o instanceof TestObject))
+                return false;
+
+            TestObject val = (TestObject)o;
+
+            return getClass().equals(o.getClass()) && this.val == val.val && enumVal == val.enumVal
+                && strVal.equals(val.strVal);
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            return val;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return getClass().getSimpleName() + "[" +
+                "val=" + val +
+                ", strVal='" + strVal + '\'' +
+                ", enumVal=" + enumVal +
+                ']';
+        }
+    }
+
+    /**
+     *
+     */
+    protected static class SerializableObject extends TestObject implements Serializable {
+        /**
+         * @param val Value.
+         */
+        public SerializableObject(int val) {
+            super(val);
+        }
+    }
+
+    /**
+     *
+     */
+    private static class ExternalizableObject extends TestObject implements Externalizable {
+        /**
+         * Default constructor.
+         */
+        ExternalizableObject() {
+            super(-1);
+        }
+
+        /**
+         * @param val Value.
+         */
+        ExternalizableObject(int val) {
+            super(val);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void writeExternal(ObjectOutput out) throws IOException {
+            out.writeInt(val);
+            out.writeObject(strVal);
+            out.writeObject(enumVal);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+            val = in.readInt();
+            strVal = (String)in.readObject();
+            enumVal = (TestEnum)in.readObject();
+        }
+    }
+
+    /**
+     * Data mode.
+     */
+    public enum DataMode {
+        /** Serializable objects. */
+        SERIALIZABLE,
+
+        /** Externalizable objects. */
+        EXTERNALIZABLE,
+
+        /** Objects without Serializable and Externalizable. */
+        PLANE_OBJECT
+    }
+
+    /**
+     *
+     */
+    private enum TestEnum {
+        /** */
+        TEST_VALUE_1,
+
+        /** */
+        TEST_VALUE_2,
+
+        /** */
+        TEST_VALUE_3
+    }
+
+    /**
+     *
+     */
+    public static interface TestRunnable {
+        /**
+         * @throws Exception If failed.
+         */
+        public void run() throws Exception;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/953b575f/modules/core/src/test/java/org/apache/ignite/testframework/test/ConfigVariationsTestSuiteBuilderTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/test/ConfigVariationsTestSuiteBuilderTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/test/ConfigVariationsTestSuiteBuilderTest.java
new file mode 100644
index 0000000..75e3010
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/test/ConfigVariationsTestSuiteBuilderTest.java
@@ -0,0 +1,112 @@
+/*
+ * 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.testframework.test;
+
+import java.util.concurrent.atomic.AtomicInteger;
+import junit.framework.TestCase;
+import junit.framework.TestSuite;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.lang.IgnitePredicate;
+import org.apache.ignite.testframework.configvariations.ConfigVariationsTestSuiteBuilder;
+import org.apache.ignite.testframework.junits.IgniteConfigVariationsAbstractTest;
+
+/**
+ *
+ */
+public class ConfigVariationsTestSuiteBuilderTest extends TestCase {
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDefaults() throws Exception {
+        TestSuite dfltSuite = new ConfigVariationsTestSuiteBuilder("testSuite", OneTestCase.class).build();
+
+        assertEquals(4, dfltSuite.countTestCases());
+
+        TestSuite dfltCacheSuite = new ConfigVariationsTestSuiteBuilder("testSuite", OneTestCase.class)
+            .withBasicCacheParams().build();
+
+        assertEquals(4 * 12, dfltCacheSuite.countTestCases());
+
+        // With clients.
+        dfltSuite = new ConfigVariationsTestSuiteBuilder("testSuite", OneTestCase.class)
+            .testedNodesCount(2).withClients().build();
+
+        assertEquals(4 * 2, dfltSuite.countTestCases());
+
+        dfltCacheSuite = new ConfigVariationsTestSuiteBuilder("testSuite", OneTestCase.class)
+            .withBasicCacheParams().testedNodesCount(3).withClients().build();
+
+        assertEquals(4 * 12 * 3, dfltCacheSuite.countTestCases());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @SuppressWarnings("serial")
+    public void testIgniteConfigFilter() throws Exception {
+        TestSuite dfltSuite = new ConfigVariationsTestSuiteBuilder("testSuite", OneTestCase.class).build();
+
+        final AtomicInteger cnt = new AtomicInteger();
+
+        TestSuite filteredSuite = new ConfigVariationsTestSuiteBuilder("testSuite", OneTestCase.class)
+            .withIgniteConfigFilters(new IgnitePredicate<IgniteConfiguration>() {
+                @Override public boolean apply(IgniteConfiguration configuration) {
+                    return cnt.getAndIncrement() % 2 == 0;
+                }
+            })
+            .build();
+
+        assertEquals(dfltSuite.countTestCases() / 2, filteredSuite.countTestCases());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @SuppressWarnings("serial")
+    public void testCacheConfigFilter() throws Exception {
+        TestSuite dfltSuite = new ConfigVariationsTestSuiteBuilder("testSuite", OneTestCase.class)
+            .withBasicCacheParams()
+            .build();
+
+        final AtomicInteger cnt = new AtomicInteger();
+
+        TestSuite filteredSuite = new ConfigVariationsTestSuiteBuilder("testSuite", OneTestCase.class)
+            .withBasicCacheParams()
+            .withCacheConfigFilters(new IgnitePredicate<CacheConfiguration>() {
+                @Override public boolean apply(CacheConfiguration configuration) {
+                    return cnt.getAndIncrement() % 2 == 0;
+                }
+            })
+            .build();
+
+        assertEquals(dfltSuite.countTestCases() / 2, filteredSuite.countTestCases());
+    }
+
+    /**
+     *
+     */
+    private static class OneTestCase extends IgniteConfigVariationsAbstractTest {
+        /**
+         * @throws Exception If failed.
+         */
+        public void test1() throws Exception {
+            // No-op.
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/953b575f/modules/core/src/test/java/org/apache/ignite/testframework/test/ParametersTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/test/ParametersTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/test/ParametersTest.java
new file mode 100644
index 0000000..2870b06
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/test/ParametersTest.java
@@ -0,0 +1,87 @@
+/*
+ * 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.testframework.test;
+
+import java.util.HashSet;
+import java.util.Set;
+import junit.framework.TestCase;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.testframework.configvariations.ConfigParameter;
+import org.apache.ignite.testframework.configvariations.Parameters;
+
+/**
+ * Test.
+ */
+public class ParametersTest extends TestCase {
+    /**
+     * @throws Exception If failed.
+     */
+    public void testEnumVariations() throws Exception {
+        ConfigParameter<CacheConfiguration>[] modes = Parameters.enumParameters("setCacheMode", CacheMode.class);
+
+        assertEquals(CacheMode.values().length, modes.length);
+
+        Set<CacheMode> res = new HashSet<>();
+
+        for (ConfigParameter<CacheConfiguration> modeApplier : modes) {
+            CacheConfiguration cfg = new CacheConfiguration();
+
+            modeApplier.apply(cfg);
+
+            CacheMode mode = cfg.getCacheMode();
+
+            res.add(mode);
+
+            System.out.println(">>> " + mode);
+        }
+
+        assertEquals(modes.length, res.size());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @SuppressWarnings("unchecked")
+    public void testEnumVariationsWithNull() throws Exception {
+        ConfigParameter<CacheConfiguration>[] cfgParam =
+            Parameters.enumParameters(true, "setCacheMode", CacheMode.class);
+
+        assertEquals(CacheMode.values().length + 1, cfgParam.length);
+
+        cfgParam[0] = null;
+
+        Set<CacheMode> set = new HashSet<>();
+
+        for (int i = 1; i < cfgParam.length; i++) {
+            ConfigParameter<CacheConfiguration> modeApplier = cfgParam[i];
+
+            CacheConfiguration cfg = new CacheConfiguration();
+
+            modeApplier.apply(cfg);
+
+            CacheMode mode = cfg.getCacheMode();
+
+            set.add(mode);
+
+            System.out.println(">>> " + mode);
+        }
+
+        assertEquals(CacheMode.values().length, set.size());
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/953b575f/modules/core/src/test/java/org/apache/ignite/testframework/test/VariationsIteratorTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/test/VariationsIteratorTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/test/VariationsIteratorTest.java
new file mode 100644
index 0000000..d8ac2b3
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/test/VariationsIteratorTest.java
@@ -0,0 +1,156 @@
+/*
+ * 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.testframework.test;
+
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Set;
+import junit.framework.TestCase;
+import org.apache.ignite.testframework.configvariations.VariationsIterator;
+
+/**
+ * Test start iterator.
+ */
+public class VariationsIteratorTest extends TestCase {
+    /**
+     * @throws Exception If failed.
+     */
+    public void test1() throws Exception {
+        Object[][] arr = new Object[][] {
+            {0, 1},
+            {0, 1},
+            {0, 1},
+        };
+
+        checkIterator(arr);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @SuppressWarnings("PointlessArithmeticExpression")
+    public void test2() throws Exception {
+        Object[][] arr = new Object[][] {
+            {0},
+            {0, 1, 2},
+            {0, 1},
+            {0, 1, 2, 3, 4, 5},
+        };
+
+        checkIterator(arr);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @SuppressWarnings("PointlessArithmeticExpression")
+    public void test3() throws Exception {
+        Object[][] arr = new Object[][] {
+            {0, 1, 2, 3, 4, 5},
+            {0, 1, 2},
+            {0, 1},
+            {0},
+        };
+
+        checkIterator(arr);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @SuppressWarnings("PointlessArithmeticExpression")
+    public void test4() throws Exception {
+        Object[][] arr = new Object[][]{
+            {0,1,2},
+            {0,1},
+            {0,1,2,4},
+            {0,1},
+            {0},
+            {0},
+            {0,1,2,4},
+        };
+
+        checkIterator(arr);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testSimple() throws Exception {
+        Object[][] arr = new Object[][] {
+            {0},
+        };
+
+        checkIterator(arr);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testSimple2() throws Exception {
+        Object[][] arr = new Object[][] {
+            {0},
+            {0},
+        };
+
+        checkIterator(arr);
+    }
+
+    /**
+     * @param arr Array.
+     */
+    private void checkIterator(Object[][] arr) {
+        int expSize = 1;
+        int significantParamsCnt = 1;
+
+        for (int i = 0; i < arr.length; i++) {
+            Object[] objects = arr[i];
+
+            System.out.println(">>> " + i + ": " + objects.length);
+
+            expSize *= objects.length;
+
+            if (objects.length > 1)
+                significantParamsCnt++;
+        }
+
+        System.out.println("Iteration info [expSize=" + expSize + ", significantParamsCnt=" + significantParamsCnt + "]");
+
+        Set<int[]> states = new HashSet<>();
+
+        int step = 0;
+
+        for (VariationsIterator it = new VariationsIterator(arr); it.hasNext(); ) {
+            int[] state = it.next();
+
+            System.out.println(Arrays.toString(state));
+
+            for (int[] state2 : states) {
+                if (Arrays.equals(state, state2))
+                    fail("Got equal states on step " + step + " [state=" + Arrays.toString(state)
+                        + ", state2=" + Arrays.toString(state2));
+            }
+
+            states.add(state);
+
+            step++;
+        }
+
+        assertEquals(expSize, states.size());
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/953b575f/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
index 3903910..9e2324c 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
@@ -51,6 +51,9 @@ import org.apache.ignite.messaging.GridMessagingSelfTest;
 import org.apache.ignite.messaging.IgniteMessagingWithClientTest;
 import org.apache.ignite.spi.GridSpiLocalHostInjectionTest;
 import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.test.ConfigVariationsTestSuiteBuilderTest;
+import org.apache.ignite.testframework.test.ParametersTest;
+import org.apache.ignite.testframework.test.VariationsIteratorTest;
 
 /**
  * Basic test suite.
@@ -121,6 +124,11 @@ public class IgniteBasicTestSuite extends TestSuite {
 
         GridTestUtils.addTestIfNeeded(suite, DynamicProxySerializationMultiJvmSelfTest.class, ignoredTests);
 
+        // Tests against configuration variations framework.
+        suite.addTestSuite(ParametersTest.class);
+        suite.addTestSuite(VariationsIteratorTest.class);
+        suite.addTestSuite(ConfigVariationsTestSuiteBuilderTest.class);
+
         return suite;
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/953b575f/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheBasicConfigVariationsFullApiTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheBasicConfigVariationsFullApiTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheBasicConfigVariationsFullApiTestSuite.java
new file mode 100644
index 0000000..85a8f59
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheBasicConfigVariationsFullApiTestSuite.java
@@ -0,0 +1,41 @@
+/*
+ * 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.testsuites;
+
+import junit.framework.TestSuite;
+import org.apache.ignite.internal.processors.cache.IgniteCacheConfigVariationsFullApiTest;
+import org.apache.ignite.testframework.configvariations.ConfigVariationsTestSuiteBuilder;
+
+/**
+ * Test suite for cache API.
+ */
+public class IgniteCacheBasicConfigVariationsFullApiTestSuite extends TestSuite {
+    /**
+     * @return Cache API test suite.
+     * @throws Exception If failed.
+     */
+    public static TestSuite suite() throws Exception {
+        return new ConfigVariationsTestSuiteBuilder(
+            "Cache New Full API Test Suite",
+            IgniteCacheConfigVariationsFullApiTest.class)
+            .withBasicCacheParams()
+            .gridsCount(5).backups(1)
+            .testedNodesCount(3).withClients()
+            .build();
+    }
+}


[3/4] ignite git commit: ignite-2521: Configuration variations tests framework + IgniteCacheBasicConfigVariationsFullApiTestSuite + ignite-2554: Fixed Affinity.mapKeyToNode() for dynamically started LOCAL cache

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/953b575f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheConfigVariationsFullApiTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheConfigVariationsFullApiTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheConfigVariationsFullApiTest.java
new file mode 100644
index 0000000..2ba7bb9
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheConfigVariationsFullApiTest.java
@@ -0,0 +1,5851 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Sets;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.Lock;
+import javax.cache.Cache;
+import javax.cache.CacheException;
+import javax.cache.event.CacheEntryEvent;
+import javax.cache.event.CacheEntryListenerException;
+import javax.cache.event.CacheEntryUpdatedListener;
+import javax.cache.expiry.Duration;
+import javax.cache.expiry.ExpiryPolicy;
+import javax.cache.expiry.TouchedExpiryPolicy;
+import javax.cache.processor.EntryProcessor;
+import javax.cache.processor.EntryProcessorException;
+import javax.cache.processor.EntryProcessorResult;
+import javax.cache.processor.MutableEntry;
+import junit.framework.AssertionFailedError;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.IgniteTransactions;
+import org.apache.ignite.cache.CacheEntryEventSerializableFilter;
+import org.apache.ignite.cache.CacheEntryProcessor;
+import org.apache.ignite.cache.CachePeekMode;
+import org.apache.ignite.cache.affinity.Affinity;
+import org.apache.ignite.cache.query.ContinuousQuery;
+import org.apache.ignite.cache.query.QueryCursor;
+import org.apache.ignite.cache.query.ScanQuery;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.events.Event;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.IgniteKernal;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager;
+import org.apache.ignite.internal.util.lang.GridAbsPredicate;
+import org.apache.ignite.internal.util.lang.GridAbsPredicateX;
+import org.apache.ignite.internal.util.typedef.CIX1;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.PA;
+import org.apache.ignite.internal.util.typedef.internal.A;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteBiPredicate;
+import org.apache.ignite.lang.IgniteClosure;
+import org.apache.ignite.lang.IgniteFuture;
+import org.apache.ignite.lang.IgnitePredicate;
+import org.apache.ignite.resources.LoggerResource;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.IgniteCacheConfigVariationsAbstractTest;
+import org.apache.ignite.transactions.Transaction;
+import org.apache.ignite.transactions.TransactionConcurrency;
+import org.apache.ignite.transactions.TransactionIsolation;
+import org.jetbrains.annotations.Nullable;
+
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
+import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheMemoryMode.OFFHEAP_TIERED;
+import static org.apache.ignite.cache.CacheMode.LOCAL;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.cache.CacheMode.REPLICATED;
+import static org.apache.ignite.cache.CachePeekMode.ALL;
+import static org.apache.ignite.cache.CachePeekMode.BACKUP;
+import static org.apache.ignite.cache.CachePeekMode.OFFHEAP;
+import static org.apache.ignite.cache.CachePeekMode.ONHEAP;
+import static org.apache.ignite.cache.CachePeekMode.PRIMARY;
+import static org.apache.ignite.events.EventType.EVT_CACHE_OBJECT_LOCKED;
+import static org.apache.ignite.events.EventType.EVT_CACHE_OBJECT_SWAPPED;
+import static org.apache.ignite.events.EventType.EVT_CACHE_OBJECT_UNLOCKED;
+import static org.apache.ignite.events.EventType.EVT_CACHE_OBJECT_UNSWAPPED;
+import static org.apache.ignite.testframework.GridTestUtils.assertThrows;
+import static org.apache.ignite.testframework.GridTestUtils.waitForCondition;
+import static org.apache.ignite.transactions.TransactionConcurrency.OPTIMISTIC;
+import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
+import static org.apache.ignite.transactions.TransactionIsolation.READ_COMMITTED;
+import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
+import static org.apache.ignite.transactions.TransactionIsolation.SERIALIZABLE;
+import static org.apache.ignite.transactions.TransactionState.COMMITTED;
+
+/**
+ * Full API cache test.
+ */
+@SuppressWarnings({"TransientFieldInNonSerializableClass", "unchecked"})
+public class IgniteCacheConfigVariationsFullApiTest extends IgniteCacheConfigVariationsAbstractTest {
+    /** Test timeout */
+    private static final long TEST_TIMEOUT = 60 * 1000;
+
+    /** */
+    public static final CacheEntryProcessor<String, Integer, String> ERR_PROCESSOR =
+        new CacheEntryProcessor<String, Integer, String>() {
+            /** */
+            private static final long serialVersionUID = 0L;
+
+            @Override public String process(MutableEntry<String, Integer> e, Object... args) {
+                throw new RuntimeException("Failed!");
+            }
+        };
+
+    /** Increment processor for invoke operations. */
+    public static final EntryProcessor<Object, Object, Object> INCR_PROCESSOR = new IncrementEntryProcessor();
+
+    /** Increment processor for invoke operations with IgniteEntryProcessor. */
+    public static final CacheEntryProcessor<Object, Object, Object> INCR_IGNITE_PROCESSOR =
+        new CacheEntryProcessor<Object, Object, Object>() {
+            /** */
+            private static final long serialVersionUID = 0L;
+
+            @Override public Object process(MutableEntry<Object, Object> e, Object... args) {
+                return INCR_PROCESSOR.process(e, args);
+            }
+        };
+
+    /** Increment processor for invoke operations. */
+    public static final EntryProcessor<Object, Object, Object> RMV_PROCESSOR = new RemoveEntryProcessor();
+
+    /** Increment processor for invoke operations with IgniteEntryProcessor. */
+    public static final CacheEntryProcessor<Object, Object, Object> RMV_IGNITE_PROCESSOR =
+        new CacheEntryProcessor<Object, Object, Object>() {
+            /** */
+            private static final long serialVersionUID = 0L;
+
+            @Override public Object process(MutableEntry<Object, Object> e, Object... args) {
+                return RMV_PROCESSOR.process(e, args);
+            }
+        };
+
+    /** {@inheritDoc} */
+    @Override protected long getTestTimeout() {
+        return TEST_TIMEOUT;
+    }
+
+    /**
+     * @throws Exception In case of error.
+     */
+    public void testSize() throws Exception {
+        assert jcache().localSize() == 0;
+
+        int size = 10;
+
+        final Map<String, Integer> map = new HashMap<>();
+
+        for (int i = 0; i < size; i++)
+            map.put("key" + i, i);
+
+        // Put in primary nodes to avoid near readers which will prevent entry from being cleared.
+        Map<ClusterNode, Collection<String>> mapped = grid(0).<String>affinity(cacheName()).mapKeysToNodes(map.keySet());
+
+        for (int i = 0; i < gridCount(); i++) {
+            Collection<String> keys = mapped.get(grid(i).localNode());
+
+            if (!F.isEmpty(keys)) {
+                for (String key : keys)
+                    jcache(i).put(key, map.get(key));
+            }
+        }
+
+        map.remove("key0");
+
+        mapped = grid(0).<String>affinity(cacheName()).mapKeysToNodes(map.keySet());
+
+        for (int i = 0; i < gridCount(); i++) {
+            // Will actually delete entry from map.
+            CU.invalidate(jcache(i), "key0");
+
+            assertNull("Failed check for grid: " + i, jcache(i).localPeek("key0", ONHEAP));
+
+            Collection<String> keysCol = mapped.get(grid(i).localNode());
+
+            assert jcache(i).localSize() != 0 || F.isEmpty(keysCol);
+        }
+
+        for (int i = 0; i < gridCount(); i++)
+            executeOnLocalOrRemoteJvm(i, new CheckCacheSizeTask(map, cacheName()));
+
+        for (int i = 0; i < gridCount(); i++) {
+            Collection<String> keysCol = mapped.get(grid(i).localNode());
+
+            assertEquals("Failed check for grid: " + i, !F.isEmpty(keysCol) ? keysCol.size() : 0,
+                jcache(i).localSize(PRIMARY));
+        }
+
+        int globalPrimarySize = map.size();
+
+        for (int i = 0; i < gridCount(); i++)
+            assertEquals(globalPrimarySize, jcache(i).size(PRIMARY));
+
+        int times = 1;
+
+        if (cacheMode() == REPLICATED)
+            times = gridCount() - clientsCount();
+        else if (cacheMode() == PARTITIONED)
+            times = Math.min(gridCount(), jcache().getConfiguration(CacheConfiguration.class).getBackups() + 1);
+
+        int globalSize = globalPrimarySize * times;
+
+        for (int i = 0; i < gridCount(); i++)
+            assertEquals(globalSize, jcache(i).size(ALL));
+    }
+
+    /**
+     * @throws Exception In case of error.
+     */
+    public void testContainsKey() throws Exception {
+        jcache().put("testContainsKey", 1);
+
+        checkContainsKey(true, "testContainsKey");
+        checkContainsKey(false, "testContainsKeyWrongKey");
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testContainsKeyTx() throws Exception {
+        if (!txEnabled())
+            return;
+
+        IgniteCache<String, Integer> cache = jcache();
+
+        IgniteTransactions txs = ignite(0).transactions();
+
+        for (int i = 0; i < 10; i++) {
+            String key = String.valueOf(i);
+
+            try (Transaction tx = txs.txStart()) {
+                assertNull(key, cache.get(key));
+
+                assertFalse(cache.containsKey(key));
+
+                tx.commit();
+            }
+
+            try (Transaction tx = txs.txStart()) {
+                assertNull(key, cache.get(key));
+
+                cache.put(key, i);
+
+                assertTrue(cache.containsKey(key));
+
+                tx.commit();
+            }
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testContainsKeysTx() throws Exception {
+        if (!txEnabled())
+            return;
+
+        IgniteCache<String, Integer> cache = jcache();
+
+        IgniteTransactions txs = ignite(0).transactions();
+
+        Set<String> keys = new HashSet<>();
+
+        for (int i = 0; i < 10; i++) {
+            String key = String.valueOf(i);
+
+            keys.add(key);
+        }
+
+        try (Transaction tx = txs.txStart()) {
+            for (String key : keys)
+                assertNull(key, cache.get(key));
+
+            assertFalse(cache.containsKeys(keys));
+
+            tx.commit();
+        }
+
+        try (Transaction tx = txs.txStart()) {
+            for (String key : keys)
+                assertNull(key, cache.get(key));
+
+            for (String key : keys)
+                cache.put(key, 0);
+
+            assertTrue(cache.containsKeys(keys));
+
+            tx.commit();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testRemoveInExplicitLocks() throws Exception {
+        if (lockingEnabled()) {
+            IgniteCache<String, Integer> cache = jcache();
+
+            cache.put("a", 1);
+
+            Lock lock = cache.lockAll(ImmutableSet.of("a", "b", "c", "d"));
+
+            lock.lock();
+
+            try {
+                cache.remove("a");
+
+                // Make sure single-key operation did not remove lock.
+                cache.putAll(F.asMap("b", 2, "c", 3, "d", 4));
+            }
+            finally {
+                lock.unlock();
+            }
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testRemoveAllSkipStore() throws Exception {
+        if (isMultiJvm())
+            fail("https://issues.apache.org/jira/browse/IGNITE-1088");
+
+        if (!storeEnabled())
+            return;
+
+        IgniteCache<String, Integer> jcache = jcache();
+
+        jcache.putAll(F.asMap("1", 1, "2", 2, "3", 3));
+
+        jcache.withSkipStore().removeAll();
+
+        assertEquals((Integer)1, jcache.get("1"));
+        assertEquals((Integer)2, jcache.get("2"));
+        assertEquals((Integer)3, jcache.get("3"));
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testAtomicOps() throws IgniteCheckedException {
+        IgniteCache<String, Integer> c = jcache();
+
+        final int cnt = 10;
+
+        for (int i = 0; i < cnt; i++)
+            assertNull(c.getAndPutIfAbsent("k" + i, i));
+
+        for (int i = 0; i < cnt; i++) {
+            boolean wrong = i % 2 == 0;
+
+            String key = "k" + i;
+
+            boolean res = c.replace(key, wrong ? i + 1 : i, -1);
+
+            assertEquals(wrong, !res);
+        }
+
+        for (int i = 0; i < cnt; i++) {
+            boolean success = i % 2 != 0;
+
+            String key = "k" + i;
+
+            boolean res = c.remove(key, -1);
+
+            assertTrue(success == res);
+        }
+    }
+
+    /**
+     * @throws Exception In case of error.
+     */
+    public void testGet() throws Exception {
+        runInAllDataModes(new TestRunnable() {
+            @Override public void run() {
+                IgniteCache cache = jcache();
+
+                cache.put(key(1), value(1));
+                cache.put(key(2), value(2));
+
+                assertEquals(value(1), cache.get(key(1)));
+                assertEquals(value(2), cache.get(key(2)));
+                // Wrong key.
+                assertNull(cache.get(key(3)));
+            }
+        });
+    }
+
+    /**
+     * @throws Exception In case of error.
+     */
+    public void testGetAsync() throws Exception {
+        IgniteCache<String, Integer> cache = jcache();
+
+        cache.put("key1", 1);
+        cache.put("key2", 2);
+
+        IgniteCache<String, Integer> cacheAsync = cache.withAsync();
+
+        cacheAsync.get("key1");
+
+        IgniteFuture<Integer> fut1 = cacheAsync.future();
+
+        cacheAsync.get("key2");
+
+        IgniteFuture<Integer> fut2 = cacheAsync.future();
+
+        cacheAsync.get("wrongKey");
+
+        IgniteFuture<Integer> fut3 = cacheAsync.future();
+
+        assert fut1.get() == 1;
+        assert fut2.get() == 2;
+        assert fut3.get() == null;
+    }
+
+    /**
+     * @throws Exception In case of error.
+     */
+    public void testGetAll() throws Exception {
+        runInAllDataModes(new TestRunnable() {
+            @Override public void run() {
+                final Object key1 = key(1);
+                final Object key2 = key(2);
+                final Object key9999 = key(9999);
+
+                final Object val1 = value(1);
+                final Object val2 = value(2);
+
+                Transaction tx = txShouldBeUsed() ? transactions().txStart() : null;
+
+                final IgniteCache<Object, Object> cache = jcache();
+
+                try {
+                    cache.put(key1, val1);
+                    cache.put(key2, val2);
+
+                    if (tx != null)
+                        tx.commit();
+                }
+                finally {
+                    if (tx != null)
+                        tx.close();
+                }
+
+                GridTestUtils.assertThrows(log, new Callable<Void>() {
+                    @Override public Void call() throws Exception {
+                        cache.getAll(null).isEmpty();
+
+                        return null;
+                    }
+                }, NullPointerException.class, null);
+
+                assert cache.getAll(Collections.<Object>emptySet()).isEmpty();
+
+                Map<Object, Object> map1 = cache.getAll(ImmutableSet.of(key1, key2, key9999));
+
+                info("Retrieved map1: " + map1);
+
+                assert 2 == map1.size() : "Invalid map: " + map1;
+
+                assertEquals(val1, map1.get(key1));
+                assertEquals(val2, map1.get(key2));
+                assertNull(map1.get(key9999));
+
+                Map<Object, Object> map2 = cache.getAll(ImmutableSet.of(key1, key2, key9999));
+
+                info("Retrieved map2: " + map2);
+
+                assert 2 == map2.size() : "Invalid map: " + map2;
+
+                assertEquals(val1, map2.get(key1));
+                assertEquals(val2, map2.get(key2));
+                assertNull(map2.get(key9999));
+
+                // Now do the same checks but within transaction.
+                if (txShouldBeUsed()) {
+                    try (Transaction tx0 = transactions().txStart()) {
+                        assert cache.getAll(Collections.<Object>emptySet()).isEmpty();
+
+                        map1 = cache.getAll(ImmutableSet.of(key1, key2, key9999));
+
+                        info("Retrieved map1: " + map1);
+
+                        assert 2 == map1.size() : "Invalid map: " + map1;
+
+                        assertEquals(val1, map2.get(key1));
+                        assertEquals(val2, map2.get(key2));
+                        assertNull(map2.get(key9999));
+
+                        map2 = cache.getAll(ImmutableSet.of(key1, key2, key9999));
+
+                        info("Retrieved map2: " + map2);
+
+                        assert 2 == map2.size() : "Invalid map: " + map2;
+
+                        assertEquals(val1, map2.get(key1));
+                        assertEquals(val2, map2.get(key2));
+                        assertNull(map2.get(key9999));
+
+                        tx0.commit();
+                    }
+                }
+            }
+        });
+    }
+
+    /**
+     * @throws Exception In case of error.
+     */
+    public void testGetAllWithNulls() throws Exception {
+        final IgniteCache<String, Integer> cache = jcache();
+
+        final Set<String> c = new HashSet<>();
+
+        c.add("key1");
+        c.add(null);
+
+        GridTestUtils.assertThrows(log, new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                cache.getAll(c);
+
+                return null;
+            }
+        }, NullPointerException.class, null);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testGetTxNonExistingKey() throws Exception {
+        if (txShouldBeUsed()) {
+            try (Transaction ignored = transactions().txStart()) {
+                assert jcache().get("key999123") == null;
+            }
+        }
+    }
+
+    /**
+     * @throws Exception In case of error.
+     */
+    public void testGetAllAsync() throws Exception {
+        final IgniteCache<String, Integer> cache = jcache();
+
+        final IgniteCache<String, Integer> cacheAsync = cache.withAsync();
+
+        cache.put("key1", 1);
+        cache.put("key2", 2);
+
+        GridTestUtils.assertThrows(log, new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                cacheAsync.getAll(null);
+
+                return null;
+            }
+        }, NullPointerException.class, null);
+
+        cacheAsync.getAll(Collections.<String>emptySet());
+        IgniteFuture<Map<String, Integer>> fut2 = cacheAsync.future();
+
+        cacheAsync.getAll(ImmutableSet.of("key1", "key2"));
+        IgniteFuture<Map<String, Integer>> fut3 = cacheAsync.future();
+
+        assert fut2.get().isEmpty();
+        assert fut3.get().size() == 2 : "Invalid map: " + fut3.get();
+        assert fut3.get().get("key1") == 1;
+        assert fut3.get().get("key2") == 2;
+    }
+
+    /**
+     * @throws Exception In case of error.
+     */
+    public void testPut() throws Exception {
+        runInAllDataModes(new TestRunnable() {
+            @Override public void run() throws Exception {
+                IgniteCache cache = jcache();
+
+                final Object key1 = key(1);
+                final Object val1 = value(1);
+                final Object key2 = key(2);
+                final Object val2 = value(2);
+
+                assert cache.getAndPut(key1, val1) == null;
+                assert cache.getAndPut(key2, val2) == null;
+
+                // Check inside transaction.
+                assertEquals(val1, cache.get(key1));
+                assertEquals(val2, cache.get(key2));
+
+                // Put again to check returned values.
+                assertEquals(val1, cache.getAndPut(key1, val1));
+                assertEquals(val2, cache.getAndPut(key2, val2));
+
+                checkContainsKey(true, key1);
+                checkContainsKey(true, key2);
+
+                assert cache.get(key1) != null;
+                assert cache.get(key2) != null;
+                assert cache.get(key(100500)) == null;
+
+                // Check outside transaction.
+                checkContainsKey(true, key1);
+                checkContainsKey(true, key2);
+
+                assertEquals(val1, cache.get(key1));
+                assertEquals(val2, cache.get(key2));
+                assert cache.get(key(100500)) == null;
+
+                assertEquals(val1, cache.getAndPut(key1, value(10)));
+                assertEquals(val2, cache.getAndPut(key2, value(11)));
+            }
+        });
+    }
+
+    /**
+     * @throws Exception In case of error.
+     */
+    public void testPutTx() throws Exception {
+        if (txShouldBeUsed()) {
+            IgniteCache<String, Integer> cache = jcache();
+
+            try (Transaction tx = transactions().txStart()) {
+                assert cache.getAndPut("key1", 1) == null;
+                assert cache.getAndPut("key2", 2) == null;
+
+                // Check inside transaction.
+                assert cache.get("key1") == 1;
+                assert cache.get("key2") == 2;
+
+                // Put again to check returned values.
+                assert cache.getAndPut("key1", 1) == 1;
+                assert cache.getAndPut("key2", 2) == 2;
+
+                assert cache.get("key1") != null;
+                assert cache.get("key2") != null;
+                assert cache.get("wrong") == null;
+
+                tx.commit();
+            }
+
+            // Check outside transaction.
+            checkContainsKey(true, "key1");
+            checkContainsKey(true, "key2");
+
+            assert cache.get("key1") == 1;
+            assert cache.get("key2") == 2;
+            assert cache.get("wrong") == null;
+
+            assertEquals((Integer)1, cache.getAndPut("key1", 10));
+            assertEquals((Integer)2, cache.getAndPut("key2", 11));
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testInvokeOptimisticReadCommitted() throws Exception {
+        runInAllDataModes(new TestRunnable() {
+            @Override public void run() throws Exception {
+                checkInvoke(OPTIMISTIC, READ_COMMITTED);
+            }
+        });
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testInvokeOptimisticRepeatableRead() throws Exception {
+        runInAllDataModes(new TestRunnable() {
+            @Override public void run() throws Exception {
+                checkInvoke(OPTIMISTIC, REPEATABLE_READ);
+            }
+        });
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testInvokePessimisticReadCommitted() throws Exception {
+        runInAllDataModes(new TestRunnable() {
+            @Override public void run() throws Exception {
+                checkInvoke(PESSIMISTIC, READ_COMMITTED);
+            }
+        });
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testInvokePessimisticRepeatableRead() throws Exception {
+        runInAllDataModes(new TestRunnable() {
+            @Override public void run() throws Exception {
+                checkInvoke(PESSIMISTIC, REPEATABLE_READ);
+            }
+        });
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testIgniteInvokeOptimisticReadCommitted1() throws Exception {
+        runInAllDataModes(new TestRunnable() {
+            @Override public void run() throws Exception {
+                checkIgniteInvoke(OPTIMISTIC, READ_COMMITTED);
+            }
+        });
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testIgniteInvokeOptimisticRepeatableRead() throws Exception {
+        runInAllDataModes(new TestRunnable() {
+            @Override public void run() throws Exception {
+                checkIgniteInvoke(OPTIMISTIC, REPEATABLE_READ);
+            }
+        });
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testIgniteInvokePessimisticReadCommitted() throws Exception {
+        runInAllDataModes(new TestRunnable() {
+            @Override public void run() throws Exception {
+                checkIgniteInvoke(PESSIMISTIC, READ_COMMITTED);
+            }
+        });
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testIgniteInvokePessimisticRepeatableRead() throws Exception {
+        runInAllDataModes(new TestRunnable() {
+            @Override public void run() throws Exception {
+                checkIgniteInvoke(PESSIMISTIC, REPEATABLE_READ);
+            }
+        });
+    }
+
+    /**
+     * @param concurrency Concurrency.
+     * @param isolation Isolation.
+     * @throws Exception If failed.
+     */
+    private void checkIgniteInvoke(TransactionConcurrency concurrency, TransactionIsolation isolation)
+        throws Exception {
+        checkInvoke(concurrency, isolation, INCR_IGNITE_PROCESSOR, RMV_IGNITE_PROCESSOR);
+    }
+
+    /**
+     * @param concurrency Transaction concurrency.
+     * @param isolation Transaction isolation.
+     * @param incrProcessor Increment processor.
+     * @param rmvProseccor Remove processor.
+     */
+    private void checkInvoke(TransactionConcurrency concurrency, TransactionIsolation isolation,
+        EntryProcessor<Object, Object, Object> incrProcessor,
+        EntryProcessor<Object, Object, Object> rmvProseccor) {
+        IgniteCache cache = jcache();
+
+        final Object key1 = key(1);
+        final Object key2 = key(2);
+        final Object key3 = key(3);
+
+        final Object val1 = value(1);
+        final Object val2 = value(2);
+        final Object val3 = value(3);
+
+        cache.put(key2, val1);
+        cache.put(key3, val3);
+
+        Transaction tx = txShouldBeUsed() ? ignite(0).transactions().txStart(concurrency, isolation) : null;
+
+        try {
+            assertNull(cache.invoke(key1, incrProcessor, dataMode));
+            assertEquals(val1, cache.invoke(key2, incrProcessor, dataMode));
+            assertEquals(val3, cache.invoke(key3, rmvProseccor));
+
+            if (tx != null)
+                tx.commit();
+        }
+        catch (Exception e) {
+            e.printStackTrace();
+
+            throw e;
+        }
+        finally {
+            if (tx != null)
+                tx.close();
+        }
+
+        assertEquals(val1, cache.get(key1));
+        assertEquals(val2, cache.get(key2));
+        assertNull(cache.get(key3));
+
+        for (int i = 0; i < gridCount(); i++)
+            assertNull("Failed for cache: " + i, jcache(i).localPeek(key3, ONHEAP));
+
+        cache.remove(key1);
+        cache.put(key2, val1);
+        cache.put(key3, val3);
+
+        assertNull(cache.invoke(key1, incrProcessor, dataMode));
+        assertEquals(val1, cache.invoke(key2, incrProcessor, dataMode));
+        assertEquals(val3, cache.invoke(key3, rmvProseccor));
+
+        assertEquals(val1, cache.get(key1));
+        assertEquals(val2, cache.get(key2));
+        assertNull(cache.get(key3));
+
+        for (int i = 0; i < gridCount(); i++)
+            assertNull(jcache(i).localPeek(key3, ONHEAP));
+    }
+
+    /**
+     * @param concurrency Concurrency.
+     * @param isolation Isolation.
+     * @throws Exception If failed.
+     */
+    private void checkInvoke(TransactionConcurrency concurrency, TransactionIsolation isolation) throws Exception {
+        checkInvoke(concurrency, isolation, INCR_PROCESSOR, RMV_PROCESSOR);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testInvokeAllOptimisticReadCommitted() throws Exception {
+        runInAllDataModes(new TestRunnable() {
+            @Override public void run() throws Exception {
+                checkInvokeAll(OPTIMISTIC, READ_COMMITTED);
+            }
+        });
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testInvokeAllOptimisticRepeatableRead() throws Exception {
+        runInAllDataModes(new TestRunnable() {
+            @Override public void run() throws Exception {
+                checkInvokeAll(OPTIMISTIC, REPEATABLE_READ);
+            }
+        });
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testInvokeAllPessimisticReadCommitted() throws Exception {
+        runInAllDataModes(new TestRunnable() {
+            @Override public void run() throws Exception {
+                checkInvokeAll(PESSIMISTIC, READ_COMMITTED);
+            }
+        });
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testInvokeAllPessimisticRepeatableRead() throws Exception {
+        runInAllDataModes(new TestRunnable() {
+            @Override public void run() throws Exception {
+                checkInvokeAll(PESSIMISTIC, REPEATABLE_READ);
+            }
+        });
+    }
+
+    /**
+     * @param concurrency Transaction concurrency.
+     * @param isolation Transaction isolation.
+     * @throws Exception If failed.
+     */
+    private void checkInvokeAll(TransactionConcurrency concurrency, TransactionIsolation isolation) throws Exception {
+        // TODO IGNITE-2664: enable tests for all modes when IGNITE-2664 will be fixed.
+        if (dataMode != DataMode.EXTERNALIZABLE && gridCount() > 1)
+            return;
+
+        final Object key1 = key(1);
+        final Object key2 = key(2);
+        final Object key3 = key(3);
+
+        final Object val1 = value(1);
+        final Object val2 = value(2);
+        final Object val3 = value(3);
+        final Object val4 = value(4);
+
+        final IgniteCache<Object, Object> cache = jcache();
+
+        cache.put(key2, val1);
+        cache.put(key3, val3);
+
+        if (txShouldBeUsed()) {
+            Map<Object, EntryProcessorResult<Object>> res;
+
+            try (Transaction tx = ignite(0).transactions().txStart(concurrency, isolation)) {
+                res = cache.invokeAll(F.asSet(key1, key2, key3), INCR_PROCESSOR, dataMode);
+
+                tx.commit();
+            }
+
+            assertEquals(val1, cache.get(key1));
+            assertEquals(val2, cache.get(key2));
+            assertEquals(val4, cache.get(key3));
+
+            assertNull(res.get(key1));
+            assertEquals(val1, res.get(key2).get());
+            assertEquals(val3, res.get(key3).get());
+
+            assertEquals(2, res.size());
+
+            cache.remove(key1);
+            cache.put(key2, val1);
+            cache.put(key3, val3);
+        }
+
+        Map<Object, EntryProcessorResult<Object>> res = cache.invokeAll(F.asSet(key1, key2, key3), RMV_PROCESSOR);
+
+        for (int i = 0; i < gridCount(); i++) {
+            assertNull(jcache(i).localPeek(key1, ONHEAP));
+            assertNull(jcache(i).localPeek(key2, ONHEAP));
+            assertNull(jcache(i).localPeek(key3, ONHEAP));
+        }
+
+        assertNull(res.get(key1));
+        assertEquals(val1, res.get(key2).get());
+        assertEquals(val3, res.get(key3).get());
+
+        assertEquals(2, res.size());
+
+        cache.remove(key1);
+        cache.put(key2, val1);
+        cache.put(key3, val3);
+
+        res = cache.invokeAll(F.asSet(key1, key2, key3), INCR_PROCESSOR, dataMode);
+
+        assertEquals(val1, cache.get(key1));
+        assertEquals(val2, cache.get(key2));
+        assertEquals(val4, cache.get(key3));
+
+        assertNull(res.get(key1));
+        assertEquals(val1, res.get(key2).get());
+        assertEquals(val3, res.get(key3).get());
+
+        assertEquals(2, res.size());
+
+        cache.remove(key1);
+        cache.put(key2, val1);
+        cache.put(key3, val3);
+
+        res = cache.invokeAll(F.asMap(key1, INCR_PROCESSOR, key2, INCR_PROCESSOR, key3, INCR_PROCESSOR), dataMode);
+
+        assertEquals(val1, cache.get(key1));
+        assertEquals(val2, cache.get(key2));
+        assertEquals(val4, cache.get(key3));
+
+        assertNull(res.get(key1));
+        assertEquals(val1, res.get(key2).get());
+        assertEquals(val3, res.get(key3).get());
+
+        assertEquals(2, res.size());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testInvokeAllWithNulls() throws Exception {
+        runInAllDataModes(new TestRunnable() {
+            @Override public void run() throws Exception {
+                final Object key1 = key(1);
+
+                final IgniteCache<Object, Object> cache = jcache();
+
+                GridTestUtils.assertThrows(log, new Callable<Void>() {
+                    @Override public Void call() throws Exception {
+                        cache.invokeAll((Set<Object>)null, INCR_PROCESSOR, dataMode);
+
+                        return null;
+                    }
+                }, NullPointerException.class, null);
+
+                GridTestUtils.assertThrows(log, new Callable<Void>() {
+                    @Override public Void call() throws Exception {
+                        cache.invokeAll(F.asSet(key1), null);
+
+                        return null;
+                    }
+                }, NullPointerException.class, null);
+
+                {
+                    final Set<Object> keys = new LinkedHashSet<>(2);
+
+                    keys.add(key1);
+                    keys.add(null);
+
+                    GridTestUtils.assertThrows(log, new Callable<Void>() {
+                        @Override public Void call() throws Exception {
+                            cache.invokeAll(keys, INCR_PROCESSOR, dataMode);
+
+                            return null;
+                        }
+                    }, NullPointerException.class, null);
+
+                    GridTestUtils.assertThrows(log, new Callable<Void>() {
+                        @Override public Void call() throws Exception {
+                            cache.invokeAll(F.asSet(key1), null);
+
+                            return null;
+                        }
+                    }, NullPointerException.class, null);
+                }
+            }
+        });
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testInvokeSequentialOptimisticNoStart() throws Exception {
+        runInAllDataModes(new TestRunnable() {
+            @Override public void run() throws Exception {
+                checkInvokeSequential0(false, OPTIMISTIC);
+            }
+        });
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testInvokeSequentialPessimisticNoStart() throws Exception {
+        runInAllDataModes(new TestRunnable() {
+            @Override public void run() throws Exception {
+                checkInvokeSequential0(false, PESSIMISTIC);
+            }
+        });
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testInvokeSequentialOptimisticWithStart() throws Exception {
+        runInAllDataModes(new TestRunnable() {
+            @Override public void run() throws Exception {
+                checkInvokeSequential0(true, OPTIMISTIC);
+            }
+        });
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testInvokeSequentialPessimisticWithStart() throws Exception {
+        runInAllDataModes(new TestRunnable() {
+            @Override public void run() throws Exception {
+                checkInvokeSequential0(true, PESSIMISTIC);
+            }
+        });
+    }
+
+    /**
+     * @param startVal Whether to put value.
+     * @param concurrency Concurrency.
+     * @throws Exception If failed.
+     */
+    private void checkInvokeSequential0(boolean startVal, TransactionConcurrency concurrency)
+        throws Exception {
+        final Object val1 = value(1);
+        final Object val2 = value(2);
+        final Object val3 = value(3);
+
+        IgniteCache<Object, Object> cache = jcache();
+
+        final Object key = primaryTestObjectKeysForCache(cache, 1).get(0);
+
+        Transaction tx = txShouldBeUsed() ? ignite(0).transactions().txStart(concurrency, READ_COMMITTED) : null;
+
+        try {
+            if (startVal)
+                cache.put(key, val2);
+            else
+                assertEquals(null, cache.get(key));
+
+            Object expRes = startVal ? val2 : null;
+
+            assertEquals(expRes, cache.invoke(key, INCR_PROCESSOR, dataMode));
+
+            expRes = startVal ? val3 : val1;
+
+            assertEquals(expRes, cache.invoke(key, INCR_PROCESSOR, dataMode));
+
+            expRes = value(valueOf(expRes) + 1);
+
+            assertEquals(expRes, cache.invoke(key, INCR_PROCESSOR, dataMode));
+
+            if (tx != null)
+                tx.commit();
+        }
+        finally {
+            if (tx != null)
+                tx.close();
+        }
+
+        Object exp = value((startVal ? 2 : 0) + 3);
+
+        assertEquals(exp, cache.get(key));
+
+        for (int i = 0; i < gridCount(); i++) {
+            if (ignite(i).affinity(cacheName()).isPrimaryOrBackup(grid(i).localNode(), key))
+                assertEquals(exp, peek(jcache(i), key));
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testInvokeAfterRemoveOptimistic() throws Exception {
+        runInAllDataModes(new TestRunnable() {
+            @Override public void run() throws Exception {
+                checkInvokeAfterRemove(OPTIMISTIC);
+            }
+        });
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testInvokeAfterRemovePessimistic() throws Exception {
+        runInAllDataModes(new TestRunnable() {
+            @Override public void run() throws Exception {
+                checkInvokeAfterRemove(PESSIMISTIC);
+            }
+        });
+    }
+
+    /**
+     * @param concurrency Concurrency.
+     * @throws Exception If failed.
+     */
+    private void checkInvokeAfterRemove(TransactionConcurrency concurrency) throws Exception {
+        IgniteCache<Object, Object> cache = jcache();
+
+        Object key = key(1);
+
+        cache.put(key, value(4));
+
+        Transaction tx = txShouldBeUsed() ? ignite(0).transactions().txStart(concurrency, READ_COMMITTED) : null;
+
+        try {
+            cache.remove(key);
+
+            cache.invoke(key, INCR_PROCESSOR, dataMode);
+            cache.invoke(key, INCR_PROCESSOR, dataMode);
+            cache.invoke(key, INCR_PROCESSOR, dataMode);
+
+            if (tx != null)
+                tx.commit();
+        }
+        finally {
+            if (tx != null)
+                tx.close();
+        }
+
+        assertEquals(value(3), cache.get(key));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testInvokeReturnValueGetOptimisticReadCommitted() throws Exception {
+        runInAllDataModes(new TestRunnable() {
+            @Override public void run() throws Exception {
+                checkInvokeReturnValue(false, OPTIMISTIC, READ_COMMITTED);
+            }
+        });
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testInvokeReturnValueGetOptimisticRepeatableRead() throws Exception {
+        runInAllDataModes(new TestRunnable() {
+            @Override public void run() throws Exception {
+                checkInvokeReturnValue(false, OPTIMISTIC, REPEATABLE_READ);
+            }
+        });
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testInvokeReturnValueGetPessimisticReadCommitted() throws Exception {
+        runInAllDataModes(new TestRunnable() {
+            @Override public void run() throws Exception {
+                checkInvokeReturnValue(false, PESSIMISTIC, READ_COMMITTED);
+            }
+        });
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testInvokeReturnValueGetPessimisticRepeatableRead() throws Exception {
+        runInAllDataModes(new TestRunnable() {
+            @Override public void run() throws Exception {
+                checkInvokeReturnValue(false, PESSIMISTIC, REPEATABLE_READ);
+            }
+        });
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testInvokeReturnValuePutInTx() throws Exception {
+        runInAllDataModes(new TestRunnable() {
+            @Override public void run() throws Exception {
+                checkInvokeReturnValue(true, OPTIMISTIC, READ_COMMITTED);
+            }
+        });
+    }
+
+    /**
+     * @param put Whether to put value.
+     * @param concurrency Concurrency.
+     * @param isolation Isolation.
+     * @throws Exception If failed.
+     */
+    private void checkInvokeReturnValue(boolean put,
+        TransactionConcurrency concurrency,
+        TransactionIsolation isolation)
+        throws Exception {
+        IgniteCache<Object, Object> cache = jcache();
+
+        Object key = key(1);
+        Object val1 = value(1);
+        Object val2 = value(2);
+
+        if (!put)
+            cache.put(key, val1);
+
+        Transaction tx = txShouldBeUsed() ? ignite(0).transactions().txStart(concurrency, isolation) : null;
+
+        try {
+            if (put)
+                cache.put(key, val1);
+
+            cache.invoke(key, INCR_PROCESSOR, dataMode);
+
+            assertEquals(val2, cache.get(key));
+
+            if (tx != null) {
+                // Second get inside tx. Make sure read value is not transformed twice.
+                assertEquals(val2, cache.get(key));
+
+                tx.commit();
+            }
+        }
+        finally {
+            if (tx != null)
+                tx.close();
+        }
+    }
+
+    /**
+     * @throws Exception In case of error.
+     */
+    public void testGetAndPutAsync() throws Exception {
+        IgniteCache<String, Integer> cache = jcache();
+
+        IgniteCache<String, Integer> cacheAsync = cache.withAsync();
+
+        cache.put("key1", 1);
+        cache.put("key2", 2);
+
+        cacheAsync.getAndPut("key1", 10);
+
+        IgniteFuture<Integer> fut1 = cacheAsync.future();
+
+        cacheAsync.getAndPut("key2", 11);
+
+        IgniteFuture<Integer> fut2 = cacheAsync.future();
+
+        assertEquals((Integer)1, fut1.get(5000));
+        assertEquals((Integer)2, fut2.get(5000));
+
+        assertEquals((Integer)10, cache.get("key1"));
+        assertEquals((Integer)11, cache.get("key2"));
+    }
+
+    /**
+     * @throws Exception In case of error.
+     */
+    public void testPutAsync0() throws Exception {
+        IgniteCache cacheAsync = jcache().withAsync();
+
+        cacheAsync.getAndPut("key1", 0);
+
+        IgniteFuture<Integer> fut1 = cacheAsync.future();
+
+        cacheAsync.getAndPut("key2", 1);
+
+        IgniteFuture<Integer> fut2 = cacheAsync.future();
+
+        assert fut1.get(5000) == null;
+        assert fut2.get(5000) == null;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testInvokeAsync() throws Exception {
+        runInAllDataModes(new TestRunnable() {
+            @Override public void run() throws Exception {
+                final Object key1 = key(1);
+                final Object key2 = key(2);
+                final Object key3 = key(3);
+
+                final Object val1 = value(1);
+                final Object val2 = value(2);
+                final Object val3 = value(3);
+
+                IgniteCache<Object, Object> cache = jcache();
+
+                cache.put(key2, val1);
+                cache.put(key3, val3);
+
+                IgniteCache<Object, Object> cacheAsync = cache.withAsync();
+
+                assertNull(cacheAsync.invoke(key1, INCR_PROCESSOR, dataMode));
+
+                IgniteFuture<?> fut0 = cacheAsync.future();
+
+                assertNull(cacheAsync.invoke(key2, INCR_PROCESSOR, dataMode));
+
+                IgniteFuture<?> fut1 = cacheAsync.future();
+
+                assertNull(cacheAsync.invoke(key3, RMV_PROCESSOR));
+
+                IgniteFuture<?> fut2 = cacheAsync.future();
+
+                fut0.get();
+                fut1.get();
+                fut2.get();
+
+                assertEquals(val1, cache.get(key1));
+                assertEquals(val2, cache.get(key2));
+                assertNull(cache.get(key3));
+
+                for (int i = 0; i < gridCount(); i++)
+                    assertNull(jcache(i).localPeek(key3, ONHEAP));
+            }
+        });
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testInvoke() throws Exception {
+        runInAllDataModes(new TestRunnable() {
+            @Override public void run() throws Exception {
+                final Object k0 = key(0);
+                final Object k1 = key(1);
+
+                final Object val1 = value(1);
+                final Object val2 = value(2);
+                final Object val3 = value(3);
+
+                final IgniteCache<Object, Object> cache = jcache();
+
+                assertNull(cache.invoke(k0, INCR_PROCESSOR, dataMode));
+
+                assertEquals(k1, cache.get(k0));
+
+                assertEquals(val1, cache.invoke(k0, INCR_PROCESSOR, dataMode));
+
+                assertEquals(val2, cache.get(k0));
+
+                cache.put(k1, val1);
+
+                assertEquals(val1, cache.invoke(k1, INCR_PROCESSOR, dataMode));
+
+                assertEquals(val2, cache.get(k1));
+
+                assertEquals(val2, cache.invoke(k1, INCR_PROCESSOR, dataMode));
+
+                assertEquals(val3, cache.get(k1));
+
+                RemoveAndReturnNullEntryProcessor c = new RemoveAndReturnNullEntryProcessor();
+
+                assertNull(cache.invoke(k1, c));
+                assertNull(cache.get(k1));
+
+                for (int i = 0; i < gridCount(); i++)
+                    assertNull(jcache(i).localPeek(k1, ONHEAP));
+
+                final EntryProcessor<Object, Object, Object> errProcessor = new FailedEntryProcessor();
+
+                GridTestUtils.assertThrows(log, new Callable<Void>() {
+                    @Override public Void call() throws Exception {
+                        cache.invoke(k1, errProcessor);
+
+                        return null;
+                    }
+                }, EntryProcessorException.class, "Test entry processor exception.");
+            }
+        });
+    }
+
+    /**
+     * @throws Exception In case of error.
+     */
+    public void testPutx() throws Exception {
+        if (txShouldBeUsed())
+            checkPut(true);
+    }
+
+    /**
+     * @throws Exception In case of error.
+     */
+    public void testPutxNoTx() throws Exception {
+        checkPut(false);
+    }
+
+    /**
+     * @param inTx Whether to start transaction.
+     * @throws Exception If failed.
+     */
+    private void checkPut(boolean inTx) throws Exception {
+        Transaction tx = inTx ? transactions().txStart() : null;
+
+        IgniteCache<String, Integer> cache = jcache();
+
+        try {
+            cache.put("key1", 1);
+            cache.put("key2", 2);
+
+            // Check inside transaction.
+            assert cache.get("key1") == 1;
+            assert cache.get("key2") == 2;
+
+            if (tx != null)
+                tx.commit();
+        }
+        finally {
+            if (tx != null)
+                tx.close();
+        }
+
+        checkSize(F.asSet("key1", "key2"));
+
+        // Check outside transaction.
+        checkContainsKey(true, "key1");
+        checkContainsKey(true, "key2");
+        checkContainsKey(false, "wrong");
+
+        assert cache.get("key1") == 1;
+        assert cache.get("key2") == 2;
+        assert cache.get("wrong") == null;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPutAsync() throws Exception {
+        Transaction tx = txShouldBeUsed() ? transactions().txStart() : null;
+
+        IgniteCache cacheAsync = jcache().withAsync();
+
+        try {
+            jcache().put("key2", 1);
+
+            cacheAsync.put("key1", 10);
+
+            IgniteFuture<?> fut1 = cacheAsync.future();
+
+            cacheAsync.put("key2", 11);
+
+            IgniteFuture<?> fut2 = cacheAsync.future();
+
+            IgniteFuture<Transaction> f = null;
+
+            if (tx != null) {
+                tx = (Transaction)tx.withAsync();
+
+                tx.commit();
+
+                f = tx.future();
+            }
+
+            assertNull(fut1.get());
+            assertNull(fut2.get());
+
+            assert f == null || f.get().state() == COMMITTED;
+        }
+        finally {
+            if (tx != null)
+                tx.close();
+        }
+
+        checkSize(F.asSet("key1", "key2"));
+
+        assert jcache().get("key1") == 10;
+        assert jcache().get("key2") == 11;
+    }
+
+    /**
+     * @throws Exception In case of error.
+     */
+    public void testPutAll() throws Exception {
+        Map<String, Integer> map = F.asMap("key1", 1, "key2", 2);
+
+        IgniteCache<String, Integer> cache = jcache();
+
+        cache.putAll(map);
+
+        checkSize(F.asSet("key1", "key2"));
+
+        assert cache.get("key1") == 1;
+        assert cache.get("key2") == 2;
+
+        map.put("key1", 10);
+        map.put("key2", 20);
+
+        cache.putAll(map);
+
+        checkSize(F.asSet("key1", "key2"));
+
+        assert cache.get("key1") == 10;
+        assert cache.get("key2") == 20;
+    }
+
+    /**
+     * @throws Exception In case of error.
+     */
+    public void testNullInTx() throws Exception {
+        if (!txShouldBeUsed())
+            return;
+
+        final IgniteCache<String, Integer> cache = jcache();
+
+        for (int i = 0; i < 100; i++) {
+            final String key = "key-" + i;
+
+            assertNull(cache.get(key));
+
+            GridTestUtils.assertThrows(log, new Callable<Void>() {
+                @Override public Void call() throws Exception {
+                    IgniteTransactions txs = transactions();
+
+                    try (Transaction tx = txs.txStart()) {
+                        cache.put(key, 1);
+
+                        cache.put(null, 2);
+
+                        tx.commit();
+                    }
+
+                    return null;
+                }
+            }, NullPointerException.class, null);
+
+            assertNull(cache.get(key));
+
+            cache.put(key, 1);
+
+            assertEquals(1, (int)cache.get(key));
+
+            GridTestUtils.assertThrows(log, new Callable<Void>() {
+                @Override public Void call() throws Exception {
+                    IgniteTransactions txs = transactions();
+
+                    try (Transaction tx = txs.txStart()) {
+                        cache.put(key, 2);
+
+                        cache.remove(null);
+
+                        tx.commit();
+                    }
+
+                    return null;
+                }
+            }, NullPointerException.class, null);
+
+            assertEquals(1, (int)cache.get(key));
+
+            cache.put(key, 2);
+
+            assertEquals(2, (int)cache.get(key));
+
+            GridTestUtils.assertThrows(log, new Callable<Void>() {
+                @Override public Void call() throws Exception {
+                    IgniteTransactions txs = transactions();
+
+                    Map<String, Integer> map = new LinkedHashMap<>();
+
+                    map.put("k1", 1);
+                    map.put("k2", 2);
+                    map.put(null, 3);
+
+                    try (Transaction tx = txs.txStart()) {
+                        cache.put(key, 1);
+
+                        cache.putAll(map);
+
+                        tx.commit();
+                    }
+
+                    return null;
+                }
+            }, NullPointerException.class, null);
+
+            assertNull(cache.get("k1"));
+            assertNull(cache.get("k2"));
+
+            assertEquals(2, (int)cache.get(key));
+
+            cache.put(key, 3);
+
+            assertEquals(3, (int)cache.get(key));
+        }
+    }
+
+    /**
+     * @throws Exception In case of error.
+     */
+    public void testPutAllWithNulls() throws Exception {
+        final IgniteCache<String, Integer> cache = jcache();
+
+        {
+            final Map<String, Integer> m = new LinkedHashMap<>(2);
+
+            m.put("key1", 1);
+            m.put(null, 2);
+
+            GridTestUtils.assertThrows(log, new Callable<Void>() {
+                @Override public Void call() throws Exception {
+                    cache.putAll(m);
+
+                    return null;
+                }
+            }, NullPointerException.class, null);
+
+            cache.put("key1", 1);
+
+            assertEquals(1, (int)cache.get("key1"));
+        }
+
+        {
+            final Map<String, Integer> m = new LinkedHashMap<>(2);
+
+            m.put("key3", 3);
+            m.put("key4", null);
+
+            GridTestUtils.assertThrows(log, new Callable<Void>() {
+                @Override public Void call() throws Exception {
+                    cache.putAll(m);
+
+                    return null;
+                }
+            }, NullPointerException.class, null);
+
+            m.put("key4", 4);
+
+            cache.putAll(m);
+
+            assertEquals(3, (int)cache.get("key3"));
+            assertEquals(4, (int)cache.get("key4"));
+        }
+
+        assertThrows(log, new Callable<Object>() {
+            @Nullable @Override public Object call() throws Exception {
+                cache.put("key1", null);
+
+                return null;
+            }
+        }, NullPointerException.class, A.NULL_MSG_PREFIX);
+
+        assertThrows(log, new Callable<Object>() {
+            @Nullable @Override public Object call() throws Exception {
+                cache.getAndPut("key1", null);
+
+                return null;
+            }
+        }, NullPointerException.class, A.NULL_MSG_PREFIX);
+
+        assertThrows(log, new Callable<Object>() {
+            @Nullable @Override public Object call() throws Exception {
+                cache.put(null, 1);
+
+                return null;
+            }
+        }, NullPointerException.class, A.NULL_MSG_PREFIX);
+
+        assertThrows(log, new Callable<Object>() {
+            @Nullable @Override public Object call() throws Exception {
+                cache.replace(null, 1);
+
+                return null;
+            }
+        }, NullPointerException.class, A.NULL_MSG_PREFIX);
+
+        assertThrows(log, new Callable<Object>() {
+            @Nullable @Override public Object call() throws Exception {
+                cache.getAndReplace(null, 1);
+
+                return null;
+            }
+        }, NullPointerException.class, A.NULL_MSG_PREFIX);
+
+        assertThrows(log, new Callable<Object>() {
+            @Nullable @Override public Object call() throws Exception {
+                cache.replace("key", null);
+
+                return null;
+            }
+        }, NullPointerException.class, A.NULL_MSG_PREFIX);
+
+        assertThrows(log, new Callable<Object>() {
+            @Nullable @Override public Object call() throws Exception {
+                cache.getAndReplace("key", null);
+
+                return null;
+            }
+        }, NullPointerException.class, A.NULL_MSG_PREFIX);
+
+        assertThrows(log, new Callable<Object>() {
+            @Nullable @Override public Object call() throws Exception {
+                cache.replace(null, 1, 2);
+
+                return null;
+            }
+        }, NullPointerException.class, A.NULL_MSG_PREFIX);
+
+        assertThrows(log, new Callable<Object>() {
+            @Nullable @Override public Object call() throws Exception {
+                cache.replace("key", null, 2);
+
+                return null;
+            }
+        }, NullPointerException.class, A.NULL_MSG_PREFIX);
+
+        assertThrows(log, new Callable<Object>() {
+            @Nullable @Override public Object call() throws Exception {
+                cache.replace("key", 1, null);
+
+                return null;
+            }
+        }, NullPointerException.class, A.NULL_MSG_PREFIX);
+    }
+
+    /**
+     * @throws Exception In case of error.
+     */
+    public void testPutAllAsync() throws Exception {
+        Map<String, Integer> map = F.asMap("key1", 1, "key2", 2);
+
+        IgniteCache<String, Integer> cache = jcache();
+
+        IgniteCache<String, Integer> cacheAsync = cache.withAsync();
+
+        cacheAsync.putAll(map);
+
+        IgniteFuture<?> f1 = cacheAsync.future();
+
+        map.put("key1", 10);
+        map.put("key2", 20);
+
+        cacheAsync.putAll(map);
+
+        IgniteFuture<?> f2 = cacheAsync.future();
+
+        assertNull(f2.get());
+        assertNull(f1.get());
+
+        checkSize(F.asSet("key1", "key2"));
+
+        assert cache.get("key1") == 10;
+        assert cache.get("key2") == 20;
+    }
+
+    /**
+     * @throws Exception In case of error.
+     */
+    public void testGetAndPutIfAbsent() throws Exception {
+        Transaction tx = txShouldBeUsed() ? transactions().txStart() : null;
+
+        IgniteCache<String, Integer> cache = jcache();
+
+        try {
+            assert cache.getAndPutIfAbsent("key", 1) == null;
+
+            assert cache.get("key") != null;
+            assert cache.get("key") == 1;
+
+            assert cache.getAndPutIfAbsent("key", 2) != null;
+            assert cache.getAndPutIfAbsent("key", 2) == 1;
+
+            assert cache.get("key") != null;
+            assert cache.get("key") == 1;
+
+            if (tx != null)
+                tx.commit();
+        }
+        finally {
+            if (tx != null)
+                tx.close();
+        }
+
+        assert cache.getAndPutIfAbsent("key", 2) != null;
+
+        for (int i = 0; i < gridCount(); i++) {
+            info("Peek on node [i=" + i + ", id=" + grid(i).localNode().id() + ", val=" +
+                grid(i).cache(cacheName()).localPeek("key", ONHEAP) + ']');
+        }
+
+        assertEquals((Integer)1, cache.getAndPutIfAbsent("key", 2));
+
+        assert cache.get("key") != null;
+        assert cache.get("key") == 1;
+
+        if (!storeEnabled())
+            return;
+
+        // Check swap.
+        cache.put("key2", 1);
+
+        cache.localEvict(Collections.singleton("key2"));
+
+        if (!isLoadPreviousValue())
+            cache.get("key2");
+
+        assertEquals((Integer)1, cache.getAndPutIfAbsent("key2", 3));
+
+        // Check db.
+        if (storeEnabled() && isLoadPreviousValue() && !isMultiJvm()) {
+            putToStore("key3", 3);
+
+            assertEquals((Integer)3, cache.getAndPutIfAbsent("key3", 4));
+
+            assertEquals((Integer)3, cache.get("key3"));
+        }
+
+        assertEquals((Integer)1, cache.get("key2"));
+
+        cache.localEvict(Collections.singleton("key2"));
+
+        if (!isLoadPreviousValue())
+            cache.get("key2");
+
+        // Same checks inside tx.
+        tx = txShouldBeUsed() ? transactions().txStart() : null;
+
+        try {
+            assertEquals((Integer)1, cache.getAndPutIfAbsent("key2", 3));
+
+            if (tx != null)
+                tx.commit();
+
+            assertEquals((Integer)1, cache.get("key2"));
+        }
+        finally {
+            if (tx != null)
+                tx.close();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testGetAndPutIfAbsentAsync() throws Exception {
+        Transaction tx = txShouldBeUsed() ? transactions().txStart() : null;
+
+        IgniteCache<String, Integer> cache = jcache();
+
+        IgniteCache<String, Integer> cacheAsync = cache.withAsync();
+
+        try {
+            cacheAsync.getAndPutIfAbsent("key", 1);
+
+            IgniteFuture<Integer> fut1 = cacheAsync.future();
+
+            assertNull(fut1.get());
+            assertEquals((Integer)1, cache.get("key"));
+
+            cacheAsync.getAndPutIfAbsent("key", 2);
+
+            IgniteFuture<Integer> fut2 = cacheAsync.future();
+
+            assertEquals((Integer)1, fut2.get());
+            assertEquals((Integer)1, cache.get("key"));
+
+            if (tx != null)
+                tx.commit();
+        }
+        finally {
+            if (tx != null)
+                tx.close();
+        }
+
+        if (!storeEnabled())
+            return;
+
+        // Check swap.
+        cache.put("key2", 1);
+
+        cache.localEvict(Collections.singleton("key2"));
+
+        if (!isLoadPreviousValue())
+            cache.get("key2");
+
+        cacheAsync.getAndPutIfAbsent("key2", 3);
+
+        assertEquals((Integer)1, cacheAsync.<Integer>future().get());
+
+        // Check db.
+        if (storeEnabled() && isLoadPreviousValue() && !isMultiJvm()) {
+            putToStore("key3", 3);
+
+            cacheAsync.getAndPutIfAbsent("key3", 4);
+
+            assertEquals((Integer)3, cacheAsync.<Integer>future().get());
+        }
+
+        cache.localEvict(Collections.singleton("key2"));
+
+        if (!isLoadPreviousValue())
+            cache.get("key2");
+
+        // Same checks inside tx.
+        tx = txShouldBeUsed() ? transactions().txStart() : null;
+
+        try {
+            cacheAsync.getAndPutIfAbsent("key2", 3);
+
+            assertEquals(1, cacheAsync.future().get());
+
+            if (tx != null)
+                tx.commit();
+
+            assertEquals((Integer)1, cache.get("key2"));
+        }
+        finally {
+            if (tx != null)
+                tx.close();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPutIfAbsent() throws Exception {
+        IgniteCache<String, Integer> cache = jcache();
+
+        assertNull(cache.get("key"));
+        assert cache.putIfAbsent("key", 1);
+        assert cache.get("key") != null && cache.get("key") == 1;
+        assert !cache.putIfAbsent("key", 2);
+        assert cache.get("key") != null && cache.get("key") == 1;
+
+        if (!storeEnabled())
+            return;
+
+        // Check swap.
+        cache.put("key2", 1);
+
+        cache.localEvict(Collections.singleton("key2"));
+
+        if (!isLoadPreviousValue())
+            cache.get("key2");
+
+        assertFalse(cache.putIfAbsent("key2", 3));
+
+        // Check db.
+        if (storeEnabled() && isLoadPreviousValue() && !isMultiJvm()) {
+            putToStore("key3", 3);
+
+            assertFalse(cache.putIfAbsent("key3", 4));
+        }
+
+        cache.localEvict(Collections.singleton("key2"));
+
+        if (!isLoadPreviousValue())
+            cache.get("key2");
+
+        // Same checks inside tx.
+        Transaction tx = txShouldBeUsed() ? transactions().txStart() : null;
+
+        try {
+            assertFalse(cache.putIfAbsent("key2", 3));
+
+            if (tx != null)
+                tx.commit();
+
+            assertEquals((Integer)1, cache.get("key2"));
+        }
+        finally {
+            if (tx != null)
+                tx.close();
+        }
+    }
+
+    /**
+     * @throws Exception In case of error.
+     */
+    public void testPutxIfAbsentAsync() throws Exception {
+        if (txShouldBeUsed())
+            checkPutxIfAbsentAsync(true);
+    }
+
+    /**
+     * @throws Exception In case of error.
+     */
+    public void testPutxIfAbsentAsyncNoTx() throws Exception {
+        checkPutxIfAbsentAsync(false);
+    }
+
+    /**
+     * @param inTx In tx flag.
+     * @throws Exception If failed.
+     */
+    private void checkPutxIfAbsentAsync(boolean inTx) throws Exception {
+        IgniteCache<String, Integer> cache = jcache();
+
+        IgniteCache<String, Integer> cacheAsync = cache.withAsync();
+
+        cacheAsync.putIfAbsent("key", 1);
+
+        IgniteFuture<Boolean> fut1 = cacheAsync.future();
+
+        assert fut1.get();
+        assert cache.get("key") != null && cache.get("key") == 1;
+
+        cacheAsync.putIfAbsent("key", 2);
+
+        IgniteFuture<Boolean> fut2 = cacheAsync.future();
+
+        assert !fut2.get();
+        assert cache.get("key") != null && cache.get("key") == 1;
+
+        if (!storeEnabled())
+            return;
+
+        // Check swap.
+        cache.put("key2", 1);
+
+        cache.localEvict(Collections.singleton("key2"));
+
+        if (!isLoadPreviousValue())
+            cache.get("key2");
+
+        cacheAsync.putIfAbsent("key2", 3);
+
+        assertFalse(cacheAsync.<Boolean>future().get());
+
+        // Check db.
+        if (storeEnabled() && isLoadPreviousValue() && !isMultiJvm()) {
+            putToStore("key3", 3);
+
+            cacheAsync.putIfAbsent("key3", 4);
+
+            assertFalse(cacheAsync.<Boolean>future().get());
+        }
+
+        cache.localEvict(Collections.singletonList("key2"));
+
+        if (!isLoadPreviousValue())
+            cache.get("key2");
+
+        // Same checks inside tx.
+        Transaction tx = inTx ? transactions().txStart() : null;
+
+        try {
+            cacheAsync.putIfAbsent("key2", 3);
+
+            assertFalse(cacheAsync.<Boolean>future().get());
+
+            if (storeEnabled() && isLoadPreviousValue() && !isMultiJvm()) {
+                cacheAsync.putIfAbsent("key3", 4);
+
+                assertFalse(cacheAsync.<Boolean>future().get());
+            }
+
+            if (tx != null)
+                tx.commit();
+        }
+        finally {
+            if (tx != null)
+                tx.close();
+        }
+
+        assertEquals((Integer)1, cache.get("key2"));
+
+        if (storeEnabled() && isLoadPreviousValue() && !isMultiJvm())
+            assertEquals((Integer)3, cache.get("key3"));
+    }
+
+    /**
+     * @throws Exception In case of error.
+     */
+    public void testPutIfAbsentAsyncConcurrent() throws Exception {
+        IgniteCache cacheAsync = jcache().withAsync();
+
+        cacheAsync.putIfAbsent("key1", 1);
+
+        IgniteFuture<Boolean> fut1 = cacheAsync.future();
+
+        cacheAsync.putIfAbsent("key2", 2);
+
+        IgniteFuture<Boolean> fut2 = cacheAsync.future();
+
+        assert fut1.get();
+        assert fut2.get();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testGetAndReplace() throws Exception {
+        IgniteCache<String, Integer> cache = jcache();
+
+        cache.put("key", 1);
+
+        assert cache.get("key") == 1;
+
+        info("key 1 -> 2");
+
+        assert cache.getAndReplace("key", 2) == 1;
+
+        assert cache.get("key") == 2;
+
+        assert cache.getAndReplace("wrong", 0) == null;
+
+        assert cache.get("wrong") == null;
+
+        info("key 0 -> 3");
+
+        assert !cache.replace("key", 0, 3);
+
+        assert cache.get("key") == 2;
+
+        info("key 0 -> 3");
+
+        assert !cache.replace("key", 0, 3);
+
+        assert cache.get("key") == 2;
+
+        info("key 2 -> 3");
+
+        assert cache.replace("key", 2, 3);
+
+        assert cache.get("key") == 3;
+
+        if (!storeEnabled())
+            return;
+
+        info("evict key");
+
+        cache.localEvict(Collections.singleton("key"));
+
+        info("key 3 -> 4");
+
+        if (!isLoadPreviousValue())
+            cache.get("key");
+
+        assert cache.replace("key", 3, 4);
+
+        assert cache.get("key") == 4;
+
+        if (storeEnabled() && isLoadPreviousValue() && !isMultiJvm()) {
+            putToStore("key2", 5);
+
+            info("key2 5 -> 6");
+
+            assert cache.replace("key2", 5, 6);
+        }
+
+        for (int i = 0; i < gridCount(); i++) {
+            info("Peek key on grid [i=" + i + ", nodeId=" + grid(i).localNode().id() +
+                ", peekVal=" + grid(i).cache(cacheName()).localPeek("key", ONHEAP) + ']');
+
+            info("Peek key2 on grid [i=" + i + ", nodeId=" + grid(i).localNode().id() +
+                ", peekVal=" + grid(i).cache(cacheName()).localPeek("key2", ONHEAP) + ']');
+        }
+
+        if (storeEnabled() && isLoadPreviousValue() && !isMultiJvm())
+            assertEquals((Integer)6, cache.get("key2"));
+
+        cache.localEvict(Collections.singleton("key"));
+
+        if (!isLoadPreviousValue())
+            cache.get("key");
+
+        Transaction tx = txShouldBeUsed() ? transactions().txStart() : null;
+
+        try {
+            assert cache.replace("key", 4, 5);
+
+            if (tx != null)
+                tx.commit();
+
+            assert cache.get("key") == 5;
+        }
+        finally {
+            if (tx != null)
+                tx.close();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testReplace() throws Exception {
+        IgniteCache<String, Integer> cache = jcache();
+
+        cache.put("key", 1);
+
+        assert cache.get("key") == 1;
+
+        assert cache.replace("key", 2);
+
+        assert cache.get("key") == 2;
+
+        assert !cache.replace("wrong", 2);
+
+        if (!storeEnabled())
+            return;
+
+        cache.localEvict(Collections.singleton("key"));
+
+        if (!isLoadPreviousValue())
+            assert cache.get("key") == 2;
+
+        assert cache.replace("key", 4);
+
+        assert cache.get("key") == 4;
+
+        if (storeEnabled() && isLoadPreviousValue() && !isMultiJvm()) {
+            putToStore("key2", 5);
+
+            cache.replace("key2", 6);
+
+            assertEquals((Integer)6, cache.get("key2"));
+        }
+
+        cache.localEvict(Collections.singleton("key"));
+
+        if (!isLoadPreviousValue())
+            assert cache.get("key") == 4;
+
+        Transaction tx = txShouldBeUsed() ? transactions().txStart() : null;
+
+        try {
+            assert cache.replace("key", 5);
+
+            if (tx != null)
+                tx.commit();
+        }
+        finally {
+            if (tx != null)
+                tx.close();
+        }
+
+        assert cache.get("key") == 5;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testGetAndReplaceAsync() throws Exception {
+        IgniteCache<String, Integer> cache = jcache();
+
+        IgniteCache<String, Integer> cacheAsync = cache.withAsync();
+
+        cache.put("key", 1);
+
+        assert cache.get("key") == 1;
+
+        cacheAsync.getAndReplace("key", 2);
+
+        assert cacheAsync.<Integer>future().get() == 1;
+
+        assert cache.get("key") == 2;
+
+        cacheAsync.getAndReplace("wrong", 0);
+
+        assert cacheAsync.future().get() == null;
+
+        assert cache.get("wrong") == null;
+
+        cacheAsync.replace("key", 0, 3);
+
+        assert !cacheAsync.<Boolean>future().get();
+
+        assert cache.get("key") == 2;
+
+        cacheAsync.replace("key", 0, 3);
+
+        assert !cacheAsync.<Boolean>future().get();
+
+        assert cache.get("key") == 2;
+
+        cacheAsync.replace("key", 2, 3);
+
+        assert cacheAsync.<Boolean>future().get();
+
+        assert cache.get("key") == 3;
+
+        if (!storeEnabled())
+            return;
+
+        cache.localEvict(Collections.singleton("key"));
+
+        if (!isLoadPreviousValue())
+            cache.get("key");
+
+        cacheAsync.replace("key", 3, 4);
+
+        assert cacheAsync.<Boolean>future().get();
+
+        assert cache.get("key") == 4;
+
+        if (storeEnabled() && isLoadPreviousValue() && !isMultiJvm()) {
+            putToStore("key2", 5);
+
+            cacheAsync.replace("key2", 5, 6);
+
+            assert cacheAsync.<Boolean>future().get();
+
+            assertEquals((Integer)6, cache.get("key2"));
+        }
+
+        cache.localEvict(Collections.singleton("key"));
+
+        if (!isLoadPreviousValue())
+            cache.get("key");
+
+        Transaction tx = txShouldBeUsed() ? transactions().txStart() : null;
+
+        try {
+            cacheAsync.replace("key", 4, 5);
+
+            assert cacheAsync.<Boolean>future().get();
+
+            if (tx != null)
+                tx.commit();
+        }
+        finally {
+            if (tx != null)
+                tx.close();
+        }
+
+        assert cache.get("key") == 5;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testReplacexAsync() throws Exception {
+        IgniteCache<String, Integer> cache = jcache();
+
+        IgniteCache<String, Integer> cacheAsync = cache.withAsync();
+
+        cache.put("key", 1);
+
+        assert cache.get("key") == 1;
+
+        cacheAsync.replace("key", 2);
+
+        assert cacheAsync.<Boolean>future().get();
+
+        info("Finished replace.");
+
+        assertEquals((Integer)2, cache.get("key"));
+
+        cacheAsync.replace("wrond", 2);
+
+        assert !cacheAsync.<Boolean>future().get();
+
+        if (!storeEnabled())
+            return;
+
+        cache.localEvict(Collections.singleton("key"));
+
+        if (!isLoadPreviousValue())
+            cache.get("key");
+
+        cacheAsync.replace("key", 4);
+
+        assert cacheAsync.<Boolean>future().get();
+
+        assert cache.get("key") == 4;
+
+        if (storeEnabled() && isLoadPreviousValue() && !isMultiJvm()) {
+            putToStore("key2", 5);
+
+            cacheAsync.replace("key2", 6);
+
+            assert cacheAsync.<Boolean>future().get();
+
+            assert cache.get("key2") == 6;
+        }
+
+        cache.localEvict(Collections.singleton("key"));
+
+        if (!isLoadPreviousValue())
+            cache.get("key");
+
+        Transaction tx = txShouldBeUsed() ? transactions().txStart() : null;
+
+        try {
+            cacheAsync.replace("key", 5);
+
+            assert cacheAsync.<Boolean>future().get();
+
+            if (tx != null)
+                tx.commit();
+        }
+        finally {
+            if (tx != null)
+                tx.close();
+        }
+
+        assert cache.get("key") == 5;
+    }
+
+    /**
+     * @throws Exception In case of error.
+     */
+    public void testGetAndRemove() throws Exception {
+        IgniteCache<String, Integer> cache = jcache();
+
+        cache.put("key1", 1);
+        cache.put("key2", 2);
+
+        assert !cache.remove("key1", 0);
+        assert cache.get("key1") != null && cache.get("key1") == 1;
+        assert cache.remove("key1", 1);
+        assert cache.get("key1") == null;
+        assert cache.getAndRemove("key2") == 2;
+        assert cache.get("key2") == null;
+        assert cache.getAndRemove("key2") == null;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testGetAndRemoveObject() throws Exception {
+        IgniteCache<String, SerializableObject> cache = ignite(0).cache(cacheName());
+
+        SerializableObject val1 = new SerializableObject(1);
+        SerializableObject val2 = new SerializableObject(2);
+
+        cache.put("key1", val1);
+        cache.put("key2", val2);
+
+        assert !cache.remove("key1", new SerializableObject(0));
+
+        SerializableObject oldVal = cache.get("key1");
+
+        assert oldVal != null && F.eq(val1, oldVal);
+
+        assert cache.remove("key1");
+
+        assert cache.get("key1") == null;
+
+        SerializableObject oldVal2 = cache.getAndRemove("key2");
+
+        assert F.eq(val2, oldVal2);
+
+        assert cache.get("key2") == null;
+        assert cache.getAndRemove("key2") == null;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testGetAndPutSerializableObject() throws Exception {
+        IgniteCache<String, SerializableObject> cache = ignite(0).cache(cacheName());
+
+        SerializableObject val1 = new SerializableObject(1);
+        SerializableObject val2 = new SerializableObject(2);
+
+        cache.put("key1", val1);
+
+        SerializableObject oldVal = cache.get("key1");
+
+        assertEquals(val1, oldVal);
+
+        oldVal = cache.getAndPut("key1", val2);
+
+        assertEquals(val1, oldVal);
+
+        SerializableObject updVal = cache.get("key1");
+
+        assertEquals(val2, updVal);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDeletedEntriesFlag() throws Exception {
+        if (cacheMode() != LOCAL && cacheMode() != REPLICATED && memoryMode() != OFFHEAP_TIERED) {
+            final int cnt = 3;
+
+            IgniteCache<String, Integer> cache = jcache();
+
+            for (int i = 0; i < cnt; i++)
+                cache.put(String.valueOf(i), i);
+
+            for (int i = 0; i < cnt; i++)
+                cache.remove(String.valueOf(i));
+
+            for (int g = 0; g < gridCount(); g++)
+                executeOnLocalOrRemoteJvm(g, new CheckEntriesDeletedTask(cnt, cacheName()));
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testRemoveLoad() throws Exception {
+        if (isMultiJvm())
+            fail("https://issues.apache.org/jira/browse/IGNITE-1088");
+
+        if (!storeEnabled())
+            return;
+
+        int cnt = 10;
+
+        Set<String> keys = new HashSet<>();
+
+        for (int i = 0; i < cnt; i++)
+            keys.add(String.valueOf(i));
+
+        jcache().removeAll(keys);
+
+        for (String key : keys)
+            putToStore(key, Integer.parseInt(key));
+
+        for (int g = 0; g < gridCount(); g++)
+            grid(g).cache(cacheName()).localLoadCache(null);
+
+        for (int g = 0; g < gridCount(); g++) {
+            for (int i = 0; i < cnt; i++) {
+                String key = String.valueOf(i);
+
+                if (grid(0).affinity(cacheName()).mapKeyToPrimaryAndBackups(key).contains(grid(g).localNode()))
+                    assertEquals((Integer)i, peek(jcache(g), key));
+                else
+                    assertNull(peek(jcache(g), key));
+            }
+        }
+    }
+
+    /**
+     * @throws Exception In case of error.
+     */
+    public void testRemoveAsync() throws Exception {
+        IgniteCache<String, Integer> cache = jcache();
+
+        IgniteCache<String, Integer> cacheAsync = cache.withAsync();
+
+        cache.put("key1", 1);
+        cache.put("key2", 2);
+
+        cacheAsync.remove("key1", 0);
+
+        assert !cacheAsync.<Boolean>future().get();
+
+        assert cache.get("key1") != null && cache.get("key1") == 1;
+
+        cacheAsync.remove("key1", 1);
+
+        assert cacheAsync.<Boolean>future().get();
+
+        assert cache.get("key1") == null;
+
+        cacheAsync.getAndRemove("key2");
+
+        assert cacheAsync.<Integer>future().get() == 2;
+
+        assert cache.get("key2") == null;
+
+        cacheAsync.getAndRemove("key2");
+
+        assert cacheAsync.future().get() == null;
+    }
+
+    /**
+     * @throws Exception In case of error.
+     */
+    public void testRemove() throws Exception {
+        IgniteCache<String, Integer> cache = jcache();
+
+        cache.put("key1", 1);
+
+        assert cache.remove("key1");
+        assert cache.get("key1") == null;
+        assert !cache.remove("key1");
+    }
+
+    /**
+     * @throws Exception In case of error.
+     */
+    public void testRemovexAsync() throws Exception {
+        IgniteCache<String, Integer> cache = jcache();
+
+        IgniteCache<String, Integer> cacheAsync = cache.withAsync();
+
+        cache.put("key1", 1);
+
+        cacheAsync.remove("key1");
+
+        assert cacheAsync.<Boolean>future().get();
+
+        assert cache.get("key1") == null;
+
+        cacheAsync.remove("key1");
+
+        assert !cacheAsync.<Boolean>future().get();
+    }
+
+    /**
+     * @throws Exception In case of error.
+     */
+    public void testGlobalRemoveAll() throws Exception {
+        globalRemoveAll(false);
+    }
+
+    /**
+     * @throws Exception In case of error.
+     */
+    public void testGlobalRemoveAllAsync() throws Exception {
+        globalRemoveAll(true);
+    }
+
+    /**
+     * @param async If {@code true} uses asynchronous operation.
+     * @throws Exception In case of error.
+     */
+    private void globalRemoveAll(boolean async) throws Exception {
+        IgniteCache<String, Integer> cache = jcache();
+
+        cache.put("key1", 1);
+        cache.put("key2", 2);
+        cache.put("key3", 3);
+
+        checkSize(F.asSet("key1", "key2", "key3"));
+
+        atomicClockModeDelay(cache);
+
+        IgniteCache<String, Integer> asyncCache = cache.withAsync();
+
+        if (async) {
+            asyncCache.removeAll(F.asSet("key1", "key2"));
+
+            asyncCache.future().get();
+        }
+        else
+            cache.removeAll(F.asSet("key1", "key2"));
+
+        checkSize(F.asSet("key3"));
+
+        checkContainsKey(false, "key1");
+        checkContainsKey(false, "key2");
+        checkContainsKey(true, "key3");
+
+        // Put values again.
+        cache.put("key1", 1);
+        cache.put("key2", 2);
+        cache.put("key3", 3);
+
+        atomicClockModeDelay(cache);
+
+        if (async) {
+            IgniteCache asyncCache0 = jcache(gridCount() > 1 ? 1 : 0).withAsync();
+
+            asyncCache0.removeAll();
+
+            asyncCache0.future().get();
+        }
+        else
+            jcache(gridCount() > 1 ? 1 : 0).removeAll();
+
+        assertEquals(0, cache.localSize());
+        long entryCnt = hugeRemoveAllEntryCount();
+
+        for (int i = 0; i < entryCnt; i++)
+            cache.put(String.valueOf(i), i);
+
+        for (int i = 0; i < entryCnt; i++)
+            assertEquals(Integer.valueOf(i), cache.get(String.valueOf(i)));
+
+        atomicClockModeDelay(cache);
+
+        if (async) {
+            asyncCache.removeAll();
+
+            asyncCache.future().get();
+        }
+        else
+            cache.removeAll();
+
+        for (int i = 0; i < entryCnt; i++)
+            assertNull(cache.get(String.valueOf(i)));
+    }
+
+    /**
+     * @return Count of entries to be removed in removeAll() test.
+     */
+    protected long hugeRemoveAllEntryCount() {
+        return 1000L;
+    }
+
+    /**
+     * @throws Exception In case of error.
+     */
+    public void testRemoveAllWithNulls() throws Exception {
+        final IgniteCache<String, Integer> cache = jcache();
+
+        final Set<String> c = new LinkedHashSet<>();
+
+        c.add("key1");
+        c.add(null);
+
+        GridTestUtils.assertThrows(log, new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                cache.removeAll(c);
+
+                return null;
+            }
+        }, NullPointerException.class, null);
+
+        assertEquals(0, jcache().localSize());
+
+        GridTestUtils.assertThrows(log, new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                cache.removeAll(null);
+
+                return null;
+            }
+        }, NullPointerException.class, null);
+
+        GridTestUtils.assertThrows(log, new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                cache.remove(null);
+
+                return null;
+            }
+        }, NullPointerException.class, null);
+
+        GridTestUtils.assertThrows(log, new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                cache.getAndRemove(null);
+
+                return null;
+            }
+        }, NullPointerException.class, null);
+
+        GridTestUtils.assertThrows(log, new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                cache.remove("key1", null);
+
+                return null;
+            }
+        }, NullPointerException.class, null);
+    }
+
+    /**
+     * @throws Exception In case of error.
+     */
+    public void testRemoveAllDuplicates() throws Exception {
+        jcache().removeAll(ImmutableSet.of("key1", "key1", "key1"));
+    }
+
+    /**
+     * @throws Exception In case of error.
+     */
+    public void testRemoveAllDuplicatesTx() throws Exception {
+        if (txShouldBeUsed()) {
+            try (Transaction tx = transactions().txStart()) {
+                jcache().removeAll(ImmutableSet.of("key1", "key1", "key1"));
+
+                tx.commit();
+            }
+        }
+    }
+
+    /**
+     * @throws Exception In case of error.
+     */
+    public void testRemoveAllEmpty() throws Exception {
+        jcache().removeAll();
+    }
+
+    /**
+     * @throws Exception In case of error.
+     */
+    public void testRemoveAllAsync() throws Exception {
+        IgniteCache<String, Integer> cache = jcache();
+
+        IgniteCache<String, Integer> cacheAsync = cache.withAsync();
+
+        cache.put("key1", 1);
+        cache.put("key2", 2);
+        cache.put("key3", 3);
+
+        checkSize(F.asSet("key1", "key2", "key3"));
+
+        cacheAsync.removeAll(F.asSet("key1", "key2"));
+
+        assertNull(cacheAsync.future().get());
+
+        checkSize(F.asSet("key3"));
+
+        checkContainsKey(false, "key1");
+        checkContainsKey(false, "key2");
+        checkContainsKey(true, "key3");
+    }
+
+    /**
+     * @throws Exception In case of error.
+     */
+    public void testLoadAll() throws Exception {
+        if (!storeEnabled())
+            return;
+
+        IgniteCache<String, Integer> cache = jcache();
+
+        Set<String> keys = new HashSet<>(primaryKeysForCache(2));
+
+        for (String key : keys)
+            assertNull(cache.localPeek(key, ONHEAP));
+
+        Map<String, Integer> vals = new HashMap<>();
+
+        int i = 0;
+
+        for (String key : keys) {
+            cache.put(key, i);
+
+            vals.put(key, i);
+
+            i++;
+        }
+
+        for (String key : keys)
+            assertEquals(vals.get(key), peek(cache, key));
+
+        cache.clear();
+
+        for (String key : keys)
+            assertNull(peek(cache, key));
+
+        loadAll(cache, keys, true);
+
+        for (String key : keys)
+            assertEquals(vals.get(key), peek(cache, key));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testRemoveAfterClear() throws Exception {
+        IgniteEx ignite = grid(0);
+
+        boolean affNode = ignite.context().cache().internalCache(cacheName()).context().affinityNode();
+
+        if (!affNode) {
+            if (gridCount() < 2)
+                return;
+
+            ignite = grid(1);
+        }
+
+        IgniteCache<Integer, Integer> cache = ignite.cache(cacheName());
+
+        int key = 0;
+
+        Collection<Integer> keys = new ArrayList<>();
+
+        for (int k = 0; k < 2; k++) {
+            while (!ignite.affinity(cacheName()).isPrimary(ignite.localNode(), key))
+                key++;
+
+            keys.add(key);
+
+            key++;
+        }
+
+        info("Keys: " + keys);
+
+        for (Integer k : keys)
+            cache.put(k, k);
+
+        cache.clear();
+
+        for (int g = 0; g < gridCount(); g++) {
+            Ignite

<TRUNCATED>

[2/4] ignite git commit: ignite-2521: Configuration variations tests framework + IgniteCacheBasicConfigVariationsFullApiTestSuite + ignite-2554: Fixed Affinity.mapKeyToNode() for dynamically started LOCAL cache

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/953b575f/modules/core/src/test/java/org/apache/ignite/testframework/configvariations/CacheStartMode.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/configvariations/CacheStartMode.java b/modules/core/src/test/java/org/apache/ignite/testframework/configvariations/CacheStartMode.java
new file mode 100644
index 0000000..ef8bdf0
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/configvariations/CacheStartMode.java
@@ -0,0 +1,29 @@
+/*
+ * 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.testframework.configvariations;
+
+/**
+ * Cache start mode.
+ */
+public enum CacheStartMode {
+    /** Start caches together with nodes (not dynamically). */
+    STATIC,
+
+    /** Starts nodes first and then starts caches dynamically. */
+    DYNAMIC
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/953b575f/modules/core/src/test/java/org/apache/ignite/testframework/configvariations/ConfigFactory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/configvariations/ConfigFactory.java b/modules/core/src/test/java/org/apache/ignite/testframework/configvariations/ConfigFactory.java
new file mode 100644
index 0000000..9541c1a
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/configvariations/ConfigFactory.java
@@ -0,0 +1,39 @@
+/*
+ * 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.testframework.configvariations;
+
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+
+/**
+ * Configuration factory.
+ */
+public interface ConfigFactory {
+    /**
+     * @param gridName Grid name.
+     * @param srcCfg Source config.
+     * @return IgniteConfiguration.
+     */
+    public IgniteConfiguration getConfiguration(String gridName, IgniteConfiguration srcCfg);
+
+    /**
+     * @param gridName Name.
+     * @return CacheConfiguration.
+     */
+    public CacheConfiguration cacheConfiguration(String gridName);
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/953b575f/modules/core/src/test/java/org/apache/ignite/testframework/configvariations/ConfigParameter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/configvariations/ConfigParameter.java b/modules/core/src/test/java/org/apache/ignite/testframework/configvariations/ConfigParameter.java
new file mode 100644
index 0000000..5a29d25
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/configvariations/ConfigParameter.java
@@ -0,0 +1,34 @@
+/*
+ * 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.testframework.configvariations;
+
+/**
+ * Configuration parameter.
+ */
+public interface ConfigParameter<T> {
+    /**
+     * @return Name
+     */
+    public String name();
+
+    /**
+     * @param cfg Configuration.
+     * @return Configuration.
+     */
+    public T apply(T cfg);
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/953b575f/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
new file mode 100644
index 0000000..e5856f0
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/configvariations/ConfigVariations.java
@@ -0,0 +1,346 @@
+/*
+ * 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.testframework.configvariations;
+
+import java.util.Collection;
+import javax.cache.Cache;
+import javax.cache.configuration.CacheEntryListenerConfiguration;
+import javax.cache.configuration.Factory;
+import javax.cache.configuration.MutableCacheEntryListenerConfiguration;
+import javax.cache.event.CacheEntryCreatedListener;
+import javax.cache.event.CacheEntryEventFilter;
+import javax.cache.event.CacheEntryListener;
+import javax.cache.event.CacheEntryListenerException;
+import org.apache.ignite.cache.CacheAtomicWriteOrderMode;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheInterceptorAdapter;
+import org.apache.ignite.cache.CacheMemoryMode;
+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;
+import org.apache.ignite.cache.store.CacheStoreSessionListener;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.NearCacheConfiguration;
+import org.apache.ignite.configuration.TopologyValidator;
+import org.apache.ignite.internal.binary.BinaryMarshaller;
+import org.apache.ignite.marshaller.optimized.OptimizedMarshaller;
+import org.apache.ignite.spi.swapspace.inmemory.GridTestSwapSpaceSpi;
+import org.apache.ignite.testframework.junits.IgniteCacheConfigVariationsAbstractTest;
+
+import static org.apache.ignite.internal.util.lang.GridFunc.asArray;
+
+/**
+ * Cache configuration variations.
+ */
+@SuppressWarnings("serial")
+public class ConfigVariations {
+    /** */
+    private static final ConfigParameter<Object> EVICTION_PARAM = Parameters.complexParameter(
+        Parameters.parameter("setEvictionPolicy", Parameters.factory(FifoEvictionPolicy.class)),
+        Parameters.parameter("setEvictionFilter", Parameters.factory(NoopEvictionFilter.class))
+    );
+
+    /** */
+    private static final ConfigParameter<Object> CACHE_STORE_PARAM = Parameters.complexParameter(
+        Parameters.parameter("setCacheStoreFactory", Parameters.factory(IgniteCacheConfigVariationsAbstractTest.TestStoreFactory.class)),
+        Parameters.parameter("setReadThrough", true),
+        Parameters.parameter("setWriteThrough", true),
+        Parameters.parameter("setCacheStoreSessionListenerFactories", noopCacheStoreSessionListenerFactory())
+    );
+
+    /** */
+    private static final ConfigParameter<Object> SIMPLE_CACHE_STORE_PARAM = Parameters.complexParameter(
+        Parameters.parameter("setCacheStoreFactory", Parameters.factory(IgniteCacheConfigVariationsAbstractTest.TestStoreFactory.class)),
+        Parameters.parameter("setReadThrough", true),
+        Parameters.parameter("setWriteThrough", true)
+    );
+
+    /** */
+    private static final ConfigParameter<Object> REBALANCING_PARAM = Parameters.complexParameter(
+        Parameters.parameter("setRebalanceBatchSize", 2028 * 1024),
+        Parameters.parameter("setRebalanceBatchesPrefetchCount", 5L),
+        Parameters.parameter("setRebalanceThreadPoolSize", 5),
+        Parameters.parameter("setRebalanceTimeout", CacheConfiguration.DFLT_REBALANCE_TIMEOUT * 2),
+        Parameters.parameter("setRebalanceDelay", 1000L)
+    );
+
+    /** */
+    private static final ConfigParameter<Object> ONHEAP_TIERED_MEMORY_PARAM =
+        Parameters.parameter("setMemoryMode", CacheMemoryMode.ONHEAP_TIERED);
+
+    /** */
+    private static final ConfigParameter<Object> OFFHEAP_TIERED_MEMORY_PARAM =
+        Parameters.parameter("setMemoryMode", CacheMemoryMode.OFFHEAP_TIERED);
+
+    /** */
+    private static final ConfigParameter<Object> OFFHEAP_VALUES_MEMORY_PARAM =
+        Parameters.parameter("setMemoryMode", CacheMemoryMode.OFFHEAP_VALUES);
+
+    /** */
+    private static final ConfigParameter<Object> OFFHEAP_ENABLED =
+        Parameters.parameter("setOffHeapMaxMemory", 10 * 1024 * 1024L);
+
+    /** */
+    @SuppressWarnings("unchecked")
+    private static final ConfigParameter<IgniteConfiguration>[][] BASIC_IGNITE_SET = new ConfigParameter[][] {
+        Parameters.objectParameters("setMarshaller", Parameters.factory(BinaryMarshaller.class), optimizedMarshallerFactory()),
+        Parameters.booleanParameters("setPeerClassLoadingEnabled"),
+        Parameters.objectParameters("setSwapSpaceSpi", Parameters.factory(GridTestSwapSpaceSpi.class)),
+    };
+
+    /** */
+    @SuppressWarnings("unchecked")
+    private static final ConfigParameter<CacheConfiguration>[][] BASIC_CACHE_SET = new ConfigParameter[][] {
+        Parameters.objectParameters("setCacheMode", CacheMode.REPLICATED, CacheMode.PARTITIONED),
+        Parameters.enumParameters("setAtomicityMode", CacheAtomicityMode.class),
+        Parameters.enumParameters("setMemoryMode", CacheMemoryMode.class),
+        // Set default parameters.
+        Parameters.objectParameters("setLoadPreviousValue", true),
+        Parameters.objectParameters("setSwapEnabled", true),
+        asArray(SIMPLE_CACHE_STORE_PARAM),
+        Parameters.objectParameters("setWriteSynchronizationMode", CacheWriteSynchronizationMode.FULL_SYNC),
+        Parameters.objectParameters("setAtomicWriteOrderMode", CacheAtomicWriteOrderMode.PRIMARY),
+        Parameters.objectParameters("setStartSize", 1024),
+    };
+
+    /** */
+    @SuppressWarnings("unchecked")
+    private static final ConfigParameter<CacheConfiguration>[][] FULL_CACHE_SET = new ConfigParameter[][] {
+        Parameters.enumParameters("setCacheMode", CacheMode.class),
+        Parameters.enumParameters("setAtomicityMode", CacheAtomicityMode.class),
+        asArray(ONHEAP_TIERED_MEMORY_PARAM,
+            Parameters.complexParameter(ONHEAP_TIERED_MEMORY_PARAM, OFFHEAP_ENABLED),
+            Parameters.complexParameter(OFFHEAP_TIERED_MEMORY_PARAM, OFFHEAP_ENABLED),
+            Parameters.complexParameter(OFFHEAP_VALUES_MEMORY_PARAM, OFFHEAP_ENABLED)
+        ),
+        Parameters.booleanParameters("setLoadPreviousValue"),
+        Parameters.booleanParameters("setReadFromBackup"),
+        Parameters.booleanParameters("setStoreKeepBinary"),
+        Parameters.objectParameters("setRebalanceMode", CacheRebalanceMode.SYNC, CacheRebalanceMode.ASYNC),
+        Parameters.booleanParameters("setSwapEnabled"),
+        Parameters.booleanParameters("setCopyOnRead"),
+        Parameters.objectParameters(true, "setNearConfiguration", nearCacheConfigurationFactory()),
+        asArray(null,
+            Parameters.complexParameter(
+                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))
+            )
+        ),
+        // Set default parameters.
+        Parameters.objectParameters("setWriteSynchronizationMode", CacheWriteSynchronizationMode.FULL_SYNC),
+        Parameters.objectParameters("setAtomicWriteOrderMode", CacheAtomicWriteOrderMode.PRIMARY),
+        Parameters.objectParameters("setStartSize", 1024),
+    };
+
+    /**
+     * Private constructor.
+     */
+    private ConfigVariations() {
+        // No-op.
+    }
+
+    /**
+     * @return Custom near cache config.
+     */
+    private static Factory nearCacheConfigurationFactory() {
+        return new Factory() {
+            @Override public Object create() {
+                NearCacheConfiguration cfg = new NearCacheConfiguration<>();
+
+                cfg.setNearEvictionPolicy(new FifoEvictionPolicy());
+
+                return cfg;
+            }
+        };
+    }
+
+    /**
+     * @return Noop cache store session listener factory.
+     */
+    private static Factory noopCacheStoreSessionListenerFactory() {
+        return new Factory() {
+            @Override public Object create() {
+                return new Factory[] {new NoopCacheStoreSessionListenerFactory()};
+            }
+        };
+    }
+
+    /**
+     * @return Default matrix of availiable variations.
+     */
+    public static ConfigParameter<CacheConfiguration>[][] cacheBasicSet() {
+        return BASIC_CACHE_SET;
+    }
+
+    /**
+     * @return Full matrix of availiable variations.
+     */
+    public static ConfigParameter<CacheConfiguration>[][] cacheFullSet() {
+        return FULL_CACHE_SET;
+    }
+
+    /**
+     * @return Default matrix of availiable variations.
+     */
+    public static ConfigParameter<IgniteConfiguration>[][] igniteBasicSet() {
+        return BASIC_IGNITE_SET;
+    }
+
+    /**
+     * @return Marshaller.
+     */
+    private static Factory<OptimizedMarshaller> optimizedMarshallerFactory() {
+        return new Factory<OptimizedMarshaller>() {
+            @Override public OptimizedMarshaller create() {
+                OptimizedMarshaller marsh = new OptimizedMarshaller(true);
+
+                marsh.setRequireSerializable(false);
+
+                return marsh;
+            }
+        };
+    }
+
+    /**
+     *
+     */
+    public static class NoopEvictionFilter implements EvictionFilter {
+        /** */
+        private static final long serialVersionUID = 0;
+
+        /** {@inheritDoc} */
+        @Override public boolean evictAllowed(Cache.Entry entry) {
+            return true;
+        }
+    }
+
+    /**
+     *
+     */
+    public static class NoopInterceptor extends CacheInterceptorAdapter {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        // No-op.
+    }
+
+    /**
+     *
+     */
+    public static class NoopCacheStoreSessionListenerFactory implements Factory<NoopCacheStoreSessionListener> {
+        /** Serial version uid. */
+        private static final long serialVersionUID = 0L;
+
+        /** {@inheritDoc} */
+        @Override public NoopCacheStoreSessionListener create() {
+            return new NoopCacheStoreSessionListener();
+        }
+    }
+
+    /**
+     *
+     */
+    public static class NoopCacheStoreSessionListener implements CacheStoreSessionListener {
+        /** {@inheritDoc} */
+        @Override public void onSessionStart(CacheStoreSession ses) {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onSessionEnd(CacheStoreSession ses, boolean commit) {
+            // No-op.
+        }
+    }
+
+    /**
+     *
+     */
+    public static class NoopTopologyValidator implements TopologyValidator {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** {@inheritDoc} */
+        @Override public boolean validate(Collection<ClusterNode> nodes) {
+            return true;
+        }
+    }
+
+    /**
+     *
+     */
+    @SuppressWarnings({"serial", "unchecked"})
+    public static class EmptyCacheEntryListenerConfiguration extends MutableCacheEntryListenerConfiguration {
+        /**
+         *
+         */
+        public EmptyCacheEntryListenerConfiguration() {
+            super(new NoopCacheEntryListenerConfiguration());
+        }
+    }
+
+    /**
+     *
+     */
+    @SuppressWarnings("serial")
+    public static class NoopCacheEntryListenerConfiguration implements CacheEntryListenerConfiguration {
+        /** {@inheritDoc} */
+        @Override public Factory<CacheEntryListener> getCacheEntryListenerFactory() {
+            return new Factory<CacheEntryListener>() {
+                @Override public CacheEntryListener create() {
+                    return new NoopCacheEntryListener();
+                }
+            };
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean isOldValueRequired() {
+            return false;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Factory<CacheEntryEventFilter> getCacheEntryEventFilterFactory() {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean isSynchronous() {
+            return false;
+        }
+    }
+
+    /**
+     *
+     */
+    public static class NoopCacheEntryListener implements CacheEntryCreatedListener {
+        /** {@inheritDoc} */
+        @Override public void onCreated(Iterable iterable) throws CacheEntryListenerException {
+            // No-op.
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/953b575f/modules/core/src/test/java/org/apache/ignite/testframework/configvariations/ConfigVariationsFactory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/configvariations/ConfigVariationsFactory.java b/modules/core/src/test/java/org/apache/ignite/testframework/configvariations/ConfigVariationsFactory.java
new file mode 100644
index 0000000..f5e7d57
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/configvariations/ConfigVariationsFactory.java
@@ -0,0 +1,197 @@
+/*
+ * 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.testframework.configvariations;
+
+import java.util.Arrays;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.util.typedef.internal.SB;
+import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Configurations variations factory.
+ */
+public class ConfigVariationsFactory implements ConfigFactory {
+    /** */
+    private final ConfigParameter<IgniteConfiguration>[][] igniteParams;
+
+    /** */
+    private final int[] igniteCfgVariation;
+
+    /** */
+    private final ConfigParameter<CacheConfiguration>[][] cacheParams;
+
+    /** */
+    private final int[] cacheCfgVariation;
+
+    /** */
+    private int backups = -1;
+
+    /**
+     * @param igniteParams Ignite Params.
+     * @param igniteCfgVariation Ignite configuration variation.
+     * @param cacheParams Cache Params.
+     * @param cacheCfgVariation Cache config variation.
+     */
+    public ConfigVariationsFactory(ConfigParameter<IgniteConfiguration>[][] igniteParams,
+        int[] igniteCfgVariation,
+        @Nullable ConfigParameter<CacheConfiguration>[][] cacheParams,
+        @Nullable int[] cacheCfgVariation) {
+        this.igniteParams = igniteParams;
+        this.igniteCfgVariation = igniteCfgVariation;
+        this.cacheParams = cacheParams;
+        this.cacheCfgVariation = cacheCfgVariation;
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override public IgniteConfiguration getConfiguration(String gridName, IgniteConfiguration srcCfg) {
+        IgniteConfiguration cfg = new IgniteConfiguration();
+
+        if (srcCfg != null)
+            copyDefaultsFromSource(cfg, srcCfg);
+
+        if (igniteParams == null)
+            return cfg;
+
+        for (int i = 0; i < igniteCfgVariation.length; i++) {
+            int var = igniteCfgVariation[i];
+
+            ConfigParameter<IgniteConfiguration> cfgC = igniteParams[i][var];
+
+            if (cfgC != null)
+                cfgC.apply(cfg);
+        }
+
+        return cfg;
+    }
+
+    /**
+     * @param cfg Config.
+     * @param srcCfg Source config.
+     */
+    private static void copyDefaultsFromSource(IgniteConfiguration cfg, IgniteConfiguration srcCfg) {
+        cfg.setGridName(srcCfg.getGridName());
+        cfg.setGridLogger(srcCfg.getGridLogger());
+        cfg.setNodeId(srcCfg.getNodeId());
+        cfg.setIgniteHome(srcCfg.getIgniteHome());
+        cfg.setMBeanServer(srcCfg.getMBeanServer());
+        cfg.setMetricsLogFrequency(srcCfg.getMetricsLogFrequency());
+        cfg.setConnectorConfiguration(srcCfg.getConnectorConfiguration());
+        cfg.setCommunicationSpi(srcCfg.getCommunicationSpi());
+        cfg.setNetworkTimeout(srcCfg.getNetworkTimeout());
+        cfg.setDiscoverySpi(srcCfg.getDiscoverySpi());
+        cfg.setCheckpointSpi(srcCfg.getCheckpointSpi());
+        cfg.setIncludeEventTypes(srcCfg.getIncludeEventTypes());
+
+        // Specials.
+        ((TcpCommunicationSpi)cfg.getCommunicationSpi()).setSharedMemoryPort(-1);
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setForceServerMode(true);
+        cfg.getTransactionConfiguration().setTxSerializableEnabled(true);
+    }
+
+    /**
+     * @return Description.
+     */
+    public String getIgniteConfigurationDescription() {
+        if (igniteParams == null)
+            return "";
+
+        SB sb = new SB("[");
+
+        for (int i = 0; i < igniteCfgVariation.length; i++) {
+            int var = igniteCfgVariation[i];
+
+            ConfigParameter<IgniteConfiguration> cfgC = igniteParams[i][var];
+
+            if (cfgC != null) {
+                sb.a(cfgC.name());
+
+                if (i + 1 < igniteCfgVariation.length)
+                    sb.a(", ");
+            }
+        }
+
+        sb.a("]");
+
+        return sb.toString();
+
+    }
+
+    /** {@inheritDoc} */
+    @Override public CacheConfiguration cacheConfiguration(String gridName) {
+        if (cacheParams == null || cacheCfgVariation == null)
+            throw new IllegalStateException("Failed to configure cache [cacheParams=" + Arrays.deepToString(cacheParams)
+                + ", cacheCfgVariation=" + Arrays.toString(cacheCfgVariation) + "]");
+
+        CacheConfiguration cfg = new CacheConfiguration();
+
+        for (int i = 0; i < cacheCfgVariation.length; i++) {
+            int var = cacheCfgVariation[i];
+
+            ConfigParameter<CacheConfiguration> cfgC = cacheParams[i][var];
+
+            if (cfgC != null)
+                cfgC.apply(cfg);
+        }
+
+        if (backups > 0)
+            cfg.setBackups(backups);
+
+        return cfg;
+    }
+
+    /**
+     * @return Description.
+     */
+    public String getCacheConfigurationDescription() {
+        if (cacheCfgVariation == null)
+            return "";
+
+        SB sb = new SB("[");
+
+        for (int i = 0; i < cacheCfgVariation.length; i++) {
+            int var = cacheCfgVariation[i];
+
+            ConfigParameter cfgC = cacheParams[i][var];
+
+            if (cfgC != null) {
+                sb.a(cfgC.name());
+
+                if (i + 1 < cacheCfgVariation.length)
+                    sb.a(", ");
+            }
+        }
+
+        if (backups > 0)
+            sb.a(", backups=").a(backups);
+
+        sb.a("]");
+
+        return sb.toString();
+    }
+
+    /**
+     * @param backups New backups.
+     */
+    public void backups(int backups) {
+        this.backups = backups;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/953b575f/modules/core/src/test/java/org/apache/ignite/testframework/configvariations/ConfigVariationsTestSuiteBuilder.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/configvariations/ConfigVariationsTestSuiteBuilder.java b/modules/core/src/test/java/org/apache/ignite/testframework/configvariations/ConfigVariationsTestSuiteBuilder.java
new file mode 100644
index 0000000..71d7987
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/configvariations/ConfigVariationsTestSuiteBuilder.java
@@ -0,0 +1,382 @@
+/*
+ * 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.testframework.configvariations;
+
+import java.util.Arrays;
+import junit.framework.TestSuite;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.testframework.junits.IgniteCacheConfigVariationsAbstractTest;
+import org.apache.ignite.lang.IgnitePredicate;
+import org.apache.ignite.testframework.junits.IgniteConfigVariationsAbstractTest;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Configuration variations test suite builder.
+ */
+public class ConfigVariationsTestSuiteBuilder {
+    /** */
+    private final TestSuite suite;
+
+    /** */
+    @SuppressWarnings("unchecked")
+    private ConfigParameter<IgniteConfiguration>[][] igniteParams =
+        ConfigVariations.igniteBasicSet();
+
+    /** */
+    @SuppressWarnings("unchecked")
+    private ConfigParameter<CacheConfiguration>[][] cacheParams;
+
+    /** */
+    private CacheStartMode cacheStartMode = CacheStartMode.DYNAMIC;
+
+    /** */
+    private boolean withClients;
+
+    /** */
+    private int gridsCnt = 3;
+
+    /** */
+    private int testedNodeCnt = 1;
+
+    /** */
+    private Class<? extends IgniteConfigVariationsAbstractTest> cls;
+
+    /** */
+    private int[] specificIgniteParam;
+
+    /** */
+    private int[] specificCacheParam;
+
+    /** */
+    private int backups = -1;
+
+    /** */
+    private IgnitePredicate<IgniteConfiguration>[] igniteCfgFilters;
+
+    /** */
+    private IgnitePredicate<CacheConfiguration>[] cacheCfgFilters;
+
+    /**
+     * @param name Name.
+     * @param cls Test class.
+     */
+    public ConfigVariationsTestSuiteBuilder(String name, Class<? extends IgniteConfigVariationsAbstractTest> cls) {
+        suite = new TestSuite(name);
+        this.cls = cls;
+    }
+
+    /**
+     * @return Test suite.
+     */
+    public TestSuite build() {
+        assert testedNodeCnt > 0;
+        assert gridsCnt > 0;
+
+        VariationsIterator igniteCfgIter;
+
+        if (specificIgniteParam == null)
+            igniteCfgIter = new VariationsIterator(igniteParams);
+        else
+            igniteCfgIter = new OneElementVariationsIterator(specificIgniteParam, igniteParams);
+
+        for (; igniteCfgIter.hasNext(); ) {
+            final int[] igniteCfgVariation = igniteCfgIter.next();
+
+            if (!passIgniteConfigFilter(igniteCfgVariation))
+                continue;
+
+            if (cacheParams == null) {
+                TestSuite addedSuite = build(igniteCfgVariation, null, true);
+
+                suite.addTest(addedSuite);
+            }
+            else {
+                VariationsIterator cacheCfgIter;
+
+                if (specificCacheParam == null)
+                    cacheCfgIter = new VariationsIterator(cacheParams);
+                else
+                    cacheCfgIter = new OneElementVariationsIterator(specificCacheParam, cacheParams);
+
+                for (; cacheCfgIter.hasNext(); ) {
+                    int[] cacheCfgVariation = cacheCfgIter.next();
+
+                    if (!passCacheConfigFilter(cacheCfgVariation))
+                        continue;
+
+                    // Stop all grids before starting new ignite configuration.
+                    boolean stopNodes = !cacheCfgIter.hasNext();
+
+                    TestSuite addedSuite = build(igniteCfgVariation, cacheCfgVariation, stopNodes);
+
+                    suite.addTest(addedSuite);
+                }
+            }
+        }
+
+        return suite;
+    }
+
+    /**
+     * @param variation Variation.
+     * @return {@code True} if variation pass filters.
+     */
+    private boolean passIgniteConfigFilter(int[] variation) {
+        ConfigVariationsFactory factory = new ConfigVariationsFactory(igniteParams, variation, null, null);
+
+        IgniteConfiguration cfg = factory.getConfiguration(null, null);
+
+        if (igniteCfgFilters != null) {
+            for (IgnitePredicate<IgniteConfiguration> filter : igniteCfgFilters) {
+                if (!filter.apply(cfg))
+                    return false;
+            }
+        }
+
+        return true;
+    }
+
+    /**
+     * @param variation Variation.
+     * @return {@code True} if variation pass filters.
+     */
+    private boolean passCacheConfigFilter(int[] variation) {
+        ConfigVariationsFactory factory = new ConfigVariationsFactory(null, null, cacheParams, variation);
+
+        CacheConfiguration cfg = factory.cacheConfiguration(null);
+
+        if (cacheCfgFilters != null) {
+            for (IgnitePredicate<CacheConfiguration> filter : cacheCfgFilters) {
+                if (!filter.apply(cfg))
+                    return false;
+            }
+        }
+
+        return true;
+    }
+
+    /**
+     * @param igniteCfgVariation Ignite Variation.
+     * @param cacheCfgVariation Cache Variation.
+     * @param stopNodes Stop nodes.
+     * @return Test suite.
+     */
+    private TestSuite build(int[] igniteCfgVariation, @Nullable int[] cacheCfgVariation, boolean stopNodes) {
+        ConfigVariationsFactory factory = new ConfigVariationsFactory(igniteParams,
+            igniteCfgVariation, cacheParams, cacheCfgVariation);
+
+        factory.backups(backups);
+
+        String clsNameSuffix = "[igniteCfgVariation=" + Arrays.toString(igniteCfgVariation)
+            + ", cacheCfgVariation=" + Arrays.toString(cacheCfgVariation)
+            + ", igniteCfg=" + factory.getIgniteConfigurationDescription()
+            + ", cacheCfg=" + factory.getCacheConfigurationDescription() + "]";
+
+        VariationsTestsConfig testCfg = new VariationsTestsConfig(factory, clsNameSuffix, stopNodes, cacheStartMode,
+            gridsCnt);
+
+        TestSuite addedSuite;
+
+        if (testedNodeCnt > 1)
+            addedSuite = createMultiNodeTestSuite((Class<? extends IgniteCacheConfigVariationsAbstractTest>)cls, 
+                testCfg, testedNodeCnt, withClients);
+        else
+            addedSuite = new IgniteConfigVariationsTestSuite(cls, testCfg);
+
+        return addedSuite;
+    }
+
+    /**
+     * @param cls Test class.
+     * @param cfg Configuration.
+     * @param testedNodeCnt Count of tested nodes.
+     */
+    private static TestSuite createMultiNodeTestSuite(Class<? extends IgniteCacheConfigVariationsAbstractTest> cls,
+        VariationsTestsConfig cfg, int testedNodeCnt, boolean withClients) {
+        TestSuite suite = new TestSuite();
+
+        if (cfg.gridCount() < testedNodeCnt)
+            throw new IllegalArgumentException("Failed to initialize test suite [nodeCnt=" + testedNodeCnt
+                + ", cfgGridCnt=" + cfg.gridCount() + "]");
+
+        for (int i = 0; i < testedNodeCnt; i++) {
+            boolean stopNodes = cfg.isStopNodes() && i + 1 == testedNodeCnt;
+            boolean startCache = i == 0;
+            boolean stopCache = i + 1 == testedNodeCnt;
+
+            VariationsTestsConfig cfg0 = new VariationsTestsConfig(cfg.configurationFactory(), cfg.description(),
+                stopNodes, startCache, stopCache, cfg.cacheStartMode(), cfg.gridCount(), i, withClients);
+
+            suite.addTest(new IgniteConfigVariationsTestSuite(cls, cfg0));
+        }
+
+        return suite;
+    }
+
+    /**
+     * @return {@code this} for chaining.
+     */
+    public ConfigVariationsTestSuiteBuilder withClients() {
+        if (testedNodeCnt < 2)
+            throw new IllegalStateException("Tested node count should be more than 1: " + testedNodeCnt);
+
+        withClients = true;
+
+        return this;
+    }
+
+    /**
+     * @param testedNodeCnt Tested node count.
+     * @return {@code this} for chaining.
+     */
+    public ConfigVariationsTestSuiteBuilder testedNodesCount(int testedNodeCnt) {
+        this.testedNodeCnt = testedNodeCnt;
+
+        return this;
+    }
+
+    /**
+     * @param cnt Count.
+     * @return {@code this} for chaining.
+     */
+    public ConfigVariationsTestSuiteBuilder gridsCount(int cnt) {
+        assert cnt > 0;
+
+        gridsCnt = cnt;
+
+        return this;
+    }
+
+    /**
+     * @param igniteParams New ignite params.
+     * @return {@code this} for chaining.
+     */
+    public ConfigVariationsTestSuiteBuilder igniteParams(
+        ConfigParameter<IgniteConfiguration>[][] igniteParams) {
+        this.igniteParams = igniteParams;
+
+        return this;
+    }
+
+    /**
+     * @param cacheParams New cache params.
+     * @return {@code this} for chaining.
+     */
+    public ConfigVariationsTestSuiteBuilder cacheParams(ConfigParameter<CacheConfiguration>[][] cacheParams) {
+        this.cacheParams = cacheParams;
+
+        return this;
+    }
+
+    /**
+     * Sets basic cache params and basic count of backups.
+     *
+     * @return {@code this} for chaining.
+     */
+    public ConfigVariationsTestSuiteBuilder withBasicCacheParams() {
+        cacheParams = ConfigVariations.cacheBasicSet();
+        backups = 1;
+
+        return this;
+    }
+
+    /**
+     * @param backups Backups.
+     * @return {@code this} for chaining.
+     */
+    public ConfigVariationsTestSuiteBuilder backups(int backups) {
+        assert backups > 0 : backups;
+
+        this.backups = backups;
+
+        return this;
+    }
+
+    /**
+     * @param singleIgniteParam Param.
+     * @return {@code this} for chaining.
+     */
+    public ConfigVariationsTestSuiteBuilder specifyIgniteParam(int... singleIgniteParam) {
+        specificIgniteParam = singleIgniteParam;
+
+        return this;
+    }
+
+    /**
+     * @param singleParam Param.
+     * @return {@code this} for chaining.
+     */
+    public ConfigVariationsTestSuiteBuilder specifyCacheParam(int... singleParam) {
+        specificCacheParam = singleParam;
+
+        return this;
+    }
+
+    /**
+     * @param filters Ignite configuration filters.
+     * @return {@code this} for chaining.
+     */
+    public ConfigVariationsTestSuiteBuilder withIgniteConfigFilters(IgnitePredicate<IgniteConfiguration>... filters) {
+        igniteCfgFilters = filters;
+
+        return this;
+    }
+
+    /**
+     * @param filters Ignite configuration filters.
+     * @return {@code this} for chaining.
+     */
+    public ConfigVariationsTestSuiteBuilder withCacheConfigFilters(IgnitePredicate<CacheConfiguration>... filters) {
+        cacheCfgFilters = filters;
+
+        return this;
+    }
+
+    /**
+     *
+     */
+    private static class OneElementVariationsIterator extends VariationsIterator {
+        /** */
+        private int[] elem;
+
+        /** */
+        private boolean hasNext = true;
+
+        /**
+         * @param elem Element.
+         */
+        OneElementVariationsIterator(int[] elem, Object[][] params) {
+            super(params);
+
+            this.elem = elem;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean hasNext() {
+            return hasNext;
+        }
+
+        /** {@inheritDoc} */
+        @Override public int[] next() {
+            hasNext = false;
+
+            return elem;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/953b575f/modules/core/src/test/java/org/apache/ignite/testframework/configvariations/IgniteConfigVariationsTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/configvariations/IgniteConfigVariationsTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testframework/configvariations/IgniteConfigVariationsTestSuite.java
new file mode 100644
index 0000000..d953c27
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/configvariations/IgniteConfigVariationsTestSuite.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.testframework.configvariations;
+
+import junit.framework.Test;
+import junit.framework.TestResult;
+import junit.framework.TestSuite;
+import org.apache.ignite.testframework.junits.IgniteConfigVariationsAbstractTest;
+
+/**
+ * Configuration variations test suite.
+ */
+public class IgniteConfigVariationsTestSuite extends TestSuite {
+    /** */
+    protected final VariationsTestsConfig cfg;
+
+    /**
+     * @param cls Test class.
+     * @param cfg Configuration.
+     */
+    public IgniteConfigVariationsTestSuite(Class<? extends IgniteConfigVariationsAbstractTest> cls,
+        VariationsTestsConfig cfg) {
+        super(cls);
+
+        this.cfg = cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void runTest(Test test, TestResult res) {
+        if (test instanceof IgniteConfigVariationsAbstractTest)
+            ((IgniteConfigVariationsAbstractTest)test).setTestsConfiguration(cfg);
+
+        super.runTest(test, res);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/953b575f/modules/core/src/test/java/org/apache/ignite/testframework/configvariations/Parameters.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/configvariations/Parameters.java b/modules/core/src/test/java/org/apache/ignite/testframework/configvariations/Parameters.java
new file mode 100644
index 0000000..27c0a48
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/configvariations/Parameters.java
@@ -0,0 +1,377 @@
+/*
+ * 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.testframework.configvariations;
+
+import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.util.ArrayDeque;
+import java.util.Arrays;
+import java.util.Queue;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import javax.cache.configuration.Factory;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.T2;
+import org.apache.ignite.internal.util.typedef.internal.A;
+import org.apache.ignite.internal.util.typedef.internal.SB;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Parameters utils.
+ */
+public class Parameters {
+    /**
+     * Private constructor.
+     */
+    private Parameters() {
+        // No-op.
+    }
+
+    /**
+     * @return Array of configuration processors for given enum.
+     */
+    @SuppressWarnings("unchecked")
+    public static <T> ConfigParameter<T>[] enumParameters(String mtdName, Class<?> enumCls) {
+        return enumParameters(false, mtdName, enumCls);
+    }
+
+    /**
+     * @return Array of configuration processors for given enum.
+     */
+    @SuppressWarnings("unchecked")
+    public static <T> ConfigParameter<T>[] enumParameters(boolean withNull, String mtdName, Class<?> enumCls) {
+        return parameters0(mtdName, withNull, enumCls.getEnumConstants());
+    }
+
+    /**
+     * @param mtdName Method name.
+     * @param values Values.
+     * @return Array of configuration paramethers.
+     */
+    @SuppressWarnings("unchecked")
+    private static <T> ConfigParameter<T>[] parameters0(String mtdName, boolean withNull, Object[] values) {
+        for (Object val : values) {
+            if (!isPrimitiveOrEnum(val) && !(val instanceof Factory))
+                throw new IllegalArgumentException("Value have to be primite, enum or factory: " + val);
+        }
+
+        if (withNull) {
+            Object[] valuesWithNull = new Object[values.length + 1];
+
+            valuesWithNull[0] = null;
+
+            System.arraycopy(values, 0, valuesWithNull, 1, valuesWithNull.length - 1);
+
+            values = valuesWithNull;
+        }
+
+        assert values != null && values.length > 0 : "MtdName:" + mtdName;
+
+        ConfigParameter<T>[] resArr = new ConfigParameter[values.length];
+
+        for (int i = 0; i < resArr.length; i++)
+            resArr[i] = new ReflectionParameter<>(mtdName, values[i]);
+
+        return resArr;
+    }
+
+    /**
+     * @param val Value.
+     * @return Primitive or enum or not.
+     */
+    private static boolean isPrimitiveOrEnum(Object val) {
+        return val.getClass().isPrimitive()
+            || val.getClass().equals(Boolean.class)
+            || val.getClass().equals(Byte.class)
+            || val.getClass().equals(Short.class)
+            || val.getClass().equals(Character.class)
+            || val.getClass().equals(Integer.class)
+            || val.getClass().equals(Long.class)
+            || val.getClass().equals(Float.class)
+            || val.getClass().equals(Double.class)
+            || val.getClass().isEnum();
+    }
+
+    /**
+     * @return Array of configuration processors for given enum.
+     */
+    @SuppressWarnings("unchecked")
+    public static <T> ConfigParameter<T>[] booleanParameters(String mtdName) {
+        return parameters0(mtdName, false, new Boolean[] {true, false});
+    }
+
+    /**
+     * @return Array of configuration processors for given enum.
+     */
+    @SuppressWarnings("unchecked")
+    public static <T> ConfigParameter<T>[] booleanParameters(boolean withNull, String mtdName) {
+        return parameters0(mtdName, withNull, new Boolean[] {true, false});
+    }
+
+    /**
+     * @param mtdName Method name.
+     * @param values Values.
+     * @return Array of configuration processors for given classes.
+     */
+    public static ConfigParameter[] objectParameters(String mtdName, Object... values) {
+        return objectParameters(false, mtdName, values);
+    }
+
+    /**
+     * @param mtdName Method name.
+     * @param values Values.
+     * @return Array of configuration processors for given classes.
+     */
+    public static ConfigParameter[] objectParameters(boolean withNull, String mtdName, Object... values) {
+        return parameters0(mtdName, withNull, values);
+    }
+
+    /**
+     * @param mtdName Method name.
+     * @param val Value.
+     * @return Configuration parameter.
+     */
+    public static <T> ConfigParameter<T> parameter(String mtdName, Object val) {
+        return new ReflectionParameter<>(mtdName, val);
+    }
+
+    /**
+     * @return Complex parameter.
+     */
+    @SuppressWarnings("unchecked")
+    public static <T> ConfigParameter<T> complexParameter(ConfigParameter<T>... params) {
+        return new ComplexParameter<T>(params);
+    }
+
+    /**
+     * @param cls Class.
+     * @return Factory that uses default constructor to initiate object by given class.
+     */
+    public static <T> Factory<T> factory(Class<?> cls) {
+        return new ReflectionFactory<>(cls);
+    }
+
+    /**
+     * Reflection configuration applier.
+     */
+    @SuppressWarnings("serial")
+    private static class ReflectionParameter<T> implements ConfigParameter<T> {
+        /** Classes of marameters cache. */
+        private static final ConcurrentMap<T2<Class, String>, Class> paramClassesCache = new ConcurrentHashMap();
+
+        /** */
+        private final String mtdName;
+
+        /** Primitive, enum or factory. */
+        private final Object val;
+
+        /**
+         * @param mtdName Method name.
+         */
+        ReflectionParameter(String mtdName, @Nullable Object val) {
+            if (val != null && !isPrimitiveOrEnum(val) && !(val instanceof Factory))
+                throw new IllegalArgumentException("Value have to be primite, enum or factory: " + val);
+
+            this.mtdName = mtdName;
+            this.val = val;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String name() {
+            String mtdName0 = mtdName;
+
+            if (mtdName0.startsWith("set") && mtdName0.length() > 3)
+                mtdName0 = mtdName0.substring(3, mtdName0.length());
+
+            String val0;
+
+            if (val == null)
+                val0 = "null";
+            else if (val instanceof Factory)
+                val0 = ((Factory)val).create().toString();
+            else
+                val0 = val.toString();
+
+            return mtdName0 + "=" + val0;
+        }
+
+        /** {@inheritDoc} */
+        @Override public T apply(T cfg) {
+            if (val == null)
+                return null;
+
+            try {
+                Object val0 = val;
+
+                if (!isPrimitiveOrEnum(val))
+                    val0 = ((Factory)val0).create();
+
+                Class<?> paramCls = paramClassesCache.get(new T2<Class, String>(cfg.getClass(), mtdName));
+
+                if (paramCls == null)
+                    paramCls = val0.getClass();
+                else if (!paramCls.isInstance(val0))
+                    throw new IgniteException("Class parameter from cache does not match value argument class " +
+                        "[paramCls=" + paramCls + ", val=" + val0 + "]");
+
+                if (val0.getClass().equals(Boolean.class))
+                    paramCls = Boolean.TYPE;
+                else if (val0.getClass().equals(Byte.class))
+                    paramCls = Byte.TYPE;
+                else if (val0.getClass().equals(Short.class))
+                    paramCls = Short.TYPE;
+                else if (val0.getClass().equals(Character.class))
+                    paramCls = Character.TYPE;
+                else if (val0.getClass().equals(Integer.class))
+                    paramCls = Integer.TYPE;
+                else if (val0.getClass().equals(Long.class))
+                    paramCls = Long.TYPE;
+                else if (val0.getClass().equals(Float.class))
+                    paramCls = Float.TYPE;
+                else if (val0.getClass().equals(Double.class))
+                    paramCls = Double.TYPE;
+
+                Method mtd;
+
+                Queue<Class> queue = new ArrayDeque<>();
+
+                boolean failed = false;
+
+                while (true) {
+                    try {
+                        mtd = cfg.getClass().getMethod(mtdName, paramCls);
+
+                        if (failed)
+                            paramClassesCache.put(new T2<Class, String>(cfg.getClass(), mtdName), paramCls);
+
+                        break;
+                    }
+                    catch (NoSuchMethodException e) {
+                        failed = true;
+
+                        U.warn(null, "Method not found [cfgCls=" + cfg.getClass() + ", mtdName=" + mtdName
+                            + ", paramCls=" + paramCls + "]");
+
+                        Class<?>[] interfaces = paramCls.getInterfaces();
+
+                        Class<?> superclass = paramCls.getSuperclass();
+
+                        if (superclass != null)
+                            queue.add(superclass);
+
+                        if (!F.isEmpty(interfaces))
+                            queue.addAll(Arrays.asList(interfaces));
+
+                        if (queue.isEmpty())
+                            throw new IgniteException("Method not found [cfgCls=" + cfg.getClass() + ", mtdName="
+                                + mtdName + ", paramCls=" + val0.getClass() + "]", e);
+
+                        paramCls = queue.remove();
+                    }
+                }
+
+                mtd.invoke(cfg, val0);
+            }
+            catch (InvocationTargetException | IllegalAccessException e) {
+                throw new IgniteException(e);
+            }
+
+            return null;
+        }
+    }
+
+    /**
+     *
+     */
+    private static class ReflectionFactory<T> implements Factory<T> {
+        /** */
+        private static final long serialVersionUID = 0;
+
+        /** */
+        private Class<?> cls;
+
+        /**
+         * @param cls Class.
+         */
+        ReflectionFactory(Class<?> cls) {
+            this.cls = cls;
+        }
+
+        /** {@inheritDoc} */
+        @Override public T create() {
+            try {
+                Constructor<?> constructor = cls.getConstructor();
+
+                return (T)constructor.newInstance();
+            }
+            catch (NoSuchMethodException | InstantiationException | InvocationTargetException |
+                IllegalAccessException e) {
+                throw new IgniteException("Failed to create object using default constructor: " + cls, e);
+            }
+        }
+    }
+
+    /**
+     *
+     */
+    private static class ComplexParameter<T> implements ConfigParameter<T> {
+        /** */
+        private final String name;
+
+        /** */
+        private ConfigParameter<T>[] params;
+
+        /**
+         * @param params Params
+         */
+        @SafeVarargs 
+        ComplexParameter(ConfigParameter<T>... params) {
+            A.notEmpty(params, "params");
+
+            this.params = params;
+
+            if (params.length == 1)
+                name = params[0].name();
+            else {
+                SB sb = new SB(params[0].name());
+
+                for (int i = 1; i < params.length; i++)
+                    sb.a('-').a(params[i]);
+
+                name = sb.toString();
+            }
+        }
+
+        /** {@inheritDoc} */
+        @Override public String name() {
+            return name;
+        }
+
+        /** {@inheritDoc} */
+        @SuppressWarnings("unchecked")
+        @Override public T apply(T cfg) {
+            for (ConfigParameter param : params)
+                param.apply(cfg);
+
+            return cfg;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/953b575f/modules/core/src/test/java/org/apache/ignite/testframework/configvariations/VariationsIterator.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/configvariations/VariationsIterator.java b/modules/core/src/test/java/org/apache/ignite/testframework/configvariations/VariationsIterator.java
new file mode 100644
index 0000000..fa1c216
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/configvariations/VariationsIterator.java
@@ -0,0 +1,174 @@
+/*
+ * 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.testframework.configvariations;
+
+import java.util.Arrays;
+import java.util.Iterator;
+
+/**
+ * Variations iterator.
+ */
+public class VariationsIterator implements Iterator<int[]> {
+    /** */
+    private final Object[][] params;
+
+    /** */
+    private final int[] vector;
+
+    /** */
+    private int position;
+
+    /** */
+    private final int expCntOfVectors;
+
+    /** */
+    private int cntOfVectors;
+
+    /**
+     * @param params Paramethers.
+     */
+    public VariationsIterator(Object[][] params) {
+        assert params != null;
+        assert params.length > 0;
+
+        for (int i = 0; i < params.length; i++) {
+            assert params[i] != null : i;
+            assert params[i].length > 0 : i;
+        }
+
+        this.params = params;
+
+        vector = new int[params.length];
+
+        for (int i = 0; i < vector.length; i++)
+            vector[i] = 0;
+
+        position = -1;
+
+        int cntOfVectors0 = 1;
+
+        for (int i = 0; i < params.length; i++)
+            cntOfVectors0 *= params[i].length;
+
+        expCntOfVectors = cntOfVectors0;
+
+        cntOfVectors = 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean hasNext() {
+        return cntOfVectors < expCntOfVectors;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int[] next() {
+        // Only first call.
+        if (position == -1) {
+            position = 0;
+
+            cntOfVectors++;
+
+            return arraycopy(vector);
+        }
+
+        if (!updateVector(vector, position)) {
+            if (position + 1 == params.length)
+                throw new IllegalStateException("[position=" + position + ", vector=" +
+                    Arrays.toString(vector) + ", params=" + Arrays.deepToString(params));
+
+            position++;
+
+            // Skip params with length 1. We cannot set 1 at this position.
+            while (position < params.length && params[position].length < 2)
+                position++;
+
+            if (position == params.length)
+                throw new IllegalStateException("[position=" + position + ", vector=" +
+                    Arrays.toString(vector) + ", params=" + Arrays.deepToString(params));
+
+            vector[position] = 1;
+
+            cntOfVectors++;
+
+            return arraycopy(vector);
+        }
+
+        cntOfVectors++;
+
+        return arraycopy(vector);
+    }
+
+    /**
+     * Updates vector starting from position.
+     *
+     * @param vector Vector.
+     * @param position Position.
+     * @return {@code True} if vector has been updated. When {@code false} is returned it means that all positions
+     *          before has been set to {@code 0}.
+     */
+    private boolean updateVector(int[] vector, int position) {
+        if (position == 0) {
+            int val = vector[0];
+
+            if (val + 1 < params[0].length) {
+                vector[0] = val + 1;
+
+                return true;
+            }
+            else {
+                vector[0] = 0;
+
+                return false;
+            }
+        }
+
+        if (updateVector(vector, position - 1))
+            return true;
+
+        int val = vector[position];
+
+        if (val + 1 < params[position].length) {
+            vector[position] = val + 1;
+
+            return true;
+        }
+        else {
+            vector[position] = 0;
+
+            return false;
+        }
+
+    }
+
+    /**
+     * @param arr Array.
+     * @return Array copy.
+     */
+    private static int[] arraycopy(int[] arr) {
+        int[] dest = new int[arr.length];
+
+        System.arraycopy(arr, 0, dest, 0, arr.length);
+
+        return dest;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void remove() {
+        throw new UnsupportedOperationException();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/953b575f/modules/core/src/test/java/org/apache/ignite/testframework/configvariations/VariationsTestsConfig.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/configvariations/VariationsTestsConfig.java b/modules/core/src/test/java/org/apache/ignite/testframework/configvariations/VariationsTestsConfig.java
new file mode 100644
index 0000000..7bcfc7f
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/configvariations/VariationsTestsConfig.java
@@ -0,0 +1,161 @@
+/*
+ * 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.testframework.configvariations;
+
+import org.apache.ignite.internal.util.typedef.internal.A;
+
+/**
+ * Immutable tests configuration.
+ */
+public class VariationsTestsConfig {
+    /** */
+    private final ConfigFactory factory;
+
+    /** */
+    private final String desc;
+
+    /** */
+    private final boolean stopNodes;
+
+    /** */
+    private final int gridCnt;
+
+    /** */
+    private final CacheStartMode cacheStartMode;
+
+    /** */
+    private final int testedNodeIdx;
+
+    /** */
+    private boolean startCache;
+
+    /** */
+    private boolean stopCache;
+
+    /** */
+    private boolean withClients;
+
+    /**
+     * @param factory Factory.
+     * @param desc Class suffix.
+     * @param stopNodes Stope nodes.
+     * @param gridCnt Grdi count.
+     */
+    public VariationsTestsConfig(
+        ConfigFactory factory,
+        String desc,
+        boolean stopNodes,
+        CacheStartMode cacheStartMode,
+        int gridCnt
+    ) {
+        this(factory, desc, stopNodes, true, true, cacheStartMode, gridCnt, 0, false);
+    }
+
+    /**
+     * @param factory Factory.
+     * @param desc Config description.
+     * @param stopNodes Stope nodes.
+     * @param gridCnt Grdi count.
+     */
+    public VariationsTestsConfig(
+        ConfigFactory factory,
+        String desc,
+        boolean stopNodes,
+        boolean startCache,
+        boolean stopCache,
+        CacheStartMode cacheStartMode,
+        int gridCnt,
+        int testedNodeIdx,
+        boolean withClients
+    ) {
+        A.ensure(gridCnt >= 1, "Grids count cannot be less then 1.");
+
+        this.factory = factory;
+        this.desc = desc;
+        this.gridCnt = gridCnt;
+        this.cacheStartMode = cacheStartMode;
+        this.testedNodeIdx = testedNodeIdx;
+        this.stopNodes = stopNodes;
+        this.startCache = startCache;
+        this.stopCache = stopCache;
+        this.withClients = withClients;
+    }
+
+    /**
+     * @return Configuration factory.
+     */
+    public ConfigFactory configurationFactory() {
+        return factory;
+    }
+
+    /**
+     * @return Configuration description..
+     */
+    public String description() {
+        return desc;
+    }
+
+    /**
+     * @return Grids count.
+     */
+    public int gridCount() {
+        return gridCnt;
+    }
+
+    /**
+     * @return Whether nodes should be stopped after tests execution or not.
+     */
+    public boolean isStopNodes() {
+        return stopNodes;
+    }
+
+    /**
+     * @return Cache start type.
+     */
+    public CacheStartMode cacheStartMode() {
+        return cacheStartMode;
+    }
+
+    /**
+     * @return Index of node which should be tested or {@code null}.
+     */
+    public int testedNodeIndex() {
+        return testedNodeIdx;
+    }
+
+    /**
+     * @return Whether cache should be started before tests execution or not.
+     */
+    public boolean isStartCache() {
+        return startCache;
+    }
+
+    /**
+     * @return Whether cache should be destroyed after tests execution or not.
+     */
+    public boolean isStopCache() {
+        return stopCache;
+    }
+
+    /**
+     * @return With clients.
+     */
+    public boolean withClients() {
+        return withClients;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/953b575f/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
index 614e634..2f8155c 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
@@ -153,7 +153,7 @@ public abstract class GridAbstractTest extends TestCase {
     private static long ts = System.currentTimeMillis();
 
     /** Starting grid name. */
-    protected final static ThreadLocal<String> startingGrid = new ThreadLocal<>();
+    protected static final ThreadLocal<String> startingGrid = new ThreadLocal<>();
 
     /**
      *
@@ -528,7 +528,7 @@ public abstract class GridAbstractTest extends TestCase {
         }
 
         if (isFirstTest()) {
-            info(">>> Starting test class: " + GridTestUtils.fullSimpleName(getClass()) + " <<<");
+            info(">>> Starting test class: " + testClassDescription() + " <<<");
 
             if (startGrid) {
                 IgniteConfiguration cfg = optimize(getConfiguration());
@@ -561,7 +561,7 @@ public abstract class GridAbstractTest extends TestCase {
             }
         }
 
-        info(">>> Starting test: " + getName() + " <<<");
+        info(">>> Starting test: " + testDescription() + " <<<");
 
         try {
             beforeTest();
@@ -581,6 +581,20 @@ public abstract class GridAbstractTest extends TestCase {
     }
 
     /**
+     * @return Test description.
+     */
+    protected String testDescription() {
+        return GridTestUtils.fullSimpleName(getClass()) + "#" + getName();
+    }
+
+    /**
+     * @return Test class description.
+     */
+    protected String testClassDescription() {
+        return GridTestUtils.fullSimpleName(getClass());
+    }
+
+    /**
      * @return Started grid.
      * @throws Exception If anything failed.
      */
@@ -738,16 +752,29 @@ public abstract class GridAbstractTest extends TestCase {
      * @throws Exception If failed.
      */
     protected Ignite startGrid(String gridName, GridSpringResourceContext ctx) throws Exception {
+        return startGrid(gridName, optimize(getConfiguration(gridName)), ctx);
+    }
+    /**
+     * Starts new grid with given name.
+     *
+     * @param gridName Grid name.
+     * @param ctx Spring context.
+     * @return Started grid.
+     * @throws Exception If failed.
+     */
+    protected Ignite startGrid(String gridName, IgniteConfiguration cfg, GridSpringResourceContext ctx)
+        throws Exception {
         if (!isRemoteJvm(gridName)) {
             startingGrid.set(gridName);
 
             try {
-                Ignite node = IgnitionEx.start(optimize(getConfiguration(gridName)), ctx);
+                Ignite node = IgnitionEx.start(cfg, ctx);
 
-                IgniteConfiguration cfg = node.configuration();
+                IgniteConfiguration nodeCfg = node.configuration();
 
                 log.info("Node started with the following configuration [id=" + node.cluster().localNode().id()
-                    + ", marshaller=" + cfg.getMarshaller() + ", binaryCfg=" + cfg.getBinaryConfiguration() + "]");
+                    + ", marshaller=" + nodeCfg.getMarshaller()
+                    + ", binaryCfg=" + nodeCfg.getBinaryConfiguration() + "]");
 
                 return node;
             }
@@ -1382,7 +1409,7 @@ public abstract class GridAbstractTest extends TestCase {
     @Override protected void tearDown() throws Exception {
         long dur = System.currentTimeMillis() - ts;
 
-        info(">>> Stopping test: " + getName() + " in " + dur + " ms <<<");
+        info(">>> Stopping test: " + testDescription() + " in " + dur + " ms <<<");
 
         TestCounters cntrs = getTestCounters();
 
@@ -1397,7 +1424,7 @@ public abstract class GridAbstractTest extends TestCase {
             serializedObj.clear();
 
             if (isLastTest()) {
-                info(">>> Stopping test class: " + GridTestUtils.fullSimpleName(getClass()) + " <<<");
+                info(">>> Stopping test class: " + testClassDescription() + " <<<");
 
                 TestCounters counters = getTestCounters();
 


[4/4] ignite git commit: ignite-2521: Configuration variations tests framework + IgniteCacheBasicConfigVariationsFullApiTestSuite + ignite-2554: Fixed Affinity.mapKeyToNode() for dynamically started LOCAL cache

Posted by sb...@apache.org.
ignite-2521: Configuration variations tests framework + IgniteCacheBasicConfigVariationsFullApiTestSuite + ignite-2554: Fixed Affinity.mapKeyToNode() for dynamically started LOCAL cache


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

Branch: refs/heads/master
Commit: 953b575f0321acd90374c2dd92b32825842999ab
Parents: 49dcd6b
Author: ashutak <as...@gridgain.com>
Authored: Mon Feb 29 14:00:45 2016 +0300
Committer: ashutak <as...@gridgain.com>
Committed: Mon Feb 29 14:00:45 2016 +0300

----------------------------------------------------------------------
 .../cache/affinity/GridCacheAffinityImpl.java   |    7 +-
 .../cache/GridCacheAbstractFullApiSelfTest.java |   10 +-
 .../IgniteCacheConfigVariationsFullApiTest.java | 5851 ++++++++++++++++++
 .../configvariations/CacheStartMode.java        |   29 +
 .../configvariations/ConfigFactory.java         |   39 +
 .../configvariations/ConfigParameter.java       |   34 +
 .../configvariations/ConfigVariations.java      |  346 ++
 .../ConfigVariationsFactory.java                |  197 +
 .../ConfigVariationsTestSuiteBuilder.java       |  382 ++
 .../IgniteConfigVariationsTestSuite.java        |   50 +
 .../configvariations/Parameters.java            |  377 ++
 .../configvariations/VariationsIterator.java    |  174 +
 .../configvariations/VariationsTestsConfig.java |  161 +
 .../testframework/junits/GridAbstractTest.java  |   43 +-
 ...IgniteCacheConfigVariationsAbstractTest.java |  583 ++
 .../IgniteConfigVariationsAbstractTest.java     |  420 ++
 .../ConfigVariationsTestSuiteBuilderTest.java   |  112 +
 .../testframework/test/ParametersTest.java      |   87 +
 .../test/VariationsIteratorTest.java            |  156 +
 .../ignite/testsuites/IgniteBasicTestSuite.java |    8 +
 ...heBasicConfigVariationsFullApiTestSuite.java |   41 +
 21 files changed, 9094 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/953b575f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/affinity/GridCacheAffinityImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/affinity/GridCacheAffinityImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/affinity/GridCacheAffinityImpl.java
index 3bc71fe..6567141 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/affinity/GridCacheAffinityImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/affinity/GridCacheAffinityImpl.java
@@ -193,7 +193,12 @@ public class GridCacheAffinityImpl<K, V> implements Affinity<K> {
 
         AffinityTopologyVersion topVer = topologyVersion();
 
-        int nodesCnt = cctx.discovery().cacheAffinityNodes(cctx.name(), topVer).size();
+        int nodesCnt;
+
+        if (!cctx.isLocal())
+            nodesCnt = cctx.discovery().cacheAffinityNodes(cctx.name(), topVer).size();
+        else
+            nodesCnt = 1;
 
         // Must return empty map if no alive nodes present or keys is empty.
         Map<ClusterNode, Collection<K>> res = new HashMap<>(nodesCnt, 1.0f);

http://git-wip-us.apache.org/repos/asf/ignite/blob/953b575f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
index 222a96b..401afbf 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
@@ -65,7 +65,6 @@ import org.apache.ignite.events.Event;
 import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.IgniteKernal;
 import org.apache.ignite.internal.IgnitionEx;
-import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager;
 import org.apache.ignite.internal.processors.resource.GridSpringResourceContext;
 import org.apache.ignite.internal.util.lang.GridAbsPredicate;
@@ -231,8 +230,10 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
                     Ignite ignite = grid(entry.getKey());
 
                     for (CacheConfiguration cfg : entry.getValue())
-                        ignite.createCache(cfg);
+                        ignite.getOrCreateCache(cfg);
                 }
+
+                awaitPartitionMapExchange();
             }
             else {
                 int cnt = gridCount();
@@ -5565,12 +5566,13 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
      *
      */
     private static class CheckCacheSizeTask extends TestIgniteIdxRunnable {
+        /** */
         private final Map<String, Integer> map;
 
         /**
          * @param map Map.
          */
-        public CheckCacheSizeTask(Map<String, Integer> map) {
+        CheckCacheSizeTask(Map<String, Integer> map) {
             this.map = map;
         }
 
@@ -5581,7 +5583,7 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
             int size = 0;
 
             for (String key : map.keySet())
-                if (ctx.affinity().localNode(key, new AffinityTopologyVersion(ctx.discovery().topologyVersion())))
+                if (ctx.affinity().localNode(key, ctx.discovery().topologyVersionEx()))
                     size++;
 
             assertEquals("Incorrect key size on cache #" + idx, size, ignite.cache(ctx.name()).localSize(ALL));