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 2015/09/09 10:53:01 UTC

[03/43] ignite git commit: ignite-971 Fix swap/unswap, added tests (do not call unswap for obsolete entry).

ignite-971 Fix swap/unswap, added tests (do not call unswap for obsolete entry).


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

Branch: refs/heads/master
Commit: 18b5be3a7f33ac382c3ea9777327c3246e247e9c
Parents: 35bc5ec
Author: sboikov <sb...@gridgain.com>
Authored: Tue Sep 8 09:51:49 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Sep 8 09:51:49 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheEntryEx.java      |  12 +-
 .../cache/GridCacheEvictionManager.java         |   2 +
 .../processors/cache/GridCacheMapEntry.java     |  11 +-
 .../processors/cache/GridCacheSwapManager.java  |   2 +-
 .../distributed/dht/GridDhtCacheAdapter.java    |  14 +-
 .../distributed/dht/GridDhtLocalPartition.java  |  23 +-
 .../distributed/dht/GridDhtLockFuture.java      |   6 +-
 .../distributed/dht/GridDhtTxLocalAdapter.java  |  22 +-
 .../dht/preloader/GridDhtPreloader.java         |  20 +-
 .../distributed/near/GridNearTxRemote.java      |   6 +-
 .../transactions/IgniteTxLocalAdapter.java      |  20 +-
 .../cache/CacheAffinityCallSelfTest.java        |  16 +-
 .../cache/CacheSwapUnswapGetTest.java           | 234 +++++++++++++++++++
 .../testsuites/IgniteCacheTestSuite4.java       |   3 +
 14 files changed, 343 insertions(+), 48 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/18b5be3a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java
index 698d043..98e86ed 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java
@@ -641,8 +641,10 @@ public interface GridCacheEntryEx {
      *
      * @return Versioned entry.
      * @throws IgniteCheckedException In case of error.
+     * @throws GridCacheEntryRemovedException If entry was removed.
      */
-    public <K, V> GridCacheVersionedEntryEx<K, V> versionedEntry() throws IgniteCheckedException;
+    public <K, V> GridCacheVersionedEntryEx<K, V> versionedEntry()
+        throws IgniteCheckedException, GridCacheEntryRemovedException;
 
     /**
      * Sets new value if passed in version matches the current version
@@ -867,8 +869,10 @@ public interface GridCacheEntryEx {
     /**
      * @return Value.
      * @throws IgniteCheckedException If failed to read from swap storage.
+     * @throws GridCacheEntryRemovedException If entry was removed.
      */
-    @Nullable public CacheObject unswap() throws IgniteCheckedException;
+    @Nullable public CacheObject unswap()
+        throws IgniteCheckedException, GridCacheEntryRemovedException;
 
     /**
      * Unswap ignoring flags.
@@ -876,8 +880,10 @@ public interface GridCacheEntryEx {
      * @param needVal If {@code false} then do not need to deserialize value during unswap.
      * @return Value.
      * @throws IgniteCheckedException If failed.
+     * @throws GridCacheEntryRemovedException If entry was removed.
      */
-    @Nullable public CacheObject unswap(boolean needVal) throws IgniteCheckedException;
+    @Nullable public CacheObject unswap(boolean needVal)
+        throws IgniteCheckedException, GridCacheEntryRemovedException;
 
     /**
      * Tests whether or not given metadata is set.

http://git-wip-us.apache.org/repos/asf/ignite/blob/18b5be3a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionManager.java
index b2fa2dd..f60c0eb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionManager.java
@@ -994,6 +994,8 @@ public class GridCacheEvictionManager extends GridCacheManagerAdapter {
                         notRemove = new HashSet<>();
 
                     notRemove.add(entry);
+
+                    continue;
                 }
 
                 if (obsoleteVer == null)

http://git-wip-us.apache.org/repos/asf/ignite/blob/18b5be3a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
index e4882e5..eb4d864 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
@@ -433,7 +433,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
     }
 
     /** {@inheritDoc} */
-    @Override public CacheObject unswap() throws IgniteCheckedException {
+    @Override public CacheObject unswap() throws IgniteCheckedException, GridCacheEntryRemovedException {
         return unswap(true);
     }
 
@@ -444,13 +444,16 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
      * @return Value.
      * @throws IgniteCheckedException If failed.
      */
-    @Nullable @Override public CacheObject unswap(boolean needVal) throws IgniteCheckedException {
+    @Nullable @Override public CacheObject unswap(boolean needVal)
+        throws IgniteCheckedException, GridCacheEntryRemovedException {
         boolean swapEnabled = cctx.swap().swapEnabled();
 
         if (!swapEnabled && !cctx.isOffHeapEnabled())
             return null;
 
         synchronized (this) {
+            checkObsolete();
+
             if (isStartVersion() && ((flags & IS_UNSWAPPED_MASK) == 0)) {
                 GridCacheSwapEntry e;
 
@@ -3056,7 +3059,8 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
     }
 
     /** {@inheritDoc} */
-    @Override public synchronized GridCacheVersionedEntryEx versionedEntry() throws IgniteCheckedException {
+    @Override public synchronized GridCacheVersionedEntryEx versionedEntry()
+        throws IgniteCheckedException, GridCacheEntryRemovedException {
         boolean isNew = isStartVersion();
 
         CacheObject val = isNew ? unswap(true) : rawGetOrUnmarshalUnlocked(false);
@@ -3757,6 +3761,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
         throws IgniteCheckedException {
         assert Thread.holdsLock(this);
         assert cctx.isSwapOrOffheapEnabled();
+        assert !obsolete();
 
         GridCacheBatchSwapEntry ret = null;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/18b5be3a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapManager.java
index 6a8a11a..7fd6013 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapManager.java
@@ -151,7 +151,7 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
                         cctx.cache().metrics0().onOffHeapEvict();
                 }
                 catch (IgniteCheckedException e) {
-                    log.error("Failed to unmarshal off-heap entry [part=" + part + ", hash=" + hash + ']', e);
+                    U.error(log, "Failed to unmarshal off-heap entry [part=" + part + ", hash=" + hash + ']', e);
                 }
             }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/18b5be3a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
index ee56b16..3ce9ee9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
@@ -839,9 +839,19 @@ public abstract class GridDhtCacheAdapter<K, V> extends GridDistributedCacheAdap
 
                 try {
                     if (swap) {
-                        entry = cache.entryEx(keys.get(i));
+                        while (true) {
+                            try {
+                                entry = cache.entryEx(keys.get(i));
 
-                        entry.unswap(false);
+                                entry.unswap(false);
+
+                                break;
+                            }
+                            catch (GridCacheEntryRemovedException e) {
+                                if (log.isDebugEnabled())
+                                    log.debug("Got removed entry: " + entry);
+                            }
+                        }
                     }
                     else
                         entry = cache.peekEx(keys.get(i));

http://git-wip-us.apache.org/repos/asf/ignite/blob/18b5be3a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
index 6d22dc7..215a1b5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
@@ -34,6 +34,7 @@ import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.IgniteInterruptedCheckedException;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.GridCacheEntryRemovedException;
 import org.apache.ignite.internal.processors.cache.GridCacheSwapEntry;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPreloader;
@@ -717,17 +718,23 @@ public class GridDhtLocalPartition implements Comparable<GridDhtLocalPartition>,
 
                 byte[] keyBytes = entry.getKey();
 
-                try {
-                    KeyCacheObject key = cctx.toCacheKeyObject(keyBytes);
+                while (true) {
+                    try {
+                        KeyCacheObject key = cctx.toCacheKeyObject(keyBytes);
 
-                    lastEntry = (GridDhtCacheEntry)cctx.cache().entryEx(key, false);
+                        lastEntry = (GridDhtCacheEntry)cctx.cache().entryEx(key, false);
 
-                    lastEntry.unswap(true);
+                        lastEntry.unswap(true);
 
-                    return lastEntry;
-                }
-                catch (IgniteCheckedException e) {
-                    throw new CacheException(e);
+                        return lastEntry;
+                    }
+                    catch (GridCacheEntryRemovedException e) {
+                        if (log.isDebugEnabled())
+                            log.debug("Got removed entry: " + lastEntry);
+                    }
+                    catch (IgniteCheckedException e) {
+                        throw new CacheException(e);
+                    }
                 }
             }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/18b5be3a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
index 4275623..2c16534 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
@@ -872,12 +872,12 @@ public final class GridDhtLockFuture extends GridCompoundIdentityFuture<Boolean>
                         for (ListIterator<GridDhtCacheEntry> it = dhtMapping.listIterator(); it.hasNext();) {
                             GridDhtCacheEntry e = it.next();
 
-                            // Must unswap entry so that isNewLocked returns correct value.
-                            e.unswap(false);
-
                             boolean needVal = false;
 
                             try {
+                                // Must unswap entry so that isNewLocked returns correct value.
+                                e.unswap(false);
+
                                 needVal = e.isNewLocked();
 
                                 if (needVal) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/18b5be3a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java
index 194da15..762d26f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java
@@ -622,11 +622,27 @@ public abstract class GridDhtTxLocalAdapter extends IgniteTxLocalAdapter {
 
                 // First time access.
                 if (txEntry == null) {
-                    GridDhtCacheEntry cached = dhtCache.entryExx(key, topVer);
+                    GridDhtCacheEntry cached;
 
-                    addActiveCache(dhtCache.context());
+                    if (dhtCache.context().isSwapOrOffheapEnabled()) {
+                        while (true) {
+                            try {
+                                cached = dhtCache.entryExx(key, topVer);
+
+                                cached.unswap(read);
+
+                                break;
+                            }
+                            catch (GridCacheEntryRemovedException e) {
+                                if (log.isDebugEnabled())
+                                    log.debug("Get removed entry: " + key);
+                            }
+                        }
+                    }
+                    else
+                        cached = dhtCache.entryExx(key, topVer);
 
-                    cached.unswap(read);
+                    addActiveCache(dhtCache.context());
 
                     txEntry = addEntry(NOOP,
                         null,

http://git-wip-us.apache.org/repos/asf/ignite/blob/18b5be3a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java
index 501d624..36c80a9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java
@@ -35,6 +35,7 @@ import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.GridCacheEntryEx;
 import org.apache.ignite.internal.processors.cache.GridCacheEntryInfo;
+import org.apache.ignite.internal.processors.cache.GridCacheEntryRemovedException;
 import org.apache.ignite.internal.processors.cache.GridCachePreloaderAdapter;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtAffinityAssignmentRequest;
@@ -378,9 +379,19 @@ public class GridDhtPreloader extends GridCachePreloaderAdapter {
                 GridCacheEntryEx entry;
 
                 if (cctx.isSwapOrOffheapEnabled()) {
-                    entry = cctx.dht().entryEx(k, true);
-
-                    entry.unswap();
+                    while (true) {
+                        try {
+                            entry = cctx.dht().entryEx(k);
+
+                            entry.unswap();
+
+                            break;
+                        }
+                        catch (GridCacheEntryRemovedException ignore) {
+                            if (log.isDebugEnabled())
+                                log.debug("Got removed entry: " + k);
+                        }
+                    }
                 }
                 else
                     entry = cctx.dht().peekEx(k);
@@ -393,6 +404,9 @@ public class GridDhtPreloader extends GridCachePreloaderAdapter {
 
                     if (info != null && !info.isNew())
                         res.addInfo(info);
+
+                    if (cctx.isSwapOrOffheapEnabled())
+                        cctx.evicts().touch(entry, msg.topologyVersion());
                 }
                 else if (log.isDebugEnabled())
                     log.debug("Key is not present in DHT cache: " + k);

http://git-wip-us.apache.org/repos/asf/ignite/blob/18b5be3a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxRemote.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxRemote.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxRemote.java
index fff6dde..4395198 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxRemote.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxRemote.java
@@ -280,9 +280,9 @@ public class GridNearTxRemote extends GridDistributedTxRemoteAdapter {
             return false;
         }
         else {
-            cached.unswap();
-
             try {
+                cached.unswap();
+
                 CacheObject val = cached.peek(true, false, false, null);
 
                 if (val == null && cached.evictInternal(false, xidVer, null)) {
@@ -313,7 +313,9 @@ public class GridNearTxRemote extends GridDistributedTxRemoteAdapter {
     }
 
     /**
+     * @param cacheCtx Cache context.
      * @param key Key to add to read set.
+     * @param op Operation.
      * @param val Value.
      * @param drVer Data center replication version.
      * @param skipStore Skip store flag.

http://git-wip-us.apache.org/repos/asf/ignite/blob/18b5be3a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
index 97e88a7..7d32401 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
@@ -2159,20 +2159,20 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
                 // First time access.
                 if (txEntry == null) {
                     while (true) {
-                        GridCacheEntryEx entry;
+                        GridCacheEntryEx entry = null;
 
-                        if (cached != null) {
-                            entry = cached;
+                        try {
+                            if (cached != null) {
+                                entry = cached;
 
-                            cached = null;
-                        }
-                        else {
-                            entry = entryEx(cacheCtx, txKey, topologyVersion());
+                                cached = null;
+                            }
+                            else {
+                                entry = entryEx(cacheCtx, txKey, topologyVersion());
 
-                            entry.unswap(false);
-                        }
+                                entry.unswap(false);
+                            }
 
-                        try {
                             // Check if lock is being explicitly acquired by the same thread.
                             if (!implicit && cctx.kernalContext().config().isCacheSanityCheckEnabled() &&
                                 entry.lockedByThread(threadId, xidVer))

http://git-wip-us.apache.org/repos/asf/ignite/blob/18b5be3a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheAffinityCallSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheAffinityCallSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheAffinityCallSelfTest.java
index c25bc7c..b9ff46c 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheAffinityCallSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheAffinityCallSelfTest.java
@@ -21,15 +21,13 @@ import java.util.concurrent.Callable;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteCompute;
-import org.apache.ignite.cluster.ClusterGroupEmptyException;
 import org.apache.ignite.cluster.ClusterNode;
-import org.apache.ignite.compute.ComputeTaskCancelledException;
+import org.apache.ignite.cluster.ClusterTopologyException;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.IgniteKernal;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
-import org.apache.ignite.internal.processors.affinity.GridAffinityProcessor;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteCallable;
 import org.apache.ignite.lang.IgniteRunnable;
@@ -147,7 +145,7 @@ public class CacheAffinityCallSelfTest extends GridCommonAbstractTest {
         final IgniteInternalFuture<Object> fut = GridTestUtils.runAsync(new Callable<Object>() {
             @Override public Object call() throws Exception {
                 for (int i = 0; i < SERVERS_COUNT; ++i)
-                    stopGrid(i);
+                    stopGrid(i, false);
 
                 return null;
             }
@@ -157,14 +155,12 @@ public class CacheAffinityCallSelfTest extends GridCommonAbstractTest {
             while (!fut.isDone())
                 client.compute().affinityCall(CACHE_NAME, key, new CheckCallable(key));
         }
-        catch (ComputeTaskCancelledException e) {
-            assertTrue("Unexpected error: " + e, e.getMessage().contains("stopping"));
+        catch (ClusterTopologyException ignore) {
+            log.info("Expected error: " + ignore);
         }
-        catch(ClusterGroupEmptyException e) {
-            assertTrue("Unexpected error: " + e, e.getMessage().contains("Topology projection is empty"));
+        finally {
+            stopAllGrids();
         }
-
-        stopAllGrids();
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/18b5be3a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheSwapUnswapGetTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheSwapUnswapGetTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheSwapUnswapGetTest.java
new file mode 100644
index 0000000..271d8b1
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheSwapUnswapGetTest.java
@@ -0,0 +1,234 @@
+/*
+ * 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.Collections;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMemoryMode;
+import org.apache.ignite.cache.eviction.lru.LruEvictionPolicy;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.util.typedef.CAX;
+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.spi.swapspace.file.FileSwapSpaceSpi;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.eclipse.jetty.util.ConcurrentHashSet;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
+
+/**
+ *
+ */
+public class CacheSwapUnswapGetTest extends GridCommonAbstractTest {
+    /** */
+    private static final TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private static final long DURATION = 30_000;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
+
+        cfg.setSwapSpaceSpi(new FileSwapSpaceSpi());
+
+        return cfg;
+    }
+
+    /**
+     * @param atomicityMode Cache atomicity mode.
+     * @param memMode Cache memory mode.
+     * @param swap {@code True} if swap enabled.
+     * @return Cache configuration.
+     */
+    private CacheConfiguration<Integer, String> cacheConfiguration(CacheAtomicityMode atomicityMode,
+        CacheMemoryMode memMode,
+        boolean swap) {
+        CacheConfiguration<Integer, String> ccfg = new CacheConfiguration<>();
+
+        ccfg.setAtomicityMode(atomicityMode);
+        ccfg.setWriteSynchronizationMode(FULL_SYNC);
+        ccfg.setMemoryMode(memMode);
+
+        if (memMode == CacheMemoryMode.ONHEAP_TIERED) {
+            LruEvictionPolicy plc = new LruEvictionPolicy();
+            plc.setMaxSize(100);
+
+            ccfg.setEvictionPolicy(plc);
+        }
+
+        if (swap) {
+            ccfg.setSwapEnabled(true);
+
+            ccfg.setOffHeapMaxMemory(1000);
+        }
+        else
+            ccfg.setOffHeapMaxMemory(0);
+
+        return ccfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        startGrid(0);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        super.afterTestsStopped();
+
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected long getTestTimeout() {
+        return DURATION + 60_000;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTxCacheOffheapEvict() throws Exception {
+        swapUnswap(TRANSACTIONAL, CacheMemoryMode.ONHEAP_TIERED, false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTxCacheOffheapTiered() throws Exception {
+        swapUnswap(TRANSACTIONAL, CacheMemoryMode.OFFHEAP_TIERED, false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAtomicCacheOffheapEvict() throws Exception {
+        swapUnswap(ATOMIC, CacheMemoryMode.ONHEAP_TIERED, false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAtomicCacheOffheapTiered() throws Exception {
+        swapUnswap(ATOMIC, CacheMemoryMode.OFFHEAP_TIERED, false);
+    }
+
+    /**
+     * @param atomicityMode Cache atomicity mode.
+     * @param memMode Cache memory mode.
+     * @param swap {@code True} if swap enabled.
+     * @throws Exception If failed.
+     */
+    private void swapUnswap(CacheAtomicityMode atomicityMode, CacheMemoryMode memMode, boolean swap) throws Exception {
+        log.info("Start test [mode=" + atomicityMode + ", swap=" + swap + ']');
+
+        int threadCnt = 20;
+        final int keyCnt = 1000;
+        final int valCnt = 10000;
+
+        final Ignite g = grid(0);
+
+        final ConcurrentHashSet<Integer> keys = new ConcurrentHashSet<>();
+
+        final AtomicBoolean done = new AtomicBoolean();
+
+        g.destroyCache(null);
+
+        final IgniteCache<Integer, String> cache = g.createCache(cacheConfiguration(atomicityMode, memMode, swap));
+
+        try {
+            IgniteInternalFuture<?> fut = multithreadedAsync(new CAX() {
+                @Override public void applyx() throws IgniteCheckedException {
+                    ThreadLocalRandom rnd = ThreadLocalRandom.current();
+
+                    while (!done.get()) {
+                        Integer key = rnd.nextInt(keyCnt);
+
+                        switch (rnd.nextInt(3)) {
+                            case 0:
+                                cache.put(key, String.valueOf(rnd.nextInt(valCnt)));
+
+                                keys.add(key);
+
+                                break;
+
+                            case 1:
+                                cache.localEvict(Collections.singletonList(key));
+
+                                break;
+
+                            case 2:
+                                if (keys.contains(key)) {
+                                    String val = cache.get(key);
+
+                                    assertNotNull(val);
+                                }
+
+                                break;
+
+                            default:
+                                assert false;
+                        }
+                    }
+                }
+            }, threadCnt, "update-thread");
+
+            IgniteInternalFuture<?> getFut = GridTestUtils.runAsync(new Callable<Void>() {
+                @Override public Void call() throws Exception {
+                    Thread.currentThread().setName("get-thread");
+
+                    while (!done.get()) {
+                        for (Integer key : keys) {
+                            String val = cache.get(key);
+
+                            assertNotNull(val);
+                        }
+                    }
+
+                    return null;
+                }
+            });
+
+            Thread.sleep(DURATION);
+
+            done.set(true);
+
+            fut.get();
+            getFut.get();
+        }
+        finally {
+            done.set(true);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/18b5be3a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
index f8ef097..88977fb 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
@@ -34,6 +34,7 @@ import org.apache.ignite.internal.processors.cache.CacheStoreUsageMultinodeDynam
 import org.apache.ignite.internal.processors.cache.CacheStoreUsageMultinodeDynamicStartTxTest;
 import org.apache.ignite.internal.processors.cache.CacheStoreUsageMultinodeStaticStartAtomicTest;
 import org.apache.ignite.internal.processors.cache.CacheStoreUsageMultinodeStaticStartTxTest;
+import org.apache.ignite.internal.processors.cache.CacheSwapUnswapGetTest;
 import org.apache.ignite.internal.processors.cache.GridCacheMarshallingNodeJoinSelfTest;
 import org.apache.ignite.internal.processors.cache.GridCacheMultinodeUpdateAtomicNearEnabledSelfTest;
 import org.apache.ignite.internal.processors.cache.GridCacheMultinodeUpdateAtomicSelfTest;
@@ -256,6 +257,8 @@ public class IgniteCacheTestSuite4 extends TestSuite {
         suite.addTestSuite(CacheVersionedEntryReplicatedAtomicOffHeapSelfTest.class);
         suite.addTestSuite(CacheVersionedEntryReplicatedTransactionalOffHeapSelfTest.class);
 
+        suite.addTestSuite(CacheSwapUnswapGetTest.class);
+
         return suite;
     }
 }
\ No newline at end of file