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 2017/01/12 11:39:55 UTC

ignite git commit: Reduced size of KeySearchRow.

Repository: ignite
Updated Branches:
  refs/heads/ignite-gg-11810 c1d5081f9 -> 33616e06a


Reduced size of KeySearchRow.


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

Branch: refs/heads/ignite-gg-11810
Commit: 33616e06a8d018bee0592dbf08a1027b6d120ed8
Parents: c1d5081
Author: sboikov <sb...@gridgain.com>
Authored: Thu Jan 12 14:39:48 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Jan 12 14:39:48 2017 +0300

----------------------------------------------------------------------
 .../cache/IgniteCacheOffheapManagerImpl.java    | 234 ++++++++++++++-----
 .../processors/cache/database/CacheDataRow.java |  13 +-
 .../cache/database/CacheDataRowAdapter.java     |  77 +-----
 .../cache/database/CacheSearchRow.java          |  40 ++++
 .../database/FreeListImplSelfTest.java          |   5 +
 .../processors/query/h2/opt/GridH2Row.java      |   5 +
 6 files changed, 230 insertions(+), 144 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/33616e06/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
index e9a7b8e..3a68b81 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
@@ -17,13 +17,11 @@
 
 package org.apache.ignite.internal.processors.cache;
 
-import java.nio.ByteBuffer;
 import java.util.Collections;
 import java.util.Iterator;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicLong;
 import javax.cache.Cache;
 import org.apache.ignite.IgniteCheckedException;
@@ -31,12 +29,14 @@ import org.apache.ignite.IgniteException;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.internal.NodeStoppingException;
 import org.apache.ignite.internal.pagemem.FullPageId;
+import org.apache.ignite.internal.pagemem.Page;
 import org.apache.ignite.internal.pagemem.PageIdUtils;
 import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.pagemem.PageUtils;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.database.CacheDataRow;
 import org.apache.ignite.internal.processors.cache.database.CacheDataRowAdapter;
+import org.apache.ignite.internal.processors.cache.database.CacheSearchRow;
 import org.apache.ignite.internal.processors.cache.database.IgniteCacheDatabaseSharedManager;
 import org.apache.ignite.internal.processors.cache.database.RootPage;
 import org.apache.ignite.internal.processors.cache.database.RowStore;
@@ -45,6 +45,8 @@ import org.apache.ignite.internal.processors.cache.database.tree.BPlusTree;
 import org.apache.ignite.internal.processors.cache.database.tree.io.BPlusIO;
 import org.apache.ignite.internal.processors.cache.database.tree.io.BPlusInnerIO;
 import org.apache.ignite.internal.processors.cache.database.tree.io.BPlusLeafIO;
+import org.apache.ignite.internal.processors.cache.database.tree.io.DataPageIO;
+import org.apache.ignite.internal.processors.cache.database.tree.io.DataPagePayload;
 import org.apache.ignite.internal.processors.cache.database.tree.io.IOVersions;
 import org.apache.ignite.internal.processors.cache.database.tree.reuse.ReuseList;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtInvalidPartitionException;
@@ -70,6 +72,8 @@ import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_IDX;
 import static org.apache.ignite.internal.pagemem.PageIdAllocator.INDEX_PARTITION;
+import static org.apache.ignite.internal.pagemem.PageIdUtils.itemId;
+import static org.apache.ignite.internal.pagemem.PageIdUtils.pageId;
 
 /**
  *
@@ -881,7 +885,7 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
             CacheObject val,
             GridCacheVersion ver,
             long expireTime) throws IgniteCheckedException {
-            DataRow dataRow = new DataRow(key.hashCode(), key, val, ver, p, expireTime);
+            DataRow dataRow = new DataRow(key, val, ver, p, expireTime);
 
             // Make sure value bytes initialized.
             key.valueBytes(cctx.cacheObjectContext());
@@ -895,7 +899,7 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
 
                 assert dataRow.link() != 0 : dataRow;
 
-                DataRow old = dataTree.put(dataRow);
+                CacheDataRow old = dataTree.put(dataRow);
 
                 if (old == null)
                     storageSize.incrementAndGet();
@@ -934,7 +938,7 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
                 throw new NodeStoppingException("Operation has been cancelled (node is stopping).");
 
             try {
-                DataRow dataRow = dataTree.remove(new KeySearchRow(key.hashCode(), key, 0));
+                CacheDataRow dataRow = dataTree.remove(new SearchRow(key));
 
                 CacheObject val = null;
                 GridCacheVersion ver = null;
@@ -971,7 +975,7 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
         /** {@inheritDoc} */
         @Override public CacheDataRow find(KeyCacheObject key)
             throws IgniteCheckedException {
-            return dataTree.findOne(new KeySearchRow(key.hashCode(), key, 0));
+            return dataTree.findOne(new SearchRow(key));
         }
 
         /** {@inheritDoc} */
@@ -982,14 +986,14 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
         /** {@inheritDoc} */
         @Override public GridCursor<? extends CacheDataRow> cursor(KeyCacheObject lower,
             KeyCacheObject upper) throws IgniteCheckedException {
-            KeySearchRow lowerRow = null;
-            KeySearchRow upperRow = null;
+            SearchRow lowerRow = null;
+            SearchRow upperRow = null;
 
             if (lower != null)
-                lowerRow = new KeySearchRow(lower.hashCode(), lower, 0);
+                lowerRow = new SearchRow(lower);
 
             if (upper != null)
-                upperRow = new KeySearchRow(upper.hashCode(), upper, 0);
+                upperRow = new SearchRow(upper);
 
             return dataTree.find(lowerRow, upperRow);
         }
@@ -1035,82 +1039,115 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
     /**
      *
      */
-    private class KeySearchRow extends CacheDataRowAdapter {
+    private static class SearchRow implements CacheSearchRow {
         /** */
-        protected int hash;
+        private final KeyCacheObject key;
+
+        /** */
+        private final int hash;
 
         /**
-         * @param hash Hash code.
          * @param key Key.
-         * @param link Link.
          */
-        KeySearchRow(int hash, KeyCacheObject key, long link) {
-            super(link);
-
+        SearchRow(KeyCacheObject key) {
             this.key = key;
-            this.hash = hash;
+
+            hash = key.hashCode();
         }
 
-        /**
-         * Init data.
-         *
-         * @param keyOnly Initialize only key.
-         */
-        protected final void initData(boolean keyOnly) {
-            if (key != null)
-                return;
+        /** {@inheritDoc} */
+        @Override public KeyCacheObject key() {
+            return key;
+        }
 
-            assert link() != 0;
+        /** {@inheritDoc} */
+        @Override public long link() {
+            throw new UnsupportedOperationException();
+        }
 
-            try {
-                initFromLink(cctx, keyOnly);
-            }
-            catch (IgniteCheckedException e) {
-                throw new IgniteException(e.getMessage(), e);
-            }
+        /** {@inheritDoc} */
+        @Override public int hash() {
+            return hash;
         }
+    }
+
+    /**
+     *
+     */
+    private static class LinkSearchRow implements CacheSearchRow {
+        /** */
+        private final int hash;
+
+        /** */
+        private final long link;
 
         /**
-         * @return Key.
+         * @param hash Key hash code.
+         * @param link Link.
          */
+        LinkSearchRow(int hash, long link) {
+            this.hash = hash;
+            this.link = link;
+        }
+
+        /** {@inheritDoc} */
         @Override public KeyCacheObject key() {
-            initData(true);
+            throw new UnsupportedOperationException();
+        }
 
-            return key;
+        /** {@inheritDoc} */
+        @Override public long link() {
+            return link;
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hash() {
+            return hash;
         }
     }
 
     /**
      *
      */
-    private class DataRow extends KeySearchRow {
+    private class DataRow extends CacheDataRowAdapter {
         /** */
         protected int part = -1;
 
+        /** */
+        protected int hash;
+
         /**
          * @param hash Hash code.
          * @param link Link.
          */
         DataRow(int hash, long link) {
-            super(hash, null, link);
+            super(link);
+
+            this.hash = hash;
 
             part = PageIdUtils.partId(link);
 
-            // We can not init data row lazily because underlying buffer can be concurrently cleared.
-            initData(false);
+            try {
+                // We can not init data row lazily because underlying buffer can be concurrently cleared.
+                initFromLink(cctx, false);
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException(e);
+            }
         }
 
         /**
-         * @param hash Hash code.
          * @param key Key.
          * @param val Value.
          * @param ver Version.
          * @param part Partition.
          * @param expireTime Expire time.
          */
-        DataRow(int hash, KeyCacheObject key, CacheObject val, GridCacheVersion ver, int part, long expireTime) {
-            super(hash, key, 0);
+        DataRow(KeyCacheObject key, CacheObject val, GridCacheVersion ver, int part, long expireTime) {
+            super(0);
 
+            this.hash = key.hashCode();
+            this.key = key;
             this.val = val;
             this.ver = ver;
             this.part = part;
@@ -1123,6 +1160,11 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
         }
 
         /** {@inheritDoc} */
+        @Override public int hash() {
+            return hash;
+        }
+
+        /** {@inheritDoc} */
         @Override public void link(long link) {
             this.link = link;
         }
@@ -1131,7 +1173,7 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
     /**
      *
      */
-    protected static class CacheDataTree extends BPlusTree<KeySearchRow, DataRow> {
+    protected static class CacheDataTree extends BPlusTree<CacheSearchRow, CacheDataRow> {
         /** */
         private final CacheDataRowStore rowStore;
 
@@ -1170,22 +1212,92 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
         }
 
         /** {@inheritDoc} */
-        @Override protected int compare(BPlusIO<KeySearchRow> io, long pageAddr, int idx, KeySearchRow row)
+        @Override protected int compare(BPlusIO<CacheSearchRow> io, long pageAddr, int idx, CacheSearchRow row)
             throws IgniteCheckedException {
             int hash = ((RowLinkIO)io).getHash(pageAddr, idx);
 
-            int cmp = Integer.compare(hash, row.hash);
+            int cmp = Integer.compare(hash, row.hash());
 
             if (cmp != 0)
                 return cmp;
 
             long link = ((RowLinkIO)io).getLink(pageAddr, idx);
 
-            return row.compareKey(cctx, link);
+            assert row.key() != null : row;
+
+            return compareKeys(row.key(), link);
+        }
+
+        /**
+         * @param key Key.
+         * @param link Link.
+         * @return Compare result.
+         * @throws IgniteCheckedException If failed.
+         */
+        private int compareKeys(KeyCacheObject key, final long link) throws IgniteCheckedException {
+            byte[] bytes = key.valueBytes(cctx.cacheObjectContext());
+
+            PageMemory pageMem = cctx.shared().database().pageMemory();
+
+            try (Page page = page(pageId(link))) {
+                long pageAddr = page.getForReadPointer(); // Non-empty data page must not be recycled.
+
+                assert pageAddr != 0L : link;
+
+                try {
+                    DataPageIO io = DataPageIO.VERSIONS.forPage(pageAddr);
+
+                    DataPagePayload data = io.readPayload(pageAddr,
+                        itemId(link),
+                        pageMem.pageSize());
+
+                    if (data.nextLink() == 0) {
+                        long addr = pageAddr + data.offset();
+
+                        int len = PageUtils.getInt(addr, 0);
+
+                        int size = Math.min(bytes.length, len);
+
+                        addr += 5; // Skip length and type byte.
+
+                        for (int i = 0; i < size; i++) {
+                            byte b1 = PageUtils.getByte(addr, i);
+                            byte b2 = bytes[i];
+
+                            if (b1 != b2)
+                                return b1 > b2 ? 1 : -1;
+                        }
+
+                        return Integer.compare(len, bytes.length);
+                    }
+                }
+                finally {
+                    page.releaseRead();
+                }
+            }
+
+            // TODO GG-11768.
+            CacheDataRowAdapter other = new CacheDataRowAdapter(link);
+            other.initFromLink(cctx, true);
+
+            byte[] bytes1 = other.key().valueBytes(cctx.cacheObjectContext());
+            byte[] bytes2 = key.valueBytes(cctx.cacheObjectContext());
+
+            int len = Math.min(bytes1.length, bytes2.length);
+
+            for (int i = 0; i < len; i++) {
+                byte b1 = bytes1[i];
+                byte b2 = bytes2[i];
+
+                if (b1 != b2)
+                    return b1 > b2 ? 1 : -1;
+            }
+
+            return Integer.compare(bytes1.length, bytes2.length);
         }
 
         /** {@inheritDoc} */
-        @Override protected DataRow getRow(BPlusIO<KeySearchRow> io, long pageAddr, int idx)
+        @Override protected CacheDataRow getRow(BPlusIO<CacheSearchRow> io, long pageAddr, int idx)
             throws IgniteCheckedException {
             int hash = ((RowLinkIO)io).getHash(pageAddr, idx);
             long link = ((RowLinkIO)io).getLink(pageAddr, idx);
@@ -1211,8 +1323,8 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
          * @param link Link.
          * @return Search row.
          */
-        private KeySearchRow keySearchRow(int hash, long link) {
-            return new KeySearchRow(hash, null, link);
+        private CacheSearchRow keySearchRow(int hash, long link) {
+            return new LinkSearchRow(hash, link);
         }
 
         /**
@@ -1220,7 +1332,7 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
          * @param link Link.
          * @return Data row.
          */
-        private DataRow dataRow(int hash, long link) {
+        private CacheDataRow dataRow(int hash, long link) {
             return new DataRow(hash, link);
         }
     }
@@ -1258,7 +1370,7 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
     /**
      *
      */
-    public static final class DataInnerIO extends BPlusInnerIO<KeySearchRow> implements RowLinkIO {
+    public static final class DataInnerIO extends BPlusInnerIO<CacheSearchRow> implements RowLinkIO {
         /** */
         public static final IOVersions<DataInnerIO> VERSIONS = new IOVersions<>(
             new DataInnerIO(1)
@@ -1272,14 +1384,14 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
         }
 
         /** {@inheritDoc} */
-        @Override public void storeByOffset(long pageAddr, int off, KeySearchRow row) {
+        @Override public void storeByOffset(long pageAddr, int off, CacheSearchRow row) {
             assert row.link() != 0;
 
-            store0(pageAddr, off, row.link(), row.hash);
+            store0(pageAddr, off, row.link(), row.hash());
         }
 
         /** {@inheritDoc} */
-        @Override public KeySearchRow getLookupRow(BPlusTree<KeySearchRow, ?> tree, long pageAddr, int idx) {
+        @Override public CacheSearchRow getLookupRow(BPlusTree<CacheSearchRow, ?> tree, long pageAddr, int idx) {
             int hash = getHash(pageAddr, idx);
             long link = getLink(pageAddr, idx);
 
@@ -1287,7 +1399,7 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
         }
 
         /** {@inheritDoc} */
-        @Override public void store(long dstPageAddr, int dstIdx, BPlusIO<KeySearchRow> srcIo, long srcPageAddr,
+        @Override public void store(long dstPageAddr, int dstIdx, BPlusIO<CacheSearchRow> srcIo, long srcPageAddr,
             int srcIdx) {
             int hash = ((RowLinkIO)srcIo).getHash(srcPageAddr, srcIdx);
             long link = ((RowLinkIO)srcIo).getLink(srcPageAddr, srcIdx);
@@ -1311,7 +1423,7 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
     /**
      *
      */
-    public static final class DataLeafIO extends BPlusLeafIO<KeySearchRow> implements RowLinkIO {
+    public static final class DataLeafIO extends BPlusLeafIO<CacheSearchRow> implements RowLinkIO {
         /** */
         public static final IOVersions<DataLeafIO> VERSIONS = new IOVersions<>(
             new DataLeafIO(1)
@@ -1325,20 +1437,20 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
         }
 
         /** {@inheritDoc} */
-        @Override public void storeByOffset(long pageAddr, int off, KeySearchRow row) {
+        @Override public void storeByOffset(long pageAddr, int off, CacheSearchRow row) {
             assert row.link() != 0;
 
-            store0(pageAddr, off, row.link(), row.hash);
+            store0(pageAddr, off, row.link(), row.hash());
         }
 
         /** {@inheritDoc} */
-        @Override public void store(long dstPageAddr, int dstIdx, BPlusIO<KeySearchRow> srcIo, long srcPageAddr,
+        @Override public void store(long dstPageAddr, int dstIdx, BPlusIO<CacheSearchRow> srcIo, long srcPageAddr,
             int srcIdx) {
             store0(dstPageAddr, offset(dstIdx), getLink(srcPageAddr, srcIdx), getHash(srcPageAddr, srcIdx));
         }
 
         /** {@inheritDoc} */
-        @Override public KeySearchRow getLookupRow(BPlusTree<KeySearchRow, ?> tree, long buf, int idx) {
+        @Override public CacheSearchRow getLookupRow(BPlusTree<CacheSearchRow, ?> tree, long buf, int idx) {
             int hash = getHash(buf, idx);
             long link = getLink(buf, idx);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/33616e06/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/CacheDataRow.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/CacheDataRow.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/CacheDataRow.java
index d4d7020..75ab8e4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/CacheDataRow.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/CacheDataRow.java
@@ -18,18 +18,12 @@
 package org.apache.ignite.internal.processors.cache.database;
 
 import org.apache.ignite.internal.processors.cache.CacheObject;
-import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 
 /**
  * Cache data row.
  */
-public interface CacheDataRow {
-    /**
-     * @return Cache key.
-     */
-    public KeyCacheObject key();
-
+public interface CacheDataRow extends CacheSearchRow {
     /**
      * @return Cache value.
      */
@@ -51,11 +45,6 @@ public interface CacheDataRow {
     public int partition();
 
     /**
-     * @return Link for this row.
-     */
-    public long link();
-
-    /**
      * @param link Link for this row.
      */
     public void link(long link);

http://git-wip-us.apache.org/repos/asf/ignite/blob/33616e06/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/CacheDataRowAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/CacheDataRowAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/CacheDataRowAdapter.java
index bb606cc..acb435e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/CacheDataRowAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/CacheDataRowAdapter.java
@@ -29,8 +29,8 @@ import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.IncompleteCacheObject;
 import org.apache.ignite.internal.processors.cache.IncompleteObject;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
-import org.apache.ignite.internal.processors.cache.database.tree.io.DataPageIO;
 import org.apache.ignite.internal.processors.cache.database.tree.io.CacheVersionIO;
+import org.apache.ignite.internal.processors.cache.database.tree.io.DataPageIO;
 import org.apache.ignite.internal.processors.cache.database.tree.io.DataPagePayload;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
@@ -73,76 +73,6 @@ public class CacheDataRowAdapter implements CacheDataRow {
     }
 
     /**
-     * Compare key with key of record, given by link.
-     *
-     * @param cctx Context.
-     * @param link Link to second record.
-     * @return compare result.
-     * @throws IgniteCheckedException if fails.
-     */
-    public int compareKey(GridCacheContext cctx, final long link) throws IgniteCheckedException {
-        byte[] bytes = key().valueBytes(cctx.cacheObjectContext());
-
-        PageMemory pageMem = cctx.shared().database().pageMemory();
-
-        try (Page page = page(pageId(link), cctx)) {
-            long pageAddr = page.getForReadPointer(); // Non-empty data page must not be recycled.
-
-            assert pageAddr != 0L : link;
-
-            try {
-                DataPageIO io = DataPageIO.VERSIONS.forPage(pageAddr);
-
-                DataPagePayload data = io.readPayload(pageAddr,
-                    itemId(link),
-                    pageMem.pageSize());
-
-                if (data.nextLink() == 0) {
-                    long addr = pageAddr + data.offset();
-
-                    int len = PageUtils.getInt(addr, 0);
-
-                    int size = Math.min(bytes.length, len);
-
-                    addr += 5; // Skip length and type byte.
-
-                    for (int i = 0; i < size; i++) {
-                        byte b1 = PageUtils.getByte(addr, i);
-                        byte b2 = bytes[i];
-
-                        if (b1 != b2)
-                            return b1 > b2 ? 1 : -1;
-                    }
-
-                    return Integer.compare(len, bytes.length);
-                }
-            }
-            finally {
-                page.releaseRead();
-            }
-        }
-
-        // TODO GG-11768.
-        CacheDataRowAdapter other = new CacheDataRowAdapter(link);
-        other.initFromLink(cctx, true);
-
-        byte[] bytes1 = other.key().valueBytes(cctx.cacheObjectContext());
-        byte[] bytes2 = key.valueBytes(cctx.cacheObjectContext());
-
-        int len = Math.min(bytes1.length, bytes2.length);
-
-        for (int i = 0; i < len; i++) {
-            byte b1 = bytes1[i];
-            byte b2 = bytes2[i];
-
-            if (b1 != b2)
-                return b1 > b2 ? 1 : -1;
-        }
-
-        return Integer.compare(bytes1.length, bytes2.length);
-    }
-
-    /**
      * Read row from data pages.
      *
      * @param cctx Cache context.
@@ -504,6 +434,11 @@ public class CacheDataRowAdapter implements CacheDataRow {
     }
 
     /** {@inheritDoc} */
+    @Override public int hash() {
+        throw new UnsupportedOperationException();
+    }
+
+    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(CacheDataRowAdapter.class, this, "link", U.hexLong(link));
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/33616e06/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/CacheSearchRow.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/CacheSearchRow.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/CacheSearchRow.java
new file mode 100644
index 0000000..d51cf0e
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/CacheSearchRow.java
@@ -0,0 +1,40 @@
+/*
+ * 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.database;
+
+import org.apache.ignite.internal.processors.cache.KeyCacheObject;
+
+/**
+ *
+ */
+public interface CacheSearchRow {
+    /**
+     * @return Cache key.
+     */
+    public KeyCacheObject key();
+
+    /**
+     * @return Link for this row.
+     */
+    public long link();
+
+    /**
+     * @return Key hash code.
+     */
+    public int hash();
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/33616e06/modules/core/src/test/java/org/apache/ignite/internal/processors/database/FreeListImplSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/FreeListImplSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/FreeListImplSelfTest.java
index 66987e2..d0d495e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/FreeListImplSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/FreeListImplSelfTest.java
@@ -396,6 +396,11 @@ public class FreeListImplSelfTest extends GridCommonAbstractTest {
         @Override public void link(long link) {
             this.link = link;
         }
+
+        /** {@inheritDoc} */
+        @Override public int hash() {
+            throw new UnsupportedOperationException();
+        }
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/33616e06/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Row.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Row.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Row.java
index ac48fcb..bdd43ef 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Row.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Row.java
@@ -164,4 +164,9 @@ public abstract class GridH2Row extends Row implements GridSearchRowPointer, Cac
     @Override public Value[] getValueList() {
         throw new UnsupportedOperationException();
     }
+
+    /** {@inheritDoc} */
+    @Override public int hash() {
+        throw new UnsupportedOperationException();
+    }
 }
\ No newline at end of file