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 2015/05/07 14:33:57 UTC

incubator-ignite git commit: #ignite-460: Add test.

Repository: incubator-ignite
Updated Branches:
  refs/heads/ignite-460 9d1869c95 -> 693bdbcc2


#ignite-460: Add test.


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

Branch: refs/heads/ignite-460
Commit: 693bdbcc2166f42347090f4d6d41c0760abd92eb
Parents: 9d1869c
Author: ivasilinets <iv...@gridgain.com>
Authored: Thu May 7 15:33:43 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Thu May 7 15:33:43 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheMapEntry.java     |  52 +++----
 .../distributed/dht/GridDhtCacheAdapter.java    |   3 -
 .../dht/GridDhtOffHeapCacheEntry.java           |   6 +-
 .../atomic/GridDhtAtomicOffHeapCacheEntry.java  |   6 +-
 .../GridDhtColocatedOffHeapCacheEntry.java      |   6 +-
 .../GridDhtDetachedOffHeapCacheEntry.java       |  55 -------
 .../near/GridNearOffHeapCacheEntry.java         |   6 +-
 .../cache/local/GridLocalCacheEntry.java        |   6 +-
 .../processors/cache/CacheMapEntrySelfTest.java | 151 +++++++++++++++++++
 .../ignite/testsuites/IgniteBasicTestSuite.java |   1 +
 10 files changed, 193 insertions(+), 99 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/693bdbcc/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 917b3ed..f3c41d5 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
@@ -185,7 +185,7 @@ public abstract class GridCacheMapEntry implements GridCacheEntryEx {
         if (cctx.cache().isIgfsDataCache() &&
             cctx.kernalContext().igfsHelper().isIgfsBlockKey(key.value(cctx.cacheObjectContext(), false))) {
             int newSize = valueLength0(val, null);
-            int oldSize = valueLength0(this.val, (this.val == null && hasValPtr()) ? valueBytes0() : null);
+            int oldSize = valueLength0(this.val, (this.val == null && hasOffHeapValuePointer()) ? valueBytes0() : null);
 
             int delta = newSize - oldSize;
 
@@ -196,7 +196,7 @@ public abstract class GridCacheMapEntry implements GridCacheEntryEx {
         if (!isOffHeapValuesOnly()) {
             this.val = val;
 
-            setValPtr(0);
+            offHeapValuePointer(0);
         }
         else {
             try {
@@ -224,12 +224,12 @@ public abstract class GridCacheMapEntry implements GridCacheEntryEx {
                 if (val != null) {
                     byte type = val.type();
 
-                    setValPtr(mem.putOffHeap(valPtr(), val.valueBytes(cctx.cacheObjectContext()), type));
+                    offHeapValuePointer(mem.putOffHeap(offHeapValuePointer(), val.valueBytes(cctx.cacheObjectContext()), type));
                 }
                 else {
-                    mem.removeOffHeap(valPtr());
+                    mem.removeOffHeap(offHeapValuePointer());
 
-                    setValPtr(0);
+                    offHeapValuePointer(0);
                 }
             }
             catch (IgniteCheckedException e) {
@@ -267,7 +267,7 @@ public abstract class GridCacheMapEntry implements GridCacheEntryEx {
 
         CacheObject val0 = val;
 
-        if (val0 == null && hasValPtr()) {
+        if (val0 == null && hasOffHeapValuePointer()) {
             IgniteBiTuple<byte[], Byte> t = valueBytes0();
 
             return cctx.cacheObjects().toCacheObject(cctx.cacheObjectContext(), t.get2(), t.get1());
@@ -431,16 +431,16 @@ public abstract class GridCacheMapEntry implements GridCacheEntryEx {
 
                     if (e != null) {
                         if (e.offheapPointer() > 0) {
-                            setValPtr(e.offheapPointer());
+                            offHeapValuePointer(e.offheapPointer());
 
                             if (needVal) {
-                                CacheObject val = cctx.fromOffheap(valPtr(), false);
+                                CacheObject val = cctx.fromOffheap(offHeapValuePointer(), false);
 
                                 e.value(val);
                             }
                         }
                         else // Read from swap.
-                            setValPtr(0);
+                            offHeapValuePointer(0);
                     }
                 }
                 else
@@ -465,7 +465,7 @@ public abstract class GridCacheMapEntry implements GridCacheEntryEx {
 
                         // Must update valPtr again since update() will reset it.
                         if (cctx.offheapTiered() && e.offheapPointer() > 0)
-                            setValPtr(e.offheapPointer());
+                            offHeapValuePointer(e.offheapPointer());
 
                         return val;
                     }
@@ -492,13 +492,13 @@ public abstract class GridCacheMapEntry implements GridCacheEntryEx {
                 if (cctx.offheapTiered()) {
                     cctx.swap().removeOffheap(key);
 
-                    setValPtr(0);
+                    offHeapValuePointer(0);
                 }
 
                 return;
             }
 
-            if (val == null && cctx.offheapTiered() && hasValPtr()) {
+            if (val == null && cctx.offheapTiered() && hasOffHeapValuePointer()) {
                 if (log.isDebugEnabled())
                     log.debug("Value did not change, skip write swap entry: " + this);
 
@@ -537,10 +537,10 @@ public abstract class GridCacheMapEntry implements GridCacheEntryEx {
     protected IgniteBiTuple<byte[], Byte> valueBytes0() {
         assert Thread.holdsLock(this);
 
-        if (hasValPtr()) {
+        if (hasOffHeapValuePointer()) {
             assert isOffHeapValuesOnly() || cctx.offheapTiered();
 
-            return cctx.unsafeMemory().get(valPtr());
+            return cctx.unsafeMemory().get(offHeapValuePointer());
         }
         else {
             assert val != null;
@@ -669,7 +669,7 @@ public abstract class GridCacheMapEntry implements GridCacheEntryEx {
 
             CacheObject val = this.val;
 
-            hasOldBytes = hasValPtr();
+            hasOldBytes = hasOffHeapValuePointer();
 
             if ((unmarshal || isOffHeapValuesOnly()) && !expired && val == null && hasOldBytes)
                 val = rawGetOrUnmarshalUnlocked(tmp);
@@ -813,7 +813,7 @@ public abstract class GridCacheMapEntry implements GridCacheEntryEx {
                         // Update indexes before actual write to entry.
                         updateIndex(ret, expTime, nextVer, prevVal);
 
-                    boolean hadValPtr = hasValPtr();
+                    boolean hadValPtr = hasOffHeapValuePointer();
 
                     // Don't change version for read-through.
                     update(ret, expTime, ttl, nextVer);
@@ -1161,7 +1161,7 @@ public abstract class GridCacheMapEntry implements GridCacheEntryEx {
             // can be updated without actually holding entry lock.
             clearIndex(old);
 
-            boolean hadValPtr = hasValPtr();
+            boolean hadValPtr = hasOffHeapValuePointer();
 
             update(null, 0, 0, newVer);
 
@@ -1518,7 +1518,7 @@ public abstract class GridCacheMapEntry implements GridCacheEntryEx {
                     // Must persist inside synchronization in non-tx mode.
                     cctx.store().remove(null, keyValue(false));
 
-                boolean hasValPtr = hasValPtr();
+                boolean hasValPtr = hasOffHeapValuePointer();
 
                 // Update index inside synchronization since it can be updated
                 // in load methods without actually holding entry lock.
@@ -2119,7 +2119,7 @@ public abstract class GridCacheMapEntry implements GridCacheEntryEx {
 
                 enqueueVer = newVer;
 
-                boolean hasValPtr = hasValPtr();
+                boolean hasValPtr = hasOffHeapValuePointer();
 
                 // Clear value on backup. Entry will be removed from cache when it got evicted from queue.
                 update(null, CU.TTL_ETERNAL, CU.EXPIRE_TIME_ETERNAL, newVer);
@@ -2926,8 +2926,8 @@ public abstract class GridCacheMapEntry implements GridCacheEntryEx {
         if (val != null)
             return val;
 
-        if (hasValPtr()) {
-            CacheObject val0 = cctx.fromOffheap(valPtr(), tmp);
+        if (hasOffHeapValuePointer()) {
+            CacheObject val0 = cctx.fromOffheap(offHeapValuePointer(), tmp);
 
             if (!tmp && cctx.kernalContext().config().isPeerClassLoadingEnabled())
                 val0.finishUnmarshal(cctx.cacheObjectContext(), cctx.deploy().globalLoader());
@@ -2949,7 +2949,7 @@ public abstract class GridCacheMapEntry implements GridCacheEntryEx {
     protected boolean hasValueUnlocked() {
         assert Thread.holdsLock(this);
 
-        return val != null || hasValPtr();
+        return val != null || hasOffHeapValuePointer();
     }
 
     /** {@inheritDoc} */
@@ -3295,7 +3295,7 @@ public abstract class GridCacheMapEntry implements GridCacheEntryEx {
             synchronized (this) {
                 CacheObject expiredVal = saveValueForIndexUnlocked();
 
-                boolean hasOldBytes = hasValPtr();
+                boolean hasOldBytes = hasOffHeapValuePointer();
 
                 boolean expired = checkExpired();
 
@@ -4101,21 +4101,21 @@ public abstract class GridCacheMapEntry implements GridCacheEntryEx {
     /**
      * @return True if entry has off-heap value pointer.
      */
-    protected boolean hasValPtr() {
+    protected boolean hasOffHeapValuePointer() {
         return false;
     }
 
     /**
      * @return Off-heap value pointer.
      */
-    protected long valPtr() {
+    protected long offHeapValuePointer() {
         return 0;
     }
 
     /**
      * @param valPtr Off-heap value pointer.
      */
-    protected void setValPtr(long valPtr) {
+    protected void offHeapValuePointer(long valPtr) {
         // No-op.
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/693bdbcc/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 86b0f8d..d604d9d 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
@@ -363,9 +363,6 @@ public abstract class GridDhtCacheAdapter<K, V> extends GridDistributedCacheAdap
      * @return Cache entry.
      */
     protected GridDistributedCacheEntry createEntry(KeyCacheObject key) {
-        if (ctx.config().getMemoryMode() == OFFHEAP_TIERED || ctx.config().getMemoryMode() == OFFHEAP_VALUES)
-            new GridDhtDetachedOffHeapCacheEntry(ctx, key, key.hashCode(), null, null, 0);
-
         return new GridDhtDetachedCacheEntry(ctx, key, key.hashCode(), null, null, 0);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/693bdbcc/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtOffHeapCacheEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtOffHeapCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtOffHeapCacheEntry.java
index 71ca0fe..7bcbee4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtOffHeapCacheEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtOffHeapCacheEntry.java
@@ -41,17 +41,17 @@ public class GridDhtOffHeapCacheEntry extends GridDhtCacheEntry {
     }
 
     /** {@inheritDoc} */
-    @Override protected boolean hasValPtr() {
+    @Override protected boolean hasOffHeapValuePointer() {
         return valPtr != 0;
     }
 
     /** {@inheritDoc} */
-    @Override protected long valPtr() {
+    @Override protected long offHeapValuePointer() {
         return valPtr;
     }
 
     /** {@inheritDoc} */
-    @Override protected void setValPtr(long valPtr) {
+    @Override protected void offHeapValuePointer(long valPtr) {
         this.valPtr = valPtr;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/693bdbcc/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicOffHeapCacheEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicOffHeapCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicOffHeapCacheEntry.java
index b5eb664..959c0de 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicOffHeapCacheEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicOffHeapCacheEntry.java
@@ -41,17 +41,17 @@ public class GridDhtAtomicOffHeapCacheEntry extends GridDhtAtomicCacheEntry {
     }
 
     /** {@inheritDoc} */
-    @Override protected boolean hasValPtr() {
+    @Override protected boolean hasOffHeapValuePointer() {
         return valPtr != 0;
     }
 
     /** {@inheritDoc} */
-    @Override protected long valPtr() {
+    @Override protected long offHeapValuePointer() {
         return valPtr;
     }
 
     /** {@inheritDoc} */
-    @Override protected void setValPtr(long valPtr) {
+    @Override protected void offHeapValuePointer(long valPtr) {
         this.valPtr = valPtr;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/693bdbcc/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedOffHeapCacheEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedOffHeapCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedOffHeapCacheEntry.java
index ed5d2c2..b8e9dfc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedOffHeapCacheEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedOffHeapCacheEntry.java
@@ -42,17 +42,17 @@ public class GridDhtColocatedOffHeapCacheEntry extends GridDhtColocatedCacheEntr
     }
 
     /** {@inheritDoc} */
-    @Override protected boolean hasValPtr() {
+    @Override protected boolean hasOffHeapValuePointer() {
         return valPtr != 0;
     }
 
     /** {@inheritDoc} */
-    @Override protected long valPtr() {
+    @Override protected long offHeapValuePointer() {
         return valPtr;
     }
 
     /** {@inheritDoc} */
-    @Override protected void setValPtr(long valPtr) {
+    @Override protected void offHeapValuePointer(long valPtr) {
         this.valPtr = valPtr;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/693bdbcc/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtDetachedOffHeapCacheEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtDetachedOffHeapCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtDetachedOffHeapCacheEntry.java
deleted file mode 100644
index ccef3bf..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtDetachedOffHeapCacheEntry.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache.distributed.dht.colocated;
-
-import org.apache.ignite.internal.processors.cache.*;
-
-/**
- * Detached cache entry for off-heap tiered or off-heap values modes.
- */
-public class GridDhtDetachedOffHeapCacheEntry extends GridDhtDetachedCacheEntry {
-    /** Off-heap value pointer. */
-    protected long valPtr;
-
-    /**
-     * @param ctx   Cache context.
-     * @param key   Cache key.
-     * @param hash  Key hash value.
-     * @param val   Entry value.
-     * @param next  Next entry in the linked list.
-     * @param hdrId Header ID.
-     */
-    public GridDhtDetachedOffHeapCacheEntry(GridCacheContext ctx, KeyCacheObject key, int hash, CacheObject val, GridCacheMapEntry next, int hdrId) {
-        super(ctx, key, hash, val, next, hdrId);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected boolean hasValPtr() {
-        return valPtr != 0;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected long valPtr() {
-        return valPtr;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void setValPtr(long valPtr) {
-        this.valPtr = valPtr;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/693bdbcc/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOffHeapCacheEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOffHeapCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOffHeapCacheEntry.java
index 8138510..e695612 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOffHeapCacheEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOffHeapCacheEntry.java
@@ -39,17 +39,17 @@ public class GridNearOffHeapCacheEntry extends GridNearCacheEntry {
     }
 
     /** {@inheritDoc} */
-    @Override protected boolean hasValPtr() {
+    @Override protected boolean hasOffHeapValuePointer() {
         return valPtr != 0;
     }
 
     /** {@inheritDoc} */
-    @Override protected long valPtr() {
+    @Override protected long offHeapValuePointer() {
         return valPtr;
     }
 
     /** {@inheritDoc} */
-    @Override protected void setValPtr(long valPtr) {
+    @Override protected void offHeapValuePointer(long valPtr) {
         this.valPtr = valPtr;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/693bdbcc/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCacheEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCacheEntry.java
index 66b11dd..abf10dd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCacheEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCacheEntry.java
@@ -389,17 +389,17 @@ public class GridLocalCacheEntry extends GridCacheMapEntry {
     }
 
     /** {@inheritDoc} */
-    @Override protected boolean hasValPtr() {
+    @Override protected boolean hasOffHeapValuePointer() {
         return valPtr != 0;
     }
 
     /** {@inheritDoc} */
-    @Override protected long valPtr() {
+    @Override protected long offHeapValuePointer() {
         return valPtr;
     }
 
     /** {@inheritDoc} */
-    @Override protected void setValPtr(long valPtr) {
+    @Override protected void offHeapValuePointer(long valPtr) {
         this.valPtr = valPtr;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/693bdbcc/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheMapEntrySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheMapEntrySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheMapEntrySelfTest.java
new file mode 100644
index 0000000..4cbfd22
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheMapEntrySelfTest.java
@@ -0,0 +1,151 @@
+/*
+ * 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 org.apache.ignite.cache.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.*;
+import org.apache.ignite.internal.processors.cache.distributed.dht.colocated.*;
+import org.apache.ignite.internal.processors.cache.distributed.near.*;
+import org.apache.ignite.internal.processors.cache.local.*;
+
+/**
+ * Cache map entry self test.
+ */
+public class CacheMapEntrySelfTest extends GridCacheAbstractSelfTest {
+    /** Cache mode. */
+    private CacheMode cacheMode;
+
+    /** Atomicity mode. */
+    private CacheAtomicityMode atomicityMode;
+
+    /** Memory mode. */
+    private CacheMemoryMode memoryMode;
+
+    /** {@inheritDoc} */
+    @Override protected int gridCount() {
+        return 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheConfiguration cacheConfiguration(String gridName) throws Exception {
+        CacheConfiguration cfg = super.cacheConfiguration(gridName);
+
+        cfg.setCacheMode(cacheMode);
+        cfg.setAtomicityMode(atomicityMode);
+        cfg.setMemoryMode(memoryMode);
+        cfg.setBackups(1);
+
+        return cfg;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCacheMapEntry() throws Exception {
+        checkCacheMapEntry(CacheMemoryMode.ONHEAP_TIERED, CacheAtomicityMode.ATOMIC, CacheMode.LOCAL,
+            GridLocalCacheEntry.class);
+        checkCacheMapEntry(CacheMemoryMode.OFFHEAP_TIERED, CacheAtomicityMode.ATOMIC, CacheMode.LOCAL,
+            GridLocalCacheEntry.class);
+        checkCacheMapEntry(CacheMemoryMode.OFFHEAP_VALUES, CacheAtomicityMode.ATOMIC, CacheMode.LOCAL,
+            GridLocalCacheEntry.class);
+        checkCacheMapEntry(CacheMemoryMode.ONHEAP_TIERED, CacheAtomicityMode.TRANSACTIONAL, CacheMode.LOCAL,
+            GridLocalCacheEntry.class);
+        checkCacheMapEntry(CacheMemoryMode.OFFHEAP_TIERED, CacheAtomicityMode.TRANSACTIONAL, CacheMode.LOCAL,
+            GridLocalCacheEntry.class);
+        checkCacheMapEntry(CacheMemoryMode.OFFHEAP_VALUES, CacheAtomicityMode.TRANSACTIONAL, CacheMode.LOCAL,
+            GridLocalCacheEntry.class);
+        checkCacheMapEntry(CacheMemoryMode.ONHEAP_TIERED, CacheAtomicityMode.ATOMIC, CacheMode.PARTITIONED,
+            GridNearCacheEntry.class);
+        checkCacheMapEntry(CacheMemoryMode.OFFHEAP_TIERED, CacheAtomicityMode.ATOMIC, CacheMode.PARTITIONED,
+            GridNearOffHeapCacheEntry.class);
+        checkCacheMapEntry(CacheMemoryMode.OFFHEAP_VALUES, CacheAtomicityMode.ATOMIC, CacheMode.PARTITIONED,
+            GridNearOffHeapCacheEntry.class);
+        checkCacheMapEntry(CacheMemoryMode.ONHEAP_TIERED, CacheAtomicityMode.TRANSACTIONAL, CacheMode.PARTITIONED,
+            GridNearCacheEntry.class);
+        checkCacheMapEntry(CacheMemoryMode.OFFHEAP_TIERED, CacheAtomicityMode.TRANSACTIONAL, CacheMode.PARTITIONED,
+            GridNearOffHeapCacheEntry.class);
+        checkCacheMapEntry(CacheMemoryMode.OFFHEAP_VALUES, CacheAtomicityMode.TRANSACTIONAL, CacheMode.PARTITIONED,
+            GridNearOffHeapCacheEntry.class);
+        checkCacheMapEntry(CacheMemoryMode.ONHEAP_TIERED, CacheAtomicityMode.ATOMIC, CacheMode.REPLICATED,
+            GridDhtAtomicCacheEntry.class);
+        checkCacheMapEntry(CacheMemoryMode.OFFHEAP_TIERED, CacheAtomicityMode.ATOMIC, CacheMode.REPLICATED,
+            GridDhtAtomicOffHeapCacheEntry.class);
+        checkCacheMapEntry(CacheMemoryMode.OFFHEAP_VALUES, CacheAtomicityMode.ATOMIC, CacheMode.REPLICATED,
+            GridDhtAtomicOffHeapCacheEntry.class);
+        checkCacheMapEntry(CacheMemoryMode.ONHEAP_TIERED, CacheAtomicityMode.TRANSACTIONAL, CacheMode.REPLICATED,
+            GridDhtColocatedCacheEntry.class);
+        checkCacheMapEntry(CacheMemoryMode.OFFHEAP_TIERED, CacheAtomicityMode.TRANSACTIONAL, CacheMode.REPLICATED,
+            GridDhtColocatedOffHeapCacheEntry.class);
+        checkCacheMapEntry(CacheMemoryMode.OFFHEAP_VALUES, CacheAtomicityMode.TRANSACTIONAL, CacheMode.REPLICATED,
+            GridDhtColocatedOffHeapCacheEntry.class);
+    }
+
+    /**
+     * @param memoryMode Cache memory mode.
+     * @param atomicityMode Cache atomicity mode.
+     * @param cacheMode Cache mode.
+     * @param entryClass Class of cache map entry.
+     * @throws Exception If failed.
+     */
+    private void checkCacheMapEntry(CacheMemoryMode memoryMode, CacheAtomicityMode atomicityMode, CacheMode cacheMode,
+        Class<?> entryClass) throws Exception {
+        this.cacheMode = cacheMode;
+        this.atomicityMode = atomicityMode;
+        this.memoryMode = memoryMode;
+
+        IgniteKernal ignite = (IgniteKernal)startGrid(2);
+
+        GridCacheAdapter<Integer, String> cache = ignite.internalCache(null);
+
+        Integer key = primaryKey(ignite.cache(null));
+
+        cache.put(key, "val");
+
+        GridCacheEntryEx entry = cache.entryEx(key);
+
+        entry.unswap(true);
+
+        assertNotNull(entry);
+
+        assertEquals(entry.getClass(), entryClass);
+
+        stopAllGrids();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/693bdbcc/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 2b33d7e..bd6e81a 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
@@ -72,6 +72,7 @@ public class IgniteBasicTestSuite extends TestSuite {
         suite.addTestSuite(GridLifecycleAwareSelfTest.class);
         suite.addTestSuite(GridMessageListenSelfTest.class);
         suite.addTestSuite(GridFailFastNodeFailureDetectionSelfTest.class);
+        suite.addTestSuite(CacheMapEntrySelfTest.class);
 
         return suite;
     }