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/06 17:06:02 UTC

[2/2] incubator-ignite git commit: #ignite-460: Move valPtr from GridCacheMapEntry.

#ignite-460: Move valPtr from GridCacheMapEntry.


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

Branch: refs/heads/ignite-460
Commit: 9d1869c95488ff1db55704f5c8e084a7f5b03781
Parents: d484af7
Author: ivasilinets <iv...@gridgain.com>
Authored: Wed May 6 18:05:49 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Wed May 6 18:05:49 2015 +0300

----------------------------------------------------------------------
 .../GridDistributedOffHeapCacheEntry.java       | 38 -------------
 .../distributed/dht/GridDhtCacheAdapter.java    | 23 ++++++--
 .../dht/GridDhtOffHeapCacheEntry.java           | 57 +++++++++++++++++++
 .../distributed/dht/GridNoStorageCacheMap.java  |  7 ++-
 .../dht/atomic/GridDhtAtomicCache.java          |  4 ++
 .../atomic/GridDhtAtomicOffHeapCacheEntry.java  | 57 +++++++++++++++++++
 .../dht/colocated/GridDhtColocatedCache.java    |  7 ++-
 .../GridDhtColocatedOffHeapCacheEntry.java      | 58 ++++++++++++++++++++
 .../GridDhtDetachedOffHeapCacheEntry.java       | 55 +++++++++++++++++++
 .../distributed/near/GridNearCacheAdapter.java  |  5 ++
 .../near/GridNearOffHeapCacheEntry.java         | 55 +++++++++++++++++++
 .../cache/local/GridLocalCacheEntry.java        | 18 ++++++
 12 files changed, 340 insertions(+), 44 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9d1869c9/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedOffHeapCacheEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedOffHeapCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedOffHeapCacheEntry.java
deleted file mode 100644
index 290d97c..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedOffHeapCacheEntry.java
+++ /dev/null
@@ -1,38 +0,0 @@
-package org.apache.ignite.internal.processors.cache.distributed;
-
-import org.apache.ignite.internal.processors.cache.*;
-
-/**
- * Entry for distributed (replicated/partitioned) cache in off-heap tiered/off-heap values memory mode.
- */
-public class GridDistributedOffHeapCacheEntry extends GridDistributedCacheEntry {
-    /** 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 Cache map header ID.
-     */
-    public GridDistributedOffHeapCacheEntry(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/9d1869c9/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 1c46fd0..86b0f8d 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
@@ -41,6 +41,7 @@ import java.io.*;
 import java.util.*;
 import java.util.concurrent.*;
 
+import static org.apache.ignite.cache.CacheMemoryMode.*;
 import static org.apache.ignite.internal.processors.dr.GridDrType.*;
 
 /**
@@ -102,6 +103,9 @@ public abstract class GridDhtCacheAdapter<K, V> extends GridDistributedCacheAdap
                 GridCacheMapEntry next,
                 int hdrId)
             {
+                if (ctx.config().getMemoryMode() == OFFHEAP_TIERED || ctx.config().getMemoryMode() == OFFHEAP_VALUES)
+                    return new GridDhtOffHeapCacheEntry(ctx, topVer, key, hash, val, next, hdrId);
+
                 return new GridDhtCacheEntry(ctx, topVer, key, hash, val, next, hdrId);
             }
         });
@@ -112,7 +116,8 @@ public abstract class GridDhtCacheAdapter<K, V> extends GridDistributedCacheAdap
         super.start();
 
         ctx.io().addHandler(ctx.cacheId(), GridCacheTtlUpdateRequest.class, new CI2<UUID, GridCacheTtlUpdateRequest>() {
-            @Override public void apply(UUID nodeId, GridCacheTtlUpdateRequest req) {
+            @Override
+            public void apply(UUID nodeId, GridCacheTtlUpdateRequest req) {
                 processTtlUpdateRequest(req);
             }
         });
@@ -343,17 +348,27 @@ public abstract class GridDhtCacheAdapter<K, V> extends GridDistributedCacheAdap
     public GridCacheEntryEx entryExx(KeyCacheObject key, AffinityTopologyVersion topVer, boolean allowDetached, boolean touch) {
         try {
             return allowDetached && !ctx.affinity().localNode(key, topVer) ?
-                new GridDhtDetachedCacheEntry(ctx, key, key.hashCode(), null, null, 0) :
-                entryEx(key, touch);
+                createEntry(key): entryEx(key, touch);
         }
         catch (GridDhtInvalidPartitionException e) {
             if (!allowDetached)
                 throw e;
 
-            return new GridDhtDetachedCacheEntry(ctx, key, key.hashCode(), null, null, 0);
+            return createEntry(key);
         }
     }
 
+    /**
+     * @param key Key for which entry should be returned.
+     * @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);
+    }
+
     /** {@inheritDoc} */
     @Override public void localLoad(Collection<? extends K> keys, final ExpiryPolicy plc)
         throws IgniteCheckedException {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9d1869c9/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
new file mode 100644
index 0000000..71ca0fe
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtOffHeapCacheEntry.java
@@ -0,0 +1,57 @@
+/*
+ * 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;
+
+import org.apache.ignite.internal.processors.affinity.*;
+import org.apache.ignite.internal.processors.cache.*;
+
+/**
+ * Replicated cache entry for off-heap tiered or off-heap values modes.
+ */
+public class GridDhtOffHeapCacheEntry extends GridDhtCacheEntry {
+    /** Off-heap value pointer. */
+    protected long valPtr;
+    /**
+     * @param ctx    Cache context.
+     * @param topVer Topology version at the time of creation (if negative, then latest topology is assumed).
+     * @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 GridDhtOffHeapCacheEntry(GridCacheContext ctx, AffinityTopologyVersion topVer, KeyCacheObject key, int hash,
+        CacheObject val, GridCacheMapEntry next, int hdrId) {
+        super(ctx, topVer, 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/9d1869c9/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridNoStorageCacheMap.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridNoStorageCacheMap.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridNoStorageCacheMap.java
index 8da4da5..4abbd74 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridNoStorageCacheMap.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridNoStorageCacheMap.java
@@ -25,6 +25,8 @@ import org.jetbrains.annotations.*;
 
 import java.util.*;
 
+import static org.apache.ignite.cache.CacheMemoryMode.*;
+
 /**
  * Empty cache map that will never store any entries.
  */
@@ -86,7 +88,10 @@ public class GridNoStorageCacheMap extends GridCacheConcurrentMap {
         boolean create)
     {
         if (create) {
-            GridCacheMapEntry entry = new GridDhtCacheEntry(ctx, topVer, key, hash(key.hashCode()), val, null, 0);
+            GridCacheMapEntry entry =
+                ctx.config().getMemoryMode() == OFFHEAP_TIERED || ctx.config().getMemoryMode() == OFFHEAP_VALUES ?
+                new GridDhtOffHeapCacheEntry(ctx, topVer, key, hash(key.hashCode()), val, null, 0) :
+                new GridDhtCacheEntry(ctx, topVer, key, hash(key.hashCode()), val, null, 0);
 
             return new GridTriple<>(entry, null, null);
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9d1869c9/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
index 905f7bf..5c506e9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
@@ -51,6 +51,7 @@ import java.util.concurrent.locks.*;
 
 import static org.apache.ignite.IgniteSystemProperties.*;
 import static org.apache.ignite.cache.CacheAtomicWriteOrderMode.*;
+import static org.apache.ignite.cache.CacheMemoryMode.*;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.*;
 import static org.apache.ignite.internal.processors.cache.GridCacheOperation.*;
 import static org.apache.ignite.internal.processors.cache.GridCacheUtils.*;
@@ -124,6 +125,9 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
                 GridCacheMapEntry next,
                 int hdrId)
             {
+                if (ctx.config().getMemoryMode() == OFFHEAP_TIERED || ctx.config().getMemoryMode() == OFFHEAP_VALUES)
+                    return new GridDhtAtomicOffHeapCacheEntry(ctx, topVer, key, hash, val, next, hdrId);
+
                 return new GridDhtAtomicCacheEntry(ctx, topVer, key, hash, val, next, hdrId);
             }
         });

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9d1869c9/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
new file mode 100644
index 0000000..b5eb664
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicOffHeapCacheEntry.java
@@ -0,0 +1,57 @@
+/*
+ * 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.atomic;
+
+import org.apache.ignite.internal.processors.affinity.*;
+import org.apache.ignite.internal.processors.cache.*;
+
+/**
+ * DHT atomic cache entry for off-heap tiered or off-heap values modes.
+ */
+public class GridDhtAtomicOffHeapCacheEntry extends GridDhtAtomicCacheEntry {
+    /** Off-heap value pointer. */
+    protected long valPtr;
+
+    /**
+     * @param ctx    Cache context.
+     * @param topVer Topology version at the time of creation (if negative, then latest topology is assumed).
+     * @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 GridDhtAtomicOffHeapCacheEntry(GridCacheContext ctx, AffinityTopologyVersion topVer, KeyCacheObject key, int hash, CacheObject val, GridCacheMapEntry next, int hdrId) {
+        super(ctx, topVer, 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/9d1869c9/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java
index c92d9ce..46b2c10 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java
@@ -38,6 +38,8 @@ import org.jetbrains.annotations.*;
 import java.io.*;
 import java.util.*;
 
+import static org.apache.ignite.cache.CacheMemoryMode.*;
+
 /**
  * Colocated cache.
  */
@@ -86,6 +88,9 @@ public class GridDhtColocatedCache<K, V> extends GridDhtTransactionalCacheAdapte
                 GridCacheMapEntry next,
                 int hdrId)
             {
+                if (ctx.config().getMemoryMode() == OFFHEAP_TIERED || ctx.config().getMemoryMode() == OFFHEAP_VALUES)
+                    return new GridDhtColocatedOffHeapCacheEntry(ctx, topVer, key, hash, val, next, hdrId);
+
                 return new GridDhtColocatedCacheEntry(ctx, topVer, key, hash, val, next, hdrId);
             }
         });
@@ -126,7 +131,7 @@ public class GridDhtColocatedCache<K, V> extends GridDhtTransactionalCacheAdapte
         boolean allowDetached
     ) {
         return allowDetached && !ctx.affinity().primary(ctx.localNode(), key, topVer) ?
-            new GridDhtDetachedCacheEntry(ctx, key, key.hashCode(), null, null, 0) : entryExx(key, topVer);
+            createEntry(key) : entryExx(key, topVer);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9d1869c9/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
new file mode 100644
index 0000000..ed5d2c2
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedOffHeapCacheEntry.java
@@ -0,0 +1,58 @@
+/*
+ * 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.affinity.*;
+import org.apache.ignite.internal.processors.cache.*;
+
+/**
+ * Cache entry for colocated cache for off-heap tiered or off-heap values modes.
+ */
+public class GridDhtColocatedOffHeapCacheEntry extends GridDhtColocatedCacheEntry {
+    /** Off-heap value pointer. */
+    protected long valPtr;
+
+    /**
+     * @param ctx    Cache context.
+     * @param topVer Topology version at the time of creation (if negative, then latest topology is assumed).
+     * @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 GridDhtColocatedOffHeapCacheEntry(GridCacheContext ctx, AffinityTopologyVersion topVer, KeyCacheObject key,
+        int hash, CacheObject val, GridCacheMapEntry next, int hdrId) {
+        super(ctx, topVer, 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/9d1869c9/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
new file mode 100644
index 0000000..ccef3bf
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtDetachedOffHeapCacheEntry.java
@@ -0,0 +1,55 @@
+/*
+ * 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/9d1869c9/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java
index 29c1d45..747eba7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java
@@ -37,6 +37,8 @@ import javax.cache.expiry.*;
 import java.io.*;
 import java.util.*;
 
+import static org.apache.ignite.cache.CacheMemoryMode.*;
+
 /**
  * Common logic for near caches.
  */
@@ -76,6 +78,9 @@ public abstract class GridNearCacheAdapter<K, V> extends GridDistributedCacheAda
             ) {
                 // Can't hold any locks here - this method is invoked when
                 // holding write-lock on the whole cache map.
+                if (ctx.config().getMemoryMode() == OFFHEAP_TIERED || ctx.config().getMemoryMode() == OFFHEAP_VALUES)
+                    return new GridNearOffHeapCacheEntry(ctx, key, hash, val, next, hdrId);
+
                 return new GridNearCacheEntry(ctx, key, hash, val, next, hdrId);
             }
         });

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9d1869c9/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
new file mode 100644
index 0000000..8138510
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOffHeapCacheEntry.java
@@ -0,0 +1,55 @@
+/*
+ * 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.near;
+
+import org.apache.ignite.internal.processors.cache.*;
+
+/**
+ * Near cache entry for off-heap tiered or off-heap values modes.
+ */
+public class GridNearOffHeapCacheEntry extends GridNearCacheEntry {
+    /** 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 GridNearOffHeapCacheEntry(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/9d1869c9/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 84d4c90..66b11dd 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
@@ -29,6 +29,9 @@ import static org.apache.ignite.events.EventType.*;
  */
 @SuppressWarnings({"NonPrivateFieldAccessedInSynchronizedContext", "TooBroadScope"})
 public class GridLocalCacheEntry extends GridCacheMapEntry {
+    /** Off-heap value pointer. */
+    protected long valPtr;
+
     /**
      * @param ctx  Cache registry.
      * @param key  Cache key.
@@ -384,4 +387,19 @@ public class GridLocalCacheEntry extends GridCacheMapEntry {
 
         return doomed != null;
     }
+
+    /** {@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;
+    }
 }