You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by vo...@apache.org on 2016/03/09 10:27:31 UTC

[16/50] [abbrv] ignite git commit: IGNITE-2650

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-1786
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;
     }