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/12/09 13:13:16 UTC

[10/20] ignite git commit: ignite-1.5 Cache map refactoring. This closes #279.

ignite-1.5 Cache map refactoring. This closes #279.


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

Branch: refs/heads/ignite-1537
Commit: 3baf4d163c6edafa7367ece7901ad1efe695a8d4
Parents: c30def8
Author: sboikov <sb...@gridgain.com>
Authored: Tue Dec 8 17:30:14 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Dec 8 17:30:14 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheAdapter.java      |  65 +-
 .../cache/GridCacheConcurrentMap.java           | 734 +++++--------------
 .../processors/cache/GridCacheMapEntry.java     |  44 +-
 .../cache/GridCacheMapEntryFactory.java         |  13 +-
 .../distributed/GridDistributedCacheEntry.java  |  14 +-
 .../distributed/dht/GridDhtCacheAdapter.java    |  13 +-
 .../distributed/dht/GridDhtCacheEntry.java      |  14 +-
 .../dht/GridDhtOffHeapCacheEntry.java           |  15 +-
 .../distributed/dht/GridNoStorageCacheMap.java  |   4 +-
 .../dht/atomic/GridDhtAtomicCache.java          |  23 +-
 .../dht/atomic/GridDhtAtomicCacheEntry.java     |  16 +-
 .../atomic/GridDhtAtomicOffHeapCacheEntry.java  |  15 +-
 .../dht/colocated/GridDhtColocatedCache.java    |  13 +-
 .../colocated/GridDhtColocatedCacheEntry.java   |  14 +-
 .../GridDhtColocatedOffHeapCacheEntry.java      |  15 +-
 .../colocated/GridDhtDetachedCacheEntry.java    |   4 +-
 .../distributed/near/GridNearCacheAdapter.java  |   8 +-
 .../distributed/near/GridNearCacheEntry.java    |  16 +-
 .../near/GridNearOffHeapCacheEntry.java         |  15 +-
 .../processors/cache/local/GridLocalCache.java  |   8 +-
 .../cache/local/GridLocalCacheEntry.java        |  15 +-
 .../local/atomic/GridLocalAtomicCache.java      |   8 +-
 .../ignite/internal/util/IgniteUtils.java       |  18 +-
 .../cache/GridCacheAlwaysEvictionPolicy.java    |   2 +-
 .../cache/GridCacheConcurrentMapSelfTest.java   |  11 -
 .../loadtests/hashmap/GridHashMapLoadTest.java  |   4 +-
 26 files changed, 357 insertions(+), 764 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/3baf4d16/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index f96954e..034640f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
@@ -925,25 +925,60 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      */
     @Nullable private GridCacheEntryEx entry0(KeyCacheObject key, AffinityTopologyVersion topVer, boolean create,
         boolean touch) {
-        GridTriple<GridCacheMapEntry> t = map.putEntryIfObsoleteOrAbsent(topVer, key, null, create);
+        GridCacheMapEntry cur = map.getEntry(key);
 
-        GridCacheEntryEx cur = t.get1();
-        GridCacheEntryEx created = t.get2();
-        GridCacheEntryEx doomed = t.get3();
+        if (cur == null || cur.obsolete()) {
+            GridTriple<GridCacheMapEntry> t = map.putEntryIfObsoleteOrAbsent(
+                topVer,
+                key,
+                null,
+                create);
 
-        if (doomed != null && ctx.events().isRecordable(EVT_CACHE_ENTRY_DESTROYED))
-            // Event notification.
-            ctx.events().addEvent(doomed.partition(), doomed.key(), locNodeId, (IgniteUuid)null, null,
-                EVT_CACHE_ENTRY_DESTROYED, null, false, null, false, null, null, null, true);
+            cur = t.get1();
 
-        if (created != null) {
-            // Event notification.
-            if (ctx.events().isRecordable(EVT_CACHE_ENTRY_CREATED))
-                ctx.events().addEvent(created.partition(), created.key(), locNodeId, (IgniteUuid)null, null,
-                    EVT_CACHE_ENTRY_CREATED, null, false, null, false, null, null, null, true);
+            GridCacheEntryEx created = t.get2();
+            GridCacheEntryEx doomed = t.get3();
 
-            if (touch)
-                ctx.evicts().touch(cur, topVer);
+            if (doomed != null && ctx.events().isRecordable(EVT_CACHE_ENTRY_DESTROYED))
+                // Event notification.
+                ctx.events().addEvent(doomed.partition(),
+                    doomed.key(),
+                    locNodeId,
+                    (IgniteUuid)null,
+                    null,
+                    EVT_CACHE_ENTRY_DESTROYED,
+                    null,
+                    false,
+                    null,
+                    false,
+                    null,
+                    null,
+                    null,
+                    true);
+
+            if (created != null) {
+                // Event notification.
+                if (ctx.events().isRecordable(EVT_CACHE_ENTRY_CREATED))
+                    ctx.events().addEvent(created.partition(),
+                        created.key(),
+                        locNodeId,
+                        (IgniteUuid)null,
+                        null,
+                        EVT_CACHE_ENTRY_CREATED,
+                        null,
+                        false,
+                        null,
+                        false,
+                        null,
+                        null,
+                        null,
+                        true);
+
+                if (touch)
+                    ctx.evicts().touch(
+                        cur,
+                        topVer);
+            }
         }
 
         return cur;

http://git-wip-us.apache.org/repos/asf/ignite/blob/3baf4d16/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentMap.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentMap.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentMap.java
index 1c64387..f78a606 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentMap.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentMap.java
@@ -22,9 +22,6 @@ import java.io.IOException;
 import java.io.ObjectInput;
 import java.io.ObjectOutput;
 import java.io.ObjectStreamException;
-import java.lang.ref.Reference;
-import java.lang.ref.ReferenceQueue;
-import java.lang.ref.WeakReference;
 import java.lang.reflect.Array;
 import java.util.AbstractCollection;
 import java.util.AbstractSet;
@@ -35,7 +32,6 @@ import java.util.Map;
 import java.util.NoSuchElementException;
 import java.util.Random;
 import java.util.Set;
-import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.locks.ReentrantLock;
 import javax.cache.Cache;
 import org.apache.ignite.IgniteCheckedException;
@@ -45,6 +41,7 @@ import org.apache.ignite.internal.IgniteKernal;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.util.F0;
 import org.apache.ignite.internal.util.lang.GridTriple;
+import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.P1;
 import org.apache.ignite.internal.util.typedef.X;
@@ -55,7 +52,6 @@ import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgnitePredicate;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
-import org.jsr166.ConcurrentHashMap8;
 import org.jsr166.LongAdder8;
 
 /**
@@ -127,82 +123,6 @@ public class GridCacheConcurrentMap {
             }
         };
 
-    /** Soft iterator queue. */
-    private final ReferenceQueue<Iterator0> itQ = new ReferenceQueue<>();
-
-    /** Soft iterator set. */
-    private final Map<WeakIterator<KeyCacheObject, CacheObject>, SegmentHeader> itMap =
-        new ConcurrentHashMap8<>();
-
-    /**
-     * Checks phantom queue.
-     */
-    private void checkWeakQueue() {
-        // If queue is empty, then it is a lock-free volatile read which should happen
-        // in most cases. Otherwise queue uses synchronization to poll elements.
-        for (Reference<? extends Iterator0> itRef = itQ.poll();
-             itRef != null; itRef = itQ.poll()) {
-            assert itRef instanceof WeakIterator;
-
-            if (DEBUG)
-                X.println("Checking weak queue [itSetSize=" + itMap.size() + ']');
-
-            SegmentHeader lastSeg = removeWeakIterator(itRef);
-
-            // Segment may be null if iterator is empty at creation time.
-            if (lastSeg != null)
-                lastSeg.onReadEnd();
-        }
-    }
-
-    /**
-     * @param itRef Iterator reference.
-     * @return Last segment.
-     */
-    private SegmentHeader removeWeakIterator(
-        Reference<? extends Iterator0> itRef) {
-        assert itRef instanceof WeakIterator;
-
-        SegmentHeader hdr = itMap.remove(itRef);
-
-        if (DEBUG) {
-            if (hdr == null)
-                X.println("Removed non-existent iterator: " + itRef);
-            else
-                X.println("Removed iterator [hdrId=" + hdr.id() + ", it=" + itRef + ", mapSize=" + itMap.size() + ']');
-        }
-
-        return hdr;
-    }
-
-    /**
-     * @param itRef Iterator reference.
-     * @param hdr Segment header.
-     */
-    private void addWeakIterator(WeakIterator itRef, SegmentHeader hdr) {
-        SegmentHeader prev = itMap.put(itRef, hdr);
-
-        if (DEBUG)
-            if (prev == null)
-                X.println("Added weak reference: " + itMap.size());
-    }
-
-
-
-    /**
-     * @return Iterator set size.
-     */
-    int iteratorMapSize() {
-        return itMap.size();
-    }
-
-    /**
-     * @return Reference queue for iterators.
-     */
-    private ReferenceQueue<Iterator0> iteratorQueue() {
-        return itQ;
-    }
-
     /**
      * Applies a supplemental hash function to a given hashCode, which
      * defends against poor quality hash functions.  This is critical
@@ -408,8 +328,6 @@ public class GridCacheConcurrentMap {
      * @return {@code True} if map contains mapping for provided key.
      */
     public boolean containsKey(Object key) {
-        checkWeakQueue();
-
         int hash = hash(key.hashCode());
 
         return segmentFor(hash).containsKey(key, hash);
@@ -422,8 +340,6 @@ public class GridCacheConcurrentMap {
      * @return a collection view of the values contained in this map.
      */
     public <K, V> Collection<V> allValues(CacheEntryPredicate[] filter) {
-        checkWeakQueue();
-
         return new Values<>(this, filter);
     }
 
@@ -431,8 +347,6 @@ public class GridCacheConcurrentMap {
      * @return Random entry out of hash map.
      */
     @Nullable public GridCacheMapEntry randomEntry() {
-        checkWeakQueue();
-
         while (true) {
             if (mapPubSize.sum() == 0)
                 return null;
@@ -475,8 +389,6 @@ public class GridCacheConcurrentMap {
     @Nullable public GridCacheMapEntry getEntry(Object key) {
         assert key != null;
 
-        checkWeakQueue();
-
         int hash = hash(key.hashCode());
 
         return segmentFor(hash).get(key, hash);
@@ -491,8 +403,6 @@ public class GridCacheConcurrentMap {
     public GridCacheMapEntry putEntry(AffinityTopologyVersion topVer, KeyCacheObject key, @Nullable CacheObject val) {
         assert key != null;
 
-        checkWeakQueue();
-
         int hash = hash(key.hashCode());
 
         return segmentFor(hash).put(key, hash, val, topVer);
@@ -514,8 +424,6 @@ public class GridCacheConcurrentMap {
     {
         assert key != null;
 
-        checkWeakQueue();
-
         int hash = hash(key.hashCode());
 
         return segmentFor(hash).putIfObsolete(key, hash, val, topVer, create);
@@ -543,8 +451,6 @@ public class GridCacheConcurrentMap {
     public boolean removeEntry(GridCacheEntryEx e) {
         assert e != null;
 
-        checkWeakQueue();
-
         KeyCacheObject key = e.key();
 
         int hash = hash(key.hashCode());
@@ -576,8 +482,6 @@ public class GridCacheConcurrentMap {
     @Nullable public GridCacheMapEntry removeEntryIfObsolete(KeyCacheObject key) {
         assert key != null;
 
-        checkWeakQueue();
-
         int hash = hash(key.hashCode());
 
         return segmentFor(hash).remove(key, hash, obsolete);
@@ -589,8 +493,6 @@ public class GridCacheConcurrentMap {
      */
     @SuppressWarnings({"unchecked"})
     public <K, V> Set<Cache.Entry<K, V>> entries(CacheEntryPredicate... filter) {
-        checkWeakQueue();
-
         return new EntrySet<>(this, filter);
     }
 
@@ -602,8 +504,6 @@ public class GridCacheConcurrentMap {
      */
     @SuppressWarnings({"unchecked"})
     public <K, V> Set<Cache.Entry<K, V>> entriesx(CacheEntryPredicate... filter) {
-        checkWeakQueue();
-
         return new EntrySet<>(this, filter, true);
     }
 
@@ -613,8 +513,6 @@ public class GridCacheConcurrentMap {
      * @return Set of the mappings contained in this map.
      */
     public Set<GridCacheEntryEx> entries0() {
-        checkWeakQueue();
-
         return new Set0<>(this, GridCacheConcurrentMap.nonInternal());
     }
 
@@ -626,8 +524,6 @@ public class GridCacheConcurrentMap {
      * @return Striped entry iterator.
      */
     public Iterator<GridCacheEntryEx> stripedEntryIterator(int id, int totalCnt) {
-        checkWeakQueue();
-
         return new Iterator0<>(this, false, GridCacheConcurrentMap.nonInternal(), id, totalCnt);
     }
 
@@ -637,8 +533,6 @@ public class GridCacheConcurrentMap {
      * @return All internal entry set, including {@link GridCacheInternal} entries.
      */
     public Set<GridCacheEntryEx> allEntries0() {
-        checkWeakQueue();
-
         return new Set0<>(this, CU.empty0());
     }
 
@@ -649,8 +543,6 @@ public class GridCacheConcurrentMap {
      * @return Set of the keys contained in this map.
      */
     public <K, V> Set<K> keySet(CacheEntryPredicate... filter) {
-        checkWeakQueue();
-
         return new KeySet<>(this, filter, false);
     }
 
@@ -661,8 +553,6 @@ public class GridCacheConcurrentMap {
      * @return Set of the keys contained in this map.
      */
     public <K, V> Set<K> keySetx(CacheEntryPredicate... filter) {
-        checkWeakQueue();
-
         return new KeySet<>(this, filter, true);
     }
 
@@ -673,8 +563,6 @@ public class GridCacheConcurrentMap {
      * @return Collection view of the values contained in this map.
      */
     public <K, V> Collection<V> values(CacheEntryPredicate... filter) {
-        checkWeakQueue();
-
         return allValues(filter);
     }
 
@@ -699,9 +587,7 @@ public class GridCacheConcurrentMap {
 
                 X.println("    Segment [idx=" + i + ", size=" + seg.size() + ']');
 
-                SegmentHeader segHdr = seg.hdr;
-
-                GridCacheMapEntry[] tab = segHdr.table();
+                HashEntry[] tab = seg.tbl;
 
                 for (int j = 0; j < tab.length; j++)
                     X.println("        Bucket [idx=" + j + ", bucket=" + tab[j] + ']');
@@ -725,13 +611,13 @@ public class GridCacheConcurrentMap {
         IgniteLogger log = ctx.logger(GridCacheConcurrentMap.class);
 
         for (Segment s : segs) {
-            SegmentHeader segHdr = s.hdr;
+            assert s.isHeldByCurrentThread();
 
-            GridCacheMapEntry[] tab = segHdr.table();
+            HashEntry[] tab = s.tbl;
 
-            for (GridCacheMapEntry b : tab) {
+            for (HashEntry b : tab) {
                 if (b != null) {
-                    GridCacheMapEntry e = b;
+                    HashEntry e = b;
 
                     assert e != null;
 
@@ -743,12 +629,12 @@ public class GridCacheConcurrentMap {
 
                         log.info("Cache map entry: " + e);
 
-                        if (!e.deleted()) {
-                            if (!(e.key instanceof GridCacheInternal))
+                        if (!e.mapEntry.deleted()) {
+                            if (!(e.mapEntry.key instanceof GridCacheInternal))
                                 pubCnt++;
                         }
 
-                        e = e.next(segHdr.id());
+                        e = e.next;
                     }
 
                     size += cnt;
@@ -813,9 +699,6 @@ public class GridCacheConcurrentMap {
          */
         private int threshold;
 
-        /** Segment header. */
-        private volatile SegmentHeader hdr;
-
         /** The number of public elements in this segment's region. */
         private final LongAdder8 pubSize = new LongAdder8();
 
@@ -827,6 +710,12 @@ public class GridCacheConcurrentMap {
          */
         private final float loadFactor;
 
+        /** Entry table. */
+        private volatile HashEntry[] tbl;
+
+        /** The number of elements in this segment's region. */
+        private volatile int size;
+
         /**
          * @param initCap Initial capacity.
          * @param lf Load factor.
@@ -835,39 +724,19 @@ public class GridCacheConcurrentMap {
         Segment(int initCap, float lf) {
             loadFactor = lf;
 
-            hdr = new SegmentHeader(initCap, 0, null);
+            tbl = new HashEntry[initCap];
 
-            threshold = (int)(hdr.length() * loadFactor);
+            threshold = (int)(initCap * loadFactor);
         }
 
         /**
          * Returns properly casted first entry for given hash.
          *
-         * @param tbl Table.
          * @param hash Hash.
          * @return Entry for hash.
          */
-        @Nullable GridCacheMapEntry getFirst(GridCacheMapEntry[] tbl, int hash) {
-            GridCacheMapEntry bin = tbl[hash & (tbl.length - 1)];
-
-            return bin != null ? bin : null;
-        }
-
-        /**
-         * @return Segment header for read operation.
-         */
-        private SegmentHeader headerForRead() {
-            while (true) {
-                SegmentHeader hdr = this.hdr;
-
-                hdr.onReadStart();
-
-                // Check if 2 rehashes didn't happen in between.
-                if (hdr == this.hdr)
-                    return hdr;
-                else
-                    hdr.onReadEnd();
-            }
+        @Nullable HashEntry getFirst(HashEntry[] tbl, int hash) {
+            return tbl[hash & (tbl.length - 1)];
         }
 
         /**
@@ -876,25 +745,18 @@ public class GridCacheConcurrentMap {
          * @return Value.
          */
         @Nullable GridCacheMapEntry get(Object key, int hash) {
-            SegmentHeader hdr = headerForRead();
-
-            try {
-                if (hdr.size() != 0) {
-                    GridCacheMapEntry e = getFirst(hdr.table(), hash);
+            if (size != 0) {
+                HashEntry e = getFirst(tbl, hash);
 
-                    while (e != null) {
-                        if (e.hash() == hash && key.equals(e.key()))
-                            return e;
+                while (e != null) {
+                    if (e.mapEntry.hash() == hash && key.equals(e.mapEntry.key()))
+                        return e.mapEntry;
 
-                        e = e.next(hdr.id());
-                    }
+                    e = e.next;
                 }
-
-                return null;
-            }
-            finally {
-                hdr.onReadEnd();
             }
+
+            return null;
         }
 
         /**
@@ -903,25 +765,18 @@ public class GridCacheConcurrentMap {
          * @return {@code True} if segment contains value.
          */
         boolean containsKey(Object key, int hash) {
-            SegmentHeader hdr = headerForRead();
-
-            try {
-                if (hdr.size() != 0) {
-                    GridCacheMapEntry e = getFirst(hdr.table(), hash);
+            if (size != 0) {
+                HashEntry e = getFirst(tbl, hash);
 
-                    while (e != null) {
-                        if (e.hash() == hash && key.equals(e.key))
-                            return true;
+                while (e != null) {
+                    if (e.mapEntry.hash() == hash && key.equals(e.mapEntry.key()))
+                        return true;
 
-                        e = e.next(hdr.id());
-                    }
+                    e = e.next;
                 }
-
-                return false;
-            }
-            finally {
-                hdr.onReadEnd();
             }
+
+            return false;
         }
 
         /**
@@ -953,58 +808,48 @@ public class GridCacheConcurrentMap {
         @SuppressWarnings({"unchecked", "SynchronizationOnLocalVariableOrMethodParameter"})
         private GridCacheMapEntry put0(KeyCacheObject key, int hash, CacheObject val, AffinityTopologyVersion topVer) {
             try {
-                SegmentHeader hdr = this.hdr;
-
-                int c = hdr.size();
+                int c = size;
 
-                if (c++ > threshold) {// Ensure capacity.
+                if (c++ > threshold) // Ensure capacity.
                     rehash();
 
-                    hdr = this.hdr;
-                }
-
-                int hdrId = hdr.id();
-
-                GridCacheMapEntry[] tab = hdr.table();
+                HashEntry[] tab = tbl;
 
                 int idx = hash & (tab.length - 1);
 
-                GridCacheMapEntry bin = tab[idx];
+                HashEntry bin = tab[idx];
 
-                GridCacheMapEntry e = bin;
+                HashEntry e = bin;
 
-                while (e != null && (e.hash() != hash || !key.equals(e.key)))
-                    e = e.next(hdrId);
+                while (e != null && (e.mapEntry.hash() != hash || !key.equals(e.mapEntry.key())))
+                    e = e.next;
 
                 GridCacheMapEntry retVal;
 
                 if (e != null) {
-                    retVal = e;
+                    retVal = e.mapEntry;
 
-                    e.rawPut(val, 0);
+                    retVal.rawPut(val, 0);
                 }
                 else {
-                    GridCacheMapEntry next = bin != null ? bin : null;
+                    HashEntry next = bin != null ? bin : null;
 
-                    GridCacheMapEntry newRoot = factory.create(ctx, topVer, key, hash, val, next, hdr.id());
+                    GridCacheMapEntry newEntry = factory.create(ctx, topVer, key, hash, val);
 
-                    // Avoiding delete (decrement) before creation (increment).
-                    synchronized (newRoot) {
-                        tab[idx] = newRoot;
+                    tab[idx] = new HashEntry(newEntry, next);
 
-                        retVal = newRoot;
+                    retVal = newEntry;
 
-                        // Modify counters.
-                        if (!retVal.isInternal()) {
-                            mapPubSize.increment();
+                    // Modify counters.
+                    if (!retVal.isInternal()) {
+                        mapPubSize.increment();
 
-                            pubSize.increment();
-                        }
+                        pubSize.increment();
                     }
 
                     mapSize.increment();
 
-                    hdr.size(c);
+                    size = c;
                 }
 
                 return retVal;
@@ -1029,20 +874,16 @@ public class GridCacheConcurrentMap {
             int hash,
             @Nullable CacheObject val,
             AffinityTopologyVersion topVer,
-            boolean create)
-        {
+            boolean create
+        ) {
             lock();
 
             try {
-                SegmentHeader hdr = this.hdr;
-
-                int hdrId = hdr.id();
-
-                GridCacheMapEntry[] tab = hdr.table();
+                HashEntry[] tab = tbl;
 
                 int idx = hash & (tab.length - 1);
 
-                GridCacheMapEntry bin = tab[idx];
+                HashEntry bin = tab[idx];
 
                 GridCacheMapEntry cur = null;
                 GridCacheMapEntry created = null;
@@ -1055,20 +896,20 @@ public class GridCacheConcurrentMap {
                     return new GridTriple<>(cur, created, doomed);
                 }
 
-                GridCacheMapEntry e = bin;
+                HashEntry e = bin;
 
-                while (e != null && (e.hash() != hash || !key.equals(e.key)))
-                    e = e.next(hdrId);
+                while (e != null && (e.mapEntry.hash() != hash || !key.equals(e.mapEntry.key())))
+                    e = e.next;
 
                 if (e != null) {
-                    if (e.obsolete()) {
+                    if (e.mapEntry.obsolete()) {
                         doomed = remove(key, hash, null);
 
                         if (create)
                             cur = created = put0(key, hash, val, topVer);
                     }
                     else
-                        cur = e;
+                        cur = e.mapEntry;
                 }
                 else if (create)
                     cur = created = put0(key, hash, val, topVer);
@@ -1085,14 +926,7 @@ public class GridCacheConcurrentMap {
          */
         @SuppressWarnings("unchecked")
         void rehash() {
-            SegmentHeader oldHdr = hdr;
-
-            if (oldHdr.previous() != null && oldHdr.previous().hasReads())
-                return; // Wait for previous header to free up.
-
-            int oldId = hdr.id();
-
-            GridCacheMapEntry[] oldTbl = oldHdr.table();
+            HashEntry[] oldTbl = tbl;
 
             int oldCap = oldTbl.length;
 
@@ -1112,32 +946,51 @@ public class GridCacheConcurrentMap {
              * reader thread that may be in the midst of traversing table
              * right now.
              */
-            SegmentHeader newHdr = new SegmentHeader(oldCap << 1, oldId + 1, oldHdr);
+            HashEntry[] newTbl = new HashEntry[oldCap << 1];
+
+            threshold = (int)(newTbl.length * loadFactor);
 
-            oldHdr.next(newHdr); // Link.
+            int sizeMask = newTbl.length - 1;
 
-            newHdr.size(oldHdr.size());
+            for (int i = 0; i < oldCap ; i++) {
+                HashEntry e = oldTbl[i];
 
-            GridCacheMapEntry[] newTbl = newHdr.table();
+                if (e != null) {
+                    HashEntry next = e.next;
 
-            threshold = (int)(newTbl.length * loadFactor);
+                    int idx = e.mapEntry.hash() & sizeMask;
 
-            int sizeMask = newTbl.length - 1;
+                    if (next == null)   //  Single node on list
+                        newTbl[idx] = e;
+                    else { // Reuse consecutive sequence at same slot
+                        HashEntry lastRun = e;
 
-            for (GridCacheMapEntry bin1 : oldTbl) {
-                // Relink all nodes.
-                for (GridCacheMapEntry e = bin1; e != null; e = e.next(oldId)) {
-                    int idx = e.hash() & sizeMask;
+                        int lastIdx = idx;
 
-                    GridCacheMapEntry bin2 = newTbl[idx];
+                        for (HashEntry last = next; last != null; last = last.next) {
+                            int k = last.mapEntry.hash() & sizeMask;
 
-                    newTbl[idx] = e;
+                            if (k != lastIdx) {
+                                lastIdx = k;
+                                lastRun = last;
+                            }
+                        }
+
+                        newTbl[lastIdx] = lastRun;
+
+                        // Clone remaining nodes
+                        for (HashEntry p = e; p != lastRun; p = p.next) {
+                            int k = p.mapEntry.hash() & sizeMask;
+
+                            HashEntry n = newTbl[k];
 
-                    e.next(newHdr.id(), bin2);
+                            newTbl[k] = new HashEntry(p.mapEntry, n);
+                        }
+                    }
                 }
             }
 
-            hdr = newHdr;
+            tbl = newTbl;
 
             if (DEBUG)
                 checkSegmentConsistency();
@@ -1157,38 +1010,34 @@ public class GridCacheConcurrentMap {
             lock();
 
             try {
-                SegmentHeader hdr = this.hdr;
-
-                GridCacheMapEntry[] tbl = hdr.table();
-
                 int idx = hash & (tbl.length - 1);
 
-                GridCacheMapEntry bin = tbl[idx];
+                HashEntry bin = tbl[idx];
 
                 if (bin == null)
                     return null;
 
-                GridCacheMapEntry prev = null;
-                GridCacheMapEntry e = bin;
+                HashEntry prev = null;
+                HashEntry e = bin;
 
-                while (e != null && (e.hash() != hash || !key.equals(e.key))) {
+                while (e != null && (e.mapEntry.hash() != hash || !key.equals(e.mapEntry.key))) {
                     prev = e;
 
-                    e = e.next(hdr.id());
+                    e = e.next;
                 }
 
                 if (e != null) {
-                    if (filter != null && !filter.apply(e))
+                    if (filter != null && !filter.apply(e.mapEntry))
                         return null;
 
                     if (prev == null)
-                        tbl[idx] = e.next(hdr.id());
+                        tbl[idx] = e.next;
                     else
-                        prev.next(hdr.id(), e.next(hdr.id()));
+                        prev.next = e.next;
 
                     // Modify counters.
                     synchronized (e) {
-                        if (!e.isInternal() && !e.deleted()) {
+                        if (!e.mapEntry.isInternal() && !e.mapEntry.deleted()) {
                             mapPubSize.decrement();
 
                             pubSize.decrement();
@@ -1197,10 +1046,12 @@ public class GridCacheConcurrentMap {
 
                     mapSize.decrement();
 
-                    hdr.decrementSize();
+                    --size;
+
+                    return e.mapEntry;
                 }
 
-                return e;
+                return null;
             }
             finally {
                 if (DEBUG)
@@ -1214,7 +1065,7 @@ public class GridCacheConcurrentMap {
          * @return Entries count within segment.
          */
         int size() {
-            return hdr.size();
+            return size;
         }
 
         /**
@@ -1242,283 +1093,97 @@ public class GridCacheConcurrentMap {
          * @return Random cache map entry from this segment.
          */
         @Nullable GridCacheMapEntry randomEntry() {
-            SegmentHeader hdr = headerForRead();
+            if (size == 0)
+                return null;
 
-            try {
-                GridCacheMapEntry[] tbl = hdr.table();
+            HashEntry[] tbl = this.tbl;
 
-                Collection<GridCacheMapEntry> entries = new ArrayList<>(3);
+            Collection<GridCacheMapEntry> entries = new ArrayList<>(3);
 
-                int pubCnt = 0;
+            int pubCnt = 0;
 
-                int start = RAND.nextInt(tbl.length);
+            int start = RAND.nextInt(tbl.length);
 
+            outerLoop:
+            {
                 for (int i = start; i < start + tbl.length; i++) {
-                    GridCacheMapEntry first = tbl[i % tbl.length];
+                    HashEntry first = tbl[i & (tbl.length - 1)];
 
                     if (first == null)
                         continue;
 
-                    entries.add(first);
-
-                    for (GridCacheMapEntry e = first; e != null; e = e.next(hdr.id()))
-                        if (!e.isInternal())
+                    for (HashEntry e = first; e != null; e = e.next) {
+                        if (!e.mapEntry.isInternal())
                             pubCnt++;
 
-                    if (entries.size() == 3)
-                        break;
+                        entries.add(e.mapEntry);
+
+                        if (entries.size() == 3)
+                            break outerLoop;
+
+                    }
                 }
+            }
 
-                if (entries.isEmpty())
-                    return null;
+            if (entries.isEmpty())
+                return null;
 
-                if (pubCnt == 0)
-                    return null;
+            if (pubCnt == 0)
+                return null;
 
-                // Desired and current indexes.
-                int idx = RAND.nextInt(pubCnt);
+            // Desired and current indexes.
+            int idx = RAND.nextInt(pubCnt);
 
-                int i = 0;
+            int i = 0;
 
-                GridCacheMapEntry retVal = null;
+            GridCacheMapEntry retVal = null;
 
-                for (GridCacheMapEntry e : entries) {
-                    for (; e != null; e = e.next(hdr.id())) {
-                        if (!(e.key instanceof GridCacheInternal)) {
-                            // In case desired entry was deleted, we return the closest one from left.
-                            retVal = e;
+            for (GridCacheMapEntry e : entries) {
+                if (!(e.key instanceof GridCacheInternal)) {
+                    // In case desired entry was deleted, we return the closest one from left.
+                    retVal = e;
 
-                            if (idx == i++)
-                                break;
-                        }
-                    }
+                    if (idx == i++)
+                        break;
                 }
-
-                return retVal;
-            }
-            finally {
-                hdr.onReadEnd();
             }
+
+            return retVal;
         }
 
         /**
          *
          */
         void checkSegmentConsistency() {
-            SegmentHeader hdr = this.hdr;
-
-            GridCacheMapEntry[] tbl = hdr.table();
+            HashEntry[] tbl = this.tbl;
 
             int cnt = 0;
             int pubCnt = 0;
 
-            for (GridCacheMapEntry b : tbl) {
+            for (HashEntry b : tbl) {
                 if (b != null) {
-                    GridCacheMapEntry e = b;
+                    HashEntry e = b;
 
                     assert e != null;
 
                     while (e != null) {
                         cnt++;
 
-                        if (!(e.key instanceof GridCacheInternal))
+                        if (!(e.mapEntry.key instanceof GridCacheInternal))
                             pubCnt++;
 
-                        e = e.next(hdr.id());
+                        e = e.next;
                     }
                 }
             }
 
-            assert cnt == hdr.size() : "Entry count and header size mismatch [cnt=" + cnt + ", hdrSize=" +
-                hdr.size() + ", segment=" + this + ", hdrId=" + hdr.id() + ']';
+            assert cnt == size : "Entry count and header size mismatch [cnt=" + cnt + ", hdrSize=" +
+                size + ", segment=" + this + ']';
             assert pubCnt == pubSize.intValue();
         }
     }
 
     /**
-     * Segment header.
-     */
-    private static class SegmentHeader {
-        /** Entry table. */
-        private final GridCacheMapEntry[] tbl;
-
-        /** Id for rehash. */
-        private final int id;
-
-        /** Reads. */
-        private final LongAdder8 reads = new LongAdder8();
-
-        /** */
-        private volatile SegmentHeader prev;
-
-        /** */
-        private volatile SegmentHeader next;
-
-        /** The number of elements in this segment's region. */
-        private volatile int size;
-
-        /** Cleaned flag. */
-        private final AtomicBoolean cleaned = new AtomicBoolean();
-
-        /**
-         * Constructs new segment header. New header is created initially and then
-         * every time during rehash operation.
-         *
-         * @param size Size of the table.
-         * @param id ID.
-         * @param prev Previous header.
-         */
-        @SuppressWarnings("unchecked")
-        private SegmentHeader(int size, int id, @Nullable SegmentHeader prev) {
-            tbl = new GridCacheMapEntry[size];
-
-            assert id >= 0;
-
-            this.id = id;
-            this.prev = prev;
-        }
-
-        /**
-         * Increment reads.
-         */
-        void onReadStart() {
-            reads.increment();
-        }
-
-        /**
-         * Decrement reads.
-         */
-        void onReadEnd() {
-            reads.decrement();
-
-            checkClean();
-        }
-
-        /**
-         * Cleans stale links if needed.
-         */
-        void checkClean() {
-            // Check if rehashing didn't occur for the next segment.
-            if (next != null && next.next() == null) {
-                long leftReads = reads.sum();
-
-                assert leftReads >= 0;
-
-                // Clean up.
-                if (leftReads == 0 && cleaned.compareAndSet(false, true)) {
-                    for (GridCacheMapEntry bin : tbl) {
-                        if (bin != null) {
-                            for (GridCacheMapEntry e = bin; e != null; ) {
-                                GridCacheMapEntry next = e.next(id);
-
-                                e.next(id, null); // Unlink.
-
-                                e = next;
-                            }
-                        }
-                    }
-                }
-            }
-        }
-
-        /**
-         * @return {@code True} if has reads.
-         */
-        boolean hasReads() {
-            return reads.sum() > 0;
-        }
-
-        /**
-         * @return Header ID.
-         */
-        int id() {
-            return id;
-        }
-
-        /**
-         * @return Table.
-         */
-        GridCacheMapEntry[] table() {
-            return tbl;
-        }
-
-        /**
-         * @return Table length.
-         */
-        int length() {
-            return tbl.length;
-        }
-
-        /**
-         * @return Next header.
-         */
-        SegmentHeader next() {
-            return next;
-        }
-
-        /**
-         * @param next Next header.
-         */
-        void next(SegmentHeader next) {
-            this.next = next;
-        }
-
-        /**
-         * @return Previous header.
-         */
-        SegmentHeader previous() {
-            return prev;
-        }
-
-        /**
-         * @param prev Previous header.
-         */
-        void previous(SegmentHeader prev) {
-            this.prev = prev;
-        }
-
-        /**
-         * @return New size.
-         */
-        int decrementSize() {
-            return --size;
-        }
-
-        /**
-         * @return Size.
-         */
-        int size() {
-            return size;
-        }
-
-        /**
-         * @param size Size.
-         */
-        void size(int size) {
-            this.size = size;
-        }
-    }
-
-    /**
-     * Phantom segment header to be used in iterators.
-     */
-    private static class WeakIterator<K, V> extends WeakReference<Iterator0<K, V>> {
-        /**
-         * Creates a new phantom reference that refers to the given segment header
-         * and is registered with the given queue.
-         *
-         * @param ref Referred segment header.
-         * @param q Reference queue.
-         */
-        WeakIterator(Iterator0<K, V> ref, ReferenceQueue<Iterator0> q) {
-            super(ref, q);
-
-            assert ref != null;
-            assert q != null;
-        }
-    }
-
-    /**
      * Iterator over {@link GridCacheEntryEx} elements.
      *
      * @param <K> Key type.
@@ -1534,17 +1199,14 @@ public class GridCacheConcurrentMap {
         /** */
         private int nextTblIdx;
 
-        /** Segment header. */
-        private SegmentHeader curSegHdr;
-
         /** */
-        private GridCacheMapEntry[] curTbl;
+        private HashEntry[] curTbl;
 
         /** */
-        private GridCacheMapEntry nextEntry;
+        private HashEntry nextEntry;
 
         /** Next entry to return. */
-        private GridCacheMapEntry next;
+        private HashEntry next;
 
         /** Next value. */
         private V nextVal;
@@ -1556,7 +1218,7 @@ public class GridCacheConcurrentMap {
         private boolean isVal;
 
         /** Current entry. */
-        private GridCacheMapEntry cur;
+        private HashEntry cur;
 
         /** Iterator filter. */
         private CacheEntryPredicate[] filter;
@@ -1567,9 +1229,6 @@ public class GridCacheConcurrentMap {
         /** Cache context. */
         private GridCacheContext<K, V> ctx;
 
-        /** Soft reference. */
-        private final WeakIterator<K, V> weakRef;
-
         /** Mod. */
         private int id;
 
@@ -1580,7 +1239,7 @@ public class GridCacheConcurrentMap {
          * Empty constructor required for {@link Externalizable}.
          */
         public Iterator0() {
-            weakRef = null;
+            // No-op.
         }
 
         /**
@@ -1605,12 +1264,7 @@ public class GridCacheConcurrentMap {
             nextSegIdx = map.segs.length - 1;
             nextTblIdx = -1;
 
-            weakRef = new WeakIterator<>(this, map.iteratorQueue());
-
             advance();
-
-            if (curSegHdr != null)
-               map.addWeakIterator(weakRef, curSegHdr); // Keep pointer to soft reference.
         }
 
         /**
@@ -1622,7 +1276,7 @@ public class GridCacheConcurrentMap {
                 return;
 
             while (nextTblIdx >= 0) {
-                GridCacheMapEntry bucket = curTbl[nextTblIdx--];
+                HashEntry bucket = curTbl[nextTblIdx--];
 
                 if (bucket != null && advanceInBucket(bucket, false))
                     return;
@@ -1633,20 +1287,11 @@ public class GridCacheConcurrentMap {
 
                 GridCacheConcurrentMap.Segment seg = map.segs[nextSegIdx0];
 
-                if (seg.size() != 0 && (id == -1 || nextSegIdx0 % totalCnt == id)) {
-                    if (curSegHdr != null)
-                        curSegHdr.onReadEnd();
-
-                    curSegHdr = seg.headerForRead();
-
-                    assert curSegHdr != null;
-
-                    map.addWeakIterator(weakRef, curSegHdr);
-
-                    curTbl = curSegHdr.table();
+                if (seg.size != 0 && (id == -1 || nextSegIdx0 % totalCnt == id)) {
+                    curTbl = seg.tbl;
 
                     for (int j = curTbl.length - 1; j >= 0; --j) {
-                        GridCacheMapEntry bucket = curTbl[j];
+                        HashEntry bucket = curTbl[j];
 
                         if (bucket != null && advanceInBucket(bucket, false)) {
                             nextTblIdx = j - 1;
@@ -1664,7 +1309,7 @@ public class GridCacheConcurrentMap {
          * @return {@code True} if advance succeeded.
          */
         @SuppressWarnings( {"unchecked"})
-        private boolean advanceInBucket(@Nullable GridCacheMapEntry e, boolean skipFirst) {
+        private boolean advanceInBucket(@Nullable HashEntry e, boolean skipFirst) {
             if (e == null)
                 return false;
 
@@ -1675,11 +1320,11 @@ public class GridCacheConcurrentMap {
                     next = nextEntry;
 
                     // Check if entry is visitable first before doing projection-aware peek.
-                    if (!next.visitable(filter))
+                    if (!next.mapEntry.visitable(filter))
                         continue;
 
                     if (isVal) {
-                        nextVal = next.<K, V>wrap().getValue();
+                        nextVal = next.mapEntry.<K, V>wrap().getValue();
 
                         if (nextVal == null)
                             continue;
@@ -1691,9 +1336,7 @@ public class GridCacheConcurrentMap {
                 // Perform checks in any case.
                 skipFirst = false;
             }
-            while ((nextEntry = nextEntry.next(curSegHdr.id())) != null);
-
-            assert nextEntry == null;
+            while ((nextEntry = nextEntry.next) != null);
 
             next = null;
             nextVal = null;
@@ -1703,17 +1346,7 @@ public class GridCacheConcurrentMap {
 
         /** {@inheritDoc} */
         @Override public boolean hasNext() {
-            boolean hasNext = next != null && (!isVal || nextVal != null);
-
-            if (!hasNext && curSegHdr != null) {
-                curSegHdr.onReadEnd();
-
-                weakRef.clear(); // Do not enqueue.
-
-                map.removeWeakIterator(weakRef); // Remove hard pointer.
-            }
-
-            return hasNext;
+            return next != null && (!isVal || nextVal != null);
         }
 
         /**
@@ -1726,7 +1359,7 @@ public class GridCacheConcurrentMap {
         /** {@inheritDoc} */
         @SuppressWarnings({"unchecked"})
         @Override public GridCacheEntryEx next() {
-            GridCacheMapEntry e = next;
+            HashEntry e = next;
             V v = nextVal;
 
             if (e == null)
@@ -1737,7 +1370,7 @@ public class GridCacheConcurrentMap {
             cur = e;
             curVal = v;
 
-            return cur;
+            return cur.mapEntry;
         }
 
         /** {@inheritDoc} */
@@ -1745,13 +1378,13 @@ public class GridCacheConcurrentMap {
             if (cur == null)
                 throw new IllegalStateException();
 
-            GridCacheMapEntry e = cur;
+            HashEntry e = cur;
 
             cur = null;
             curVal = null;
 
             try {
-                ((IgniteKernal)ctx.grid()).getCache(ctx.name()).getAndRemove(e.key());
+                ((IgniteKernal)ctx.grid()).getCache(ctx.name()).getAndRemove(e.mapEntry.key);
             }
             catch (IgniteCheckedException ex) {
                 throw new IgniteException(ex);
@@ -2383,4 +2016,33 @@ public class GridCacheConcurrentMap {
             set = (Set0<K, V>)in.readObject();
         }
     }
+
+    /**
+     *
+     */
+    private static class HashEntry {
+        /** */
+        private final GridCacheMapEntry mapEntry;
+
+        /** */
+        @GridToStringExclude
+        private volatile HashEntry next;
+
+        /**
+         * @param mapEntry Entry.
+         * @param next Next.
+         */
+        private HashEntry(
+            GridCacheMapEntry mapEntry,
+            HashEntry next
+        ) {
+            this.mapEntry = mapEntry;
+            this.next = next;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(HashEntry.class, this, super.toString());
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/3baf4d16/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 aa06a24..ae0b412 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
@@ -136,14 +136,6 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
     @GridToStringInclude
     protected GridCacheVersion ver;
 
-    /** Next entry in the linked list. */
-    @GridToStringExclude
-    private volatile GridCacheMapEntry next0;
-
-    /** Next entry in the linked list. */
-    @GridToStringExclude
-    private volatile GridCacheMapEntry next1;
-
     /** Key hash code. */
     @GridToStringInclude
     private final int hash;
@@ -167,16 +159,13 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
      * @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.
      */
-    protected GridCacheMapEntry(GridCacheContext<?, ?> cctx,
+    protected GridCacheMapEntry(
+        GridCacheContext<?, ?> cctx,
         KeyCacheObject key,
         int hash,
-        CacheObject val,
-        GridCacheMapEntry next,
-        int hdrId)
-    {
+        CacheObject val
+    ) {
         if (log == null)
             log = U.logger(cctx.kernalContext(), logRef, GridCacheMapEntry.class);
 
@@ -194,8 +183,6 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
             value(val);
         }
 
-        next(hdrId, next);
-
         ver = cctx.versions().next();
 
         startVer = ver.order();
@@ -3009,29 +2996,6 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
         return hash;
     }
 
-    /**
-     * Gets next entry in bucket linked list within a hash map segment.
-     *
-     * @param segId Segment ID.
-     * @return Next entry.
-     */
-    GridCacheMapEntry next(int segId) {
-        return (segId & 1) == 0 ? next0 : next1;
-    }
-
-    /**
-     * Sets next entry in bucket linked list within a hash map segment.
-     *
-     * @param segId Segment ID.
-     * @param next Next entry.
-     */
-    void next(int segId, @Nullable GridCacheMapEntry next) {
-        if ((segId & 1) == 0)
-            next0 = next;
-        else
-            next1 = next;
-    }
-
     /** {@inheritDoc} */
     @Nullable @Override public CacheObject peek(boolean heap,
         boolean offheap,

http://git-wip-us.apache.org/repos/asf/ignite/blob/3baf4d16/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntryFactory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntryFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntryFactory.java
index 0f8dae3..4ee9385 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntryFactory.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntryFactory.java
@@ -18,7 +18,6 @@
 package org.apache.ignite.internal.processors.cache;
 
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
-import org.jetbrains.annotations.Nullable;
 
 /**
  * Factory for cache entries.
@@ -30,15 +29,13 @@ public interface GridCacheMapEntryFactory {
      * @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.
      * @return New cache entry.
      */
-    public GridCacheMapEntry create(GridCacheContext ctx,
+    public GridCacheMapEntry create(
+        GridCacheContext ctx,
         AffinityTopologyVersion topVer,
         KeyCacheObject key,
         int hash,
-        CacheObject val,
-        @Nullable GridCacheMapEntry next,
-        int hdrId);
-}
\ No newline at end of file
+        CacheObject val
+    );
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/3baf4d16/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheEntry.java
index f95aa87..2d1b02e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheEntry.java
@@ -22,7 +22,6 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
 import java.util.UUID;
-import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
@@ -54,17 +53,14 @@ public class GridDistributedCacheEntry extends GridCacheMapEntry {
      * @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 GridDistributedCacheEntry(GridCacheContext ctx,
+    public GridDistributedCacheEntry(
+        GridCacheContext ctx,
         KeyCacheObject key,
         int hash,
-        CacheObject val,
-        GridCacheMapEntry next,
-        int hdrId)
-    {
-        super(ctx, key, hash, val, next, hdrId);
+        CacheObject val
+    ) {
+        super(ctx, key, hash, val);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/3baf4d16/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 7648f10..ff8d315 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
@@ -178,18 +178,17 @@ public abstract class GridDhtCacheAdapter<K, V> extends GridDistributedCacheAdap
     @Override protected void init() {
         map.setEntryFactory(new GridCacheMapEntryFactory() {
             /** {@inheritDoc} */
-            @Override public GridCacheMapEntry create(GridCacheContext ctx,
+            @Override public GridCacheMapEntry create(
+                GridCacheContext ctx,
                 AffinityTopologyVersion topVer,
                 KeyCacheObject key,
                 int hash,
-                CacheObject val,
-                GridCacheMapEntry next,
-                int hdrId)
-            {
+                CacheObject val
+            ) {
                 if (ctx.useOffheapEntry())
-                    return new GridDhtOffHeapCacheEntry(ctx, topVer, key, hash, val, next, hdrId);
+                    return new GridDhtOffHeapCacheEntry(ctx, topVer, key, hash, val);
 
-                return new GridDhtCacheEntry(ctx, topVer, key, hash, val, next, hdrId);
+                return new GridDhtCacheEntry(ctx, topVer, key, hash, val);
             }
         });
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/3baf4d16/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheEntry.java
index b9207db..14e3d3e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheEntry.java
@@ -30,7 +30,6 @@ import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.GridCacheEntryRemovedException;
-import org.apache.ignite.internal.processors.cache.GridCacheMapEntry;
 import org.apache.ignite.internal.processors.cache.GridCacheMultiTxFuture;
 import org.apache.ignite.internal.processors.cache.GridCacheMvcc;
 import org.apache.ignite.internal.processors.cache.GridCacheMvccCandidate;
@@ -78,18 +77,15 @@ public class GridDhtCacheEntry extends GridDistributedCacheEntry {
      * @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 GridDhtCacheEntry(GridCacheContext ctx,
+    public GridDhtCacheEntry(
+        GridCacheContext ctx,
         AffinityTopologyVersion topVer,
         KeyCacheObject key,
         int hash,
-        CacheObject val,
-        GridCacheMapEntry next,
-        int hdrId)
-    {
-        super(ctx, key, hash, val, next, hdrId);
+        CacheObject val
+    ) {
+        super(ctx, key, hash, val);
 
         // Record this entry with partition.
         locPart = ctx.dht().topology().onAdded(topVer, this);

http://git-wip-us.apache.org/repos/asf/ignite/blob/3baf4d16/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 8cb3fa4..07272ae 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
@@ -20,7 +20,6 @@ package org.apache.ignite.internal.processors.cache.distributed.dht;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
-import org.apache.ignite.internal.processors.cache.GridCacheMapEntry;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 
 /**
@@ -36,17 +35,15 @@ public class GridDhtOffHeapCacheEntry extends GridDhtCacheEntry {
      * @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,
+    public GridDhtOffHeapCacheEntry(
+        GridCacheContext ctx,
         AffinityTopologyVersion topVer,
         KeyCacheObject key,
         int hash,
-        CacheObject val,
-        GridCacheMapEntry next,
-        int hdrId) {
-        super(ctx, topVer, key, hash, val, next, hdrId);
+        CacheObject val
+    ) {
+        super(ctx, topVer, key, hash, val);
     }
 
     /** {@inheritDoc} */
@@ -63,4 +60,4 @@ public class GridDhtOffHeapCacheEntry extends GridDhtCacheEntry {
     @Override protected void offHeapPointer(long valPtr) {
         this.valPtr = valPtr;
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/3baf4d16/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 c6b969d..274701f 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
@@ -91,8 +91,8 @@ public class GridNoStorageCacheMap extends GridCacheConcurrentMap {
     {
         if (create) {
             GridCacheMapEntry entry = ctx.useOffheapEntry() ?
-                new GridDhtOffHeapCacheEntry(ctx, topVer, key, hash(key.hashCode()), val, null, 0) :
-                new GridDhtCacheEntry(ctx, topVer, key, hash(key.hashCode()), val, null, 0);
+                new GridDhtOffHeapCacheEntry(ctx, topVer, key, hash(key.hashCode()), val) :
+                new GridDhtCacheEntry(ctx, topVer, key, hash(key.hashCode()), val);
 
             return new GridTriple<>(entry, null, null);
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/3baf4d16/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 c5ec258..d5e849e 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
@@ -179,18 +179,17 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
     @Override protected void init() {
         map.setEntryFactory(new GridCacheMapEntryFactory() {
             /** {@inheritDoc} */
-            @Override public GridCacheMapEntry create(GridCacheContext ctx,
+            @Override public GridCacheMapEntry create(
+                GridCacheContext ctx,
                 AffinityTopologyVersion topVer,
                 KeyCacheObject key,
                 int hash,
-                CacheObject val,
-                GridCacheMapEntry next,
-                int hdrId)
-            {
+                CacheObject val
+            ) {
                 if (ctx.useOffheapEntry())
-                    return new GridDhtAtomicOffHeapCacheEntry(ctx, topVer, key, hash, val, next, hdrId);
+                    return new GridDhtAtomicOffHeapCacheEntry(ctx, topVer, key, hash, val);
 
-                return new GridDhtAtomicCacheEntry(ctx, topVer, key, hash, val, next, hdrId);
+                return new GridDhtAtomicCacheEntry(ctx, topVer, key, hash, val);
             }
         });
 
@@ -2970,16 +2969,6 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
      *
      */
     private static class FinishedLockFuture extends GridFinishedFuture<Boolean> implements GridDhtFuture<Boolean> {
-        /** */
-        private static final long serialVersionUID = 0L;
-
-        /**
-         * Empty constructor required by {@link Externalizable}.
-         */
-        public FinishedLockFuture() {
-            // No-op.
-        }
-
         /**
          * @param err Error.
          */

http://git-wip-us.apache.org/repos/asf/ignite/blob/3baf4d16/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCacheEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCacheEntry.java
index 78870c6..3f014d5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCacheEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCacheEntry.java
@@ -20,7 +20,6 @@ package org.apache.ignite.internal.processors.cache.distributed.dht.atomic;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
-import org.apache.ignite.internal.processors.cache.GridCacheMapEntry;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheEntry;
 import org.apache.ignite.internal.util.typedef.internal.CU;
@@ -36,18 +35,15 @@ public class GridDhtAtomicCacheEntry extends GridDhtCacheEntry {
      * @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 GridDhtAtomicCacheEntry(GridCacheContext ctx,
+    public GridDhtAtomicCacheEntry(
+        GridCacheContext ctx,
         AffinityTopologyVersion topVer,
         KeyCacheObject key,
         int hash,
-        CacheObject val,
-        GridCacheMapEntry next,
-        int hdrId)
-    {
-        super(ctx, topVer, key, hash, val, next, hdrId);
+        CacheObject val
+    ) {
+        super(ctx, topVer, key, hash, val);
     }
 
     /** {@inheritDoc} */
@@ -59,4 +55,4 @@ public class GridDhtAtomicCacheEntry extends GridDhtCacheEntry {
     @Override public synchronized String toString() {
         return S.toString(GridDhtAtomicCacheEntry.class, this, super.toString());
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/3baf4d16/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 bd3dc10..85cfb80 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
@@ -20,7 +20,6 @@ package org.apache.ignite.internal.processors.cache.distributed.dht.atomic;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
-import org.apache.ignite.internal.processors.cache.GridCacheMapEntry;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 
 /**
@@ -36,17 +35,15 @@ public class GridDhtAtomicOffHeapCacheEntry extends GridDhtAtomicCacheEntry {
      * @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,
+    public GridDhtAtomicOffHeapCacheEntry(
+        GridCacheContext ctx,
         AffinityTopologyVersion topVer,
         KeyCacheObject key,
         int hash,
-        CacheObject val,
-        GridCacheMapEntry next,
-        int hdrId) {
-        super(ctx, topVer, key, hash, val, next, hdrId);
+        CacheObject val
+    ) {
+        super(ctx, topVer, key, hash, val);
     }
 
     /** {@inheritDoc} */
@@ -63,4 +60,4 @@ public class GridDhtAtomicOffHeapCacheEntry extends GridDhtAtomicCacheEntry {
     @Override protected void offHeapPointer(long valPtr) {
         this.valPtr = valPtr;
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/3baf4d16/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 b69b42c..19fefdb 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
@@ -119,18 +119,17 @@ public class GridDhtColocatedCache<K, V> extends GridDhtTransactionalCacheAdapte
     @Override protected void init() {
         map.setEntryFactory(new GridCacheMapEntryFactory() {
             /** {@inheritDoc} */
-            @Override public GridCacheMapEntry create(GridCacheContext ctx,
+            @Override public GridCacheMapEntry create(
+                GridCacheContext ctx,
                 AffinityTopologyVersion topVer,
                 KeyCacheObject key,
                 int hash,
-                CacheObject val,
-                GridCacheMapEntry next,
-                int hdrId)
-            {
+                CacheObject val
+            ) {
                 if (ctx.useOffheapEntry())
-                    return new GridDhtColocatedOffHeapCacheEntry(ctx, topVer, key, hash, val, next, hdrId);
+                    return new GridDhtColocatedOffHeapCacheEntry(ctx, topVer, key, hash, val);
 
-                return new GridDhtColocatedCacheEntry(ctx, topVer, key, hash, val, next, hdrId);
+                return new GridDhtColocatedCacheEntry(ctx, topVer, key, hash, val);
             }
         });
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/3baf4d16/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCacheEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCacheEntry.java
index f5cd961..cc71e11 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCacheEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCacheEntry.java
@@ -20,7 +20,6 @@ package org.apache.ignite.internal.processors.cache.distributed.dht.colocated;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
-import org.apache.ignite.internal.processors.cache.GridCacheMapEntry;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheEntry;
 import org.apache.ignite.internal.util.typedef.internal.S;
@@ -35,18 +34,15 @@ public class GridDhtColocatedCacheEntry extends GridDhtCacheEntry {
      * @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 GridDhtColocatedCacheEntry(GridCacheContext ctx,
+    public GridDhtColocatedCacheEntry(
+        GridCacheContext ctx,
         AffinityTopologyVersion topVer,
         KeyCacheObject key,
         int hash,
-        CacheObject val,
-        GridCacheMapEntry next,
-        int hdrId
+        CacheObject val
     ) {
-        super(ctx, topVer, key, hash, val, next, hdrId);
+        super(ctx, topVer, key, hash, val);
     }
 
     /** {@inheritDoc} */
@@ -58,4 +54,4 @@ public class GridDhtColocatedCacheEntry extends GridDhtCacheEntry {
     @Override public synchronized String toString() {
         return S.toString(GridDhtColocatedCacheEntry.class, this, super.toString());
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/3baf4d16/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 ce17474..ea368d2 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
@@ -20,7 +20,6 @@ package org.apache.ignite.internal.processors.cache.distributed.dht.colocated;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
-import org.apache.ignite.internal.processors.cache.GridCacheMapEntry;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 
 /**
@@ -36,17 +35,15 @@ public class GridDhtColocatedOffHeapCacheEntry extends GridDhtColocatedCacheEntr
      * @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,
+    public GridDhtColocatedOffHeapCacheEntry(
+        GridCacheContext ctx,
         AffinityTopologyVersion topVer,
         KeyCacheObject key,
         int hash,
-        CacheObject val,
-        GridCacheMapEntry next,
-        int hdrId) {
-        super(ctx, topVer, key, hash, val, next, hdrId);
+        CacheObject val
+    ) {
+        super(ctx, topVer, key, hash, val);
     }
 
     /** {@inheritDoc} */
@@ -63,4 +60,4 @@ public class GridDhtColocatedOffHeapCacheEntry extends GridDhtColocatedCacheEntr
     @Override protected void offHeapPointer(long valPtr) {
         this.valPtr = valPtr;
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/3baf4d16/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtDetachedCacheEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtDetachedCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtDetachedCacheEntry.java
index c08f956..c06f68b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtDetachedCacheEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtDetachedCacheEntry.java
@@ -41,7 +41,7 @@ public class GridDhtDetachedCacheEntry extends GridDistributedCacheEntry {
      */
     public GridDhtDetachedCacheEntry(GridCacheContext ctx, KeyCacheObject key, int hash, CacheObject val,
         GridCacheMapEntry next, int hdrId) {
-        super(ctx, key, hash, val, next, hdrId);
+        super(ctx, key, hash, val);
     }
 
     /**
@@ -97,4 +97,4 @@ public class GridDhtDetachedCacheEntry extends GridDistributedCacheEntry {
         // No-op for detached cache entry.
         return true;
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/3baf4d16/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 c92e4e8..27ef996 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
@@ -102,16 +102,14 @@ public abstract class GridNearCacheAdapter<K, V> extends GridDistributedCacheAda
                 AffinityTopologyVersion topVer,
                 KeyCacheObject key,
                 int hash,
-                CacheObject val,
-                GridCacheMapEntry next,
-                int hdrId
+                CacheObject val
             ) {
                 // Can't hold any locks here - this method is invoked when
                 // holding write-lock on the whole cache map.
                 if (ctx.useOffheapEntry())
-                    return new GridNearOffHeapCacheEntry(ctx, key, hash, val, next, hdrId);
+                    return new GridNearOffHeapCacheEntry(ctx, key, hash, val);
 
-                return new GridNearCacheEntry(ctx, key, hash, val, next, hdrId);
+                return new GridNearCacheEntry(ctx, key, hash, val);
             }
         });
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/3baf4d16/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheEntry.java
index afdc5f1..6520f3d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheEntry.java
@@ -25,7 +25,6 @@ import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.GridCacheEntryInfo;
 import org.apache.ignite.internal.processors.cache.GridCacheEntryRemovedException;
-import org.apache.ignite.internal.processors.cache.GridCacheMapEntry;
 import org.apache.ignite.internal.processors.cache.GridCacheMvcc;
 import org.apache.ignite.internal.processors.cache.GridCacheMvccCandidate;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
@@ -66,17 +65,14 @@ public class GridNearCacheEntry extends GridDistributedCacheEntry {
      * @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 GridNearCacheEntry(GridCacheContext ctx,
+    public GridNearCacheEntry(
+        GridCacheContext ctx,
         KeyCacheObject key,
         int hash,
-        CacheObject val,
-        GridCacheMapEntry next,
-        int hdrId)
-    {
-        super(ctx, key, hash, val, next, hdrId);
+        CacheObject val
+    ) {
+        super(ctx, key, hash, val);
 
         part = ctx.affinity().partition(key);
     }
@@ -719,4 +715,4 @@ public class GridNearCacheEntry extends GridDistributedCacheEntry {
     @Override public synchronized String toString() {
         return S.toString(GridNearCacheEntry.class, this, "super", super.toString());
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/3baf4d16/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 c3f3e02..1558f4c 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
@@ -19,7 +19,6 @@ package org.apache.ignite.internal.processors.cache.distributed.near;
 
 import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
-import org.apache.ignite.internal.processors.cache.GridCacheMapEntry;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 
 /**
@@ -34,16 +33,14 @@ public class GridNearOffHeapCacheEntry extends GridNearCacheEntry {
      * @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,
+    public GridNearOffHeapCacheEntry(
+        GridCacheContext ctx,
         KeyCacheObject key,
         int hash,
-        CacheObject val,
-        GridCacheMapEntry next,
-        int hdrId) {
-        super(ctx, key, hash, val, next, hdrId);
+        CacheObject val
+    ) {
+        super(ctx, key, hash, val);
     }
 
     /** {@inheritDoc} */
@@ -60,4 +57,4 @@ public class GridNearOffHeapCacheEntry extends GridNearCacheEntry {
     @Override protected void offHeapPointer(long valPtr) {
         this.valPtr = valPtr;
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/3baf4d16/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCache.java
index 821455a..4ce1f36 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCache.java
@@ -88,11 +88,9 @@ public class GridLocalCache<K, V> extends GridCacheAdapter<K, V> {
                 AffinityTopologyVersion topVer,
                 KeyCacheObject key,
                 int hash,
-                CacheObject val,
-                GridCacheMapEntry next,
-                int hdrId
+                CacheObject val
             ) {
-                return new GridLocalCacheEntry(ctx, key, hash, val, next, hdrId);
+                return new GridLocalCacheEntry(ctx, key, hash, val);
             }
         });
     }
@@ -236,4 +234,4 @@ public class GridLocalCache<K, V> extends GridCacheAdapter<K, V> {
                 log().debug("Explicitly removed future from map of futures: " + fut);
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/3baf4d16/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 76bfc46..6ddd2e5 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
@@ -45,17 +45,14 @@ public class GridLocalCacheEntry extends GridCacheMapEntry {
      * @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 GridLocalCacheEntry(GridCacheContext ctx,
+    public GridLocalCacheEntry(
+        GridCacheContext ctx,
         KeyCacheObject key,
         int hash,
-        CacheObject val,
-        GridCacheMapEntry next,
-        int hdrId)
-    {
-        super(ctx, key, hash, val, next, hdrId);
+        CacheObject val
+    ) {
+        super(ctx, key, hash, val);
     }
 
     /** {@inheritDoc} */
@@ -440,4 +437,4 @@ public class GridLocalCacheEntry extends GridCacheMapEntry {
     @Override public synchronized String toString() {
         return S.toString(GridLocalCacheEntry.class, this, super.toString());
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/3baf4d16/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
index 0afd6bc..dda5ed2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
@@ -119,11 +119,9 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
                 AffinityTopologyVersion topVer,
                 KeyCacheObject key,
                 int hash,
-                CacheObject val,
-                @Nullable GridCacheMapEntry next,
-                int hdrId
+                CacheObject val
             ) {
-                return new GridLocalCacheEntry(ctx, key, hash, val, next, hdrId);
+                return new GridLocalCacheEntry(ctx, key, hash, val);
             }
         });
     }
@@ -1647,4 +1645,4 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
     @Override public void onDeferredDelete(GridCacheEntryEx entry, GridCacheVersion ver) {
         assert false : "Should not be called";
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/3baf4d16/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
index 121cd46..e74b3f0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
@@ -8208,15 +8208,15 @@ public abstract class IgniteUtils {
      * @return Hash value.
      */
     public static int hash(int h) {
-        // Apply base step of MurmurHash; see http://code.google.com/p/smhasher/
-        // Despite two multiplies, this is often faster than others
-        // with comparable bit-spread properties.
-        h ^= h >>> 16;
-        h *= 0x85ebca6b;
-        h ^= h >>> 13;
-        h *= 0xc2b2ae35;
-
-        return (h >>> 16) ^ h;
+        // Spread bits to regularize both segment and index locations,
+        // using variant of single-word Wang/Jenkins hash.
+        h += (h <<  15) ^ 0xffffcd7d;
+        h ^= (h >>> 10);
+        h += (h <<   3);
+        h ^= (h >>>  6);
+        h += (h <<   2) + (h << 14);
+
+        return h ^ (h >>> 16);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/3baf4d16/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAlwaysEvictionPolicy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAlwaysEvictionPolicy.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAlwaysEvictionPolicy.java
index 92e1bab..e1b2fa3 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAlwaysEvictionPolicy.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAlwaysEvictionPolicy.java
@@ -45,4 +45,4 @@ public class GridCacheAlwaysEvictionPolicy<K, V> implements EvictionPolicy<K, V>
     @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
         // No-op.
     }
-}
\ No newline at end of file
+}