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 2016/12/22 15:32:45 UTC

ignite git commit: ignite-gg-11810

Repository: ignite
Updated Branches:
  refs/heads/ignite-11810 8fcac6f29 -> 94f22a0c0


ignite-gg-11810


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

Branch: refs/heads/ignite-11810
Commit: 94f22a0c08971e3bb8ff13c2cc8e32f748f1c3e4
Parents: 8fcac6f
Author: sboikov <sb...@gridgain.com>
Authored: Thu Dec 22 11:06:51 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Dec 22 11:19:27 2016 +0300

----------------------------------------------------------------------
 .../cache/IgniteCacheOffheapManagerImpl.java    |  99 ++++++-------
 .../cache/database/freelist/FreeListImpl.java   |   6 +-
 .../cache/database/tree/BPlusTree.java          | 147 +++++++++----------
 .../cache/database/tree/io/DataPageIO.java      | 129 ++++++++--------
 .../database/tree/reuse/ReuseListImpl.java      |   2 +-
 .../processors/query/h2/database/H2Tree.java    |   2 +-
 6 files changed, 189 insertions(+), 196 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/94f22a0c/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 02a068d..4b1d464 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
@@ -31,6 +31,7 @@ import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.internal.pagemem.FullPageId;
 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;
@@ -1085,7 +1086,7 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
         }
 
         /** {@inheritDoc} */
-        @Override protected int compare(BPlusIO<KeySearchRow> io, ByteBuffer buf, int idx, KeySearchRow row)
+        @Override protected int compare(BPlusIO<KeySearchRow> io, long buf, int idx, KeySearchRow row)
             throws IgniteCheckedException {
             int hash = ((RowLinkIO)io).getHash(buf, idx);
 
@@ -1100,7 +1101,7 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
         }
 
         /** {@inheritDoc} */
-        @Override protected DataRow getRow(BPlusIO<KeySearchRow> io, ByteBuffer buf, int idx)
+        @Override protected DataRow getRow(BPlusIO<KeySearchRow> io, long buf, int idx)
             throws IgniteCheckedException {
             int hash = ((RowLinkIO)io).getHash(buf, idx);
             long link = ((RowLinkIO)io).getLink(buf, idx);
@@ -1169,9 +1170,9 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
      * @param link Link.
      * @param hash Hash.
      */
-    private static void store0(ByteBuffer buf, int off, long link, int hash) {
-        buf.putLong(off, link);
-        buf.putInt(off + 8, hash);
+    private static void store0(long buf, int off, long link, int hash) {
+        PageUtils.putLong(buf, off, link);
+        PageUtils.putInt(buf, off + 8, hash);
     }
 
     /**
@@ -1183,14 +1184,14 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
          * @param idx Index.
          * @return Row link.
          */
-        public long getLink(ByteBuffer buf, int idx);
+        public long getLink(long buf, int idx);
 
         /**
          * @param buf Buffer.
          * @param idx Index.
          * @return Key hash code.
          */
-        public int getHash(ByteBuffer buf, int idx);
+        public int getHash(long buf, int idx);
     }
 
     /**
@@ -1210,14 +1211,14 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
         }
 
         /** {@inheritDoc} */
-        @Override public void storeByOffset(ByteBuffer buf, int off, KeySearchRow row) {
+        @Override public void storeByOffset(long buf, int off, KeySearchRow row) {
             assert row.link() != 0;
 
             store0(buf, off, row.link(), row.hash);
         }
 
         /** {@inheritDoc} */
-        @Override public KeySearchRow getLookupRow(BPlusTree<KeySearchRow, ?> tree, ByteBuffer buf, int idx) {
+        @Override public KeySearchRow getLookupRow(BPlusTree<KeySearchRow, ?> tree, long buf, int idx) {
             int hash = getHash(buf, idx);
             long link = getLink(buf, idx);
 
@@ -1225,7 +1226,7 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
         }
 
         /** {@inheritDoc} */
-        @Override public void store(ByteBuffer dst, int dstIdx, BPlusIO<KeySearchRow> srcIo, ByteBuffer src,
+        @Override public void store(long dst, int dstIdx, BPlusIO<KeySearchRow> srcIo, long src,
             int srcIdx) {
             int hash = ((RowLinkIO)srcIo).getHash(src, srcIdx);
             long link = ((RowLinkIO)srcIo).getLink(src, srcIdx);
@@ -1234,15 +1235,15 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
         }
 
         /** {@inheritDoc} */
-        @Override public long getLink(ByteBuffer buf, int idx) {
+        @Override public long getLink(long buf, int idx) {
             assert idx < getCount(buf) : idx;
 
-            return buf.getLong(offset(idx));
+            return PageUtils.getLong(buf, offset(idx));
         }
 
         /** {@inheritDoc} */
-        @Override public int getHash(ByteBuffer buf, int idx) {
-            return buf.getInt(offset(idx) + 8);
+        @Override public int getHash(long buf, int idx) {
+            return PageUtils.getInt(buf, offset(idx) + 8);
         }
     }
 
@@ -1263,20 +1264,20 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
         }
 
         /** {@inheritDoc} */
-        @Override public void storeByOffset(ByteBuffer buf, int off, KeySearchRow row) {
+        @Override public void storeByOffset(long buf, int off, KeySearchRow row) {
             assert row.link() != 0;
 
             store0(buf, off, row.link(), row.hash);
         }
 
         /** {@inheritDoc} */
-        @Override public void store(ByteBuffer dst, int dstIdx, BPlusIO<KeySearchRow> srcIo, ByteBuffer src,
+        @Override public void store(long dst, int dstIdx, BPlusIO<KeySearchRow> srcIo, long src,
             int srcIdx) {
             store0(dst, offset(dstIdx), getLink(src, srcIdx), getHash(src, srcIdx));
         }
 
         /** {@inheritDoc} */
-        @Override public KeySearchRow getLookupRow(BPlusTree<KeySearchRow, ?> tree, ByteBuffer buf, int idx) {
+        @Override public KeySearchRow getLookupRow(BPlusTree<KeySearchRow, ?> tree, long buf, int idx) {
             int hash = getHash(buf, idx);
             long link = getLink(buf, idx);
 
@@ -1284,15 +1285,15 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
         }
 
         /** {@inheritDoc} */
-        @Override public long getLink(ByteBuffer buf, int idx) {
+        @Override public long getLink(long buf, int idx) {
             assert idx < getCount(buf) : idx;
 
-            return buf.getLong(offset(idx));
+            return PageUtils.getLong(buf, offset(idx));
         }
 
         /** {@inheritDoc} */
-        @Override public int getHash(ByteBuffer buf, int idx) {
-            return buf.getInt(offset(idx) + 8);
+        @Override public int getHash(long buf, int idx) {
+            return PageUtils.getInt(buf, offset(idx) + 8);
         }
     }
 
@@ -1387,7 +1388,7 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
         }
 
         /** {@inheritDoc} */
-        @Override protected int compare(BPlusIO<PendingRow> io, ByteBuffer buf, int idx, PendingRow row)
+        @Override protected int compare(BPlusIO<PendingRow> io, long buf, int idx, PendingRow row)
             throws IgniteCheckedException {
             long expireTime = ((PendingRowIO)io).getExpireTime(buf, idx);
 
@@ -1405,7 +1406,7 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
         }
 
         /** {@inheritDoc} */
-        @Override protected PendingRow getRow(BPlusIO<PendingRow> io, ByteBuffer buf, int idx)
+        @Override protected PendingRow getRow(BPlusIO<PendingRow> io, long buf, int idx)
             throws IgniteCheckedException {
             return io.getLookupRow(this, buf, idx);
         }
@@ -1420,14 +1421,14 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
          * @param idx Index.
          * @return Expire time.
          */
-        long getExpireTime(ByteBuffer buf, int idx);
+        long getExpireTime(long buf, int idx);
 
         /**
          * @param buf Buffer.
          * @param idx Index.
          * @return Link.
          */
-        long getLink(ByteBuffer buf, int idx);
+        long getLink(long buf, int idx);
     }
 
     /**
@@ -1447,43 +1448,43 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
         }
 
         /** {@inheritDoc} */
-        @Override public void storeByOffset(ByteBuffer buf, int off, PendingRow row) throws IgniteCheckedException {
+        @Override public void storeByOffset(long buf, int off, PendingRow row) throws IgniteCheckedException {
             assert row.link != 0;
             assert row.expireTime != 0;
 
-            buf.putLong(off, row.expireTime);
-            buf.putLong(off + 8, row.link);
+            PageUtils.putLong(buf, off, row.expireTime);
+            PageUtils.putLong(buf, off + 8, row.link);
         }
 
         /** {@inheritDoc} */
-        @Override public void store(ByteBuffer dst,
+        @Override public void store(long dst,
             int dstIdx,
             BPlusIO<PendingRow> srcIo,
-            ByteBuffer src,
+            long src,
             int srcIdx) throws IgniteCheckedException {
             int dstOff = offset(dstIdx);
 
             long link = ((PendingRowIO)srcIo).getLink(src, srcIdx);
             long expireTime = ((PendingRowIO)srcIo).getExpireTime(src, srcIdx);
 
-            dst.putLong(dstOff, expireTime);
-            dst.putLong(dstOff + 8, link);
+            PageUtils.putLong(dst, dstOff, expireTime);
+            PageUtils.putLong(dst, dstOff + 8, link);
         }
 
         /** {@inheritDoc} */
-        @Override public PendingRow getLookupRow(BPlusTree<PendingRow, ?> tree, ByteBuffer buf, int idx)
+        @Override public PendingRow getLookupRow(BPlusTree<PendingRow, ?> tree, long buf, int idx)
             throws IgniteCheckedException {
             return PendingRow.createRowWithKey(((PendingEntriesTree)tree).cctx, getExpireTime(buf, idx), getLink(buf, idx));
         }
 
         /** {@inheritDoc} */
-        @Override public long getExpireTime(ByteBuffer buf, int idx) {
-            return buf.getLong(offset(idx));
+        @Override public long getExpireTime(long buf, int idx) {
+            return PageUtils.getLong(buf, offset(idx));
         }
 
         /** {@inheritDoc} */
-        @Override public long getLink(ByteBuffer buf, int idx) {
-            return buf.getLong(offset(idx) + 8);
+        @Override public long getLink(long buf, int idx) {
+            return PageUtils.getLong(buf, offset(idx) + 8);
         }
     }
 
@@ -1504,43 +1505,43 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
         }
 
         /** {@inheritDoc} */
-        @Override public void storeByOffset(ByteBuffer buf, int off, PendingRow row) throws IgniteCheckedException {
+        @Override public void storeByOffset(long buf, int off, PendingRow row) throws IgniteCheckedException {
             assert row.link != 0;
             assert row.expireTime != 0;
 
-            buf.putLong(off, row.expireTime);
-            buf.putLong(off + 8, row.link);
+            PageUtils.putLong(buf, off, row.expireTime);
+            PageUtils.putLong(buf, off + 8, row.link);
         }
 
         /** {@inheritDoc} */
-        @Override public void store(ByteBuffer dst,
+        @Override public void store(long dst,
             int dstIdx,
             BPlusIO<PendingRow> srcIo,
-            ByteBuffer src,
+            long src,
             int srcIdx) throws IgniteCheckedException {
             int dstOff = offset(dstIdx);
 
             long link = ((PendingRowIO)srcIo).getLink(src, srcIdx);
             long expireTime = ((PendingRowIO)srcIo).getExpireTime(src, srcIdx);
 
-            dst.putLong(dstOff, expireTime);
-            dst.putLong(dstOff + 8, link);
+            PageUtils.putLong(dst, dstOff, expireTime);
+            PageUtils.putLong(dst, dstOff + 8, link);
         }
 
         /** {@inheritDoc} */
-        @Override public PendingRow getLookupRow(BPlusTree<PendingRow, ?> tree, ByteBuffer buf, int idx)
+        @Override public PendingRow getLookupRow(BPlusTree<PendingRow, ?> tree, long buf, int idx)
             throws IgniteCheckedException {
             return PendingRow.createRowWithKey(((PendingEntriesTree)tree).cctx, getExpireTime(buf, idx), getLink(buf, idx));
         }
 
         /** {@inheritDoc} */
-        @Override public long getExpireTime(ByteBuffer buf, int idx) {
-            return buf.getLong(offset(idx));
+        @Override public long getExpireTime(long buf, int idx) {
+            return PageUtils.getLong(buf, offset(idx));
         }
 
         /** {@inheritDoc} */
-        @Override public long getLink(ByteBuffer buf, int idx) {
-            return buf.getLong(offset(idx) + 8);
+        @Override public long getLink(long buf, int idx) {
+            return PageUtils.getLong(buf, offset(idx) + 8);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/94f22a0c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/FreeListImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/FreeListImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/FreeListImpl.java
index af20d15..1d2524d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/FreeListImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/FreeListImpl.java
@@ -109,7 +109,7 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
              */
             private int addRow(
                 Page page,
-                ByteBuffer buf,
+                long buf,
                 DataPageIO io,
                 CacheDataRow row,
                 int rowSize
@@ -149,7 +149,7 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
              */
             private int addRowFragment(
                 Page page,
-                ByteBuffer buf,
+                long buf,
                 DataPageIO io,
                 CacheDataRow row,
                 int written,
@@ -179,7 +179,7 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
 
     /** */
     private final PageHandler<Void, Long> rmvRow = new PageHandler<Void, Long>() {
-        @Override public Long run(Page page, PageIO iox, ByteBuffer buf, Void arg, int itemId)
+        @Override public Long run(Page page, PageIO iox, long buf, Void arg, int itemId)
             throws IgniteCheckedException {
             DataPageIO io = (DataPageIO)iox;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/94f22a0c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
index 6ea1bff..b8a1004 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
@@ -18,7 +18,6 @@
 package org.apache.ignite.internal.processors.cache.database.tree;
 
 import java.io.Externalizable;
-import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
@@ -127,7 +126,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
                 return null;
 
             try (Page page = page(pageId)) {
-                ByteBuffer buf = readLock(page); // No correctness guaranties.
+                long buf = readLock(page); // No correctness guaranties.
 
                 try {
                     BPlusIO io = io(buf);
@@ -174,7 +173,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
                 return "<Zero>";
 
             try (Page page = page(pageId)) {
-                ByteBuffer buf = readLock(page); // No correctness guaranties.
+                long buf = readLock(page); // No correctness guaranties.
 
                 try {
                     BPlusIO<L> io = io(buf);
@@ -193,7 +192,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
 
     /** */
     private final GetPageHandler<Get> askNeighbor = new GetPageHandler<Get>() {
-        @Override public Result run0(Page page, ByteBuffer buf, BPlusIO<L> io, Get g, int isBack) {
+        @Override public Result run0(Page page, long buf, BPlusIO<L> io, Get g, int isBack) {
             assert !io.isLeaf(); // Inner page.
 
             boolean back = isBack == TRUE.ordinal();
@@ -220,7 +219,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
 
     /** */
     private final GetPageHandler<Get> search = new GetPageHandler<Get>() {
-        @Override public Result run0(Page page, ByteBuffer buf, BPlusIO<L> io, Get g, int lvl)
+        @Override public Result run0(Page page, long buf, BPlusIO<L> io, Get g, int lvl)
             throws IgniteCheckedException {
             // Check the triangle invariant.
             if (io.getForward(buf) != g.fwdId)
@@ -296,7 +295,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
 
     /** */
     private final GetPageHandler<Put> replace = new GetPageHandler<Put>() {
-        @Override public Result run0(Page page, ByteBuffer buf, BPlusIO<L> io, Put p, int lvl)
+        @Override public Result run0(Page page, long buf, BPlusIO<L> io, Put p, int lvl)
             throws IgniteCheckedException {
             // Check the triangle invariant.
             if (io.getForward(buf) != p.fwdId)
@@ -338,7 +337,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
 
     /** */
     private final GetPageHandler<Put> insert = new GetPageHandler<Put>() {
-        @Override public Result run0(Page page, ByteBuffer buf, BPlusIO<L> io, Put p, int lvl)
+        @Override public Result run0(Page page, long buf, BPlusIO<L> io, Put p, int lvl)
             throws IgniteCheckedException {
             assert p.btmLvl == lvl : "we must always insert at the bottom level: " + p.btmLvl + " " + lvl;
 
@@ -378,7 +377,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
 
     /** */
     private final GetPageHandler<Remove> removeFromLeaf = new GetPageHandler<Remove>() {
-        @Override public Result run0(Page leaf, ByteBuffer buf, BPlusIO<L> io, Remove r, int lvl)
+        @Override public Result run0(Page leaf, long buf, BPlusIO<L> io, Remove r, int lvl)
             throws IgniteCheckedException {
             assert lvl == 0 : lvl; // Leaf.
 
@@ -450,7 +449,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
 
     /** */
     private final GetPageHandler<Remove> lockBackAndRemoveFromLeaf = new GetPageHandler<Remove>() {
-        @Override protected Result run0(Page back, ByteBuffer buf, BPlusIO<L> io, Remove r, int lvl)
+        @Override protected Result run0(Page back, long buf, BPlusIO<L> io, Remove r, int lvl)
             throws IgniteCheckedException {
             // Check that we have consistent view of the world.
             if (io.getForward(buf) != r.pageId)
@@ -469,7 +468,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
 
     /** */
     private final GetPageHandler<Remove> lockBackAndTail = new GetPageHandler<Remove>() {
-        @Override public Result run0(Page back, ByteBuffer buf, BPlusIO<L> io, Remove r, int lvl)
+        @Override public Result run0(Page back, long buf, BPlusIO<L> io, Remove r, int lvl)
             throws IgniteCheckedException {
             // Check that we have consistent view of the world.
             if (io.getForward(buf) != r.pageId)
@@ -487,7 +486,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
 
     /** */
     private final GetPageHandler<Remove> lockTailForward = new GetPageHandler<Remove>() {
-        @Override protected Result run0(Page page, ByteBuffer buf, BPlusIO<L> io, Remove r, int lvl)
+        @Override protected Result run0(Page page, long buf, BPlusIO<L> io, Remove r, int lvl)
             throws IgniteCheckedException {
             r.addTail(page, buf, io, lvl, Tail.FORWARD);
 
@@ -497,7 +496,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
 
     /** */
     private final GetPageHandler<Remove> lockTail = new GetPageHandler<Remove>() {
-        @Override public Result run0(Page page, ByteBuffer buf, BPlusIO<L> io, Remove r, int lvl)
+        @Override public Result run0(Page page, long buf, BPlusIO<L> io, Remove r, int lvl)
             throws IgniteCheckedException {
             assert lvl > 0 : lvl; // We are not at the bottom.
 
@@ -521,7 +520,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
 
     /** */
     private final PageHandler<Void, Bool> cutRoot = new PageHandler<Void, Bool>() {
-        @Override public Bool run(Page meta, PageIO iox, ByteBuffer buf, Void ignore, int lvl)
+        @Override public Bool run(Page meta, PageIO iox, long buf, Void ignore, int lvl)
             throws IgniteCheckedException {
             // Safe cast because we should never recycle meta page until the tree is destroyed.
             BPlusMetaIO io = (BPlusMetaIO)iox;
@@ -539,7 +538,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
 
     /** */
     private final PageHandler<Long, Bool> addRoot = new PageHandler<Long, Bool>() {
-        @Override public Bool run(Page meta, PageIO iox, ByteBuffer buf, Long rootPageId, int lvl)
+        @Override public Bool run(Page meta, PageIO iox, long buf, Long rootPageId, int lvl)
             throws IgniteCheckedException {
             assert rootPageId != null;
 
@@ -559,7 +558,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
 
     /** */
     private final PageHandler<Long, Bool> initRoot = new PageHandler<Long, Bool>() {
-        @Override public Bool run(Page meta, PageIO iox, ByteBuffer buf, Long rootId, int lvl)
+        @Override public Bool run(Page meta, PageIO iox, long buf, Long rootId, int lvl)
             throws IgniteCheckedException {
             assert rootId != null;
 
@@ -657,7 +656,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
      * @return Root level.
      */
     private int getRootLevel(Page meta) {
-        ByteBuffer buf = readLock(meta); // Meta can't be removed.
+        long buf = readLock(meta); // Meta can't be removed.
 
         try {
             return BPlusMetaIO.VERSIONS.forPage(buf).getRootLevel(buf);
@@ -673,7 +672,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
      * @return Page ID.
      */
     private long getFirstPageId(Page meta, int lvl) {
-        ByteBuffer buf = readLock(meta); // Meta can't be removed.
+        long buf = readLock(meta); // Meta can't be removed.
 
         try {
             BPlusMetaIO io = BPlusMetaIO.VERSIONS.forPage(buf);
@@ -705,7 +704,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
         }
 
         try (Page first = page(firstPageId)) {
-            ByteBuffer buf = readLock(first); // We always merge pages backwards, the first page is never removed.
+            long buf = readLock(first); // We always merge pages backwards, the first page is never removed.
 
             try {
                 cursor.init(buf, io(buf), 0);
@@ -922,7 +921,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
      */
     private void validateDownKeys(long pageId, L minRow) throws IgniteCheckedException {
         try (Page page = page(pageId)) {
-            ByteBuffer buf = readLock(page); // No correctness guaranties.
+            long buf = readLock(page); // No correctness guaranties.
 
             try {
                 BPlusIO<L> io = io(buf);
@@ -983,7 +982,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
      */
     private L getGreatestRowInSubTree(long pageId) throws IgniteCheckedException {
         try (Page page = page(pageId)) {
-            ByteBuffer buf = readLock(page); // No correctness guaranties.
+            long buf = readLock(page); // No correctness guaranties.
 
             try {
                 BPlusIO<L> io = io(buf);
@@ -1038,7 +1037,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
         long leftmostChildId;
 
         try (Page page = page(pageId)) {
-            ByteBuffer buf = readLock(page); // No correctness guaranties.
+            long buf = readLock(page); // No correctness guaranties.
 
             try {
                 BPlusIO<L> io = io(buf);
@@ -1068,7 +1067,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
      */
     private void validateDownPages(Page meta, long pageId, long fwdId, final int lvl) throws IgniteCheckedException {
         try (Page page = page(pageId)) {
-            ByteBuffer buf = readLock(page); // No correctness guaranties.
+            long buf = readLock(page); // No correctness guaranties.
 
             try {
                 long realPageId = BPlusIO.getPageId(buf);
@@ -1103,7 +1102,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
                     if (fwdId != 0) {
                         // For the rightmost child ask neighbor.
                         try (Page fwd = page(fwdId)) {
-                            ByteBuffer fwdBuf = readLock(fwd); // No correctness guaranties.
+                            long fwdBuf = readLock(fwd); // No correctness guaranties.
 
                             try {
                                 if (io(fwdBuf) != io)
@@ -1135,7 +1134,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
      * @return String.
      * @throws IgniteCheckedException If failed.
      */
-    private String printPage(BPlusIO<L> io, ByteBuffer buf, boolean keys) throws IgniteCheckedException {
+    private String printPage(BPlusIO<L> io, long buf, boolean keys) throws IgniteCheckedException {
         StringBuilder b = new StringBuilder();
 
         b.append(formatPageId(PageIO.getPageId(buf)));
@@ -1170,7 +1169,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
      * @return Keys as String.
      * @throws IgniteCheckedException If failed.
      */
-    private String printPageKeys(BPlusIO<L> io, ByteBuffer buf) throws IgniteCheckedException {
+    private String printPageKeys(BPlusIO<L> io, long buf) throws IgniteCheckedException {
         int cnt = io.getCount(buf);
 
         StringBuilder b = new StringBuilder();
@@ -1488,7 +1487,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
 
         while (pageId != 0) {
             try (Page page = page(pageId)) {
-                ByteBuffer buf = readLock(page); // No correctness guaranties.
+                long buf = readLock(page); // No correctness guaranties.
 
                 try {
                     if (io == null) {
@@ -1598,7 +1597,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
         long pagesCnt = 0;
 
         try (Page meta = page(metaPageId)) {
-            ByteBuffer metaBuf = writeLock(meta); // No checks, we must be out of use.
+            long metaBuf = writeLock(meta); // No checks, we must be out of use.
 
             try {
                 for (long pageId : getFirstPageIds(metaBuf)) {
@@ -1606,7 +1605,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
 
                     do {
                         try (Page page = page(pageId)) {
-                            ByteBuffer buf = writeLock(page); // No checks, we must be out of use.
+                            long buf = writeLock(page); // No checks, we must be out of use.
 
                             try {
                                 BPlusIO<L> io = io(buf);
@@ -1658,7 +1657,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
      * @param metaBuf Meta page buffer.
      * @return First page IDs.
      */
-    protected Iterable<Long> getFirstPageIds(ByteBuffer metaBuf) {
+    protected Iterable<Long> getFirstPageIds(long metaBuf) {
         List<Long> result = new ArrayList<>();
 
         BPlusMetaIO mio = BPlusMetaIO.VERSIONS.forPage(metaBuf);
@@ -1677,7 +1676,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
      * @return Recycled page ID.
      * @throws IgniteCheckedException If failed.
      */
-    private long recyclePage(long pageId, Page page, ByteBuffer buf) throws IgniteCheckedException {
+    private long recyclePage(long pageId, Page page, long buf) throws IgniteCheckedException {
         // Rotate page ID to avoid concurrency issues with reused pages.
         pageId = PageIdUtils.rotatePageId(pageId);
 
@@ -1704,10 +1703,10 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
     private boolean splitPage(
         BPlusIO io,
         Page page,
-        ByteBuffer buf,
+        long buf,
         long fwdId,
         Page fwd,
-        ByteBuffer fwdBuf,
+        long fwdBuf,
         int idx
     ) throws IgniteCheckedException {
         int cnt = io.getCount(buf);
@@ -1740,7 +1739,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
      * @param page Page.
      * @param buf Buffer.
      */
-    private void writeUnlockAndClose(Page page, ByteBuffer buf) {
+    private void writeUnlockAndClose(Page page, long buf) {
         try {
             writeUnlock(page, buf, true);
         }
@@ -1860,7 +1859,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
      * @param back Backward page.
      * @return Page ID.
      */
-    private long doAskNeighbor(BPlusIO<L> io, ByteBuffer buf, boolean back) {
+    private long doAskNeighbor(BPlusIO<L> io, long buf, boolean back) {
         long res;
 
         if (back) {
@@ -1938,9 +1937,9 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
             int rootLvl;
             long rootId;
 
-            ByteBuffer buf = readLock(meta); // Meta can't be removed.
+            long buf = readLock(meta); // Meta can't be removed.
 
-            assert buf != null : "Failed to read lock meta page [page=" + meta + ", metaPageId=" +
+            assert buf != 0 : "Failed to read lock meta page [page=" + meta + ", metaPageId=" +
                 U.hexLong(metaPageId) + ']';
 
             try {
@@ -1975,7 +1974,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
          * @return {@code true} If we need to stop.
          * @throws IgniteCheckedException If failed.
          */
-        boolean found(BPlusIO<L> io, ByteBuffer buf, int idx, int lvl) throws IgniteCheckedException {
+        boolean found(BPlusIO<L> io, long buf, int idx, int lvl) throws IgniteCheckedException {
             assert lvl >= 0;
 
             return lvl == 0; // Stop if we are at the bottom.
@@ -1989,7 +1988,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
          * @return {@code true} If we need to stop.
          * @throws IgniteCheckedException If failed.
          */
-        boolean notFound(BPlusIO<L> io, ByteBuffer buf, int idx, int lvl) throws IgniteCheckedException {
+        boolean notFound(BPlusIO<L> io, long buf, int idx, int lvl) throws IgniteCheckedException {
             assert lvl >= 0;
 
             return lvl == 0; // Stop if we are at the bottom.
@@ -2027,7 +2026,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
         }
 
         /** {@inheritDoc} */
-        @Override boolean found(BPlusIO<L> io, ByteBuffer buf, int idx, int lvl) throws IgniteCheckedException {
+        @Override boolean found(BPlusIO<L> io, long buf, int idx, int lvl) throws IgniteCheckedException {
             // Check if we are on an inner page and can't get row from it.
             if (lvl != 0 && !canGetRowFromInner)
                 return false;
@@ -2060,12 +2059,12 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
         }
 
         /** {@inheritDoc} */
-        @Override boolean found(BPlusIO<L> io, ByteBuffer buf, int idx, int lvl) throws IgniteCheckedException {
+        @Override boolean found(BPlusIO<L> io, long buf, int idx, int lvl) throws IgniteCheckedException {
             throw new IllegalStateException(); // Must never be called because we always have a shift.
         }
 
         /** {@inheritDoc} */
-        @Override boolean notFound(BPlusIO<L> io, ByteBuffer buf, int idx, int lvl) throws IgniteCheckedException {
+        @Override boolean notFound(BPlusIO<L> io, long buf, int idx, int lvl) throws IgniteCheckedException {
             if (lvl != 0)
                 return false;
 
@@ -2093,7 +2092,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
         private Page tail;
 
         /** */
-        private ByteBuffer tailBuf;
+        private long tailBuf;
 
         /**
          * Bottom level for insertion (insert can't go deeper). Will be incremented on split on each level.
@@ -2117,7 +2116,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
         }
 
         /** {@inheritDoc} */
-        @Override boolean found(BPlusIO<L> io, ByteBuffer buf, int idx, int lvl) {
+        @Override boolean found(BPlusIO<L> io, long buf, int idx, int lvl) {
             if (lvl == 0) // Leaf: need to stop.
                 return true;
 
@@ -2131,7 +2130,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
         }
 
         /** {@inheritDoc} */
-        @Override boolean notFound(BPlusIO<L> io, ByteBuffer buf, int idx, int lvl) {
+        @Override boolean notFound(BPlusIO<L> io, long buf, int idx, int lvl) {
             assert btmLvl >= 0 : btmLvl;
             assert lvl >= btmLvl : lvl;
 
@@ -2142,8 +2141,8 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
          * @param tail Tail page.
          * @param tailBuf Tail buffer.
          */
-        private void tail(Page tail, ByteBuffer tailBuf) {
-            assert (tail == null) == (tailBuf == null);
+        private void tail(Page tail, long tailBuf) {
+            assert (tail == null) == (tailBuf == 0L);
 
             if (this.tail != null)
                 writeUnlockAndClose(this.tail, this.tailBuf);
@@ -2164,7 +2163,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
             row = null;
             rightId = 0;
 
-            tail(null, null);
+            tail(null, 0L);
         }
 
         /**
@@ -2183,7 +2182,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
          * @return Move up row.
          * @throws IgniteCheckedException If failed.
          */
-        private L insert(Page page, BPlusIO<L> io, ByteBuffer buf, int idx, int lvl)
+        private L insert(Page page, BPlusIO<L> io, long buf, int idx, int lvl)
             throws IgniteCheckedException {
             int maxCnt = io.getMaxCount(pageMem.pageSize(), buf);
             int cnt = io.getCount(buf);
@@ -2203,7 +2202,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
          * @param idx Index.
          * @throws IgniteCheckedException If failed.
          */
-        private void insertSimple(Page page, BPlusIO<L> io, ByteBuffer buf, int idx)
+        private void insertSimple(Page page, BPlusIO<L> io, long buf, int idx)
             throws IgniteCheckedException {
             io.insert(buf, idx, row, null, rightId);
 
@@ -2220,7 +2219,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
          * @return Move up row.
          * @throws IgniteCheckedException If failed.
          */
-        private L insertWithSplit(Page page, BPlusIO<L> io, final ByteBuffer buf, int idx, int lvl)
+        private L insertWithSplit(Page page, BPlusIO<L> io, final long buf, int idx, int lvl)
             throws IgniteCheckedException {
             long fwdId = allocatePage(bag);
 
@@ -2228,9 +2227,9 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
                 // Need to check this before the actual split, because after the split we will have new forward page here.
                 boolean hadFwd = io.getForward(buf) != 0;
 
-                ByteBuffer fwdBuf = writeLock(fwd); // Initial write, no need to check for concurrent modification.
+                long fwdBuf = writeLock(fwd); // Initial write, no need to check for concurrent modification.
 
-                assert fwdBuf != null;
+                assert fwdBuf != 0L;
 
                 try {
                     // Never write full forward page, because it is known to be new.
@@ -2275,9 +2274,9 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
                             if (io.isLeaf())
                                 io = latestInnerIO();
 
-                            ByteBuffer newRootBuf = writeLock(newRoot); // Initial write.
+                            long newRootBuf = writeLock(newRoot); // Initial write.
 
-                            assert newRootBuf != null;
+                            assert newRootBuf != 0L;
 
                             try {
                                 // Never write full new root page, because it is known to be new.
@@ -2405,7 +2404,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
         }
 
         /** {@inheritDoc} */
-        @Override boolean notFound(BPlusIO<L> io, ByteBuffer buf, int idx, int lvl) {
+        @Override boolean notFound(BPlusIO<L> io, long buf, int idx, int lvl) {
             if (lvl == 0) {
                 assert tail == null;
 
@@ -2772,7 +2771,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
          * @param idx Index to remove.
          * @throws IgniteCheckedException If failed.
          */
-        private void removeDataRowFromLeaf(Page page, BPlusIO<L> io, ByteBuffer buf, int cnt, int idx)
+        private void removeDataRowFromLeaf(Page page, BPlusIO<L> io, long buf, int cnt, int idx)
             throws IgniteCheckedException {
             assert idx >= 0 && idx < cnt: idx;
             assert io.isLeaf(): "inner";
@@ -2794,7 +2793,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
          * @param idx Index to remove.
          * @throws IgniteCheckedException If failed.
          */
-        private void doRemove(Page page, BPlusIO<L> io, ByteBuffer buf, int cnt, int idx)
+        private void doRemove(Page page, BPlusIO<L> io, long buf, int cnt, int idx)
             throws IgniteCheckedException {
             assert cnt > 0 : cnt;
             assert idx >= 0 && idx < cnt : idx + " " + cnt;
@@ -2966,7 +2965,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
          * @param release Release write lock and release page.
          * @throws IgniteCheckedException If failed.
          */
-        private void freePage(Page page, ByteBuffer buf, boolean release)
+        private void freePage(Page page, long buf, boolean release)
             throws IgniteCheckedException {
             long pageId = page.id();
 
@@ -3165,7 +3164,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
          * @param type Type.
          * @return Added tail.
          */
-        private Tail<L> addTail(Page page, ByteBuffer buf, BPlusIO<L> io, int lvl, byte type) {
+        private Tail<L> addTail(Page page, long buf, BPlusIO<L> io, int lvl, byte type) {
             final Tail<L> t = new Tail<>(page, buf, io, type, lvl);
 
             if (tail == null)
@@ -3273,7 +3272,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
         private final Page page;
 
         /** */
-        private final ByteBuffer buf;
+        private final long buf;
 
         /** */
         private final BPlusIO<L> io;
@@ -3300,7 +3299,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
          * @param type Type.
          * @param lvl Level.
          */
-        private Tail(Page page, ByteBuffer buf, BPlusIO<L> io, byte type, int lvl) {
+        private Tail(Page page, long buf, BPlusIO<L> io, byte type, int lvl) {
             assert type == BACK || type == EXACT || type == FORWARD : type;
             assert lvl >= 0 && lvl <= Byte.MAX_VALUE : lvl;
             assert page != null;
@@ -3353,7 +3352,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
      * @param shift Shift if equal.
      * @return Insertion point as in {@link Arrays#binarySearch(Object[], Object, Comparator)}.
      */
-    private int findInsertionPoint(BPlusIO<L> io, ByteBuffer buf, int low, int cnt, L row, int shift)
+    private int findInsertionPoint(BPlusIO<L> io, long buf, int low, int cnt, L row, int shift)
         throws IgniteCheckedException {
         assert row != null;
 
@@ -3383,8 +3382,8 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
      * @param buf Buffer.
      * @return IO.
      */
-    private BPlusIO<L> io(ByteBuffer buf) {
-        assert buf != null;
+    private BPlusIO<L> io(long buf) {
+        assert buf != 0;
 
         int type = PageIO.getType(buf);
         int ver = PageIO.getVersion(buf);
@@ -3429,7 +3428,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
      * @return Comparison result as in {@link Comparator#compare(Object, Object)}.
      * @throws IgniteCheckedException If failed.
      */
-    protected abstract int compare(BPlusIO<L> io, ByteBuffer buf, int idx, L row) throws IgniteCheckedException;
+    protected abstract int compare(BPlusIO<L> io, long buf, int idx, L row) throws IgniteCheckedException;
 
     /**
      * Get the full detached row. Can be called on inner page only if {@link #canGetRowFromInner} is {@code true}.
@@ -3440,7 +3439,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
      * @return Full detached data row.
      * @throws IgniteCheckedException If failed.
      */
-    protected abstract T getRow(BPlusIO<L> io, ByteBuffer buf, int idx) throws IgniteCheckedException;
+    protected abstract T getRow(BPlusIO<L> io, long buf, int idx) throws IgniteCheckedException;
 
     /**
      * Forward cursor.
@@ -3480,7 +3479,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
          * @param startIdx Start index.
          * @throws IgniteCheckedException If failed.
          */
-        private void init(ByteBuffer buf, BPlusIO<L> io, int startIdx) throws IgniteCheckedException {
+        private void init(long buf, BPlusIO<L> io, int startIdx) throws IgniteCheckedException {
             nextPageId = 0;
             row = -1;
 
@@ -3509,7 +3508,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
          * @return Adjusted to lower bound start index.
          * @throws IgniteCheckedException If failed.
          */
-        private int findLowerBound(ByteBuffer buf, BPlusIO<L> io, int cnt) throws IgniteCheckedException {
+        private int findLowerBound(long buf, BPlusIO<L> io, int cnt) throws IgniteCheckedException {
             // Compare with the first row on the page.
             int cmp = compare(io, buf, 0, lowerBound);
 
@@ -3532,7 +3531,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
          * @return Corrected number of rows with respect to upper bound.
          * @throws IgniteCheckedException If failed.
          */
-        private int findUpperBound(ByteBuffer buf, BPlusIO<L> io, int low, int cnt) throws IgniteCheckedException {
+        private int findUpperBound(long buf, BPlusIO<L> io, int low, int cnt) throws IgniteCheckedException {
             // Compare with the last row on the page.
             int cmp = compare(io, buf, cnt - 1, upperBound);
 
@@ -3558,7 +3557,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
          * @throws IgniteCheckedException If failed.
          */
         @SuppressWarnings("unchecked")
-        private boolean fillFromBuffer(ByteBuffer buf, BPlusIO<L> io, int startIdx, int cnt)
+        private boolean fillFromBuffer(long buf, BPlusIO<L> io, int startIdx, int cnt)
             throws IgniteCheckedException {
             assert io.isLeaf();
             assert cnt != 0: cnt; // We can not see empty pages (empty tree handled in init).
@@ -3665,10 +3664,10 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
                 }
 
                 try (Page next = page(nextPageId)) {
-                    ByteBuffer buf = readLock(next); // Doing explicit null check.
+                    long buf = readLock(next); // Doing explicit null check.
 
                     // If concurrent merge occurred we have to reinitialize cursor from the last returned row.
-                    if (buf == null)
+                    if (buf == 0L)
                         break;
 
                     try {
@@ -3715,7 +3714,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
     private abstract class GetPageHandler<G extends Get> extends PageHandler<G, Result> {
         /** {@inheritDoc} */
         @SuppressWarnings("unchecked")
-        @Override public final Result run(Page page, PageIO iox, ByteBuffer buf, G g, int lvl)
+        @Override public final Result run(Page page, PageIO iox, long buf, G g, int lvl)
             throws IgniteCheckedException {
             assert PageIO.getPageId(buf) == page.id();
 
@@ -3739,7 +3738,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure {
          * @return Result code.
          * @throws IgniteCheckedException If failed.
          */
-        protected abstract Result run0(Page page, ByteBuffer buf, BPlusIO<L> io, G g, int lvl)
+        protected abstract Result run0(Page page, long buf, BPlusIO<L> io, G g, int lvl)
             throws IgniteCheckedException;
 
         /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/94f22a0c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPageIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPageIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPageIO.java
index e1bab5f..ca8ea39 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPageIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPageIO.java
@@ -88,7 +88,7 @@ public class DataPageIO extends PageIO {
     }
 
     /** {@inheritDoc} */
-    @Override public void initNewPage(ByteBuffer buf, long pageId) {
+    @Override public void initNewPage(long buf, long pageId) {
         super.initNewPage(buf, pageId);
 
         setEmptyPage(buf);
@@ -98,7 +98,7 @@ public class DataPageIO extends PageIO {
     /**
      * @param buf Buffer.
      */
-    private void setEmptyPage(ByteBuffer buf) {
+    private void setEmptyPage(long buf) {
         setDirectCount(buf, 0);
         setIndirectCount(buf, 0);
         setFirstEntryOffset(buf, buf.capacity());
@@ -127,8 +127,8 @@ public class DataPageIO extends PageIO {
      * @param show What elements of data page entry to show in the result.
      * @return Data page entry size.
      */
-    private int getPageEntrySize(ByteBuffer buf, int dataOff, int show) {
-        int payloadLen = buf.getShort(dataOff) & 0xFFFF;
+    private int getPageEntrySize(long buf, int dataOff, int show) {
+        int payloadLen = PageUtils.getShort(buf, dataOff) & 0xFFFF;
 
         if ((payloadLen & FRAGMENTED_FLAG) != 0)
             payloadLen &= ~FRAGMENTED_FLAG; // We are fragmented and have a link.
@@ -164,28 +164,28 @@ public class DataPageIO extends PageIO {
      * @param buf Buffer.
      * @param dataOff Entry data offset.
      */
-    private void setFirstEntryOffset(ByteBuffer buf, int dataOff) {
+    private void setFirstEntryOffset(long buf, int dataOff) {
         assert dataOff >= ITEMS_OFF + ITEM_SIZE && dataOff <= buf.capacity(): dataOff;
 
-        buf.putShort(FIRST_ENTRY_OFF, (short)dataOff);
+        PageUtils.putShort(buf, FIRST_ENTRY_OFF, (short)dataOff);
     }
 
     /**
      * @param buf Buffer.
      * @return Entry data offset.
      */
-    private int getFirstEntryOffset(ByteBuffer buf) {
-        return buf.getShort(FIRST_ENTRY_OFF) & 0xFFFF;
+    private int getFirstEntryOffset(long buf) {
+        return PageUtils.getShort(buf, FIRST_ENTRY_OFF) & 0xFFFF;
     }
 
     /**
      * @param buf Buffer.
      * @param freeSpace Free space.
      */
-    private void setRealFreeSpace(ByteBuffer buf, int freeSpace) {
+    private void setRealFreeSpace(long buf, int freeSpace) {
         assert freeSpace == actualFreeSpace(buf): freeSpace + " != " + actualFreeSpace(buf);
 
-        buf.putShort(FREE_SPACE_OFF, (short)freeSpace);
+        PageUtils.putShort(buf, FREE_SPACE_OFF, (short)freeSpace);
     }
 
     /**
@@ -195,7 +195,7 @@ public class DataPageIO extends PageIO {
      * @param buf Buffer.
      * @return Free space.
      */
-    public int getFreeSpace(ByteBuffer buf) {
+    public int getFreeSpace(long buf) {
         if (getFreeItemSlots(buf) == 0)
             return 0;
 
@@ -218,23 +218,23 @@ public class DataPageIO extends PageIO {
     }
 
     /**
-     * Equivalent for {@link #actualFreeSpace(ByteBuffer)} but reads saved value.
+     * Equivalent for {@link #actualFreeSpace(long)} but reads saved value.
      *
      * @param buf Buffer.
      * @return Free space.
      */
-    private int getRealFreeSpace(ByteBuffer buf) {
-        return buf.getShort(FREE_SPACE_OFF);
+    private int getRealFreeSpace(long buf) {
+        return PageUtils.getShort(buf, FREE_SPACE_OFF);
     }
 
     /**
      * @param buf Buffer.
      * @param cnt Direct count.
      */
-    private void setDirectCount(ByteBuffer buf, int cnt) {
+    private void setDirectCount(long buf, int cnt) {
         assert checkCount(cnt): cnt;
 
-        buf.put(DIRECT_CNT_OFF, (byte)cnt);
+        PageUtils.putByte(buf, DIRECT_CNT_OFF, (byte)cnt);
     }
 
     /**
@@ -249,10 +249,10 @@ public class DataPageIO extends PageIO {
      * @param buf Buffer.
      * @param cnt Indirect count.
      */
-    private void setIndirectCount(ByteBuffer buf, int cnt) {
+    private void setIndirectCount(long buf, int cnt) {
         assert checkCount(cnt): cnt;
 
-        buf.put(INDIRECT_CNT_OFF, (byte)cnt);
+        PageUtils.putByte(buf, INDIRECT_CNT_OFF, (byte)cnt);
     }
 
     /**
@@ -275,15 +275,15 @@ public class DataPageIO extends PageIO {
      * @param buf Buffer.
      * @return Indirect count.
      */
-    private int getIndirectCount(ByteBuffer buf) {
-        return buf.get(INDIRECT_CNT_OFF) & 0xFF;
+    private int getIndirectCount(long buf) {
+        return PageUtils.getByte(buf, INDIRECT_CNT_OFF) & 0xFF;
     }
 
     /**
      * @param buf Buffer.
      * @return Number of free entry slots.
      */
-    private int getFreeItemSlots(ByteBuffer buf) {
+    private int getFreeItemSlots(long buf) {
         return 0xFF - getDirectCount(buf);
     }
 
@@ -294,7 +294,7 @@ public class DataPageIO extends PageIO {
      * @param indirectCnt Indirect items count.
      * @return Found index of indirect item.
      */
-    private int findIndirectItemIndex(ByteBuffer buf, int itemId, int directCnt, int indirectCnt) {
+    private int findIndirectItemIndex(long buf, int itemId, int directCnt, int indirectCnt) {
         int low = directCnt;
         int high = directCnt + indirectCnt - 1;
 
@@ -318,7 +318,7 @@ public class DataPageIO extends PageIO {
      * @param buf Buffer.
      * @return String representation.
      */
-    public String printPageLayout(ByteBuffer buf) {
+    public String printPageLayout(long buf) {
         int directCnt = getDirectCount(buf);
         int indirectCnt = getIndirectCount(buf);
         int free = getRealFreeSpace(buf);
@@ -395,7 +395,7 @@ public class DataPageIO extends PageIO {
      * @param itemId Fixed item ID (the index used for referencing an entry from the outside).
      * @return Data entry offset in bytes.
      */
-    private int getDataOffset(ByteBuffer buf, int itemId) {
+    private int getDataOffset(long buf, int itemId) {
         assert checkIndex(itemId): itemId;
 
         int directCnt = getDirectCount(buf);
@@ -427,10 +427,10 @@ public class DataPageIO extends PageIO {
      * @param dataOff Points to the entry start.
      * @return Link to the next entry fragment or 0 if no fragments left or if entry is not fragmented.
      */
-    private long getNextFragmentLink(ByteBuffer buf, int dataOff) {
+    private long getNextFragmentLink(long buf, int dataOff) {
         assert isFragmented(buf, dataOff);
 
-        return buf.getLong(dataOff + PAYLOAD_LEN_SIZE);
+        return PageUtils.getLong(buf, dataOff + PAYLOAD_LEN_SIZE);
     }
 
     /**
@@ -438,8 +438,8 @@ public class DataPageIO extends PageIO {
      * @param dataOff Data offset.
      * @return {@code true} If the data row is fragmented across multiple pages.
      */
-    private boolean isFragmented(ByteBuffer buf, int dataOff) {
-        return (buf.getShort(dataOff) & FRAGMENTED_FLAG) != 0;
+    private boolean isFragmented(long buf, int dataOff) {
+        return (PageUtils.getShort(buf, dataOff) & FRAGMENTED_FLAG) != 0;
     }
 
     /**
@@ -449,7 +449,7 @@ public class DataPageIO extends PageIO {
      * @param itemId Item to position on.
      * @return Link to the next fragment or {@code 0} if it is the last fragment or the data row is not fragmented.
      */
-    public long setPositionAndLimitOnPayload(final ByteBuffer buf, final int itemId) {
+    public long setPositionAndLimitOnPayload(final long buf, final int itemId) {
         int dataOff = getDataOffset(buf, itemId);
 
         boolean fragmented = isFragmented(buf, dataOff);
@@ -468,8 +468,8 @@ public class DataPageIO extends PageIO {
      * @param idx Item index.
      * @return Item.
      */
-    private short getItem(ByteBuffer buf, int idx) {
-        return buf.getShort(itemOffset(idx));
+    private short getItem(long buf, int idx) {
+        return PageUtils.getShort(buf, itemOffset(idx));
     }
 
     /**
@@ -477,8 +477,8 @@ public class DataPageIO extends PageIO {
      * @param idx Item index.
      * @param item Item.
      */
-    private void setItem(ByteBuffer buf, int idx, short item) {
-        buf.putShort(itemOffset(idx), item);
+    private void setItem(long buf, int idx, short item) {
+        PageUtils.putShort(buf, itemOffset(idx), item);
     }
 
     /**
@@ -546,7 +546,7 @@ public class DataPageIO extends PageIO {
      * @param indirectCnt Indirect items count.
      * @return {@code true} If the last direct item already had corresponding indirect item.
      */
-    private boolean moveLastItem(ByteBuffer buf, int freeDirectIdx, int directCnt, int indirectCnt) {
+    private boolean moveLastItem(long buf, int freeDirectIdx, int directCnt, int indirectCnt) {
         int lastIndirectId = findIndirectIndexForLastDirect(buf, directCnt, indirectCnt);
 
         int lastItemId = directCnt - 1;
@@ -577,7 +577,7 @@ public class DataPageIO extends PageIO {
      * @param indirectCnt Indirect items count.
      * @return Index of indirect item for the last direct item.
      */
-    private int findIndirectIndexForLastDirect(ByteBuffer buf, int directCnt, int indirectCnt) {
+    private int findIndirectIndexForLastDirect(long buf, int directCnt, int indirectCnt) {
         int lastDirectId = directCnt - 1;
 
         for (int i = directCnt, end = directCnt + indirectCnt; i < end; i++) {
@@ -596,7 +596,7 @@ public class DataPageIO extends PageIO {
      * @return Next link for fragmented entries or {@code 0} if none.
      * @throws IgniteCheckedException If failed.
      */
-    public long removeRow(ByteBuffer buf, int itemId) throws IgniteCheckedException {
+    public long removeRow(long buf, int itemId) throws IgniteCheckedException {
         assert checkIndex(itemId) : itemId;
 
         final int dataOff = getDataOffset(buf, itemId);
@@ -675,7 +675,7 @@ public class DataPageIO extends PageIO {
      * @param cnt Count.
      * @param step Step.
      */
-    private void moveItems(ByteBuffer buf, int idx, int cnt, int step) {
+    private void moveItems(long buf, int idx, int cnt, int step) {
         assert cnt >= 0: cnt;
 
         if (cnt != 0)
@@ -702,7 +702,7 @@ public class DataPageIO extends PageIO {
      * @throws IgniteCheckedException If failed.
      */
     public void addRow(
-        ByteBuffer buf,
+        long buf,
         CacheDataRow row,
         int rowSize
     ) throws IgniteCheckedException {
@@ -729,7 +729,7 @@ public class DataPageIO extends PageIO {
      * @throws IgniteCheckedException If failed.
      */
     public void addRow(
-        ByteBuffer buf,
+        long buf,
         byte[] payload
     ) throws IgniteCheckedException {
         assert payload.length <= getFreeSpace(buf): "can't call addRow if not enough space for the whole row";
@@ -755,7 +755,7 @@ public class DataPageIO extends PageIO {
      * @return First entry offset after compaction.
      */
     private int compactIfNeed(
-        final ByteBuffer buf,
+        final long buf,
         final int entryFullSize,
         final int directCnt,
         final int indirectCnt,
@@ -780,7 +780,7 @@ public class DataPageIO extends PageIO {
      * @param dataOff Data offset.
      * @return Item ID.
      */
-    private int addItem(final ByteBuffer buf, final int fullEntrySize, final int directCnt,
+    private int addItem(final long buf, final int fullEntrySize, final int directCnt,
         final int indirectCnt, final int dataOff) {
         setFirstEntryOffset(buf, dataOff);
 
@@ -803,7 +803,7 @@ public class DataPageIO extends PageIO {
      * @param indirectCnt Indirect items count.
      * @return Offset in the buffer where the entry must be written.
      */
-    private int getDataOffsetForWrite(ByteBuffer buf, int fullEntrySize, int directCnt, int indirectCnt) {
+    private int getDataOffsetForWrite(long buf, int fullEntrySize, int directCnt, int indirectCnt) {
         int dataOff = getFirstEntryOffset(buf);
 
         // Compact if we do not have enough space for entry.
@@ -826,7 +826,7 @@ public class DataPageIO extends PageIO {
      * @throws IgniteCheckedException If failed.
      */
     public int addRowFragment(
-        ByteBuffer buf,
+        long buf,
         CacheDataRow row,
         int written,
         int rowSize
@@ -843,7 +843,7 @@ public class DataPageIO extends PageIO {
      * @throws IgniteCheckedException If failed.
      */
     public void addRowFragment(
-        ByteBuffer buf,
+        long buf,
         byte[] payload,
         long lastLink
     ) throws IgniteCheckedException {
@@ -863,7 +863,7 @@ public class DataPageIO extends PageIO {
      * @throws IgniteCheckedException If failed.
      */
     private int addRowFragment(
-        ByteBuffer buf,
+        long buf,
         int written,
         int rowSize,
         long lastLink,
@@ -881,23 +881,16 @@ public class DataPageIO extends PageIO {
         int fullEntrySize = getPageEntrySize(payloadSize, SHOW_PAYLOAD_LEN | SHOW_LINK | SHOW_ITEM);
         int dataOff = getDataOffsetForWrite(buf, fullEntrySize, directCnt, indirectCnt);
 
-        try {
-            buf.position(dataOff);
-
-            buf.putShort((short) (payloadSize | FRAGMENTED_FLAG));
-            buf.putLong(lastLink);
+        PageUtils.putShort(buf, dataOff, (short)(payloadSize | FRAGMENTED_FLAG));
+        PageUtils.putLong(buf, dataOff + 2, lastLink);
 
-            if (payload == null) {
-                int rowOff = rowSize - written - payloadSize;
+        if (payload == null) {
+            int rowOff = rowSize - written - payloadSize;
 
-                writeFragmentData(row, buf, rowOff, payloadSize);
-            }
-            else
-                buf.put(payload);
-        }
-        finally {
-            buf.position(0);
+            writeFragmentData(row, buf + dataOff + 10, rowOff, payloadSize);
         }
+        else
+            PageUtils.putBytes(buf, dataOff + 10, payload);
 
         int itemId = addItem(buf, fullEntrySize, directCnt, indirectCnt, dataOff);
 
@@ -912,7 +905,7 @@ public class DataPageIO extends PageIO {
      * @param buf Page buffer.
      * @param itemId Item ID.
      */
-    private void setLink(CacheDataRow row, ByteBuffer buf, int itemId) {
+    private void setLink(CacheDataRow row, long buf, int itemId) {
         row.link(PageIdUtils.link(getPageId(buf), itemId));
     }
 
@@ -927,7 +920,7 @@ public class DataPageIO extends PageIO {
      */
     private void writeFragmentData(
         final CacheDataRow row,
-        final ByteBuffer buf,
+        final long buf,
         final int rowOff,
         final int payloadSize
     ) throws IgniteCheckedException {
@@ -953,7 +946,7 @@ public class DataPageIO extends PageIO {
      */
     private int writeFragment(
         final CacheDataRow row,
-        final ByteBuffer buf,
+        final long buf,
         final int rowOff,
         final int payloadSize,
         final EntryPart type,
@@ -1089,7 +1082,7 @@ public class DataPageIO extends PageIO {
      * @param indirectCnt Indirect items count.
      * @return Item ID (insertion index).
      */
-    private int insertItem(ByteBuffer buf, int dataOff, int directCnt, int indirectCnt) {
+    private int insertItem(long buf, int dataOff, int directCnt, int indirectCnt) {
         if (indirectCnt > 0) {
             // If the first indirect item is on correct place to become the last direct item, do the transition
             // and insert the new item into the free slot which was referenced by this first indirect item.
@@ -1124,7 +1117,7 @@ public class DataPageIO extends PageIO {
      * @param directCnt Direct items count.
      * @return New first entry offset.
      */
-    private int compactDataEntries(ByteBuffer buf, int directCnt) {
+    private int compactDataEntries(long buf, int directCnt) {
         assert checkCount(directCnt): directCnt;
 
         int[] offs = new int[directCnt];
@@ -1173,7 +1166,7 @@ public class DataPageIO extends PageIO {
      * @param buf Buffer to scan.
      * @return Actual free space in the buffer.
      */
-    private int actualFreeSpace(ByteBuffer buf) {
+    private int actualFreeSpace(long buf) {
         int directCnt = getDirectCount(buf);
 
         int entriesSize = 0;
@@ -1195,7 +1188,7 @@ public class DataPageIO extends PageIO {
      * @param cnt Count.
      * @param step Step.
      */
-    private void moveBytes(ByteBuffer buf, int off, int cnt, int step) {
+    private void moveBytes(long buf, int off, int cnt, int step) {
         assert step != 0: step;
         assert off + step >= 0;
         assert off + step + cnt <= buf.capacity() : "[off=" + off + ", step=" + step + ", cnt=" + cnt +
@@ -1212,7 +1205,7 @@ public class DataPageIO extends PageIO {
      * @throws IgniteCheckedException If failed.
      */
     private void writeRowData(
-        ByteBuffer buf,
+        long buf,
         int dataOff,
         int payloadSize,
         CacheDataRow row
@@ -1245,7 +1238,7 @@ public class DataPageIO extends PageIO {
      * @param payload Payload
      */
     private void writeRowData(
-        ByteBuffer buf,
+        long buf,
         int dataOff,
         byte[] payload
     ) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/94f22a0c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/reuse/ReuseListImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/reuse/ReuseListImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/reuse/ReuseListImpl.java
index d2813c9..ce8266a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/reuse/ReuseListImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/reuse/ReuseListImpl.java
@@ -65,7 +65,7 @@ public class ReuseListImpl extends PagesList implements ReuseList {
 
     /** {@inheritDoc} */
     @Override public void addForRecycle(ReuseBag bag) throws IgniteCheckedException {
-        put(bag, null, null, 0);
+        put(bag, null, 0, 0);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/94f22a0c/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2Tree.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2Tree.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2Tree.java
index 0593c60..5f9d0fa 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2Tree.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2Tree.java
@@ -76,7 +76,7 @@ public abstract class H2Tree extends BPlusTree<SearchRow, GridH2Row> {
     }
 
     /** {@inheritDoc} */
-    @Override protected GridH2Row getRow(BPlusIO<SearchRow> io, ByteBuffer buf, int idx)
+    @Override protected GridH2Row getRow(BPlusIO<SearchRow> io, long buf, int idx)
         throws IgniteCheckedException {
         return (GridH2Row)io.getLookupRow(this, buf, idx);
     }