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/03/02 09:11:20 UTC

[01/19] ignite git commit: ignite-2720 Need call 'initializeLocalAddresses' before starting client message worker.

Repository: ignite
Updated Branches:
  refs/heads/ignite-1232 71286d7c3 -> 4fe6db71f


ignite-2720 Need call 'initializeLocalAddresses' before starting client message worker.


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

Branch: refs/heads/ignite-1232
Commit: 49dcd6b96fef901cdd8b20da1e9136688e59a7f4
Parents: c3f5168
Author: sboikov <sb...@gridgain.com>
Authored: Fri Feb 26 16:49:37 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Feb 29 09:40:08 2016 +0300

----------------------------------------------------------------------
 .../java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java   | 6 +++---
 .../java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java   | 2 +-
 .../tcp/ipfinder/multicast/TcpDiscoveryMulticastIpFinder.java  | 1 +
 3 files changed, 5 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/49dcd6b9/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
index 9b2bf46..65b94ca 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
@@ -242,12 +242,12 @@ class ClientImpl extends TcpDiscoveryImpl {
         sockReader = new SocketReader();
         sockReader.start();
 
-        msgWorker = new MessageWorker();
-        msgWorker.start();
-
         if (spi.ipFinder.isShared())
             registerLocalNodeAddress();
 
+        msgWorker = new MessageWorker();
+        msgWorker.start();
+
         try {
             joinLatch.await();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/49dcd6b9/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
index 3ce983e..d6db66f 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
@@ -313,7 +313,7 @@ class ServerImpl extends TcpDiscoveryImpl {
         else {
             if (F.isEmpty(spi.ipFinder.getRegisteredAddresses()))
                 throw new IgniteSpiException("Non-shared IP finder must have IP addresses specified in " +
-                    "GridTcpDiscoveryIpFinder.getRegisteredAddresses() configuration property " +
+                    "TcpDiscoveryIpFinder.getRegisteredAddresses() configuration property " +
                     "(specify list of IP addresses in configuration).");
 
             ipFinderHasLocAddr = spi.ipFinderHasLocalAddress();

http://git-wip-us.apache.org/repos/asf/ignite/blob/49dcd6b9/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/multicast/TcpDiscoveryMulticastIpFinder.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/multicast/TcpDiscoveryMulticastIpFinder.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/multicast/TcpDiscoveryMulticastIpFinder.java
index 75b5f91..d5b3dae 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/multicast/TcpDiscoveryMulticastIpFinder.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/multicast/TcpDiscoveryMulticastIpFinder.java
@@ -136,6 +136,7 @@ public class TcpDiscoveryMulticastIpFinder extends TcpDiscoveryVmIpFinder {
     /** */
     private boolean mcastErr;
 
+    /** */
     @GridToStringExclude
     private Set<InetSocketAddress> locNodeAddrs;
 


[11/19] ignite git commit: Test fixed

Posted by sb...@apache.org.
Test fixed


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

Branch: refs/heads/ignite-1232
Commit: 74fd4008abb1c0e13ffcf0c5e7224b98eb1eee7e
Parents: 9777198
Author: agura <ag...@gridgain.com>
Authored: Tue Mar 1 17:46:35 2016 +0300
Committer: agura <ag...@gridgain.com>
Committed: Tue Mar 1 17:46:35 2016 +0300

----------------------------------------------------------------------
 .../GridCacheOnCopyFlagAbstractSelfTest.java    | 32 ++++++++++++--------
 1 file changed, 19 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/74fd4008/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOnCopyFlagAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOnCopyFlagAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOnCopyFlagAbstractSelfTest.java
index 8d8aa02..027f411 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOnCopyFlagAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOnCopyFlagAbstractSelfTest.java
@@ -25,6 +25,7 @@ import javax.cache.processor.EntryProcessor;
 import javax.cache.processor.EntryProcessorException;
 import javax.cache.processor.MutableEntry;
 import org.apache.ignite.IgniteCache;
+import org.apache.ignite.binary.BinaryObject;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheInterceptor;
 import org.apache.ignite.cache.CacheInterceptorAdapter;
@@ -180,10 +181,7 @@ public abstract class GridCacheOnCopyFlagAbstractSelfTest extends GridCommonAbst
                     ((GridCacheAdapter)((IgniteCacheProxy)cache).delegate()).peekEx(key).peekVisibleValue();
 
                 // Check thar internal entry wasn't changed.
-                if (storeValue(cache))
-                    assertEquals(i, ((TestValue)U.field(obj, "val")).val());
-                else
-                    assertEquals(i, CU.<TestValue>value(obj, ((IgniteCacheProxy)cache).context(), false).val());
+                assertEquals(i, getValue(obj, cache));
 
                 final TestValue newTestVal = new TestValue(-i);
 
@@ -216,10 +214,7 @@ public abstract class GridCacheOnCopyFlagAbstractSelfTest extends GridCommonAbst
                 obj = ((GridCacheAdapter)((IgniteCacheProxy)cache).delegate()).peekEx(key).peekVisibleValue();
 
                 // Check thar internal entry wasn't changed.
-                if (storeValue(cache))
-                    assertEquals(-i, ((TestValue)U.field(obj, "val")).val());
-                else
-                    assertEquals(-i, CU.<TestValue>value(obj, ((IgniteCacheProxy)cache).context(), false).val());
+                assertEquals(-i, getValue(obj, cache));
 
                 interceptor.delegate(new CacheInterceptorAdapter<TestKey, TestValue>() {
                     @Override public IgniteBiTuple onBeforeRemove(Cache.Entry<TestKey, TestValue> entry) {
@@ -297,11 +292,7 @@ public abstract class GridCacheOnCopyFlagAbstractSelfTest extends GridCommonAbst
                 CacheObject obj =
                     ((GridCacheAdapter)((IgniteCacheProxy)cache).delegate()).peekEx(key).peekVisibleValue();
 
-                if (storeValue(cache))
-                    assertNotEquals(WRONG_VALUE, ((TestValue)U.field(obj, "val")).val());
-                else
-                    assertNotEquals(WRONG_VALUE,
-                        CU.<TestValue>value(obj, ((IgniteCacheProxy)cache).context(), false).val());
+                assertNotEquals(WRONG_VALUE, getValue(obj, cache));
             }
         }
         finally {
@@ -556,6 +547,21 @@ public abstract class GridCacheOnCopyFlagAbstractSelfTest extends GridCommonAbst
     }
 
     /**
+     * @param obj Object.
+     * @param cache Cache.
+     */
+    private static Object getValue(CacheObject obj, IgniteCache cache) {
+        if (obj instanceof BinaryObject)
+            return ((BinaryObject)obj).field("val");
+        else {
+            if (storeValue(cache))
+                return ((TestValue)U.field(obj, "val")).val();
+            else
+                return CU.<TestValue>value(obj, ((IgniteCacheProxy)cache).context(), false).val();
+        }
+    }
+
+    /**
      *
      */
     public static class TestValue implements Serializable {


[18/19] ignite git commit: Merge remote-tracking branch 'remotes/origin/master' into ignite-1232

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/master' into ignite-1232


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

Branch: refs/heads/ignite-1232
Commit: 3a51439f449fe720129c8f0333947d090e55731c
Parents: 0bf14f0 4e108ad
Author: sboikov <sb...@gridgain.com>
Authored: Wed Mar 2 11:02:42 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Mar 2 11:02:42 2016 +0300

----------------------------------------------------------------------
 bin/ignite.bat                                  |    7 +-
 bin/ignite.sh                                   |    6 +-
 .../processors/cache/CacheLazyEntry.java        |    3 +-
 .../processors/cache/GridCacheProcessor.java    |    8 +
 .../processors/cache/GridCacheUtils.java        |   15 +
 .../cache/affinity/GridCacheAffinityImpl.java   |    7 +-
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |   15 +
 .../ignite/spi/IgniteSpiConsistencyChecked.java |    8 +
 .../ignite/spi/discovery/tcp/ClientImpl.java    |    6 +-
 .../ignite/spi/discovery/tcp/ServerImpl.java    |    2 +-
 .../TcpDiscoveryMulticastIpFinder.java          |    1 +
 .../spi/swapspace/file/FileSwapSpaceSpi.java    |    2 +
 .../spi/swapspace/noop/NoopSwapSpaceSpi.java    |    2 +
 .../cache/CacheEntryProcessorCopySelfTest.java  |  213 +
 .../cache/GridCacheAbstractFullApiSelfTest.java |   10 +-
 .../GridCacheOnCopyFlagAbstractSelfTest.java    |  479 +-
 ...ridCacheSwapSpaceSpiConsistencySelfTest.java |  146 +
 .../IgniteCacheConfigVariationsFullApiTest.java | 5851 ++++++++++++++++++
 .../GridSwapSpaceSpiConsistencySelfTest.java    |  131 +
 .../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 +
 .../ignite/testsuites/IgniteCacheTestSuite.java |    2 +
 .../testsuites/IgniteCacheTestSuite5.java       |    2 +
 .../IgniteSpiSwapSpaceSelfTestSuite.java        |    2 +
 .../processors/query/h2/IgniteH2Indexing.java   |    8 +
 .../cache/CacheQueryNewClientSelfTest.java      |   43 +-
 .../Apache.Ignite.Core.Tests.csproj             |    1 +
 .../ProjectFilesTest.cs                         |   94 +
 .../yardstick/IgniteBenchmarkArguments.java     |   11 +
 .../org/apache/ignite/yardstick/IgniteNode.java |    2 +
 .../cache/IgniteCacheAbstractBenchmark.java     |   54 +
 modules/zookeeper/pom.xml                       |   50 +-
 .../tcp/ipfinder/zk/ZookeeperIpFinderTest.java  |   24 +-
 49 files changed, 10196 insertions(+), 248 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/3a51439f/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
----------------------------------------------------------------------


[16/19] ignite git commit: Cleanup indexing twoStepCache on cache destroy.

Posted by sb...@apache.org.
Cleanup indexing twoStepCache on cache destroy.


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

Branch: refs/heads/ignite-1232
Commit: 4e108ad5194eb70671533ff4fe440a5963b72b85
Parents: a4391d7
Author: sboikov <sb...@gridgain.com>
Authored: Wed Mar 2 10:59:16 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Mar 2 10:59:16 2016 +0300

----------------------------------------------------------------------
 .../processors/query/h2/IgniteH2Indexing.java   |  8 ++++
 .../cache/CacheQueryNewClientSelfTest.java      | 43 ++++++++++++--------
 2 files changed, 33 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/4e108ad5/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
index a0e9c58..caf49e8 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
@@ -1626,6 +1626,14 @@ public class IgniteH2Indexing implements GridQueryIndexing {
             catch (IgniteCheckedException e) {
                 U.error(log, "Failed to drop schema on cache stop (will ignore): " + U.maskName(ccfg.getName()), e);
             }
+
+            for (Iterator<Map.Entry<T3<String, String, Boolean>, TwoStepCachedQuery>> it = twoStepCache.entrySet().iterator();
+                 it.hasNext();) {
+                Map.Entry<T3<String, String, Boolean>, TwoStepCachedQuery> e = it.next();
+
+                if (F.eq(e.getKey().get1(), ccfg.getName()))
+                    it.remove();
+            }
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/4e108ad5/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheQueryNewClientSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheQueryNewClientSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheQueryNewClientSelfTest.java
index 6e7a038..a9f5b51 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheQueryNewClientSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheQueryNewClientSelfTest.java
@@ -54,39 +54,46 @@ public class CacheQueryNewClientSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testQueryFromNewClient() throws Exception {
-        Ignite server = startGrid("server");
+        Ignite srv = startGrid("server");
 
-        IgniteCache<Integer, Integer> cache1 = server.createCache(new CacheConfiguration<Integer, Integer>().
-            setName("cache1").setIndexedTypes(Integer.class, Integer.class));
-        IgniteCache<Integer, Integer> cache2 = server.createCache(new CacheConfiguration<Integer, Integer>().
-            setName("cache2").setIndexedTypes(Integer.class, Integer.class));
+        for (int iter = 0; iter < 2; iter++) {
+            log.info("Iteration: " + iter);
 
-        for (int i = 0; i < 10; i++) {
-            cache1.put(i, i);
-            cache2.put(i, i);
-        }
+            IgniteCache<Integer, Integer> cache1 = srv.createCache(new CacheConfiguration<Integer, Integer>().
+                setName("cache1").setIndexedTypes(Integer.class, Integer.class));
+            IgniteCache<Integer, Integer> cache2 = srv.createCache(new CacheConfiguration<Integer, Integer>().
+                setName("cache2").setIndexedTypes(Integer.class, Integer.class));
 
-        Ignition.setClientMode(true);
+            for (int i = 0; i < 10; i++) {
+                cache1.put(i, i);
+                cache2.put(i, i);
+            }
 
-        Ignite client = startGrid("client");
+            Ignition.setClientMode(true);
 
-        IgniteCache<Integer, Integer> cache = client.cache("cache1");
+            Ignite client = (iter == 0) ? startGrid("client") : grid("client");
 
-        List<List<?>> res = cache.query(new SqlFieldsQuery(
-            "select i1._val, i2._val from Integer i1 cross join \"cache2\".Integer i2")).getAll();
+            IgniteCache<Integer, Integer> cache = client.cache("cache1");
 
-        assertEquals(100, res.size());
+            List<List<?>> res = cache.query(new SqlFieldsQuery(
+                "select i1._val, i2._val from Integer i1 cross join \"cache2\".Integer i2")).getAll();
+
+            assertEquals(100, res.size());
+
+            srv.destroyCache(cache1.getName());
+            srv.destroyCache(cache2.getName());
+        }
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testQueryFromNewClientCustomSchemaName() throws Exception {
-        Ignite server = startGrid("server");
+        Ignite srv = startGrid("server");
 
-        IgniteCache<Integer, Integer> cache1 = server.createCache(new CacheConfiguration<Integer, Integer>().
+        IgniteCache<Integer, Integer> cache1 = srv.createCache(new CacheConfiguration<Integer, Integer>().
             setName("cache1").setSqlSchema("cache1_sql").setIndexedTypes(Integer.class, Integer.class));
-        IgniteCache<Integer, Integer> cache2 = server.createCache(new CacheConfiguration<Integer, Integer>().
+        IgniteCache<Integer, Integer> cache2 = srv.createCache(new CacheConfiguration<Integer, Integer>().
             setName("cache2").setSqlSchema("cache2_sql").setIndexedTypes(Integer.class, Integer.class));
 
         for (int i = 0; i < 10; i++) {


[06/19] ignite git commit: Added output cache configuration in yardstick.

Posted by sb...@apache.org.
Added output cache configuration in yardstick.


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

Branch: refs/heads/ignite-1232
Commit: 62c26d5a1257eb581b35d766098bf75fd8515d68
Parents: 953b575
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Mon Feb 29 16:27:09 2016 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Mon Feb 29 16:27:09 2016 +0300

----------------------------------------------------------------------
 .../src/main/java/org/apache/ignite/yardstick/IgniteNode.java      | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/62c26d5a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteNode.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteNode.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteNode.java
index 33656fb..eed4450 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteNode.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteNode.java
@@ -136,6 +136,8 @@ public class IgniteNode implements BenchmarkServer {
             cc.setWriteThrough(args.isStoreEnabled());
 
             cc.setWriteBehindEnabled(args.isWriteBehind());
+
+            BenchmarkUtils.println(cfg, "Cache configured with the following parameters: " + cc);
         }
 
         TransactionConfiguration tc = c.getTransactionConfiguration();


[05/19] 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/ignite-1232
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));


[08/19] ignite git commit: Added print partition stats.

Posted by sb...@apache.org.
Added print partition stats.


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

Branch: refs/heads/ignite-1232
Commit: c4d6f3cddf0a7ef89d6126eff432775e713e797a
Parents: 071498f
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Mon Feb 29 22:37:38 2016 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Mon Feb 29 22:37:38 2016 +0300

----------------------------------------------------------------------
 .../ignite/yardstick/cache/IgniteCacheAbstractBenchmark.java | 8 +++++---
 1 file changed, 5 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/c4d6f3cd/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteCacheAbstractBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteCacheAbstractBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteCacheAbstractBenchmark.java
index c9a4b9c..6c25ec0 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteCacheAbstractBenchmark.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteCacheAbstractBenchmark.java
@@ -87,9 +87,11 @@ public abstract class IgniteCacheAbstractBenchmark<K, V> extends IgniteAbstractB
                 List<Integer> primary = e.getValue().get1();
                 List<Integer> backup = e.getValue().get2();
 
-                BenchmarkUtils.println(cfg, e.getKey().id() + " " + primary.size() + " " + primary.size() * 1. /
-                    aff.partitions() + " " + backup.size() + " "
-                    + backup.size() * 1. / (aff.partitions() * args.backups() == 0 ? 1 : args.backups())
+                BenchmarkUtils.println(cfg, e.getKey().id() + "  "
+                    + primary.size() + "  " + primary.size() * 1. /aff.partitions() + "  "
+                    + backup.size() + "  "
+                    + backup.size() * 1. / (aff.partitions() * (args.backups() == 0 ? 1 : args.backups())) + "  "
+                    + (primary.size() + backup.size()) + "  "
                     + (primary.size() + backup.size() * 1.) / (aff.partitions() * args.backups() + aff.partitions())
                 );
             }


[10/19] ignite git commit: ignite-2719 Value is not copied in entry processor

Posted by sb...@apache.org.
ignite-2719 Value is not copied in entry processor


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

Branch: refs/heads/ignite-1232
Commit: 9777198aa25b9127764c5c539d77c39f5a770661
Parents: 8917269
Author: agura <ag...@gridgain.com>
Authored: Sun Feb 28 19:43:58 2016 +0300
Committer: agura <ag...@gridgain.com>
Committed: Tue Mar 1 13:54:51 2016 +0300

----------------------------------------------------------------------
 .../processors/cache/CacheLazyEntry.java        |   3 +-
 .../cache/CacheEntryProcessorCopySelfTest.java  | 213 +++++++++
 .../GridCacheOnCopyFlagAbstractSelfTest.java    | 473 +++++++++++--------
 .../ignite/testsuites/IgniteCacheTestSuite.java |   2 +
 4 files changed, 491 insertions(+), 200 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/9777198a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLazyEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLazyEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLazyEntry.java
index 30933e5..6ec17c0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLazyEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLazyEntry.java
@@ -114,9 +114,10 @@ public class CacheLazyEntry<K, V> implements Cache.Entry<K, V> {
      * @param keepBinary Flag to keep binary if needed.
      * @return the value corresponding to this entry
      */
+    @SuppressWarnings("unchecked")
     public V getValue(boolean keepBinary) {
         if (val == null)
-            val = (V)cctx.unwrapBinaryIfNeeded(valObj, keepBinary, false);
+            val = (V)cctx.unwrapBinaryIfNeeded(valObj, keepBinary, true);
 
         return val;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/9777198a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheEntryProcessorCopySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheEntryProcessorCopySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheEntryProcessorCopySelfTest.java
new file mode 100644
index 0000000..21395e6
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheEntryProcessorCopySelfTest.java
@@ -0,0 +1,213 @@
+/*
+ * 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 java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.util.concurrent.atomic.AtomicInteger;
+import javax.cache.processor.MutableEntry;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.binary.BinaryObject;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheEntryProcessor;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+/**
+ * Test for value copy in entry processor.
+ */
+public class CacheEntryProcessorCopySelfTest extends GridCommonAbstractTest {
+    /** Old value. */
+    private static final int OLD_VAL = 100;
+
+    /** New value. */
+    private static final int NEW_VAL = 200;
+
+    /** Empty array. */
+    private static final int[] EMPTY_ARR = new int[0];
+
+    /** Deserializations counter. */
+    private static final AtomicInteger cnt = new AtomicInteger();
+
+    /** p2p enabled. */
+    private boolean p2pEnabled;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cfg.setIncludeEventTypes(EMPTY_ARR);
+
+        cfg.setPeerClassLoadingEnabled(p2pEnabled);
+
+        return cfg;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMutableEntryWithP2PEnabled() throws Exception {
+        doTestMutableEntry(true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMutableEntryWithP2PDisabled() throws Exception {
+        doTestMutableEntry(false);
+    }
+
+    /**
+     *
+     */
+    private void doTestMutableEntry(boolean p2pEnabled) throws Exception {
+        this.p2pEnabled = p2pEnabled;
+
+        Ignite grid = startGrid();
+
+        assertEquals(p2pEnabled, grid.configuration().isPeerClassLoadingEnabled());
+
+        try {
+            // One deserialization due to copyOnRead == true.
+            doTest(true, false, OLD_VAL, 1);
+
+            // One deserialization due to copyOnRead == true.
+            // Additional deserialization in case p2p enabled due to storeValue == true on update entry.
+            doTest(true, true, NEW_VAL, p2pEnabled ? 2 : 1);
+
+            // No deserialization.
+            doTest(false, false, NEW_VAL, 0);
+
+            // One deserialization due to storeValue == true.
+            doTest(false, true, NEW_VAL, 1);
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
+
+    /**
+     * @param cpOnRead Copy on read.
+     * @param mutate Mutate.
+     * @param expVal Expected value.
+     * @param expCnt Expected deserializations count.
+     */
+    @SuppressWarnings("unchecked")
+    private void doTest(boolean cpOnRead, final boolean mutate, int expVal, int expCnt) throws Exception {
+
+        Ignite ignite = grid();
+
+        CacheConfiguration ccfg = defaultCacheConfiguration();
+        ccfg.setAtomicityMode(CacheAtomicityMode.ATOMIC);
+        ccfg.setCopyOnRead(cpOnRead);
+        ccfg.setNearConfiguration(null);
+
+        IgniteCache<Integer, Value> cache = null;
+
+        try {
+            cache = ignite.createCache(ccfg);
+
+            cache.put(0, new Value(OLD_VAL));
+
+            cache.get(0);
+
+            cnt.set(0);
+
+            cache.invoke(0, new CacheEntryProcessor<Integer, Value, Object>() {
+                @Override public Object process(MutableEntry<Integer, Value> entry, Object... args) {
+                    Value val = entry.getValue();
+
+                    val.i = NEW_VAL;
+
+                    if (mutate)
+                        entry.setValue(val);
+
+                    return null;
+                }
+            });
+
+            CacheObject obj = ((GridCacheAdapter)((IgniteCacheProxy)cache).delegate()).peekEx(0).peekVisibleValue();
+
+            int actCnt = cnt.get();
+
+            if (obj instanceof BinaryObject)
+                if (cpOnRead)
+                    assertEquals(expVal, (int)((BinaryObject)obj).field("i"));
+                else
+                    assertEquals(expVal, ((Value)U.field(obj, "obj")).i);
+            else {
+                if (storeValue(cache))
+                    assertEquals(expVal, U.<Value>field(obj, "val").i);
+                else
+                    assertEquals(expVal, CU.<Value>value(obj, ((IgniteCacheProxy)cache).context(), false).i);
+            }
+
+            assertEquals(expCnt, actCnt);
+        }
+        finally {
+            if (cache != null)
+                cache.destroy();
+        }
+    }
+
+    /**
+     * @param cache Cache.
+     */
+    private static boolean storeValue(IgniteCache cache) {
+        return ((IgniteCacheProxy)cache).context().cacheObjectContext().storeValue();
+    }
+
+    /**
+     *
+     */
+    private static class Value implements Externalizable {
+        /**  */
+        private int i;
+
+        /**
+         * Default constructor (required by Externalizable).
+         */
+        public Value() {
+        }
+
+        /**
+         * @param i I.
+         */
+        public Value(int i) {
+            this.i = i;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void writeExternal(ObjectOutput out) throws IOException {
+            out.writeInt(i);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+            i = in.readInt();
+
+            cnt.incrementAndGet();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/9777198a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOnCopyFlagAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOnCopyFlagAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOnCopyFlagAbstractSelfTest.java
index 6f73e0f..8d8aa02 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOnCopyFlagAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOnCopyFlagAbstractSelfTest.java
@@ -17,10 +17,6 @@
 
 package org.apache.ignite.internal.processors.cache;
 
-import java.io.Externalizable;
-import java.io.IOException;
-import java.io.ObjectInput;
-import java.io.ObjectOutput;
 import java.io.Serializable;
 import java.util.HashMap;
 import java.util.Map;
@@ -32,12 +28,17 @@ import org.apache.ignite.IgniteCache;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheInterceptor;
 import org.apache.ignite.cache.CacheInterceptorAdapter;
+import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.jetbrains.annotations.Nullable;
 
 import static org.junit.Assert.assertNotEquals;
@@ -45,7 +46,7 @@ import static org.junit.Assert.assertNotEquals;
 /**
  * Tests that cache value is copied for get, interceptor and invoke closure.
  */
-public abstract class GridCacheOnCopyFlagAbstractSelfTest extends GridCacheAbstractSelfTest {
+public abstract class GridCacheOnCopyFlagAbstractSelfTest extends GridCommonAbstractTest {
     /** */
     private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
 
@@ -61,36 +62,21 @@ public abstract class GridCacheOnCopyFlagAbstractSelfTest extends GridCacheAbstr
     /** */
     private static boolean noInterceptor;
 
-    /** {@inheritDoc} */
-    @Override protected int gridCount() {
-        return 1;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        interceptor = new Interceptor();
-
-        super.beforeTestsStarted();
-
-        awaitPartitionMapExchange();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTest() throws Exception {
-        super.beforeTest();
-
-        noInterceptor = false;
-    }
+    /** p2p enabled. */
+    private boolean p2pEnabled;
 
-    /** {@inheritDoc} */
-    @Override protected void afterTest() throws Exception {
-        super.afterTest();
+    /**
+     * Returns cache mode for tests.
+     * @return cache mode.
+     */
+    protected abstract CacheMode cacheMode();
 
-        interceptor.delegate(new CacheInterceptorAdapter<TestKey, TestValue>());
+    /**
+     * Returns cache atomicity mode for cache.
+     * @return cache atomicity mode.
+     */
+    protected abstract CacheAtomicityMode atomicityMode();
 
-        for (int i = 0; i < gridCount(); i++)
-            jcache(i, null).localClearAll(keySet(jcache(i, null)));
-    }
 
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
@@ -102,7 +88,7 @@ public abstract class GridCacheOnCopyFlagAbstractSelfTest extends GridCacheAbstr
 
         c.setDiscoverySpi(spi);
 
-        c.setPeerClassLoadingEnabled(false);
+        c.setPeerClassLoadingEnabled(p2pEnabled);
 
         c.getTransactionConfiguration().setTxSerializableEnabled(true);
 
@@ -111,12 +97,12 @@ public abstract class GridCacheOnCopyFlagAbstractSelfTest extends GridCacheAbstr
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override protected CacheConfiguration cacheConfiguration(String gridName) throws Exception {
-        CacheConfiguration ccfg = super.cacheConfiguration(gridName);
+    protected CacheConfiguration cacheConfiguration() throws Exception {
+        CacheConfiguration ccfg = defaultCacheConfiguration();
 
         assertTrue(ccfg.isCopyOnRead());
 
-        assertNotNull(interceptor);
+        interceptor = new Interceptor();
 
         ccfg.setInterceptor(interceptor);
 
@@ -127,288 +113,370 @@ public abstract class GridCacheOnCopyFlagAbstractSelfTest extends GridCacheAbstr
         return ccfg;
     }
 
-    /** {@inheritDoc} */
-    @Override protected abstract CacheAtomicityMode atomicityMode();
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCopyOnReadFlagP2PEnabled() throws Exception {
+        doTest(true);
+    }
 
     /**
      * @throws Exception If failed.
      */
-    public void testInterceptor() throws Exception {
-        IgniteCache<TestKey, TestValue> cache = grid(0).cache(null);
+    public void testCopyOnReadFlagP2PDisbaled() throws Exception {
+        doTest(false);
+    }
 
-        for (int i = 0; i < ITER_CNT; i++) {
-            final TestValue val = new TestValue(i);
-            final TestKey key = new TestKey(i, i);
+    /**
+     * @param p2pEnabled P 2 p enabled.
+     */
+    private void doTest(boolean p2pEnabled) throws Exception {
+        this.p2pEnabled = p2pEnabled;
 
-            interceptor.delegate(new CacheInterceptorAdapter<TestKey, TestValue>() {
-                @Override public void onAfterPut(Cache.Entry<TestKey, TestValue> entry) {
-                    assertNotSame(key, entry.getKey());
+        IgniteEx grid = startGrid(0);
 
-                    assertSame(entry.getValue(), entry.getValue());
-                    assertSame(entry.getKey(), entry.getKey());
+        assertEquals(p2pEnabled, grid.configuration().isPeerClassLoadingEnabled());
 
-                    // Try change value.
-                    entry.getValue().val(WRONG_VALUE);
-                }
-            });
+        try {
+            interceptor();
+            invokeAndInterceptor();
+            putGet();
+            putGetByteArray();
+            putGetKnownImmutable();
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
 
-            cache.put(key, val);
+    /**
+     * @throws Exception If failed.
+     */
+    private void interceptor() throws Exception {
+        noInterceptor = false;
 
-            Cache.Entry<Object, Object> entry = grid(0).cache(null).localEntries().iterator().next();
+        IgniteCache<TestKey, TestValue> cache = grid(0).createCache(cacheConfiguration());
 
-            // Check thar internal entry wasn't changed.
-            assertEquals(i, ((TestKey)entry.getKey()).field());
-            assertEquals(i, ((TestValue)entry.getValue()).val());
+        try {
+            for (int i = 0; i < ITER_CNT; i++) {
+                final TestValue val = new TestValue(i);
+                final TestKey key = new TestKey(i, i);
 
-            final TestValue newTestVal = new TestValue(-i);
+                interceptor.delegate(new CacheInterceptorAdapter<TestKey, TestValue>() {
+                    @Override public void onAfterPut(Cache.Entry<TestKey, TestValue> entry) {
+                        assertNotSame(key, entry.getKey());
 
-            interceptor.delegate(new CacheInterceptorAdapter<TestKey, TestValue>() {
-                @Override public TestValue onBeforePut(Cache.Entry<TestKey, TestValue> entry, TestValue newVal) {
-                    assertNotSame(key, entry.getKey());
-                    assertNotSame(val, entry.getValue());
+                        assertSame(entry.getValue(), entry.getValue());
+                        assertSame(entry.getKey(), entry.getKey());
 
-                    assertEquals(newTestVal, newVal);
+                        // Try change value.
+                        entry.getValue().val(WRONG_VALUE);
+                    }
+                });
 
-                    // Try change value.
-                    entry.getValue().val(WRONG_VALUE);
+                cache.put(key, val);
 
-                    return newVal;
-                }
+                CacheObject obj =
+                    ((GridCacheAdapter)((IgniteCacheProxy)cache).delegate()).peekEx(key).peekVisibleValue();
 
-                @Override public void onAfterPut(Cache.Entry<TestKey, TestValue> entry) {
-                    assertNotSame(key, entry.getKey());
+                // Check thar internal entry wasn't changed.
+                if (storeValue(cache))
+                    assertEquals(i, ((TestValue)U.field(obj, "val")).val());
+                else
+                    assertEquals(i, CU.<TestValue>value(obj, ((IgniteCacheProxy)cache).context(), false).val());
 
-                    assertSame(entry.getValue(), entry.getValue());
-                    assertSame(entry.getKey(), entry.getKey());
+                final TestValue newTestVal = new TestValue(-i);
 
-                    // Try change value.
-                    entry.getValue().val(WRONG_VALUE);
-                }
-            });
+                interceptor.delegate(new CacheInterceptorAdapter<TestKey, TestValue>() {
+                    @Override public TestValue onBeforePut(Cache.Entry<TestKey, TestValue> entry, TestValue newVal) {
+                        assertNotSame(key, entry.getKey());
+                        assertNotSame(val, entry.getValue());
 
-            cache.put(key, newTestVal);
+                        assertEquals(newTestVal, newVal);
 
-            entry = grid(0).cache(null).localEntries().iterator().next();
+                        // Try change value.
+                        entry.getValue().val(WRONG_VALUE);
 
-            // Check thar internal entry wasn't changed.
-            assertEquals(i, ((TestKey)entry.getKey()).field());
-            assertEquals(-i, ((TestValue)entry.getValue()).val());
+                        return newVal;
+                    }
 
-            interceptor.delegate(new CacheInterceptorAdapter<TestKey, TestValue>() {
-                @Override public IgniteBiTuple onBeforeRemove(Cache.Entry<TestKey, TestValue> entry) {
-                    assertNotSame(key, entry.getKey());
-                    assertNotSame(newTestVal, entry.getValue());
+                    @Override public void onAfterPut(Cache.Entry<TestKey, TestValue> entry) {
+                        assertNotSame(key, entry.getKey());
 
-                    return super.onBeforeRemove(entry);
-                }
+                        assertSame(entry.getValue(), entry.getValue());
+                        assertSame(entry.getKey(), entry.getKey());
 
-                @Override public void onAfterRemove(Cache.Entry<TestKey, TestValue> entry) {
-                    assertNotSame(key, entry.getKey());
-                    assertNotSame(newTestVal, entry.getValue());
-                }
-            });
+                        // Try change value.
+                        entry.getValue().val(WRONG_VALUE);
+                    }
+                });
+
+                cache.put(key, newTestVal);
+
+                obj = ((GridCacheAdapter)((IgniteCacheProxy)cache).delegate()).peekEx(key).peekVisibleValue();
+
+                // Check thar internal entry wasn't changed.
+                if (storeValue(cache))
+                    assertEquals(-i, ((TestValue)U.field(obj, "val")).val());
+                else
+                    assertEquals(-i, CU.<TestValue>value(obj, ((IgniteCacheProxy)cache).context(), false).val());
+
+                interceptor.delegate(new CacheInterceptorAdapter<TestKey, TestValue>() {
+                    @Override public IgniteBiTuple onBeforeRemove(Cache.Entry<TestKey, TestValue> entry) {
+                        assertNotSame(key, entry.getKey());
+                        assertNotSame(newTestVal, entry.getValue());
+
+                        return super.onBeforeRemove(entry);
+                    }
 
-            cache.remove(key);
+                    @Override public void onAfterRemove(Cache.Entry<TestKey, TestValue> entry) {
+                        assertNotSame(key, entry.getKey());
+                        assertNotSame(newTestVal, entry.getValue());
+                    }
+                });
+
+                cache.remove(key);
+            }
+        }
+        finally {
+            if (cache != null)
+                cache.destroy();
         }
     }
 
     /**
      * @throws Exception If failed.
      */
-    public void testInvokeAndInterceptor() throws Exception {
-        IgniteCache<TestKey, TestValue> cache = grid(0).cache(null);
-
-        for (int i = 0; i < ITER_CNT; i++)
-            cache.put(new TestKey(i, i), new TestValue(i));
-
-        interceptor.delegate(new CacheInterceptorAdapter<TestKey, TestValue>(){
-            @Override public TestValue onBeforePut(Cache.Entry<TestKey, TestValue> entry, TestValue newVal) {
-                // Check that we have correct value and key.
-                assertEquals(entry.getKey().key(), entry.getKey().field());
+    private void invokeAndInterceptor() throws Exception {
+        noInterceptor = false;
 
-                // Try changed entry.
-                entry.getValue().val(WRONG_VALUE);
+        IgniteCache<TestKey, TestValue> cache = grid(0).createCache(cacheConfiguration());
 
-                return super.onBeforePut(entry, newVal);
-            }
+        try {
+            for (int i = 0; i < ITER_CNT; i++)
+                cache.put(new TestKey(i, i), new TestValue(i));
 
-            @Override public void onAfterPut(Cache.Entry<TestKey, TestValue> entry) {
-                assertEquals(entry.getKey().key(), entry.getKey().field());
+            interceptor.delegate(new CacheInterceptorAdapter<TestKey, TestValue>(){
+                @Override public TestValue onBeforePut(Cache.Entry<TestKey, TestValue> entry, TestValue newVal) {
+                    // Check that we have correct value and key.
+                    assertEquals(entry.getKey().key(), entry.getKey().field());
 
-                entry.getValue().val(WRONG_VALUE);
+                    // Try changed entry.
+                    entry.getValue().val(WRONG_VALUE);
 
-                super.onAfterPut(entry);
-            }
-        });
+                    return super.onBeforePut(entry, newVal);
+                }
 
-        for (int i = 0; i < ITER_CNT; i++)
-            cache.invoke(new TestKey(i, i), new EntryProcessor<TestKey, TestValue, Object>() {
-                @Override public Object process(MutableEntry<TestKey, TestValue> entry, Object... arguments)
-                    throws EntryProcessorException {
-                    // Check that we have correct value.
-                    assertEquals(entry.getKey().key(), entry.getValue().val());
+                @Override public void onAfterPut(Cache.Entry<TestKey, TestValue> entry) {
+                    assertEquals(entry.getKey().key(), entry.getKey().field());
 
-                    // Try changed entry.
                     entry.getValue().val(WRONG_VALUE);
 
-                    return -1;
+                    super.onAfterPut(entry);
                 }
             });
 
-        // Check that entries weren't changed.
-        for (Cache.Entry<Object, Object> e : grid(0).cache(null).localEntries()) {
-            assertNotEquals(WRONG_VALUE, ((TestKey)e.getKey()).field());
-            assertNotEquals(WRONG_VALUE, ((TestValue)e.getValue()).val());
+            for (int i = 0; i < ITER_CNT; i++) {
+                TestKey key = new TestKey(i, i);
+
+                cache.invoke(key, new EntryProcessor<TestKey, TestValue, Object>() {
+                    @Override public Object process(MutableEntry<TestKey, TestValue> entry, Object... arguments)
+                        throws EntryProcessorException {
+                        TestValue val = entry.getValue();
+
+                        // Check that we have correct value.
+                        assertEquals(entry.getKey().key(), val.val());
+
+                        // Try changed entry.
+                        val.val(WRONG_VALUE);
+
+                        return -1;
+                    }
+                });
+
+                CacheObject obj =
+                    ((GridCacheAdapter)((IgniteCacheProxy)cache).delegate()).peekEx(key).peekVisibleValue();
+
+                if (storeValue(cache))
+                    assertNotEquals(WRONG_VALUE, ((TestValue)U.field(obj, "val")).val());
+                else
+                    assertNotEquals(WRONG_VALUE,
+                        CU.<TestValue>value(obj, ((IgniteCacheProxy)cache).context(), false).val());
+            }
+        }
+        finally {
+            if (cache != null)
+                cache.destroy();
         }
     }
 
     /**
      * @throws Exception If failed.
      */
-    public void testPutGet() throws Exception {
+    private void putGet() throws Exception {
         noInterceptor = true;
 
-        IgniteCache<TestKey, TestValue> cache = grid(0).cache(null);
+        IgniteCache<TestKey, TestValue> cache = grid(0).createCache(cacheConfiguration());
 
-        Map<TestKey, TestValue> map = new HashMap<>();
+        try {
+            Map<TestKey, TestValue> map = new HashMap<>();
 
-        for (int i = 0; i < ITER_CNT; i++) {
-            TestKey key = new TestKey(i, i);
-            TestValue val = new TestValue(i);
+            for (int i = 0; i < ITER_CNT; i++) {
+                TestKey key = new TestKey(i, i);
+                TestValue val = new TestValue(i);
 
-            cache.put(key, val);
+                cache.put(key, val);
 
-            map.put(key, val);
-        }
+                map.put(key, val);
+            }
 
-        GridCacheAdapter cache0 = internalCache(cache);
+            GridCacheAdapter cache0 = internalCache(cache);
 
-        GridCacheContext cctx = cache0.context();
+            GridCacheContext cctx = cache0.context();
 
-        boolean binary = cctx.cacheObjects().isBinaryEnabled(null);
+            boolean binary = cctx.cacheObjects().isBinaryEnabled(null);
 
-        for (Map.Entry<TestKey, TestValue> e : map.entrySet()) {
-            GridCacheEntryEx entry = cache0.peekEx(e.getKey());
+            for (Map.Entry<TestKey, TestValue> e : map.entrySet()) {
+                GridCacheEntryEx entry = cache0.peekEx(e.getKey());
 
-            assertNotNull("No entry for key: " + e.getKey(), entry);
+                assertNotNull("No entry for key: " + e.getKey(), entry);
 
-            TestKey key0 = entry.key().value(cctx.cacheObjectContext(), false);
+                TestKey key0 = entry.key().value(cctx.cacheObjectContext(), false);
 
-            assertNotSame(key0, e.getKey());
+                assertNotSame(key0, e.getKey());
 
-            TestKey key1 = entry.key().value(cctx.cacheObjectContext(), true);
+                TestKey key1 = entry.key().value(cctx.cacheObjectContext(), true);
 
-            if (!binary)
-                assertSame(key0, key1);
-            else
-                assertNotSame(key0, key1);
+                if (!binary)
+                    assertSame(key0, key1);
+                else
+                    assertNotSame(key0, key1);
 
-            TestValue val0 = entry.rawGet().value(cctx.cacheObjectContext(), false);
+                TestValue val0 = entry.rawGet().value(cctx.cacheObjectContext(), false);
 
-            assertNotSame(val0, e.getValue());
+                assertNotSame(val0, e.getValue());
 
-            TestValue val1 = entry.rawGet().value(cctx.cacheObjectContext(), true);
+                TestValue val1 = entry.rawGet().value(cctx.cacheObjectContext(), true);
 
-            assertNotSame(val0, val1);
+                assertNotSame(val0, val1);
+            }
+        }
+        finally {
+            if (cache != null)
+                cache.destroy();
         }
     }
 
     /**
      * @throws Exception If failed.
      */
-    public void testPutGetByteArray() throws Exception {
+    private void putGetByteArray() throws Exception {
         noInterceptor = true;
 
-        IgniteCache<TestKey, byte[]> cache = grid(0).cache(null);
+        IgniteCache<TestKey, byte[]> cache = grid(0).createCache(cacheConfiguration());
 
-        Map<TestKey, byte[]> map = new HashMap<>();
+        try {
+            Map<TestKey, byte[]> map = new HashMap<>();
 
-        for (int i = 0; i < ITER_CNT; i++) {
-            TestKey key = new TestKey(i, i);
-            byte[] val = new byte[10];
+            for (int i = 0; i < ITER_CNT; i++) {
+                TestKey key = new TestKey(i, i);
+                byte[] val = new byte[10];
 
-            cache.put(key, val);
+                cache.put(key, val);
 
-            map.put(key, val);
-        }
+                map.put(key, val);
+            }
 
-        GridCacheAdapter cache0 = internalCache(cache);
+            GridCacheAdapter cache0 = internalCache(cache);
 
-        GridCacheContext cctx = cache0.context();
+            GridCacheContext cctx = cache0.context();
 
-        boolean binary = cctx.cacheObjects().isBinaryEnabled(null);
+            boolean binary = cctx.cacheObjects().isBinaryEnabled(null);
 
-        for (Map.Entry<TestKey, byte[]> e : map.entrySet()) {
-            GridCacheEntryEx entry = cache0.peekEx(e.getKey());
+            for (Map.Entry<TestKey, byte[]> e : map.entrySet()) {
+                GridCacheEntryEx entry = cache0.peekEx(e.getKey());
 
-            assertNotNull("No entry for key: " + e.getKey(), entry);
+                assertNotNull("No entry for key: " + e.getKey(), entry);
 
-            TestKey key0 = entry.key().value(cctx.cacheObjectContext(), false);
+                TestKey key0 = entry.key().value(cctx.cacheObjectContext(), false);
 
-            assertNotSame(key0, e.getKey());
+                assertNotSame(key0, e.getKey());
 
-            TestKey key1 = entry.key().value(cctx.cacheObjectContext(), true);
+                TestKey key1 = entry.key().value(cctx.cacheObjectContext(), true);
 
-            if (!binary)
-                assertSame(key0, key1);
-            else
-                assertNotSame(key0, key1);
+                if (!binary)
+                    assertSame(key0, key1);
+                else
+                    assertNotSame(key0, key1);
 
-            byte[] val0 = entry.rawGet().value(cctx.cacheObjectContext(), false);
+                byte[] val0 = entry.rawGet().value(cctx.cacheObjectContext(), false);
 
-            assertNotSame(val0, e.getValue());
+                assertNotSame(val0, e.getValue());
 
-            byte[] val1 = entry.rawGet().value(cctx.cacheObjectContext(), true);
+                byte[] val1 = entry.rawGet().value(cctx.cacheObjectContext(), true);
 
-            assertNotSame(val0, val1);
+                assertNotSame(val0, val1);
+            }
+        }
+        finally {
+            if (cache != null)
+                cache.destroy();
         }
     }
 
     /**
      * @throws Exception If failed.
      */
-    public void testPutGetKnownImmutable() throws Exception {
+    private void putGetKnownImmutable() throws Exception {
         noInterceptor = true;
 
-        IgniteCache<String, Long> cache = grid(0).cache(null);
+        IgniteCache<String, Long> cache = grid(0).createCache(cacheConfiguration());
 
-        Map<String, Long> map = new HashMap<>();
+        try {
+            Map<String, Long> map = new HashMap<>();
 
-        for (int i = 0; i < ITER_CNT; i++) {
-            String key = String.valueOf(i);
-            Long val = Long.MAX_VALUE - i;
+            for (int i = 0; i < ITER_CNT; i++) {
+                String key = String.valueOf(i);
+                Long val = Long.MAX_VALUE - i;
 
-            cache.put(key, val);
+                cache.put(key, val);
 
-            map.put(key, val);
-        }
+                map.put(key, val);
+            }
 
-        GridCacheAdapter cache0 = internalCache(cache);
+            GridCacheAdapter cache0 = internalCache(cache);
 
-        GridCacheContext cctx = cache0.context();
+            GridCacheContext cctx = cache0.context();
 
-        for (Map.Entry<String, Long> e : map.entrySet()) {
-            GridCacheEntryEx entry = cache0.peekEx(e.getKey());
+            for (Map.Entry<String, Long> e : map.entrySet()) {
+                GridCacheEntryEx entry = cache0.peekEx(e.getKey());
 
-            assertNotNull("No entry for key: " + e.getKey(), entry);
+                assertNotNull("No entry for key: " + e.getKey(), entry);
 
-            String key0 = entry.key().value(cctx.cacheObjectContext(), false);
+                String key0 = entry.key().value(cctx.cacheObjectContext(), false);
 
-            assertSame(key0, e.getKey());
+                assertSame(key0, e.getKey());
 
-            String key1 = entry.key().value(cctx.cacheObjectContext(), true);
+                String key1 = entry.key().value(cctx.cacheObjectContext(), true);
 
-            assertSame(key0, key1);
+                assertSame(key0, key1);
 
-            Long val0 = entry.rawGet().value(cctx.cacheObjectContext(), false);
+                if (!storeValue(cache)) {
+                    Long val0 = entry.rawGet().value(cctx.cacheObjectContext(), false);
 
-            assertNotSame(val0, e.getValue());
+                    assertNotSame(val0, e.getValue());
 
-            Long val1 = entry.rawGet().value(cctx.cacheObjectContext(), true);
+                    Long val1 = entry.rawGet().value(cctx.cacheObjectContext(), true);
 
-            assertNotSame(val0, val1);
+                    assertNotSame(val0, val1);
 
-            assertNotSame(e.getValue(), cache.get(e.getKey()));
+                    assertNotSame(e.getValue(), cache.get(e.getKey()));
+                }
+            }
+        }
+        finally {
+            if (cache != null)
+                cache.destroy();
         }
     }
 
@@ -481,6 +549,13 @@ public abstract class GridCacheOnCopyFlagAbstractSelfTest extends GridCacheAbstr
     }
 
     /**
+     * @param cache Cache.
+     */
+    private static boolean storeValue(IgniteCache cache) {
+        return ((IgniteCacheProxy)cache).context().cacheObjectContext().storeValue();
+    }
+
+    /**
      *
      */
     public static class TestValue implements Serializable {

http://git-wip-us.apache.org/repos/asf/ignite/blob/9777198a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
index 5af37a6..045ff6f 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
@@ -37,6 +37,7 @@ import org.apache.ignite.cache.store.jdbc.GridCacheJdbcBlobStoreMultithreadedSel
 import org.apache.ignite.cache.store.jdbc.GridCacheJdbcBlobStoreSelfTest;
 import org.apache.ignite.internal.processors.cache.CacheAffinityCallSelfTest;
 import org.apache.ignite.internal.processors.cache.CacheDeferredDeleteSanitySelfTest;
+import org.apache.ignite.internal.processors.cache.CacheEntryProcessorCopySelfTest;
 import org.apache.ignite.internal.processors.cache.CacheFutureExceptionSelfTest;
 import org.apache.ignite.internal.processors.cache.CacheNamesSelfTest;
 import org.apache.ignite.internal.processors.cache.CachePutEventListenerErrorSelfTest;
@@ -178,6 +179,7 @@ public class IgniteCacheTestSuite extends TestSuite {
         suite.addTestSuite(IgniteCacheAtomicLocalWithStoreInvokeTest.class);
         suite.addTestSuite(IgniteCacheTxInvokeTest.class);
         suite.addTestSuite(IgniteCacheEntryProcessorCallTest.class);
+        suite.addTestSuite(CacheEntryProcessorCopySelfTest.class);
         suite.addTestSuite(IgniteCacheTxNearEnabledInvokeTest.class);
         suite.addTestSuite(IgniteCacheTxLocalInvokeTest.class);
         suite.addTestSuite(IgniteCrossCacheTxStoreSelfTest.class);


[07/19] ignite git commit: Added print partition stats.

Posted by sb...@apache.org.
Added print partition stats.


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

Branch: refs/heads/ignite-1232
Commit: 071498f10ab1857759bc3c180199b6c7efbce81a
Parents: 62c26d5
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Mon Feb 29 22:20:15 2016 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Mon Feb 29 22:20:15 2016 +0300

----------------------------------------------------------------------
 .../yardstick/IgniteBenchmarkArguments.java     | 11 +++++
 .../cache/IgniteCacheAbstractBenchmark.java     | 52 ++++++++++++++++++++
 2 files changed, 63 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/071498f1/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteBenchmarkArguments.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteBenchmarkArguments.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteBenchmarkArguments.java
index 1ecfa0f..36551cc 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteBenchmarkArguments.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteBenchmarkArguments.java
@@ -136,6 +136,17 @@ public class IgniteBenchmarkArguments {
     @Parameter(names = {"-kpt", "--keysPerThread"}, description = "Use not intersecting keys in putAll benchmark")
     private boolean keysPerThread;
 
+    /** */
+    @Parameter(names = {"-pp", "--printPartitionStats"}, description = "Print partition statistics")
+    private boolean printPartStats;
+
+    /**
+     * @return If {@code true} when need to print partition statistics.
+     */
+    public boolean printPartitionStatistics() {
+        return printPartStats;
+    }
+
     /**
      * @return JDBC url.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/071498f1/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteCacheAbstractBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteCacheAbstractBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteCacheAbstractBenchmark.java
index 3efa4a5..c9a4b9c 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteCacheAbstractBenchmark.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteCacheAbstractBenchmark.java
@@ -17,9 +17,18 @@
 
 package org.apache.ignite.yardstick.cache;
 
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
 import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.affinity.Affinity;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.util.typedef.T2;
+import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.yardstick.IgniteAbstractBenchmark;
 import org.yardstickframework.BenchmarkConfiguration;
 import org.yardstickframework.BenchmarkUtils;
@@ -42,6 +51,49 @@ public abstract class IgniteCacheAbstractBenchmark<K, V> extends IgniteAbstractB
         super.setUp(cfg);
 
         cache = cache();
+
+        if (args.printPartitionStatistics()) {
+            Map<ClusterNode, T2<List<Integer>, List<Integer>>> parts = new HashMap<>();
+
+            for (ClusterNode node : ignite().cluster().nodes())
+                parts.put(node,
+                    new T2<List<Integer>, List<Integer>>(new ArrayList<Integer>(), new ArrayList<Integer>()));
+
+            U.sleep(5000);
+
+            Affinity<Object> aff = ignite().affinity(cache.getName());
+
+            for (int p = 0; p < aff.partitions(); p++) {
+                Collection<ClusterNode> nodes = aff.mapPartitionToPrimaryAndBackups(p);
+
+                boolean primary = true;
+
+                for (ClusterNode node : nodes) {
+                    if (primary) {
+                        parts.get(node).get1().add(p);
+
+                        primary = false;
+                    }
+                    else
+                        parts.get(node).get2().add(p);
+                }
+            }
+
+            BenchmarkUtils.println(cfg, "Partition stats. [cacheName: "+ cache.getName() +", topVer: "
+                + ignite().cluster().topologyVersion() + "]");
+            BenchmarkUtils.println(cfg, "(Node id,  Number of Primary, Percent, Number of Backup, Percent, Total, Percent)");
+
+            for (Map.Entry<ClusterNode, T2<List<Integer>, List<Integer>>> e : parts.entrySet()) {
+                List<Integer> primary = e.getValue().get1();
+                List<Integer> backup = e.getValue().get2();
+
+                BenchmarkUtils.println(cfg, e.getKey().id() + " " + primary.size() + " " + primary.size() * 1. /
+                    aff.partitions() + " " + backup.size() + " "
+                    + backup.size() * 1. / (aff.partitions() * args.backups() == 0 ? 1 : args.backups())
+                    + (primary.size() + backup.size() * 1.) / (aff.partitions() * args.backups() + aff.partitions())
+                );
+            }
+        }
     }
 
     /**


[17/19] ignite git commit: ignite-1232 Improved test.

Posted by sb...@apache.org.
ignite-1232 Improved test.


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

Branch: refs/heads/ignite-1232
Commit: 0bf14f09aa7f7099fc4b4dfbe9d8c74db221f824
Parents: 71286d7
Author: sboikov <sb...@gridgain.com>
Authored: Wed Mar 2 10:08:13 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Mar 2 11:02:01 2016 +0300

----------------------------------------------------------------------
 ...niteCrossCachesDistributedJoinQueryTest.java | 142 ++++++++++++-------
 1 file changed, 93 insertions(+), 49 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/0bf14f09/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCrossCachesDistributedJoinQueryTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCrossCachesDistributedJoinQueryTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCrossCachesDistributedJoinQueryTest.java
index 648a4c7..dbe339c 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCrossCachesDistributedJoinQueryTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCrossCachesDistributedJoinQueryTest.java
@@ -28,23 +28,18 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.ThreadLocalRandom;
-import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
-import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.cache.QueryEntity;
 import org.apache.ignite.cache.query.SqlFieldsQuery;
 import org.apache.ignite.cache.query.annotations.QuerySqlField;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.util.lang.GridAbsPredicateX;
-import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.T4;
 import org.apache.ignite.internal.util.typedef.internal.SB;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
-import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
@@ -168,12 +163,14 @@ public class IgniteCrossCachesDistributedJoinQueryTest extends GridCommonAbstrac
      * @param personCacheType Person cache type.
      * @param accountCacheType Account cache type.
      * @param orgCacheType Organization cache type.
+     * @throws Exception If failed.
      */
     private void checkDistributedCrossCacheJoin(final TestCacheType personCacheType,
         final TestCacheType accountCacheType,
         final TestCacheType orgCacheType) throws Exception {
-        info("Checking distributed cross cache join [personCache=" + personCacheType + ", accCache=" + accountCacheType
-            + ", orgCache=" + orgCacheType + "]");
+        info("Checking distributed cross cache join [personCache=" + personCacheType +
+            ", accCache=" + accountCacheType +
+            ", orgCache=" + orgCacheType + "]");
 
         Collection<TestCacheType> cacheTypes = new ArrayList<TestCacheType>() {{
             add(personCacheType);
@@ -182,7 +179,13 @@ public class IgniteCrossCachesDistributedJoinQueryTest extends GridCommonAbstrac
         }};
 
         for (TestCacheType type : cacheTypes) {
-            CacheConfiguration cc = cacheConfiguration(type.cacheName, type.cacheMode, type.backups);
+            CacheConfiguration cc = cacheConfiguration(type.cacheName,
+                type.cacheMode,
+                type.backups,
+                type == accountCacheType,
+                type == personCacheType,
+                type == orgCacheType
+            );
 
             ignite(0).getOrCreateCache(cc);
 
@@ -209,32 +212,36 @@ public class IgniteCrossCachesDistributedJoinQueryTest extends GridCommonAbstrac
             for (Organization org : data.orgs)
                 orgCache.put(org.id, org);
 
+            List<String> cacheNames = new ArrayList<>();
+
+            cacheNames.add(personCacheType.cacheName);
+            cacheNames.add(orgCacheType.cacheName);
+            cacheNames.add(accountCacheType.cacheName);
+
             for (int i = 0; i < NODES; i++) {
                 log.info("Test node: " + i);
 
-                checkPersonAccountsJoin(orgCache, data.accountsCntForPerson, accCache.getName(), personCache.getName());
+                for (String cacheName : cacheNames) {
+                    IgniteCache cache = ignite(i).cache(cacheName);
+
+                    log.info("Use cache: " + cache.getName());
 
-                checkOrganizationPersonsJoin(accCache, data.personsCntAtOrg,
-                    orgCacheType.cacheName, personCacheType.cacheName);
+                    checkPersonAccountsJoin(cache,
+                        data.accountsCntForPerson,
+                        accCache.getName(),
+                        personCache.getName());
+
+                    checkOrganizationPersonsJoin(cache,
+                        data.personsCntAtOrg,
+                        orgCacheType.cacheName,
+                        personCacheType.cacheName);
+                }
             }
         }
         finally {
             ignite(0).destroyCache(accountCacheType.cacheName);
             ignite(0).destroyCache(personCacheType.cacheName);
             ignite(0).destroyCache(orgCacheType.cacheName);
-
-            assertTrue(GridTestUtils.waitForCondition(new GridAbsPredicateX() {
-                @Override public boolean applyx() throws IgniteCheckedException {
-                    for (int i = 0; i < NODES; i++) {
-                        if (grid(i).cache(accountCacheType.cacheName) != null
-                            || grid(i).cache(personCacheType.cacheName) != null
-                            || grid(i).cache(orgCacheType.cacheName) != null)
-                            return false;
-                    }
-
-                    return true;
-                }
-            }, 10_000));
         }
     }
 
@@ -253,7 +260,7 @@ public class IgniteCrossCachesDistributedJoinQueryTest extends GridCommonAbstrac
         Collection<Person> persons = new ArrayList<>();
         Collection<Account> accounts = new ArrayList<>();
 
-        final int ORG_CNT = 1;
+        final int ORG_CNT = 10;
 
         for (int id = 0; id < ORG_CNT; id++)
             orgs.add(new Organization(id, "org-" + id));
@@ -262,7 +269,7 @@ public class IgniteCrossCachesDistributedJoinQueryTest extends GridCommonAbstrac
         Set<Integer> accountIds = new HashSet<>();
 
         for (int orgId = 0; orgId < ORG_CNT; orgId++) {
-            int personsCnt = ThreadLocalRandom.current().nextInt(100);
+            int personsCnt = ThreadLocalRandom.current().nextInt(20);
 
             for (int p = 0; p < personsCnt; p++) {
                 int personId = ThreadLocalRandom.current().nextInt(10, 10_000);
@@ -292,18 +299,26 @@ public class IgniteCrossCachesDistributedJoinQueryTest extends GridCommonAbstrac
         }
 
         return new Data(orgs, persons, accounts, personsCntAtOrg, accountsCntForPerson);
-
     }
 
     /**
+     * @param cacheName Cache name.
      * @param cacheMode Cache mode.
      * @param backups Number of backups.
+     * @param accountIdx Account index flag.
+     * @param personIdx Person index flag.
+     * @param orgIdx Organization index flag.
      * @return Cache configuration.
      */
-    private CacheConfiguration cacheConfiguration(String cacheName, CacheMode cacheMode, int backups) {
+    private CacheConfiguration cacheConfiguration(String cacheName,
+        CacheMode cacheMode,
+        int backups,
+        boolean accountIdx,
+        boolean personIdx,
+        boolean orgIdx) {
         CacheConfiguration ccfg = new CacheConfiguration();
 
-        ccfg.setName(String.valueOf(cacheName));
+        ccfg.setName(cacheName);
 
         ccfg.setCacheMode(cacheMode);
 
@@ -312,25 +327,39 @@ public class IgniteCrossCachesDistributedJoinQueryTest extends GridCommonAbstrac
 
         ccfg.setWriteSynchronizationMode(FULL_SYNC);
 
-        QueryEntity account = new QueryEntity();
-        account.setKeyType(Integer.class.getName());
-        account.setValueType(Account.class.getName());
-        account.addQueryField("personId", Integer.class.getName(), null);
+        List<QueryEntity> entities = new ArrayList<>();
+
+        if (accountIdx) {
+            QueryEntity account = new QueryEntity();
+            account.setKeyType(Integer.class.getName());
+            account.setValueType(Account.class.getName());
+            account.addQueryField("personId", Integer.class.getName(), null);
+
+            entities.add(account);
+        }
+
+        if (personIdx) {
+            QueryEntity person = new QueryEntity();
+            person.setKeyType(Integer.class.getName());
+            person.setValueType(Person.class.getName());
+            person.addQueryField("orgId", Integer.class.getName(), null);
+            person.addQueryField("id", Integer.class.getName(), null);
+            person.addQueryField("name", String.class.getName(), null);
 
-        QueryEntity person = new QueryEntity();
-        person.setKeyType(Integer.class.getName());
-        person.setValueType(Person.class.getName());
-        person.addQueryField("orgId", Integer.class.getName(), null);
-        person.addQueryField("id", Integer.class.getName(), null);
-        person.addQueryField("name", String.class.getName(), null);
+            entities.add(person);
+        }
 
-        QueryEntity org = new QueryEntity();
-        org.setKeyType(Integer.class.getName());
-        org.setValueType(Organization.class.getName());
-        org.addQueryField("id", Integer.class.getName(), null);
-        org.addQueryField("name", String.class.getName(), null);
+        if (orgIdx) {
+            QueryEntity org = new QueryEntity();
+            org.setKeyType(Integer.class.getName());
+            org.setValueType(Organization.class.getName());
+            org.addQueryField("id", Integer.class.getName(), null);
+            org.addQueryField("name", String.class.getName(), null);
 
-        ccfg.setQueryEntities(F.asList(account, person, org));
+            entities.add(org);
+        }
+
+        ccfg.setQueryEntities(entities);
 
         return ccfg;
     }
@@ -338,8 +367,12 @@ public class IgniteCrossCachesDistributedJoinQueryTest extends GridCommonAbstrac
     /**
      * @param cache Cache.
      * @param cnts Organizations per person counts.
+     * @param orgCacheName Organization cache name.
+     * @param personCacheName Person cache name.
      */
-    private void checkOrganizationPersonsJoin(IgniteCache cache, Map<Integer, Integer> cnts, String orgCacheName,
+    private void checkOrganizationPersonsJoin(IgniteCache cache,
+        Map<Integer, Integer> cnts,
+        String orgCacheName,
         String personCacheName) {
         SqlFieldsQuery qry = new SqlFieldsQuery("select o.name, p.name " +
             "from \"" + orgCacheName + "\".Organization o, \"" + personCacheName + "\".Person p " +
@@ -373,8 +406,12 @@ public class IgniteCrossCachesDistributedJoinQueryTest extends GridCommonAbstrac
     /**
      * @param cache Cache.
      * @param cnts Accounts per person counts.
+     * @param accCacheName Account cache name.
+     * @param personCacheName Person cache name.
      */
-    private void checkPersonAccountsJoin(IgniteCache cache, Map<Integer, Integer> cnts, String accCacheName,
+    private void checkPersonAccountsJoin(IgniteCache cache,
+        Map<Integer, Integer> cnts,
+        String accCacheName,
         String personCacheName) {
         SqlFieldsQuery qry1 = new SqlFieldsQuery("select p.name " +
             "from \"" + personCacheName + "\".Person p, \"" + accCacheName + "\".Account a " +
@@ -388,8 +425,6 @@ public class IgniteCrossCachesDistributedJoinQueryTest extends GridCommonAbstrac
 
         qry2.setDistributedJoins(true);
 
-        Ignite ignite = (Ignite)cache.unwrap(Ignite.class);
-
         long total = 0;
 
         for (Map.Entry<Integer, Integer> e : cnts.entrySet()) {
@@ -527,6 +562,10 @@ public class IgniteCrossCachesDistributedJoinQueryTest extends GridCommonAbstrac
         @QuerySqlField
         private int personId;
 
+        /**
+         * @param id ID.
+         * @param personId Person ID.
+         */
         Account(int id, int personId) {
             this.id = id;
             this.personId = personId;
@@ -548,6 +587,11 @@ public class IgniteCrossCachesDistributedJoinQueryTest extends GridCommonAbstrac
         @QuerySqlField
         String name;
 
+        /**
+         * @param id ID.
+         * @param orgId Organization ID.
+         * @param name Name.
+         */
         public Person(int id, int orgId, String name) {
             this.id = id;
             this.orgId = orgId;


[04/19] 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>

[14/19] ignite git commit: IGNITE-2718: Added Zookeeper IP finder dependencies. - Fixes #522.

Posted by sb...@apache.org.
IGNITE-2718: Added Zookeeper IP finder dependencies. - Fixes #522.

Signed-off-by: shtykh_roman <rs...@yahoo.com>


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

Branch: refs/heads/ignite-1232
Commit: 3b44cc4fec50b17ecd9ab4d155edf461f02361fc
Parents: 452673d
Author: shtykh_roman <rs...@yahoo.com>
Authored: Wed Mar 2 12:02:27 2016 +0900
Committer: shtykh_roman <rs...@yahoo.com>
Committed: Wed Mar 2 12:02:27 2016 +0900

----------------------------------------------------------------------
 modules/zookeeper/pom.xml                       | 50 +++++++++++++++++++-
 .../tcp/ipfinder/zk/ZookeeperIpFinderTest.java  | 24 ++++++----
 2 files changed, 63 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/3b44cc4f/modules/zookeeper/pom.xml
----------------------------------------------------------------------
diff --git a/modules/zookeeper/pom.xml b/modules/zookeeper/pom.xml
index e296b55..b749c30 100644
--- a/modules/zookeeper/pom.xml
+++ b/modules/zookeeper/pom.xml
@@ -20,7 +20,8 @@
 <!--
     POM file.
 -->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
     <modelVersion>4.0.0</modelVersion>
 
     <parent>
@@ -55,6 +56,53 @@
 
         <dependency>
             <groupId>org.apache.curator</groupId>
+            <artifactId>curator-client</artifactId>
+            <version>${curator.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>com.google.guava</groupId>
+            <artifactId>guava</artifactId>
+            <version>${guava16.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.codehaus.jackson</groupId>
+            <artifactId>jackson-core-asl</artifactId>
+            <version>${jackson.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.codehaus.jackson</groupId>
+            <artifactId>jackson-mapper-asl</artifactId>
+            <version>${jackson.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.zookeeper</groupId>
+            <artifactId>zookeeper</artifactId>
+            <version>${zookeeper.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-api</artifactId>
+            <version>${slf4j.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>slf4j-log4j12</artifactId>
+            <version>${slf4j.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>log4j</groupId>
+            <artifactId>log4j</artifactId>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.curator</groupId>
             <artifactId>curator-test</artifactId>
             <version>${curator.version}</version>
             <scope>test</scope>

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b44cc4f/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/zk/ZookeeperIpFinderTest.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/zk/ZookeeperIpFinderTest.java b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/zk/ZookeeperIpFinderTest.java
index 1b91163..3f9b1ff 100644
--- a/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/zk/ZookeeperIpFinderTest.java
+++ b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/zk/ZookeeperIpFinderTest.java
@@ -27,6 +27,7 @@ import org.apache.curator.retry.ExponentialBackoffRetry;
 import org.apache.curator.retry.RetryNTimes;
 import org.apache.curator.test.InstanceSpec;
 import org.apache.curator.test.TestingCluster;
+import org.apache.curator.utils.CloseableUtils;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.events.Event;
@@ -65,6 +66,7 @@ public class ZookeeperIpFinderTest extends GridCommonAbstractTest {
 
     /**
      * Before test.
+     *
      * @throws Exception
      */
     @Override public void beforeTest() throws Exception {
@@ -73,6 +75,9 @@ public class ZookeeperIpFinderTest extends GridCommonAbstractTest {
         // remove stale system properties
         System.getProperties().remove(TcpDiscoveryZookeeperIpFinder.PROP_ZK_CONNECTION_STRING);
 
+        // disable JMX for tests
+        System.setProperty("zookeeper.jmx.log4j.disable", "true");
+
         // start the ZK cluster
         zkCluster = new TestingCluster(ZK_CLUSTER_SIZE);
         zkCluster.start();
@@ -84,18 +89,17 @@ public class ZookeeperIpFinderTest extends GridCommonAbstractTest {
 
     /**
      * After test.
+     *
      * @throws Exception
      */
     @Override public void afterTest() throws Exception {
         super.afterTest();
 
         if (zkCurator != null)
-            zkCurator.close();
+            CloseableUtils.closeQuietly(zkCurator);
 
-        if (zkCluster != null) {
-            zkCluster.stop();
-            zkCluster.close();
-        }
+        if (zkCluster != null)
+            CloseableUtils.closeQuietly(zkCluster);
 
         stopAllGrids();
     }
@@ -110,7 +114,7 @@ public class ZookeeperIpFinderTest extends GridCommonAbstractTest {
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration configuration = super.getConfiguration(gridName);
 
-        TcpDiscoverySpi tcpDisco = (TcpDiscoverySpi) configuration.getDiscoverySpi();
+        TcpDiscoverySpi tcpDisco = (TcpDiscoverySpi)configuration.getDiscoverySpi();
         TcpDiscoveryZookeeperIpFinder zkIpFinder = new TcpDiscoveryZookeeperIpFinder();
         zkIpFinder.setAllowDuplicateRegistrations(allowDuplicateRegistrations);
 
@@ -147,7 +151,7 @@ public class ZookeeperIpFinderTest extends GridCommonAbstractTest {
         startGrid(0);
 
         // set up an event listener to expect one NODE_JOINED event
-        CountDownLatch latch =  expectJoinEvents(grid(0), 1);
+        CountDownLatch latch = expectJoinEvents(grid(0), 1);
 
         // start the other node
         startGrid(1);
@@ -170,7 +174,7 @@ public class ZookeeperIpFinderTest extends GridCommonAbstractTest {
         startGrid(0);
 
         // set up an event listener to expect one NODE_JOINED event
-        CountDownLatch latch =  expectJoinEvents(grid(0), 2);
+        CountDownLatch latch = expectJoinEvents(grid(0), 2);
 
         // start the 2nd node
         startGrid(1);
@@ -201,7 +205,7 @@ public class ZookeeperIpFinderTest extends GridCommonAbstractTest {
         startGrid(0);
 
         // set up an event listener to expect one NODE_JOINED event
-        CountDownLatch latch =  expectJoinEvents(grid(0), 3);
+        CountDownLatch latch = expectJoinEvents(grid(0), 3);
 
         // start the 2nd node
         startGrid(1);
@@ -392,4 +396,4 @@ public class ZookeeperIpFinderTest extends GridCommonAbstractTest {
 
         return latch;
     }
-}
\ No newline at end of file
+}


[12/19] ignite git commit: IGNITE-2521: fix JCache TCK tests

Posted by sb...@apache.org.
IGNITE-2521: fix JCache TCK tests


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

Branch: refs/heads/ignite-1232
Commit: 3472a842d68f1a52a0eb21fa3108e7b0f66e438a
Parents: 74fd400
Author: ashutak <as...@gridgain.com>
Authored: Tue Mar 1 20:13:14 2016 +0300
Committer: ashutak <as...@gridgain.com>
Committed: Tue Mar 1 20:13:14 2016 +0300

----------------------------------------------------------------------
 .../ConfigVariationsTestSuiteBuilderTest.java     | 18 +++++++++---------
 1 file changed, 9 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/3472a842/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
index 75e3010..d237345 100644
--- 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
@@ -34,22 +34,22 @@ public class ConfigVariationsTestSuiteBuilderTest extends TestCase {
      * @throws Exception If failed.
      */
     public void testDefaults() throws Exception {
-        TestSuite dfltSuite = new ConfigVariationsTestSuiteBuilder("testSuite", OneTestCase.class).build();
+        TestSuite dfltSuite = new ConfigVariationsTestSuiteBuilder("testSuite", NoopTest.class).build();
 
         assertEquals(4, dfltSuite.countTestCases());
 
-        TestSuite dfltCacheSuite = new ConfigVariationsTestSuiteBuilder("testSuite", OneTestCase.class)
+        TestSuite dfltCacheSuite = new ConfigVariationsTestSuiteBuilder("testSuite", NoopTest.class)
             .withBasicCacheParams().build();
 
         assertEquals(4 * 12, dfltCacheSuite.countTestCases());
 
         // With clients.
-        dfltSuite = new ConfigVariationsTestSuiteBuilder("testSuite", OneTestCase.class)
+        dfltSuite = new ConfigVariationsTestSuiteBuilder("testSuite", NoopTest.class)
             .testedNodesCount(2).withClients().build();
 
         assertEquals(4 * 2, dfltSuite.countTestCases());
 
-        dfltCacheSuite = new ConfigVariationsTestSuiteBuilder("testSuite", OneTestCase.class)
+        dfltCacheSuite = new ConfigVariationsTestSuiteBuilder("testSuite", NoopTest.class)
             .withBasicCacheParams().testedNodesCount(3).withClients().build();
 
         assertEquals(4 * 12 * 3, dfltCacheSuite.countTestCases());
@@ -60,11 +60,11 @@ public class ConfigVariationsTestSuiteBuilderTest extends TestCase {
      */
     @SuppressWarnings("serial")
     public void testIgniteConfigFilter() throws Exception {
-        TestSuite dfltSuite = new ConfigVariationsTestSuiteBuilder("testSuite", OneTestCase.class).build();
+        TestSuite dfltSuite = new ConfigVariationsTestSuiteBuilder("testSuite", NoopTest.class).build();
 
         final AtomicInteger cnt = new AtomicInteger();
 
-        TestSuite filteredSuite = new ConfigVariationsTestSuiteBuilder("testSuite", OneTestCase.class)
+        TestSuite filteredSuite = new ConfigVariationsTestSuiteBuilder("testSuite", NoopTest.class)
             .withIgniteConfigFilters(new IgnitePredicate<IgniteConfiguration>() {
                 @Override public boolean apply(IgniteConfiguration configuration) {
                     return cnt.getAndIncrement() % 2 == 0;
@@ -80,13 +80,13 @@ public class ConfigVariationsTestSuiteBuilderTest extends TestCase {
      */
     @SuppressWarnings("serial")
     public void testCacheConfigFilter() throws Exception {
-        TestSuite dfltSuite = new ConfigVariationsTestSuiteBuilder("testSuite", OneTestCase.class)
+        TestSuite dfltSuite = new ConfigVariationsTestSuiteBuilder("testSuite", NoopTest.class)
             .withBasicCacheParams()
             .build();
 
         final AtomicInteger cnt = new AtomicInteger();
 
-        TestSuite filteredSuite = new ConfigVariationsTestSuiteBuilder("testSuite", OneTestCase.class)
+        TestSuite filteredSuite = new ConfigVariationsTestSuiteBuilder("testSuite", NoopTest.class)
             .withBasicCacheParams()
             .withCacheConfigFilters(new IgnitePredicate<CacheConfiguration>() {
                 @Override public boolean apply(CacheConfiguration configuration) {
@@ -101,7 +101,7 @@ public class ConfigVariationsTestSuiteBuilderTest extends TestCase {
     /**
      *
      */
-    private static class OneTestCase extends IgniteConfigVariationsAbstractTest {
+    private static class NoopTest extends IgniteConfigVariationsAbstractTest {
         /**
          * @throws Exception If failed.
          */


[13/19] ignite git commit: IGNITE-255: Jvm8 warns that MaxPermSize is ignored

Posted by sb...@apache.org.
IGNITE-255: Jvm8 warns that MaxPermSize is ignored


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

Branch: refs/heads/ignite-1232
Commit: 452673d3e2d3c8d0fe1e72d177437841f412df9a
Parents: 3472a84
Author: Saikat Maitra <sa...@gmail.com>
Authored: Wed Mar 2 01:55:20 2016 +0300
Committer: Denis Magda <dm...@gridgain.com>
Committed: Wed Mar 2 01:55:20 2016 +0300

----------------------------------------------------------------------
 bin/ignite.bat | 7 ++++++-
 bin/ignite.sh  | 6 +++++-
 2 files changed, 11 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/452673d3/bin/ignite.bat
----------------------------------------------------------------------
diff --git a/bin/ignite.bat b/bin/ignite.bat
index 788a85f..b999b86 100644
--- a/bin/ignite.bat
+++ b/bin/ignite.bat
@@ -159,7 +159,12 @@ if "%JMX_PORT%" == "" (
 ::
 :: ADD YOUR/CHANGE ADDITIONAL OPTIONS HERE
 ::
-if "%JVM_OPTS%" == "" set JVM_OPTS=-Xms1g -Xmx1g -server -XX:+AggressiveOpts -XX:MaxPermSize=256m
+"%JAVA_HOME%\bin\java.exe" -version 2>&1 | findstr "1\.[7]\." > nul
+if %ERRORLEVEL% equ 0 (
+    if "%JVM_OPTS%" == "" set JVM_OPTS=-Xms1g -Xmx1g -server -XX:+AggressiveOpts -XX:MaxPermSize=256m
+) else (
+    if "%JVM_OPTS%" == "" set JVM_OPTS=-Xms1g -Xmx1g -server -XX:+AggressiveOpts -XX:MaxMetaspaceSize=256m
+)
 
 ::
 :: Uncomment the following GC settings if you see spikes in your throughput due to Garbage Collection.

http://git-wip-us.apache.org/repos/asf/ignite/blob/452673d3/bin/ignite.sh
----------------------------------------------------------------------
diff --git a/bin/ignite.sh b/bin/ignite.sh
index 7fbca57..05d2d5f 100755
--- a/bin/ignite.sh
+++ b/bin/ignite.sh
@@ -90,7 +90,11 @@ fi
 # ADD YOUR/CHANGE ADDITIONAL OPTIONS HERE
 #
 if [ -z "$JVM_OPTS" ] ; then
-    JVM_OPTS="-Xms1g -Xmx1g -server -XX:+AggressiveOpts -XX:MaxPermSize=256m"
+    if [[ `"$JAVA" -version 2>&1 | egrep "1\.[7]\."` ]]; then
+        JVM_OPTS="-Xms1g -Xmx1g -server -XX:+AggressiveOpts -XX:MaxPermSize=256m"
+    else
+        JVM_OPTS="-Xms1g -Xmx1g -server -XX:+AggressiveOpts -XX:MaxMetaspaceSize=256m"
+    fi
 fi
 
 #


[15/19] ignite git commit: IGNITE-2705: .NET: Added test for tools version. This closes #529.

Posted by sb...@apache.org.
IGNITE-2705: .NET: Added test for tools version. This closes #529.


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

Branch: refs/heads/ignite-1232
Commit: a4391d7935199a658d6dd4ac3fc8fa601e36c3f5
Parents: 3b44cc4
Author: Pavel Tupitsyn <pt...@gridgain.com>
Authored: Wed Mar 2 09:11:06 2016 +0300
Committer: thatcoach <pp...@list.ru>
Committed: Wed Mar 2 09:11:06 2016 +0300

----------------------------------------------------------------------
 .../Apache.Ignite.Core.Tests.csproj             |  1 +
 .../ProjectFilesTest.cs                         | 94 ++++++++++++++++++++
 2 files changed, 95 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/a4391d79/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
index 6f0e630..4ba05e1 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
@@ -130,6 +130,7 @@
     <Compile Include="BinaryConfigurationTest.cs" />
     <Compile Include="Binary\BinaryStructureTest.cs" />
     <Compile Include="ProcessExtensions.cs" />
+    <Compile Include="ProjectFilesTest.cs" />
     <Compile Include="ReconnectTest.cs" />
     <Compile Include="SerializationTest.cs" />
     <Compile Include="IgniteStartStopTest.cs" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/a4391d79/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ProjectFilesTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ProjectFilesTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ProjectFilesTest.cs
new file mode 100644
index 0000000..081dd89
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ProjectFilesTest.cs
@@ -0,0 +1,94 @@
+/*
+ * 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.
+ */
+
+namespace Apache.Ignite.Core.Tests
+{
+    using System;
+    using System.IO;
+    using System.Linq;
+    using System.Reflection;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Verifies source files.
+    /// </summary>
+    public class ProjectFilesTest
+    {
+        /// <summary>
+        /// Tests that tools version is compatible with VS2010.
+        /// </summary>
+        [Test]
+        public void TestCsprojToolsVersion()
+        {
+            var projFiles = GetDotNetSourceDir().GetFiles("*.csproj", SearchOption.AllDirectories);
+            Assert.GreaterOrEqual(projFiles.Length, 7);
+
+            var invalidFiles =
+                projFiles.Where(x =>
+                {
+                    var text = File.ReadAllText(x.FullName);
+
+                    return !text.Contains("ToolsVersion=\"4.0\"") ||
+                           text.IndexOf("AnyCPU", StringComparison.OrdinalIgnoreCase) >= 0;
+                }).ToArray();
+
+            Assert.AreEqual(0, invalidFiles.Length,
+                "Invalid csproj files: " + string.Join(", ", invalidFiles.Select(x => x.FullName)));
+        }
+
+        /// <summary>
+        /// Tests that tools version is compatible with VS2010.
+        /// </summary>
+        [Test]
+        public void TestSlnToolsVersion()
+        {
+            var slnFiles = GetDotNetSourceDir().GetFiles("*.sln", SearchOption.AllDirectories);
+            Assert.GreaterOrEqual(slnFiles.Length, 2);
+
+            var invalidFiles =
+                slnFiles.Where(x =>
+                {
+                    var text = File.ReadAllText(x.FullName);
+
+                    return !text.Contains("# Visual Studio 2010") ||
+                           !text.Contains("Microsoft Visual Studio Solution File, Format Version 11.00");
+                }).ToArray();
+
+            Assert.AreEqual(0, invalidFiles.Length,
+                "Invalid sln files: " + string.Join(", ", invalidFiles.Select(x => x.FullName)));
+        }
+
+        /// <summary>
+        /// Gets the dot net source dir.
+        /// </summary>
+        private static DirectoryInfo GetDotNetSourceDir()
+        {
+            // ReSharper disable once AssignNullToNotNullAttribute
+            var dir = new DirectoryInfo(Path.GetDirectoryName(Assembly.GetExecutingAssembly().Location));
+
+            while (dir != null)
+            {
+                if (dir.GetFiles().Any(x => x.Name == "Apache.Ignite.sln"))
+                    return dir;
+
+                dir = dir.Parent;
+            }
+
+            throw new InvalidOperationException("Could not resolve Ignite.NET source directory.");
+        }
+    }
+}


[02/19] 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/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();
+    }
+}


[03/19] 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();
 


[19/19] ignite git commit: ignite-1232 Merge from master.

Posted by sb...@apache.org.
ignite-1232 Merge from master.


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

Branch: refs/heads/ignite-1232
Commit: 4fe6db71fd537b599c7b9c1cf168887cd8bdef90
Parents: 3a51439
Author: sboikov <sb...@gridgain.com>
Authored: Wed Mar 2 11:08:11 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Mar 2 11:08:11 2016 +0300

----------------------------------------------------------------------
 .../ignite/internal/processors/query/h2/IgniteH2Indexing.java  | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/4fe6db71/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
index 382ba2d..5cbd02b 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
@@ -1817,11 +1817,11 @@ public class IgniteH2Indexing implements GridQueryIndexing {
                 U.error(log, "Failed to drop schema on cache stop (will ignore): " + U.maskName(ccfg.getName()), e);
             }
 
-            for (Iterator<Map.Entry<T3<String, String, Boolean>, TwoStepCachedQuery>> it = twoStepCache.entrySet().iterator();
+            for (Iterator<Map.Entry<TwoStepCachedQueryKey, TwoStepCachedQuery>> it = twoStepCache.entrySet().iterator();
                  it.hasNext();) {
-                Map.Entry<T3<String, String, Boolean>, TwoStepCachedQuery> e = it.next();
+                Map.Entry<TwoStepCachedQueryKey, TwoStepCachedQuery> e = it.next();
 
-                if (F.eq(e.getKey().get1(), ccfg.getName()))
+                if (F.eq(e.getKey().space, ccfg.getName()))
                     it.remove();
             }
         }


[09/19] ignite git commit: IGNITE-2650

Posted by sb...@apache.org.
IGNITE-2650


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

Branch: refs/heads/ignite-1232
Commit: 891726916390c2de5c4c28cc308b0fac0a3c9974
Parents: c4d6f3c
Author: ruskim <ru...@gmail.com>
Authored: Tue Mar 1 12:13:14 2016 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Tue Mar 1 12:13:14 2016 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheProcessor.java    |   8 +
 .../processors/cache/GridCacheUtils.java        |  15 ++
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |  15 ++
 .../ignite/spi/IgniteSpiConsistencyChecked.java |   8 +
 .../spi/swapspace/file/FileSwapSpaceSpi.java    |   2 +
 .../spi/swapspace/noop/NoopSwapSpaceSpi.java    |   2 +
 ...ridCacheSwapSpaceSpiConsistencySelfTest.java | 146 +++++++++++++++++++
 .../GridSwapSpaceSpiConsistencySelfTest.java    | 131 +++++++++++++++++
 .../testsuites/IgniteCacheTestSuite5.java       |   2 +
 .../IgniteSpiSwapSpaceSelfTestSuite.java        |   2 +
 10 files changed, 331 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/89172691/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index a8f205b..c0fc32a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -2271,6 +2271,14 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             req.startCacheConfiguration(ccfg);
         }
 
+        // Fail cache with swap enabled creation on grid without swap space SPI.
+        if (ccfg.isSwapEnabled())
+            for (ClusterNode n : ctx.discovery().allNodes())
+                if (!GridCacheUtils.clientNode(n) && !GridCacheUtils.isSwapEnabled(n))
+                    return new GridFinishedFuture<>(new IgniteCheckedException("Failed to start cache " +
+                        cacheName + " with swap enabled: Remote Node with ID " + n.id().toString().toUpperCase() +
+                        " has not swap SPI configured"));
+
         if (nearCfg != null)
             req.nearCacheConfiguration(nearCfg);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/89172691/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
index cd21794..dce5357 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
@@ -90,6 +90,7 @@ import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.lang.IgniteReducer;
 import org.apache.ignite.lifecycle.LifecycleAware;
 import org.apache.ignite.plugin.CachePluginConfiguration;
+import org.apache.ignite.spi.swapspace.noop.NoopSwapSpaceSpi;
 import org.apache.ignite.transactions.Transaction;
 import org.apache.ignite.transactions.TransactionConcurrency;
 import org.apache.ignite.transactions.TransactionIsolation;
@@ -283,6 +284,10 @@ public class GridCacheUtils {
         }
     };
 
+    /** NoopSwapSpaceSpi used attribute. */
+    private static final String NOOP_SWAP_SPACE_SPI_ATTR_NAME = U.spiAttribute(new NoopSwapSpaceSpi(),
+        IgniteNodeAttributes.ATTR_SPI_CLASS);
+
     /**
      * Ensure singleton.
      */
@@ -1836,4 +1841,14 @@ public class GridCacheUtils {
 
         return res;
     }
+
+    /**
+     * Checks if swap is enabled on node.
+     *
+     * @param node Node
+     * @return {@code true} if swap is enabled, {@code false} otherwise.
+     */
+    public static boolean isSwapEnabled(ClusterNode node) {
+        return !node.attributes().containsKey(NOOP_SWAP_SPACE_SPI_ATTR_NAME);
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/89172691/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
index 1cb202c..f31aae7 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
@@ -39,10 +39,12 @@ import org.apache.ignite.internal.IgniteKernal;
 import org.apache.ignite.internal.IgniteNodeAttributes;
 import org.apache.ignite.internal.managers.communication.GridMessageListener;
 import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener;
+import org.apache.ignite.internal.processors.cache.GridCacheUtils;
 import org.apache.ignite.internal.processors.timeout.GridSpiTimeoutObject;
 import org.apache.ignite.internal.util.IgniteExceptionRegistry;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.X;
+import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.SB;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteFuture;
@@ -448,6 +450,15 @@ public abstract class IgniteSpiAdapter implements IgniteSpi, IgniteSpiManagement
     }
 
     /**
+     * @return {@code true} if client cluster nodes should be checked.
+     */
+    private boolean checkClient() {
+        IgniteSpiConsistencyChecked ann = U.getAnnotation(getClass(), IgniteSpiConsistencyChecked.class);
+
+        return ann != null && ann.checkClient();
+    }
+
+    /**
      * Method which is called in the end of checkConfigurationConsistency() method. May be overriden in SPIs.
      *
      * @param spiCtx SPI context.
@@ -480,10 +491,14 @@ public abstract class IgniteSpiAdapter implements IgniteSpi, IgniteSpiManagement
          */
         boolean optional = checkOptional();
         boolean enabled = checkEnabled();
+        boolean checkClient = checkClient();
 
         if (!enabled)
             return;
 
+        if (!checkClient && (CU.clientNode(getLocalNode()) || CU.clientNode(node)))
+            return;
+
         String clsAttr = createSpiAttributeName(IgniteNodeAttributes.ATTR_SPI_CLASS);
 
         String name = getName();

http://git-wip-us.apache.org/repos/asf/ignite/blob/89172691/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiConsistencyChecked.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiConsistencyChecked.java b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiConsistencyChecked.java
index 157d9ce..81ab21e 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiConsistencyChecked.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiConsistencyChecked.java
@@ -40,4 +40,12 @@ public @interface IgniteSpiConsistencyChecked {
      */
     @SuppressWarnings("JavaDoc")
     public boolean optional();
+
+    /**
+     * If false, skip consistency checks for client cluster nodes. Could be useful
+     * for SwapSpaceSpi for example, since client nodes has no data at all, so they
+     * don't need to be consistent with server nodes.
+     */
+    @SuppressWarnings("JavaDoc")
+    public boolean checkClient() default true;
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/89172691/modules/core/src/main/java/org/apache/ignite/spi/swapspace/file/FileSwapSpaceSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/swapspace/file/FileSwapSpaceSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/swapspace/file/FileSwapSpaceSpi.java
index df35ed3..8809f08 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/swapspace/file/FileSwapSpaceSpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/swapspace/file/FileSwapSpaceSpi.java
@@ -61,6 +61,7 @@ import org.apache.ignite.resources.LoggerResource;
 import org.apache.ignite.spi.IgniteSpiAdapter;
 import org.apache.ignite.spi.IgniteSpiCloseableIterator;
 import org.apache.ignite.spi.IgniteSpiConfiguration;
+import org.apache.ignite.spi.IgniteSpiConsistencyChecked;
 import org.apache.ignite.spi.IgniteSpiException;
 import org.apache.ignite.spi.IgniteSpiMultipleInstancesSupport;
 import org.apache.ignite.spi.IgniteSpiThread;
@@ -137,6 +138,7 @@ import static org.apache.ignite.events.EventType.EVT_SWAP_SPACE_DATA_STORED;
  * @see org.apache.ignite.spi.swapspace.SwapSpaceSpi
  */
 @IgniteSpiMultipleInstancesSupport(true)
+@IgniteSpiConsistencyChecked(optional = false, checkClient = false)
 @SuppressWarnings({"PackageVisibleInnerClass", "PackageVisibleField"})
 public class FileSwapSpaceSpi extends IgniteSpiAdapter implements SwapSpaceSpi, FileSwapSpaceSpiMBean {
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/89172691/modules/core/src/main/java/org/apache/ignite/spi/swapspace/noop/NoopSwapSpaceSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/swapspace/noop/NoopSwapSpaceSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/swapspace/noop/NoopSwapSpaceSpi.java
index 13622d9..df73eb0 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/swapspace/noop/NoopSwapSpaceSpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/swapspace/noop/NoopSwapSpaceSpi.java
@@ -30,6 +30,7 @@ import org.apache.ignite.lang.IgniteInClosure;
 import org.apache.ignite.resources.LoggerResource;
 import org.apache.ignite.spi.IgniteSpiAdapter;
 import org.apache.ignite.spi.IgniteSpiCloseableIterator;
+import org.apache.ignite.spi.IgniteSpiConsistencyChecked;
 import org.apache.ignite.spi.IgniteSpiException;
 import org.apache.ignite.spi.IgniteSpiMultipleInstancesSupport;
 import org.apache.ignite.spi.IgniteSpiNoop;
@@ -44,6 +45,7 @@ import org.jetbrains.annotations.Nullable;
  */
 @IgniteSpiNoop
 @IgniteSpiMultipleInstancesSupport(true)
+@IgniteSpiConsistencyChecked(optional = false, checkClient = false)
 public class NoopSwapSpaceSpi extends IgniteSpiAdapter implements SwapSpaceSpi {
     /** Logger. */
     @LoggerResource

http://git-wip-us.apache.org/repos/asf/ignite/blob/89172691/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheSwapSpaceSpiConsistencySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheSwapSpaceSpiConsistencySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheSwapSpaceSpiConsistencySelfTest.java
new file mode 100644
index 0000000..74ac2fc
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheSwapSpaceSpiConsistencySelfTest.java
@@ -0,0 +1,146 @@
+/*
+ * 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 java.util.concurrent.Callable;
+import javax.cache.CacheException;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.spi.swapspace.file.FileSwapSpaceSpi;
+import org.apache.ignite.spi.swapspace.noop.NoopSwapSpaceSpi;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.apache.ignite.testframework.junits.common.GridCommonTest;
+
+/**
+ * Check creation of cache with swap space enabled on grids with and without swap space spi
+ */
+@SuppressWarnings({"ProhibitedExceptionDeclared"})
+@GridCommonTest(group = "Kernal")
+public class GridCacheSwapSpaceSpiConsistencySelfTest extends GridCommonAbstractTest {
+    /** */
+    protected static final String GRID_WITHOUT_SWAP_SPACE = "grid-without-swap-space";
+
+    /** */
+    protected static final String GRID_WITH_SWAP_SPACE = "grid-with-swap-space";
+
+    /** */
+    protected static final String GRID_CLIENT = "grid-client";
+
+    /** */
+    protected static final String CACHE_NAME = "TestCache";
+
+    /**
+     *
+     */
+    public GridCacheSwapSpaceSpiConsistencySelfTest() {
+        super(false);
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings({"unchecked"})
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        if (gridName.startsWith(GRID_WITHOUT_SWAP_SPACE))
+            cfg.setSwapSpaceSpi(new NoopSwapSpaceSpi());
+
+        if (gridName.startsWith(GRID_WITH_SWAP_SPACE))
+            cfg.setSwapSpaceSpi(new FileSwapSpaceSpi());
+
+        if (GRID_CLIENT.equals(gridName))
+            cfg.setClientMode(true);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     *
+     */
+    public void testInconsistentCacheCreationFromClient() throws Exception {
+        testInconsistentCacheCreation(true);
+    }
+
+    /**
+     *
+     */
+    public void testInconsistentCacheCreationFromServer() throws Exception {
+        testInconsistentCacheCreation(false);
+    }
+
+    /**
+     * It should be impossible to create cache with swap enabled on grid without swap.
+     */
+    public void testInconsistentCacheCreation(boolean fromClient) throws Exception {
+        startGrid(GRID_WITHOUT_SWAP_SPACE);
+
+        final Ignite ignite = startGrid(fromClient ? GRID_CLIENT : GRID_WITHOUT_SWAP_SPACE + "2");
+
+        final CacheConfiguration<Integer, String> cfg = new CacheConfiguration<>();
+
+        cfg.setSwapEnabled(true);
+        cfg.setName(CACHE_NAME);
+
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                return ignite.createCache(cfg);
+            }
+        }, CacheException.class, "Failed to start cache " + CACHE_NAME + " with swap enabled:");
+    }
+
+    /**
+     *
+     */
+    public void testConsistentCacheCreationFromClient() throws Exception {
+        testConsistentCacheCreation(true);
+    }
+
+    /**
+     *
+     */
+    public void testConsistentCacheCreationFromServer() throws Exception {
+        testConsistentCacheCreation(false);
+    }
+
+    /**
+     * It should ok to create cache with swap enabled on grid with swap.
+     */
+    public void testConsistentCacheCreation(boolean fromClient) throws Exception {
+        startGrid(GRID_WITH_SWAP_SPACE);
+
+        final Ignite ignite = startGrid(fromClient ? GRID_CLIENT : GRID_WITH_SWAP_SPACE + "2");
+
+        final CacheConfiguration<Integer, String> cfg = new CacheConfiguration<>();
+
+        cfg.setSwapEnabled(true);
+        cfg.setName(CACHE_NAME);
+
+        IgniteCache<Integer, String> cache = ignite.createCache(cfg);
+
+        cache.put(1, "one");
+
+        assert cache.get(1).equals("one");
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/89172691/modules/core/src/test/java/org/apache/ignite/spi/swapspace/GridSwapSpaceSpiConsistencySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/swapspace/GridSwapSpaceSpiConsistencySelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/swapspace/GridSwapSpaceSpiConsistencySelfTest.java
new file mode 100644
index 0000000..8efafdf
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/spi/swapspace/GridSwapSpaceSpiConsistencySelfTest.java
@@ -0,0 +1,131 @@
+/*
+ * 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.spi.swapspace;
+
+import java.util.concurrent.Callable;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.spi.swapspace.file.FileSwapSpaceSpi;
+import org.apache.ignite.spi.swapspace.noop.NoopSwapSpaceSpi;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+/**
+ * Check that all server nodes in grid have configured the same swap space spi. Check that client nodes could have any
+ * swap space spi.
+ */
+@SuppressWarnings({"ProhibitedExceptionDeclared"})
+public class GridSwapSpaceSpiConsistencySelfTest extends GridCommonAbstractTest {
+    /** */
+    protected static final String GRID_WITHOUT_SWAP_SPACE = "grid-without-swap-space";
+
+    /** */
+    protected static final String GRID_WITH_SWAP_SPACE = "grid-with-swap-space";
+
+    /** */
+    protected static final String GRID_CLIENT_WITHOUT_SWAP_SPACE = "grid-client-without-swap-space";
+
+    /** */
+    protected static final String GRID_CLIENT_WITH_SWAP_SPACE = "grid-client-with-swap-space";
+
+    /** */
+    protected static final String CACHE_NAME = "TestCache";
+
+    /**
+     *
+     */
+    public GridSwapSpaceSpiConsistencySelfTest() {
+        super(false);
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings({"unchecked"})
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        if (GRID_WITHOUT_SWAP_SPACE.equals(gridName))
+            cfg.setSwapSpaceSpi(new NoopSwapSpaceSpi());
+
+        if (GRID_WITH_SWAP_SPACE.equals(gridName))
+            cfg.setSwapSpaceSpi(new FileSwapSpaceSpi());
+
+        if (GRID_CLIENT_WITHOUT_SWAP_SPACE.equals(gridName)) {
+            cfg.setClientMode(true);
+            cfg.setSwapSpaceSpi(new NoopSwapSpaceSpi());
+        }
+
+        if (GRID_CLIENT_WITH_SWAP_SPACE.equals(gridName)) {
+            cfg.setClientMode(true);
+            cfg.setSwapSpaceSpi(new FileSwapSpaceSpi());
+        }
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     * Node with swap enabled should not start after node without swap
+     */
+    public void testServerNodeIncompatibleSwapSpaceSpi1() throws Exception {
+        startGrid(GRID_WITHOUT_SWAP_SPACE);
+
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                return startGrid(GRID_WITH_SWAP_SPACE);
+            }
+        }, IgniteCheckedException.class, "Failed to initialize SPI context");
+    }
+
+    /**
+     * Node without swap should not start after node with swap enabled
+     */
+    public void testServerNodeIncompatibleSwapSpaceSpi2() throws Exception {
+        startGrid(GRID_WITH_SWAP_SPACE);
+
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                return startGrid(GRID_WITHOUT_SWAP_SPACE);
+            }
+        }, IgniteCheckedException.class, "Failed to initialize SPI context");
+    }
+
+    /**
+     * Client nodes should join to grid with any swap policy
+     */
+    public void testClientNodeAnySwapSpaceSpi() throws Exception {
+        startGrid(GRID_WITHOUT_SWAP_SPACE);
+
+        Ignite client1 = startGrid(GRID_CLIENT_WITH_SWAP_SPACE);
+
+        Ignite client2 = startGrid(GRID_CLIENT_WITHOUT_SWAP_SPACE);
+
+        IgniteCache<Integer, String> cache1 = client1.createCache("TestCache");
+
+        cache1.put(1, "one");
+
+        IgniteCache<Integer, String> cache2 = client2.getOrCreateCache("TestCache");
+
+        assert cache2.get(1).equals("one");
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/89172691/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite5.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite5.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite5.java
index 3eb0b13..305f5cd 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite5.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite5.java
@@ -20,6 +20,7 @@ package org.apache.ignite.testsuites;
 import junit.framework.TestSuite;
 import org.apache.ignite.internal.processors.cache.CacheNearReaderUpdateTest;
 import org.apache.ignite.internal.processors.cache.CacheSerializableTransactionsTest;
+import org.apache.ignite.internal.processors.cache.GridCacheSwapSpaceSpiConsistencySelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCachePutStackOverflowSelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheStoreCollectionTest;
 import org.apache.ignite.internal.processors.cache.store.IgniteCacheWriteBehindNoUpdateSelfTest;
@@ -40,6 +41,7 @@ public class IgniteCacheTestSuite5 extends TestSuite {
         suite.addTestSuite(IgniteCacheStoreCollectionTest.class);
         suite.addTestSuite(IgniteCacheWriteBehindNoUpdateSelfTest.class);
         suite.addTestSuite(IgniteCachePutStackOverflowSelfTest.class);
+        suite.addTestSuite(GridCacheSwapSpaceSpiConsistencySelfTest.class);
 
         return suite;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/89172691/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiSwapSpaceSelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiSwapSpaceSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiSwapSpaceSelfTestSuite.java
index 6d44934..3bf1b1b 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiSwapSpaceSelfTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiSwapSpaceSelfTestSuite.java
@@ -21,6 +21,7 @@ import junit.framework.TestSuite;
 import org.apache.ignite.spi.swapspace.file.GridFileSwapCompactionSelfTest;
 import org.apache.ignite.spi.swapspace.file.GridFileSwapSpaceSpiSelfTest;
 import org.apache.ignite.spi.swapspace.noop.GridNoopSwapSpaceSpiSelfTest;
+import org.apache.ignite.spi.swapspace.GridSwapSpaceSpiConsistencySelfTest;
 
 /**
  *
@@ -36,6 +37,7 @@ public class IgniteSpiSwapSpaceSelfTestSuite {
         suite.addTest(new TestSuite(GridFileSwapCompactionSelfTest.class));
         suite.addTest(new TestSuite(GridFileSwapSpaceSpiSelfTest.class));
         suite.addTest(new TestSuite(GridNoopSwapSpaceSpiSelfTest.class));
+        suite.addTest(new TestSuite(GridSwapSpaceSpiConsistencySelfTest.class));
 
         return suite;
     }