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/17 11:52:11 UTC

[4/7] ignite git commit: ignite-3477 PageMemory optimizations - use page address instead of ByteBuffer to work with page memory - got rid of pages pin/unpin - do not copy byte array for cache key comparison - reduced size of data tree search row

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/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 aef7217..e91dda9 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;
@@ -126,15 +125,15 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                 return null;
 
             try (Page page = page(pageId)) {
-                ByteBuffer buf = readLock(page); // No correctness guaranties.
+                long pageAddr = readLock(page); // No correctness guaranties.
 
                 try {
-                    BPlusIO io = io(buf);
+                    BPlusIO io = io(pageAddr);
 
                     if (io.isLeaf())
                         return null;
 
-                    int cnt = io.getCount(buf);
+                    int cnt = io.getCount(pageAddr);
 
                     assert cnt >= 0 : cnt;
 
@@ -144,12 +143,12 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                         res = new ArrayList<>(cnt + 1);
 
                         for (int i = 0; i < cnt; i++)
-                            res.add(inner(io).getLeft(buf, i));
+                            res.add(inner(io).getLeft(pageAddr, i));
 
-                        res.add(inner(io).getRight(buf, cnt - 1));
+                        res.add(inner(io).getRight(pageAddr, cnt - 1));
                     }
                     else {
-                        long left = inner(io).getLeft(buf, 0);
+                        long left = inner(io).getLeft(pageAddr, 0);
 
                         res = left == 0 ? Collections.<Long>emptyList() : Collections.singletonList(left);
                     }
@@ -157,7 +156,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                     return res;
                 }
                 finally {
-                    readUnlock(page, buf);
+                    readUnlock(page, pageAddr);
                 }
             }
             catch (IgniteCheckedException e) {
@@ -173,15 +172,15 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                 return "<Zero>";
 
             try (Page page = page(pageId)) {
-                ByteBuffer buf = readLock(page); // No correctness guaranties.
+                long pageAddr = readLock(page); // No correctness guaranties.
 
                 try {
-                    BPlusIO<L> io = io(buf);
+                    BPlusIO<L> io = io(pageAddr);
 
-                    return printPage(io, buf, keys);
+                    return printPage(io, pageAddr, keys);
                 }
                 finally {
-                    readUnlock(page, buf);
+                    readUnlock(page, pageAddr);
                 }
             }
             catch (IgniteCheckedException e) {
@@ -192,17 +191,17 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
     /** */
     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 pageAddr, BPlusIO<L> io, Get g, int isBack) {
             assert !io.isLeaf(); // Inner page.
 
             boolean back = isBack == TRUE.ordinal();
 
-            long res = doAskNeighbor(io, buf, back);
+            long res = doAskNeighbor(io, pageAddr, back);
 
             if (back) {
                 assert g.getClass() == Remove.class;
 
-                if (io.getForward(buf) != g.backId) // See how g.backId is setup in removeDown for this check.
+                if (io.getForward(pageAddr) != g.backId) // See how g.backId is setup in removeDown for this check.
                     return RETRY;
 
                 g.backId = res;
@@ -219,25 +218,25 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
     /** */
     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 pageAddr, BPlusIO<L> io, Get g, int lvl)
             throws IgniteCheckedException {
             // Check the triangle invariant.
-            if (io.getForward(buf) != g.fwdId)
+            if (io.getForward(pageAddr) != g.fwdId)
                 return RETRY;
 
             boolean needBackIfRouting = g.backId != 0;
 
             g.backId = 0; // Usually we'll go left down and don't need it.
 
-            int cnt = io.getCount(buf);
-            int idx = findInsertionPoint(io, buf, 0, cnt, g.row, g.shift);
+            int cnt = io.getCount(pageAddr);
+            int idx = findInsertionPoint(io, pageAddr, 0, cnt, g.row, g.shift);
 
             boolean found = idx >= 0;
 
             if (found) { // Found exact match.
                 assert g.getClass() != GetCursor.class;
 
-                if (g.found(io, buf, idx, lvl))
+                if (g.found(io, pageAddr, idx, lvl))
                     return FOUND;
 
                 // Else we need to reach leaf page, go left down.
@@ -245,20 +244,20 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             else {
                 idx = fix(idx);
 
-                if (g.notFound(io, buf, idx, lvl)) // No way down, stop here.
+                if (g.notFound(io, pageAddr, idx, lvl)) // No way down, stop here.
                     return NOT_FOUND;
             }
 
             assert !io.isLeaf();
 
             // If idx == cnt then we go right down, else left down: getLeft(cnt) == getRight(cnt - 1).
-            g.pageId = inner(io).getLeft(buf, idx);
+            g.pageId = inner(io).getLeft(pageAddr, idx);
 
             // If we see the tree in consistent state, then our right down page must be forward for our left down page,
             // we need to setup fwdId and/or backId to be able to check this invariant on lower level.
             if (idx < cnt) {
                 // Go left down here.
-                g.fwdId = inner(io).getRight(buf, idx);
+                g.fwdId = inner(io).getRight(pageAddr, idx);
             }
             else {
                 // Go right down here or it is an empty branch.
@@ -267,7 +266,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                 // Here child's forward is unknown to us (we either go right or it is an empty "routing" page),
                 // need to ask our forward about the child's forward (it must be leftmost child of our forward page).
                 // This is ok from the locking standpoint because we take all locks in the forward direction.
-                long fwdId = io.getForward(buf);
+                long fwdId = io.getForward(pageAddr);
 
                 // Setup fwdId.
                 if (fwdId == 0)
@@ -282,7 +281,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
                 // Setup backId.
                 if (cnt != 0) // It is not a routing page and we are going to the right, can get backId here.
-                    g.backId = inner(io).getLeft(buf, cnt - 1);
+                    g.backId = inner(io).getLeft(pageAddr, cnt - 1);
                 else if (needBackIfRouting) {
                     // Can't get backId here because of possible deadlock and it is only needed for remove operation.
                     return GO_DOWN_X;
@@ -295,16 +294,16 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
     /** */
     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 pageAddr, BPlusIO<L> io, Put p, int lvl)
             throws IgniteCheckedException {
             // Check the triangle invariant.
-            if (io.getForward(buf) != p.fwdId)
+            if (io.getForward(pageAddr) != p.fwdId)
                 return RETRY;
 
             assert p.btmLvl == 0 : "split is impossible with replace";
 
-            final int cnt = io.getCount(buf);
-            final int idx = findInsertionPoint(io, buf, 0, cnt, p.row, 0);
+            final int cnt = io.getCount(pageAddr);
+            final int idx = findInsertionPoint(io, pageAddr, 0, cnt, p.row, 0);
 
             if (idx < 0) // Not found, split or merge happened.
                 return RETRY;
@@ -318,7 +317,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                 assert p.oldRow == null;
 
                 // Get old row in leaf page to reduce contention at upper level.
-                p.oldRow = getRow(io, buf, idx);
+                p.oldRow = getRow(io, pageAddr, idx);
 
                 p.finish();
 
@@ -326,7 +325,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                 assert p.needReplaceInner == FALSE || p.needReplaceInner == DONE : p.needReplaceInner;
             }
 
-            io.store(buf, idx, newRow, null);
+            io.store(pageAddr, idx, newRow, null);
 
             if (needWalDeltaRecord(page))
                 wal.log(new ReplaceRecord<>(cacheId, page.id(), io, newRow, null, idx));
@@ -337,16 +336,16 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
     /** */
     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 pageAddr, BPlusIO<L> io, Put p, int lvl)
             throws IgniteCheckedException {
             assert p.btmLvl == lvl : "we must always insert at the bottom level: " + p.btmLvl + " " + lvl;
 
             // Check triangle invariant.
-            if (io.getForward(buf) != p.fwdId)
+            if (io.getForward(pageAddr) != p.fwdId)
                 return RETRY;
 
-            int cnt = io.getCount(buf);
-            int idx = findInsertionPoint(io, buf, 0, cnt, p.row, 0);
+            int cnt = io.getCount(pageAddr);
+            int idx = findInsertionPoint(io, pageAddr, 0, cnt, p.row, 0);
 
             if (idx >= 0) // We do not support concurrent put of the same key.
                 throw new IllegalStateException("Duplicate row in index.");
@@ -354,7 +353,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             idx = fix(idx);
 
             // Do insert.
-            L moveUpRow = p.insert(page, io, buf, idx, lvl);
+            L moveUpRow = p.insert(page, io, pageAddr, idx, lvl);
 
             // Check if split happened.
             if (moveUpRow != null) {
@@ -363,8 +362,8 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
                 // Here forward page can't be concurrently removed because we keep write lock on tail which is the only
                 // page who knows about the forward page, because it was just produced by split.
-                p.rightId = io.getForward(buf);
-                p.tail(page, buf);
+                p.rightId = io.getForward(pageAddr);
+                p.tail(page, pageAddr);
 
                 assert p.rightId != 0;
             }
@@ -376,16 +375,16 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
     };
 
     /** */
-    private final GetPageHandler<Remove> removeFromLeaf = new GetPageHandler<Remove>() {
-        @Override public Result run0(Page leaf, ByteBuffer buf, BPlusIO<L> io, Remove r, int lvl)
+    private final GetPageHandler<Remove> rmvFromLeaf = new GetPageHandler<Remove>() {
+        @Override public Result run0(Page leaf, long pageAddr, BPlusIO<L> io, Remove r, int lvl)
             throws IgniteCheckedException {
             assert lvl == 0 : lvl; // Leaf.
 
-            final int cnt = io.getCount(buf);
+            final int cnt = io.getCount(pageAddr);
 
             assert cnt <= Short.MAX_VALUE: cnt;
 
-            int idx = findInsertionPoint(io, buf, 0, cnt, r.row, 0);
+            int idx = findInsertionPoint(io, pageAddr, 0, cnt, r.row, 0);
 
             if (idx < 0) {
                 if (!r.ceil) // We've found exact match on search but now it's gone.
@@ -401,14 +400,14 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
             // Need to do inner replace when we remove the rightmost element and the leaf have no forward page,
             // i.e. it is not the rightmost leaf of the tree.
-            boolean needReplaceInner = canGetRowFromInner && idx == cnt - 1 && io.getForward(buf) != 0;
+            boolean needReplaceInner = canGetRowFromInner && idx == cnt - 1 && io.getForward(pageAddr) != 0;
 
             // !!! Before modifying state we have to make sure that we will not go for retry.
 
             // We may need to replace inner key or want to merge this leaf with sibling after the remove -> keep lock.
             if (needReplaceInner ||
                 // We need to make sure that we have back or forward to be able to merge.
-                ((r.fwdId != 0 || r.backId != 0) && mayMerge(cnt - 1, io.getMaxCount(buf)))) {
+                ((r.fwdId != 0 || r.backId != 0) && mayMerge(cnt - 1, io.getMaxCount(pageAddr, pageSize())))) {
                 // If we have backId then we've already locked back page, nothing to do here.
                 if (r.fwdId != 0 && r.backId == 0) {
                     Result res = r.lockForward(0);
@@ -432,7 +431,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                 if (needReplaceInner)
                     r.needReplaceInner = TRUE;
 
-                Tail<L> t = r.addTail(leaf, buf, io, 0, Tail.EXACT);
+                Tail<L> t = r.addTail(leaf, pageAddr, io, 0, Tail.EXACT);
 
                 t.idx = (short)idx;
 
@@ -441,18 +440,18 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                 return FOUND;
             }
 
-            r.removeDataRowFromLeaf(leaf, io, buf, cnt, idx);
+            r.removeDataRowFromLeaf(leaf, io, pageAddr, cnt, idx);
 
             return FOUND;
         }
     };
 
     /** */
-    private final GetPageHandler<Remove> lockBackAndRemoveFromLeaf = new GetPageHandler<Remove>() {
-        @Override protected Result run0(Page back, ByteBuffer buf, BPlusIO<L> io, Remove r, int lvl)
+    private final GetPageHandler<Remove> lockBackAndRmvFromLeaf = new GetPageHandler<Remove>() {
+        @Override protected Result run0(Page back, long pageAddr, BPlusIO<L> io, Remove r, int lvl)
             throws IgniteCheckedException {
             // Check that we have consistent view of the world.
-            if (io.getForward(buf) != r.pageId)
+            if (io.getForward(pageAddr) != r.pageId)
                 return RETRY;
 
             // Correct locking order: from back to forward.
@@ -460,7 +459,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
             // Keep locks on back and leaf pages for subsequent merges.
             if (res == FOUND && r.tail != null)
-                r.addTail(back, buf, io, lvl, Tail.BACK);
+                r.addTail(back, pageAddr, io, lvl, Tail.BACK);
 
             return res;
         }
@@ -468,17 +467,17 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
     /** */
     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 pageAddr, BPlusIO<L> io, Remove r, int lvl)
             throws IgniteCheckedException {
             // Check that we have consistent view of the world.
-            if (io.getForward(buf) != r.pageId)
+            if (io.getForward(pageAddr) != r.pageId)
                 return RETRY;
 
             // Correct locking order: from back to forward.
             Result res = r.doLockTail(lvl);
 
             if (res == FOUND)
-                r.addTail(back, buf, io, lvl, Tail.BACK);
+                r.addTail(back, pageAddr, io, lvl, Tail.BACK);
 
             return res;
         }
@@ -486,9 +485,9 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
     /** */
     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 pageAddr, BPlusIO<L> io, Remove r, int lvl)
             throws IgniteCheckedException {
-            r.addTail(page, buf, io, lvl, Tail.FORWARD);
+            r.addTail(page, pageAddr, io, lvl, Tail.FORWARD);
 
             return FOUND;
         }
@@ -496,12 +495,12 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
     /** */
     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 pageAddr, BPlusIO<L> io, Remove r, int lvl)
             throws IgniteCheckedException {
             assert lvl > 0 : lvl; // We are not at the bottom.
 
             // Check that we have a correct view of the world.
-            if (io.getForward(buf) != r.fwdId)
+            if (io.getForward(pageAddr) != r.fwdId)
                 return RETRY;
 
             // We don't have a back page, need to lock our forward and become a back for it.
@@ -512,7 +511,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                     return res; // Retry.
             }
 
-            r.addTail(page, buf, io, lvl, Tail.EXACT);
+            r.addTail(page, pageAddr, io, lvl, Tail.EXACT);
 
             return FOUND;
         }
@@ -520,14 +519,14 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
     /** */
     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 pageAddr, Void ignore, int lvl)
             throws IgniteCheckedException {
             // Safe cast because we should never recycle meta page until the tree is destroyed.
             BPlusMetaIO io = (BPlusMetaIO)iox;
 
-            assert lvl == io.getRootLevel(buf); // Can drop only root.
+            assert lvl == io.getRootLevel(pageAddr); // Can drop only root.
 
-            io.cutRoot(buf);
+            io.cutRoot(pageAddr, pageSize());
 
             if (needWalDeltaRecord(meta))
                 wal.log(new MetaPageCutRootRecord(cacheId, meta.id()));
@@ -538,16 +537,16 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
     /** */
     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 pageAddr, Long rootPageId, int lvl)
             throws IgniteCheckedException {
             assert rootPageId != null;
 
             // Safe cast because we should never recycle meta page until the tree is destroyed.
             BPlusMetaIO io = (BPlusMetaIO)iox;
 
-            assert lvl == io.getLevelsCount(buf);
+            assert lvl == io.getLevelsCount(pageAddr);
 
-            io.addRoot(buf, rootPageId);
+            io.addRoot(pageAddr, rootPageId, pageSize());
 
             if (needWalDeltaRecord(meta))
                 wal.log(new MetaPageAddRootRecord(cacheId, meta.id(), rootPageId));
@@ -558,14 +557,14 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
     /** */
     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 pageAddr, Long rootId, int lvl)
             throws IgniteCheckedException {
             assert rootId != null;
 
             // Safe cast because we should never recycle meta page until the tree is destroyed.
             BPlusMetaIO io = (BPlusMetaIO)iox;
 
-            io.initRoot(buf, rootId);
+            io.initRoot(pageAddr, rootId, pageSize());
 
             if (needWalDeltaRecord(meta))
                 wal.log(new MetaPageInitRootRecord(cacheId, meta.id(), rootId));
@@ -579,10 +578,12 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
      * @param cacheId Cache ID.
      * @param pageMem Page memory.
      * @param wal Write ahead log manager.
+     * @param globalRmvId Remove ID.
      * @param metaPageId Meta page ID.
      * @param reuseList Reuse list.
      * @param innerIos Inner IO versions.
      * @param leafIos Leaf IO versions.
+     * @throws IgniteCheckedException If failed.
      */
     public BPlusTree(
         String name,
@@ -641,28 +642,29 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         long rootId = allocatePage(null);
 
         try (Page root = page(rootId)) {
-            initPage(root, this, latestLeafIO(), wal);
+            initPage(pageMem, root, this, latestLeafIO(), wal);
         }
 
         // Initialize meta page with new root page.
         try (Page meta = page(metaPageId)) {
-            Bool res = writePage(meta, this, initRoot, BPlusMetaIO.VERSIONS.latest(), wal, rootId, 0, FALSE);
+            Bool res = writePage(pageMem, meta, this, initRoot, BPlusMetaIO.VERSIONS.latest(), wal, rootId, 0, FALSE);
 
             assert res == TRUE: res;
         }
     }
 
     /**
+     * @param meta Meta page.
      * @return Root level.
      */
     private int getRootLevel(Page meta) {
-        ByteBuffer buf = readLock(meta); // Meta can't be removed.
+        long pageAddr = readLock(meta); // Meta can't be removed.
 
         try {
-            return BPlusMetaIO.VERSIONS.forPage(buf).getRootLevel(buf);
+            return BPlusMetaIO.VERSIONS.forPage(pageAddr).getRootLevel(pageAddr);
         }
         finally {
-            readUnlock(meta, buf);
+            readUnlock(meta, pageAddr);
         }
     }
 
@@ -672,27 +674,28 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
      * @return Page ID.
      */
     private long getFirstPageId(Page meta, int lvl) {
-        ByteBuffer buf = readLock(meta); // Meta can't be removed.
+        long pageAddr = readLock(meta); // Meta can't be removed.
 
         try {
-            BPlusMetaIO io = BPlusMetaIO.VERSIONS.forPage(buf);
+            BPlusMetaIO io = BPlusMetaIO.VERSIONS.forPage(pageAddr);
 
             if (lvl < 0)
-                lvl = io.getRootLevel(buf);
+                lvl = io.getRootLevel(pageAddr);
 
-            if (lvl >= io.getLevelsCount(buf))
+            if (lvl >= io.getLevelsCount(pageAddr))
                 return 0;
 
-            return io.getFirstPageId(buf, lvl);
+            return io.getFirstPageId(pageAddr, lvl);
         }
         finally {
-            readUnlock(meta, buf);
+            readUnlock(meta, pageAddr);
         }
     }
 
     /**
      * @param upper Upper bound.
      * @return Cursor.
+     * @throws IgniteCheckedException If failed.
      */
     private GridCursor<T> findLowerUnbounded(L upper) throws IgniteCheckedException {
         ForwardCursor cursor = new ForwardCursor(null, upper);
@@ -704,13 +707,13 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         }
 
         try (Page first = page(firstPageId)) {
-            ByteBuffer buf = readLock(first); // We always merge pages backwards, the first page is never removed.
+            long pageAddr = readLock(first); // We always merge pages backwards, the first page is never removed.
 
             try {
-                cursor.init(buf, io(buf), 0);
+                cursor.init(pageAddr, io(pageAddr), 0);
             }
             finally {
-                readUnlock(first, buf);
+                readUnlock(first, pageAddr);
             }
         }
 
@@ -758,6 +761,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
     /**
      * @param row Lookup row for exact match.
      * @return Found row.
+     * @throws IgniteCheckedException If failed.
      */
     @SuppressWarnings("unchecked")
     @Override public final T findOne(L row) throws IgniteCheckedException {
@@ -783,6 +787,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
     /**
      * @param g Get.
+     * @throws IgniteCheckedException If failed.
      */
     private void doFind(Get g) throws IgniteCheckedException {
         try {
@@ -921,22 +926,22 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
      */
     private void validateDownKeys(long pageId, L minRow) throws IgniteCheckedException {
         try (Page page = page(pageId)) {
-            ByteBuffer buf = readLock(page); // No correctness guaranties.
+            long pageAddr = readLock(page); // No correctness guaranties.
 
             try {
-                BPlusIO<L> io = io(buf);
+                BPlusIO<L> io = io(pageAddr);
 
-                int cnt = io.getCount(buf);
+                int cnt = io.getCount(pageAddr);
 
                 if (cnt < 0)
                     fail("Negative count: " + cnt);
 
                 if (io.isLeaf()) {
                     for (int i = 0; i < cnt; i++) {
-                        if (minRow != null && compare(io, buf, i, minRow) <= 0)
+                        if (minRow != null && compare(io, pageAddr, i, minRow) <= 0)
                             fail("Wrong sort order: " + U.hexLong(pageId) + " , at " + i + " , minRow: " + minRow);
 
-                        minRow = io.getLookupRow(this, buf, i);
+                        minRow = io.getLookupRow(this, pageAddr, i);
                     }
 
                     return;
@@ -944,16 +949,16 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
                 // To find our inner key we have to go left and then always go to the right.
                 for (int i = 0; i < cnt; i++) {
-                    L row = io.getLookupRow(this, buf, i);
+                    L row = io.getLookupRow(this, pageAddr, i);
 
-                    if (minRow != null && compare(io, buf, i, minRow) <= 0)
+                    if (minRow != null && compare(io, pageAddr, i, minRow) <= 0)
                         fail("Min row violated: " + row + " , minRow: " + minRow);
 
-                    long leftId = inner(io).getLeft(buf, i);
+                    long leftId = inner(io).getLeft(pageAddr, i);
 
                     L leafRow = getGreatestRowInSubTree(leftId);
 
-                    int cmp = compare(io, buf, i, leafRow);
+                    int cmp = compare(io, pageAddr, i, leafRow);
 
                     if (cmp < 0 || (cmp != 0 && canGetRowFromInner))
                         fail("Wrong inner row: " + U.hexLong(pageId) + " , at: " + i + " , leaf:  " + leafRow +
@@ -965,12 +970,12 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                 }
 
                 // Need to handle the rightmost child subtree separately or handle empty routing page.
-                long rightId = inner(io).getLeft(buf, cnt); // The same as getRight(cnt - 1)
+                long rightId = inner(io).getLeft(pageAddr, cnt); // The same as getRight(cnt - 1)
 
                 validateDownKeys(rightId, minRow);
             }
             finally {
-                readUnlock(page, buf);
+                readUnlock(page, pageAddr);
             }
         }
     }
@@ -982,26 +987,26 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
      */
     private L getGreatestRowInSubTree(long pageId) throws IgniteCheckedException {
         try (Page page = page(pageId)) {
-            ByteBuffer buf = readLock(page); // No correctness guaranties.
+            long pageAddr = readLock(page); // No correctness guaranties.
 
             try {
-                BPlusIO<L> io = io(buf);
+                BPlusIO<L> io = io(pageAddr);
 
-                int cnt = io.getCount(buf);
+                int cnt = io.getCount(pageAddr);
 
                 if (io.isLeaf()) {
                     if (cnt <= 0) // This code is called only if the tree is not empty, so we can't see empty leaf.
                         fail("Invalid leaf count: " + cnt + " " + U.hexLong(pageId));
 
-                    return io.getLookupRow(this, buf, cnt - 1);
+                    return io.getLookupRow(this, pageAddr, cnt - 1);
                 }
 
-                long rightId = inner(io).getLeft(buf, cnt);// The same as getRight(cnt - 1), but good for routing pages.
+                long rightId = inner(io).getLeft(pageAddr, cnt);// The same as getRight(cnt - 1), but good for routing pages.
 
                 return getGreatestRowInSubTree(rightId);
             }
             finally {
-                readUnlock(page, buf);
+                readUnlock(page, pageAddr);
             }
         }
     }
@@ -1037,18 +1042,18 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         long leftmostChildId;
 
         try (Page page = page(pageId)) {
-            ByteBuffer buf = readLock(page); // No correctness guaranties.
+            long pageAddr = readLock(page); // No correctness guaranties.
 
             try {
-                BPlusIO<L> io = io(buf);
+                BPlusIO<L> io = io(pageAddr);
 
                 if (io.isLeaf())
                     fail("Leaf.");
 
-                leftmostChildId = inner(io).getLeft(buf, 0);
+                leftmostChildId = inner(io).getLeft(pageAddr, 0);
             }
             finally {
-                readUnlock(page, buf);
+                readUnlock(page, pageAddr);
             }
         }
 
@@ -1067,25 +1072,25 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
      */
     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 pageAddr = readLock(page); // No correctness guaranties.
 
             try {
-                long realPageId = BPlusIO.getPageId(buf);
+                long realPageId = BPlusIO.getPageId(pageAddr);
 
                 if (realPageId != pageId)
                     fail(new SB("ABA on page ID: ref ").appendHex(pageId).a(", buf ").appendHex(realPageId));
 
-                BPlusIO<L> io = io(buf);
+                BPlusIO<L> io = io(pageAddr);
 
                 if (io.isLeaf() != (lvl == 0)) // Leaf pages only at the level 0.
                     fail("Leaf level mismatch: " + lvl);
 
-                long actualFwdId = io.getForward(buf);
+                long actualFwdId = io.getForward(pageAddr);
 
                 if (actualFwdId != fwdId)
                     fail(new SB("Triangle: expected fwd ").appendHex(fwdId).a(", actual fwd ").appendHex(actualFwdId));
 
-                int cnt = io.getCount(buf);
+                int cnt = io.getCount(pageAddr);
 
                 if (cnt < 0)
                     fail("Negative count: " + cnt);
@@ -1097,66 +1102,66 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                 else {
                     // Recursively go down if we are on inner level.
                     for (int i = 0; i < cnt; i++)
-                        validateDownPages(meta, inner(io).getLeft(buf, i), inner(io).getRight(buf, i), lvl - 1);
+                        validateDownPages(meta, inner(io).getLeft(pageAddr, i), inner(io).getRight(pageAddr, i), lvl - 1);
 
                     if (fwdId != 0) {
                         // For the rightmost child ask neighbor.
                         try (Page fwd = page(fwdId)) {
-                            ByteBuffer fwdBuf = readLock(fwd); // No correctness guaranties.
+                            long fwdPageAddr = readLock(fwd); // No correctness guaranties.
 
                             try {
-                                if (io(fwdBuf) != io)
+                                if (io(fwdPageAddr) != io)
                                     fail("IO on the same level must be the same");
 
-                                fwdId = inner(io).getLeft(fwdBuf, 0);
+                                fwdId = inner(io).getLeft(fwdPageAddr, 0);
                             }
                             finally {
-                                readUnlock(fwd, fwdBuf);
+                                readUnlock(fwd, fwdPageAddr);
                             }
                         }
                     }
 
-                    pageId = inner(io).getLeft(buf, cnt); // The same as io.getRight(cnt - 1) but works for routing pages.
+                    pageId = inner(io).getLeft(pageAddr, cnt); // The same as io.getRight(cnt - 1) but works for routing pages.
 
                     validateDownPages(meta, pageId, fwdId, lvl - 1);
                 }
             }
             finally {
-                readUnlock(page, buf);
+                readUnlock(page, pageAddr);
             }
         }
     }
 
     /**
      * @param io IO.
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param keys Keys.
      * @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 pageAddr, boolean keys) throws IgniteCheckedException {
         StringBuilder b = new StringBuilder();
 
-        b.append(formatPageId(PageIO.getPageId(buf)));
+        b.append(formatPageId(PageIO.getPageId(pageAddr)));
 
         b.append(" [ ");
         b.append(io.isLeaf() ? "L " : "I ");
 
-        int cnt = io.getCount(buf);
-        long fwdId = io.getForward(buf);
+        int cnt = io.getCount(pageAddr);
+        long fwdId = io.getForward(pageAddr);
 
         b.append("cnt=").append(cnt).append(' ');
         b.append("fwd=").append(formatPageId(fwdId)).append(' ');
 
         if (!io.isLeaf()) {
-            b.append("lm=").append(formatPageId(inner(io).getLeft(buf, 0))).append(' ');
+            b.append("lm=").append(formatPageId(inner(io).getLeft(pageAddr, 0))).append(' ');
 
             if (cnt > 0)
-                b.append("rm=").append(formatPageId(inner(io).getRight(buf, cnt - 1))).append(' ');
+                b.append("rm=").append(formatPageId(inner(io).getRight(pageAddr, cnt - 1))).append(' ');
         }
 
         if (keys)
-            b.append("keys=").append(printPageKeys(io, buf)).append(' ');
+            b.append("keys=").append(printPageKeys(io, pageAddr)).append(' ');
 
         b.append(']');
 
@@ -1165,12 +1170,12 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
     /**
      * @param io IO.
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return Keys as String.
      * @throws IgniteCheckedException If failed.
      */
-    private String printPageKeys(BPlusIO<L> io, ByteBuffer buf) throws IgniteCheckedException {
-        int cnt = io.getCount(buf);
+    private String printPageKeys(BPlusIO<L> io, long pageAddr) throws IgniteCheckedException {
+        int cnt = io.getCount(pageAddr);
 
         StringBuilder b = new StringBuilder();
 
@@ -1180,7 +1185,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             if (i != 0)
                 b.append(',');
 
-            b.append(io.isLeaf() || canGetRowFromInner ? getRow(io, buf, i) : io.getLookupRow(this, buf, i));
+            b.append(io.isLeaf() || canGetRowFromInner ? getRow(io, pageAddr, i) : io.getLookupRow(this, pageAddr, i));
         }
 
         b.append(']');
@@ -1487,21 +1492,21 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
         while (pageId != 0) {
             try (Page page = page(pageId)) {
-                ByteBuffer buf = readLock(page); // No correctness guaranties.
+                long pageAddr = readLock(page); // No correctness guaranties.
 
                 try {
                     if (io == null) {
-                        io = io(buf);
+                        io = io(pageAddr);
 
                         assert io.isLeaf();
                     }
 
-                    cnt += io.getCount(buf);
+                    cnt += io.getCount(pageAddr);
 
-                    pageId = io.getForward(buf);
+                    pageId = io.getForward(pageAddr);
                 }
                 finally {
-                    readUnlock(page, buf);
+                    readUnlock(page, pageAddr);
                 }
             }
         }
@@ -1531,9 +1536,9 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             for (;;) { // Go down with retries.
                 p.init();
 
-                Result result = putDown(p, p.rootId, 0L, p.rootLvl);
+                Result res = putDown(p, p.rootId, 0L, p.rootLvl);
 
-                switch (result) {
+                switch (res) {
                     case RETRY:
                     case RETRY_ROOT:
                         checkInterrupted();
@@ -1556,7 +1561,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                         return p.oldRow;
 
                     default:
-                        throw new IllegalStateException("Result: " + result);
+                        throw new IllegalStateException("Result: " + res);
                 }
             }
         }
@@ -1595,28 +1600,28 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         long pagesCnt = 0;
 
         try (Page meta = page(metaPageId)) {
-            ByteBuffer metaBuf = writeLock(meta); // No checks, we must be out of use.
+            long metaPageAddr = writeLock(meta); // No checks, we must be out of use.
 
             try {
-                for (long pageId : getFirstPageIds(metaBuf)) {
+                for (long pageId : getFirstPageIds(metaPageAddr)) {
                     assert pageId != 0;
 
                     do {
                         try (Page page = page(pageId)) {
-                            ByteBuffer buf = writeLock(page); // No checks, we must be out of use.
+                            long pageAddr = writeLock(page); // No checks, we must be out of use.
 
                             try {
-                                BPlusIO<L> io = io(buf);
+                                BPlusIO<L> io = io(pageAddr);
 
-                                long fwdPageId = io.getForward(buf);
+                                long fwdPageId = io.getForward(pageAddr);
 
-                                bag.addFreePage(recyclePage(pageId, page, buf));
+                                bag.addFreePage(recyclePage(pageId, page, pageAddr));
                                 pagesCnt++;
 
                                 pageId = fwdPageId;
                             }
                             finally {
-                                writeUnlock(page, buf, true);
+                                writeUnlock(page, pageAddr, true);
                             }
                         }
 
@@ -1629,11 +1634,11 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                     while (pageId != 0);
                 }
 
-                bag.addFreePage(recyclePage(metaPageId, meta, metaBuf));
+                bag.addFreePage(recyclePage(metaPageId, meta, metaPageAddr));
                 pagesCnt++;
             }
             finally {
-                writeUnlock(meta, metaBuf, true);
+                writeUnlock(meta, metaPageAddr, true);
             }
         }
 
@@ -1647,39 +1652,38 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
     /**
      * @return {@code True} if state was changed.
      */
-    protected final boolean markDestroyed() {
+    private final boolean markDestroyed() {
         return destroyed.compareAndSet(false, true);
     }
 
     /**
-     * @param metaBuf Meta page buffer.
+     * @param pageAddr Meta page address.
      * @return First page IDs.
      */
-    protected Iterable<Long> getFirstPageIds(ByteBuffer metaBuf) {
-        List<Long> result = new ArrayList<>();
+    protected Iterable<Long> getFirstPageIds(long pageAddr) {
+        List<Long> res = new ArrayList<>();
 
-        BPlusMetaIO mio = BPlusMetaIO.VERSIONS.forPage(metaBuf);
+        BPlusMetaIO mio = BPlusMetaIO.VERSIONS.forPage(pageAddr);
 
-        for (int lvl = mio.getRootLevel(metaBuf); lvl >= 0; lvl--) {
-            result.add(mio.getFirstPageId(metaBuf, lvl));
-        }
+        for (int lvl = mio.getRootLevel(pageAddr); lvl >= 0; lvl--)
+            res.add(mio.getFirstPageId(pageAddr, lvl));
 
-        return result;
+        return res;
     }
 
     /**
      * @param pageId Page ID.
      * @param page Page.
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @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 pageAddr) throws IgniteCheckedException {
         // Rotate page ID to avoid concurrency issues with reused pages.
         pageId = PageIdUtils.rotatePageId(pageId);
 
         // Update page ID inside of the buffer, Page.id() will always return the original page ID.
-        PageIO.setPageId(buf, pageId);
+        PageIO.setPageId(pageAddr, pageId);
 
         if (needWalDeltaRecord(page))
             wal.log(new RecycleRecord(cacheId, page.id(), pageId));
@@ -1701,10 +1705,10 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
     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);
@@ -1719,7 +1723,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         }
 
         // Update forward page.
-        io.splitForwardPage(buf, fwdId, fwdBuf, mid, cnt);
+        io.splitForwardPage(buf, fwdId, fwdBuf, mid, cnt, pageSize());
 
         // TODO GG-11640 log a correct forward page record.
         fwd.fullPageWalRecordPolicy(Boolean.TRUE);
@@ -1735,11 +1739,11 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
     /**
      * @param page Page.
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      */
-    private void writeUnlockAndClose(Page page, ByteBuffer buf) {
+    private void writeUnlockAndClose(Page page, long pageAddr) {
         try {
-            writeUnlock(page, buf, true);
+            writeUnlock(page, pageAddr, true);
         }
         finally {
             page.close();
@@ -1798,7 +1802,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                             p.fwdId = fwdId;
                             p.pageId = pageId;
 
-                            res = writePage(page, this, replace, p, lvl, RETRY);
+                            res = writePage(pageMem, page, this, replace, p, lvl, RETRY);
 
                             // Restore args.
                             p.pageId = oldPageId;
@@ -1828,7 +1832,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                         p.pageId = pageId;
                         p.fwdId = fwdId;
 
-                        return writePage(page, this, replace, p, lvl, RETRY);
+                        return writePage(pageMem, page, this, replace, p, lvl, RETRY);
 
                     case NOT_FOUND: // Do insert.
                         assert lvl == p.btmLvl : "must insert at the bottom level";
@@ -1838,7 +1842,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                         p.pageId = pageId;
                         p.fwdId = fwdId;
 
-                        return writePage(page, this, insert, p, lvl, RETRY);
+                        return writePage(pageMem, page, this, insert, p, lvl, RETRY);
 
                     default:
                         return res;
@@ -1853,25 +1857,25 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
     /**
      * @param io IO.
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param back Backward page.
      * @return Page ID.
      */
-    private long doAskNeighbor(BPlusIO<L> io, ByteBuffer buf, boolean back) {
+    private long doAskNeighbor(BPlusIO<L> io, long pageAddr, boolean back) {
         long res;
 
         if (back) {
             // Count can be 0 here if it is a routing page, in this case we have a single child.
-            int cnt = io.getCount(buf);
+            int cnt = io.getCount(pageAddr);
 
             // We need to do get the rightmost child: io.getRight(cnt - 1),
             // here io.getLeft(cnt) is the same, but handles negative index if count is 0.
-            res = inner(io).getLeft(buf, cnt);
+            res = inner(io).getLeft(pageAddr, cnt);
         }
         else // Leftmost child.
-            res = inner(io).getLeft(buf, 0);
+            res = inner(io).getLeft(pageAddr, 0);
 
-        assert res != 0 : "inner page with no route down: " + U.hexLong(PageIO.getPageId(buf));
+        assert res != 0 : "inner page with no route down: " + U.hexLong(PageIO.getPageId(pageAddr));
 
         return res;
     }
@@ -1935,19 +1939,19 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             int rootLvl;
             long rootId;
 
-            ByteBuffer buf = readLock(meta); // Meta can't be removed.
+            long pageAddr = readLock(meta); // Meta can't be removed.
 
-            assert buf != null : "Failed to read lock meta page [page=" + meta + ", metaPageId=" +
+            assert pageAddr != 0 : "Failed to read lock meta page [page=" + meta + ", metaPageId=" +
                 U.hexLong(metaPageId) + ']';
 
             try {
-                BPlusMetaIO io = BPlusMetaIO.VERSIONS.forPage(buf);
+                BPlusMetaIO io = BPlusMetaIO.VERSIONS.forPage(pageAddr);
 
-                rootLvl = io.getRootLevel(buf);
-                rootId = io.getFirstPageId(buf, rootLvl);
+                rootLvl = io.getRootLevel(pageAddr);
+                rootId = io.getFirstPageId(pageAddr, rootLvl);
             }
             finally {
-                readUnlock(meta, buf);
+                readUnlock(meta, pageAddr);
             }
 
             restartFromRoot(rootId, rootLvl, globalRmvId.get());
@@ -1966,13 +1970,13 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
         /**
          * @param io IO.
-         * @param buf Buffer.
+         * @param pageAddr Page address.
          * @param idx Index of found entry.
          * @param lvl Level.
          * @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 pageAddr, int idx, int lvl) throws IgniteCheckedException {
             assert lvl >= 0;
 
             return lvl == 0; // Stop if we are at the bottom.
@@ -1980,13 +1984,13 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
         /**
          * @param io IO.
-         * @param buf Buffer.
+         * @param pageAddr Page address.
          * @param idx Insertion point.
          * @param lvl Level.
          * @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 pageAddr, int idx, int lvl) throws IgniteCheckedException {
             assert lvl >= 0;
 
             return lvl == 0; // Stop if we are at the bottom.
@@ -2024,12 +2028,12 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         }
 
         /** {@inheritDoc} */
-        @Override boolean found(BPlusIO<L> io, ByteBuffer buf, int idx, int lvl) throws IgniteCheckedException {
+        @Override boolean found(BPlusIO<L> io, long pageAddr, 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;
 
-            row = getRow(io, buf, idx);
+            row = getRow(io, pageAddr, idx);
 
             return true;
         }
@@ -2057,16 +2061,16 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         }
 
         /** {@inheritDoc} */
-        @Override boolean found(BPlusIO<L> io, ByteBuffer buf, int idx, int lvl) throws IgniteCheckedException {
+        @Override boolean found(BPlusIO<L> io, long pageAddr, 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 pageAddr, int idx, int lvl) throws IgniteCheckedException {
             if (lvl != 0)
                 return false;
 
-            cursor.init(buf, io, idx);
+            cursor.init(pageAddr, io, idx);
 
             return true;
         }
@@ -2090,7 +2094,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         private Page tail;
 
         /** */
-        private ByteBuffer tailBuf;
+        private long tailPageAddr;
 
         /**
          * Bottom level for insertion (insert can't go deeper). Will be incremented on split on each level.
@@ -2114,7 +2118,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         }
 
         /** {@inheritDoc} */
-        @Override boolean found(BPlusIO<L> io, ByteBuffer buf, int idx, int lvl) {
+        @Override boolean found(BPlusIO<L> io, long pageAddr, int idx, int lvl) {
             if (lvl == 0) // Leaf: need to stop.
                 return true;
 
@@ -2128,7 +2132,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         }
 
         /** {@inheritDoc} */
-        @Override boolean notFound(BPlusIO<L> io, ByteBuffer buf, int idx, int lvl) {
+        @Override boolean notFound(BPlusIO<L> io, long pageAddr, int idx, int lvl) {
             assert btmLvl >= 0 : btmLvl;
             assert lvl >= btmLvl : lvl;
 
@@ -2137,16 +2141,16 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
         /**
          * @param tail Tail page.
-         * @param tailBuf Tail buffer.
+         * @param tailPageAddr Tail page address.
          */
-        private void tail(Page tail, ByteBuffer tailBuf) {
-            assert (tail == null) == (tailBuf == null);
+        private void tail(Page tail, long tailPageAddr) {
+            assert (tail == null) == (tailPageAddr == 0L);
 
             if (this.tail != null)
-                writeUnlockAndClose(this.tail, this.tailBuf);
+                writeUnlockAndClose(this.tail, this.tailPageAddr);
 
             this.tail = tail;
-            this.tailBuf = tailBuf;
+            this.tailPageAddr = tailPageAddr;
         }
 
         /** {@inheritDoc} */
@@ -2161,7 +2165,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             row = null;
             rightId = 0;
 
-            tail(null, null);
+            tail(null, 0L);
         }
 
         /**
@@ -2174,21 +2178,21 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         /**
          * @param page Page.
          * @param io IO.
-         * @param buf Buffer.
+         * @param pageAddr Page address.
          * @param idx Index.
          * @param lvl Level.
          * @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 pageAddr, int idx, int lvl)
             throws IgniteCheckedException {
-            int maxCnt = io.getMaxCount(buf);
-            int cnt = io.getCount(buf);
+            int maxCnt = io.getMaxCount(pageAddr, pageSize());
+            int cnt = io.getCount(pageAddr);
 
             if (cnt == maxCnt) // Need to split page.
-                return insertWithSplit(page, io, buf, idx, lvl);
+                return insertWithSplit(page, io, pageAddr, idx, lvl);
 
-            insertSimple(page, io, buf, idx);
+            insertSimple(page, io, pageAddr, idx);
 
             return null;
         }
@@ -2196,13 +2200,13 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         /**
          * @param page Page.
          * @param io IO.
-         * @param buf Buffer.
+         * @param pageAddr Page address.
          * @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 pageAddr, int idx)
             throws IgniteCheckedException {
-            io.insert(buf, idx, row, null, rightId);
+            io.insert(pageAddr, idx, row, null, rightId);
 
             if (needWalDeltaRecord(page))
                 wal.log(new InsertRecord<>(cacheId, page.id(), io, idx, row, null, rightId));
@@ -2211,55 +2215,55 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         /**
          * @param page Page.
          * @param io IO.
-         * @param buf Buffer.
+         * @param pageAddr Page address.
          * @param idx Index.
          * @param lvl Level.
          * @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 pageAddr, int idx, int lvl)
             throws IgniteCheckedException {
             long fwdId = allocatePage(bag);
 
             try (Page fwd = page(fwdId)) {
                 // 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;
+                boolean hadFwd = io.getForward(pageAddr) != 0;
 
-                ByteBuffer fwdBuf = writeLock(fwd); // Initial write, no need to check for concurrent modification.
+                long fwdPageAddr = writeLock(fwd); // Initial write, no need to check for concurrent modification.
 
-                assert fwdBuf != null;
+                assert fwdPageAddr != 0L;
 
                 try {
                     // Never write full forward page, because it is known to be new.
                     fwd.fullPageWalRecordPolicy(Boolean.FALSE);
 
-                    boolean midShift = splitPage(io, page, buf, fwdId, fwd, fwdBuf, idx);
+                    boolean midShift = splitPage(io, page, pageAddr, fwdId, fwd, fwdPageAddr, idx);
 
                     // Do insert.
-                    int cnt = io.getCount(buf);
+                    int cnt = io.getCount(pageAddr);
 
                     if (idx < cnt || (idx == cnt && !midShift)) { // Insert into back page.
-                        insertSimple(page, io, buf, idx);
+                        insertSimple(page, io, pageAddr, idx);
 
                         // Fix leftmost child of forward page, because newly inserted row will go up.
                         if (idx == cnt && !io.isLeaf()) {
-                            inner(io).setLeft(fwdBuf, 0, rightId);
+                            inner(io).setLeft(fwdPageAddr, 0, rightId);
 
                             if (needWalDeltaRecord(fwd)) // Rare case, we can afford separate WAL record to avoid complexity.
                                 wal.log(new FixLeftmostChildRecord(cacheId, fwd.id(), rightId));
                         }
                     }
                     else // Insert into newly allocated forward page.
-                        insertSimple(fwd, io, fwdBuf, idx - cnt);
+                        insertSimple(fwd, io, fwdPageAddr, idx - cnt);
 
                     // Do move up.
-                    cnt = io.getCount(buf);
+                    cnt = io.getCount(pageAddr);
 
                     // Last item from backward row goes up.
-                    L moveUpRow = io.getLookupRow(BPlusTree.this, buf, cnt - 1);
+                    L moveUpRow = io.getLookupRow(BPlusTree.this, pageAddr, cnt - 1);
 
                     if (!io.isLeaf()) { // Leaf pages must contain all the links, inner pages remove moveUpLink.
-                        io.setCount(buf, cnt - 1);
+                        io.setCount(pageAddr, cnt - 1);
 
                         if (needWalDeltaRecord(page)) // Rare case, we can afford separate WAL record to avoid complexity.
                             wal.log(new FixCountRecord(cacheId, page.id(), cnt - 1));
@@ -2272,28 +2276,34 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                             if (io.isLeaf())
                                 io = latestInnerIO();
 
-                            ByteBuffer newRootBuf = writeLock(newRoot); // Initial write.
+                            long newRootPageAddr = writeLock(newRoot); // Initial write.
 
-                            assert newRootBuf != null;
+                            assert newRootPageAddr != 0L;
 
                             try {
                                 // Never write full new root page, because it is known to be new.
                                 newRoot.fullPageWalRecordPolicy(Boolean.FALSE);
 
-                                long pageId = PageIO.getPageId(buf);
+                                long pageId = PageIO.getPageId(pageAddr);
 
-                                inner(io).initNewRoot(newRootBuf, newRootId, pageId, moveUpRow, null, fwdId);
+                                inner(io).initNewRoot(newRootPageAddr,
+                                    newRootId,
+                                    pageId,
+                                    moveUpRow,
+                                    null,
+                                    fwdId,
+                                    pageSize());
 
                                 if (needWalDeltaRecord(newRoot))
                                     wal.log(new NewRootInitRecord<>(cacheId, newRoot.id(), newRootId,
                                         inner(io), pageId, moveUpRow, null, fwdId));
                             }
                             finally {
-                                writeUnlock(newRoot, newRootBuf, true);
+                                writeUnlock(newRoot, newRootPageAddr, true);
                             }
                         }
 
-                        Bool res = writePage(meta, BPlusTree.this, addRoot, newRootId, lvl + 1, FALSE);
+                        Bool res = writePage(pageMem, meta, BPlusTree.this, addRoot, newRootId, lvl + 1, FALSE);
 
                         assert res == TRUE: res;
 
@@ -2304,7 +2314,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                     return moveUpRow;
                 }
                 finally {
-                    writeUnlock(fwd, fwdBuf, true);
+                    writeUnlock(fwd, fwdPageAddr, true);
                 }
             }
         }
@@ -2402,7 +2412,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         }
 
         /** {@inheritDoc} */
-        @Override boolean notFound(BPlusIO<L> io, ByteBuffer buf, int idx, int lvl) {
+        @Override boolean notFound(BPlusIO<L> io, long pageAddr, int idx, int lvl) {
             if (lvl == 0) {
                 assert tail == null;
 
@@ -2622,7 +2632,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                         // Exit: we are done.
                     }
                     else if (tail.sibling != null &&
-                        tail.getCount() + tail.sibling.getCount() < tail.io.getMaxCount(tail.buf)) {
+                        tail.getCount() + tail.sibling.getCount() < tail.io.getMaxCount(tail.buf, pageSize())) {
                         // Release everything lower than tail, we've already merged this path.
                         doReleaseTail(tail.down);
                         tail.down = null;
@@ -2678,7 +2688,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             Page back = page(backId);
 
             try {
-                return writePage(back, BPlusTree.this, lockBackAndRemoveFromLeaf, this, 0, RETRY);
+                return writePage(pageMem, back, BPlusTree.this, lockBackAndRmvFromLeaf, this, 0, RETRY);
             }
             finally {
                 if (canRelease(back, 0))
@@ -2693,7 +2703,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         private Result doRemoveFromLeaf() throws IgniteCheckedException {
             assert page != null;
 
-            return writePage(page, BPlusTree.this, removeFromLeaf, this, 0, RETRY);
+            return writePage(pageMem, page, BPlusTree.this, rmvFromLeaf, this, 0, RETRY);
         }
 
         /**
@@ -2704,7 +2714,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         private Result doLockTail(int lvl) throws IgniteCheckedException {
             assert page != null;
 
-            return writePage(page, BPlusTree.this, lockTail, this, lvl, RETRY);
+            return writePage(pageMem, page, BPlusTree.this, lockTail, this, lvl, RETRY);
         }
 
         /**
@@ -2732,7 +2742,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             Page back = page(backId);
 
             try {
-                return writePage(back, BPlusTree.this, lockBackAndTail, this, lvl, RETRY);
+                return writePage(pageMem, back, BPlusTree.this, lockBackAndTail, this, lvl, RETRY);
             }
             finally {
                 if (canRelease(back, lvl))
@@ -2752,7 +2762,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             Page fwd = page(fwdId);
 
             try {
-                return writePage(fwd, BPlusTree.this, lockTailForward, this, lvl, RETRY);
+                return writePage(pageMem, fwd, BPlusTree.this, lockTailForward, this, lvl, RETRY);
             }
             finally {
                 // If we were not able to lock forward page as tail, release the page.
@@ -2764,21 +2774,21 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         /**
          * @param page Page.
          * @param io IO.
-         * @param buf Buffer.
+         * @param pageAddr Page address.
          * @param cnt Count.
          * @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 pageAddr, int cnt, int idx)
             throws IgniteCheckedException {
             assert idx >= 0 && idx < cnt: idx;
             assert io.isLeaf(): "inner";
             assert !isRemoved(): "already removed";
 
             // Detach the row.
-            removed = getRow(io, buf, idx);
+            removed = getRow(io, pageAddr, idx);
 
-            doRemove(page, io, buf, cnt, idx);
+            doRemove(page, io, pageAddr, cnt, idx);
 
             assert isRemoved();
         }
@@ -2786,17 +2796,17 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         /**
          * @param page Page.
          * @param io IO.
-         * @param buf Buffer.
+         * @param pageAddr Page address.
          * @param cnt Count.
          * @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 pageAddr, int cnt, int idx)
             throws IgniteCheckedException {
             assert cnt > 0 : cnt;
             assert idx >= 0 && idx < cnt : idx + " " + cnt;
 
-            io.remove(buf, idx, cnt);
+            io.remove(pageAddr, idx, cnt);
 
             if (needWalDeltaRecord(page))
                 wal.log(new RemoveRecord(cacheId, page.id(), idx, cnt));
@@ -2937,7 +2947,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
             boolean emptyBranch = needMergeEmptyBranch == TRUE || needMergeEmptyBranch == READY;
 
-            if (!left.io.merge(prnt.io, prnt.buf, prntIdx, left.buf, right.buf, emptyBranch))
+            if (!left.io.merge(prnt.io, prnt.buf, prntIdx, left.buf, right.buf, emptyBranch, pageSize()))
                 return false;
 
             // Invalidate indexes after successful merge.
@@ -2959,23 +2969,23 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
         /**
          * @param page Page.
-         * @param buf Buffer.
+         * @param pageAddr Page address.
          * @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 pageAddr, boolean release)
             throws IgniteCheckedException {
             long pageId = page.id();
 
             long effectivePageId = PageIdUtils.effectivePageId(pageId);
 
-            pageId = recyclePage(pageId, page, buf);
+            pageId = recyclePage(pageId, page, pageAddr);
 
             if (effectivePageId != PageIdUtils.effectivePageId(pageId))
                 throw new IllegalStateException("Effective page ID must stay the same.");
 
             if (release)
-                writeUnlockAndClose(page, buf);
+                writeUnlockAndClose(page, pageAddr);
 
             bag().addFreePage(pageId);
         }
@@ -2985,7 +2995,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
          * @throws IgniteCheckedException If failed.
          */
         private void cutRoot(int lvl) throws IgniteCheckedException {
-            Bool res = writePage(meta, BPlusTree.this, cutRoot, null, lvl, FALSE);
+            Bool res = writePage(pageMem, meta, BPlusTree.this, cutRoot, null, lvl, FALSE);
 
             assert res == TRUE: res;
         }
@@ -3156,14 +3166,14 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
         /**
          * @param page Page.
-         * @param buf Buffer.
+         * @param pageAddr Page address.
          * @param io IO.
          * @param lvl Level.
          * @param type Type.
          * @return Added tail.
          */
-        private Tail<L> addTail(Page page, ByteBuffer buf, BPlusIO<L> io, int lvl, byte type) {
-            final Tail<L> t = new Tail<>(page, buf, io, type, lvl);
+        private Tail<L> addTail(Page page, long pageAddr, BPlusIO<L> io, int lvl, byte type) {
+            final Tail<L> t = new Tail<>(page, pageAddr, io, type, lvl);
 
             if (tail == null)
                 tail = t;
@@ -3270,7 +3280,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         private final Page page;
 
         /** */
-        private final ByteBuffer buf;
+        private final long buf;
 
         /** */
         private final BPlusIO<L> io;
@@ -3297,7 +3307,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
          * @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;
@@ -3350,7 +3360,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
      * @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;
 
@@ -3377,14 +3387,14 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return IO.
      */
-    private BPlusIO<L> io(ByteBuffer buf) {
-        assert buf != null;
+    private BPlusIO<L> io(long pageAddr) {
+        assert pageAddr != 0;
 
-        int type = PageIO.getType(buf);
-        int ver = PageIO.getVersion(buf);
+        int type = PageIO.getType(pageAddr);
+        int ver = PageIO.getVersion(pageAddr);
 
         if (innerIos.getType() == type)
             return innerIos.forVersion(ver);
@@ -3392,7 +3402,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         if (leafIos.getType() == type)
             return leafIos.forVersion(ver);
 
-        throw new IllegalStateException("Unknown page type: " + type + " pageId: " + U.hexLong(PageIO.getPageId(buf)));
+        throw new IllegalStateException("Unknown page type: " + type + " pageId: " + U.hexLong(PageIO.getPageId(pageAddr)));
     }
 
     /**
@@ -3420,24 +3430,24 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param idx Index of row in the given buffer.
      * @param row Lookup row.
      * @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 pageAddr, int idx, L row) throws IgniteCheckedException;
 
     /**
      * Get the full detached row. Can be called on inner page only if {@link #canGetRowFromInner} is {@code true}.
      *
      * @param io IO.
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param idx Index.
      * @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 pageAddr, int idx) throws IgniteCheckedException;
 
     /**
      * Forward cursor.
@@ -3472,24 +3482,24 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         }
 
         /**
-         * @param buf Buffer.
+         * @param pageAddr Page address.
          * @param io IO.
          * @param startIdx Start index.
          * @throws IgniteCheckedException If failed.
          */
-        private void init(ByteBuffer buf, BPlusIO<L> io, int startIdx) throws IgniteCheckedException {
+        private void init(long pageAddr, BPlusIO<L> io, int startIdx) throws IgniteCheckedException {
             nextPageId = 0;
             row = -1;
 
-            int cnt = io.getCount(buf);
+            int cnt = io.getCount(pageAddr);
 
             // If we see an empty page here, it means that it is an empty tree.
             if (cnt == 0) {
-                assert io.getForward(buf) == 0L;
+                assert io.getForward(pageAddr) == 0L;
 
                 rows = null;
             }
-            else if (!fillFromBuffer(buf, io, startIdx, cnt)) {
+            else if (!fillFromBuffer(pageAddr, io, startIdx, cnt)) {
                 if (rows != EMPTY) {
                     assert rows.length > 0; // Otherwise it makes no sense to create an array.
 
@@ -3500,18 +3510,18 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         }
 
         /**
-         * @param buf Buffer.
+         * @param pageAddr Page address.
          * @param io IO.
          * @param cnt Count.
          * @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 pageAddr, BPlusIO<L> io, int cnt) throws IgniteCheckedException {
             // Compare with the first row on the page.
-            int cmp = compare(io, buf, 0, lowerBound);
+            int cmp = compare(io, pageAddr, 0, lowerBound);
 
             if (cmp < 0 || (cmp == 0 && lowerShift == 1)) {
-                int idx = findInsertionPoint(io, buf, 0, cnt, lowerBound, lowerShift);
+                int idx = findInsertionPoint(io, pageAddr, 0, cnt, lowerBound, lowerShift);
 
                 assert idx < 0;
 
@@ -3522,19 +3532,19 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         }
 
         /**
-         * @param buf Buffer.
+         * @param pageAddr Page address.
          * @param io IO.
          * @param low Start index.
          * @param cnt Number of rows in the buffer.
          * @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 pageAddr, 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);
+            int cmp = compare(io, pageAddr, cnt - 1, upperBound);
 
             if (cmp > 0) {
-                int idx = findInsertionPoint(io, buf, low, cnt, upperBound, 1);
+                int idx = findInsertionPoint(io, pageAddr, low, cnt, upperBound, 1);
 
                 assert idx < 0;
 
@@ -3547,7 +3557,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         }
 
         /**
-         * @param buf Buffer.
+         * @param pageAddr Page address.
          * @param io IO.
          * @param startIdx Start index.
          * @param cnt Number of rows in the buffer.
@@ -3555,7 +3565,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
          * @throws IgniteCheckedException If failed.
          */
         @SuppressWarnings("unchecked")
-        private boolean fillFromBuffer(ByteBuffer buf, BPlusIO<L> io, int startIdx, int cnt)
+        private boolean fillFromBuffer(long pageAddr, BPlusIO<L> io, int startIdx, int cnt)
             throws IgniteCheckedException {
             assert io.isLeaf() : io;
             assert cnt != 0 : cnt; // We can not see empty pages (empty tree handled in init).
@@ -3564,13 +3574,13 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
             checkDestroyed();
 
-            nextPageId = io.getForward(buf);
+            nextPageId = io.getForward(pageAddr);
 
             if (lowerBound != null && startIdx == 0)
-                startIdx = findLowerBound(buf, io, cnt);
+                startIdx = findLowerBound(pageAddr, io, cnt);
 
             if (upperBound != null && cnt != startIdx)
-                cnt = findUpperBound(buf, io, startIdx, cnt);
+                cnt = findUpperBound(pageAddr, io, startIdx, cnt);
 
             cnt -= startIdx;
 
@@ -3581,7 +3591,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                 rows = (T[])new Object[cnt];
 
             for (int i = 0; i < cnt; i++) {
-                T r = getRow(io, buf, startIdx + i);
+                T r = getRow(io, pageAddr, startIdx + i);
 
                 rows = GridArrays.set(rows, i, r);
             }
@@ -3662,22 +3672,22 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                 }
 
                 try (Page next = page(nextPageId)) {
-                    ByteBuffer buf = readLock(next); // Doing explicit null check.
+                    long pageAddr = readLock(next); // Doing explicit null check.
 
                     // If concurrent merge occurred we have to reinitialize cursor from the last returned row.
-                    if (buf == null)
+                    if (pageAddr == 0L)
                         break;
 
                     try {
-                        BPlusIO<L> io = io(buf);
+                        BPlusIO<L> io = io(pageAddr);
 
-                        if (fillFromBuffer(buf, io, 0, io.getCount(buf)))
+                        if (fillFromBuffer(pageAddr, io, 0, io.getCount(pageAddr)))
                             return true;
 
                         // Continue fetching forward.
                     }
                     finally {
-                        readUnlock(next, buf);
+                        readUnlock(next, pageAddr);
                     }
                 }
             }
@@ -3712,31 +3722,31 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
     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 pageAddr, G g, int lvl)
             throws IgniteCheckedException {
-            assert PageIO.getPageId(buf) == page.id();
+            assert PageIO.getPageId(pageAddr) == page.id();
 
             // If we've passed the check for correct page ID, we can safely cast.
             BPlusIO<L> io = (BPlusIO<L>)iox;
 
             // In case of intersection with inner replace in remove operation
             // we need to restart our operation from the tree root.
-            if (lvl == 0 && g.rmvId < io.getRemoveId(buf))
+            if (lvl == 0 && g.rmvId < io.getRemoveId(pageAddr))
                 return RETRY_ROOT;
 
-            return run0(page, buf, io, g, lvl);
+            return run0(page, pageAddr, io, g, lvl);
         }
 
         /**
          * @param page Page.
-         * @param buf Buffer.
+         * @param pageAddr Page address.
          * @param io IO.
          * @param g Operation.
          * @param lvl Level.
          * @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 pageAddr, BPlusIO<L> io, G g, int lvl)
             throws IgniteCheckedException;
 
         /** {@inheritDoc} */
@@ -3774,13 +3784,39 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
      * Operation result.
      */
     enum Result {
-        GO_DOWN, GO_DOWN_X, FOUND, NOT_FOUND, RETRY, RETRY_ROOT
+        /** */
+        GO_DOWN,
+
+        /** */
+        GO_DOWN_X,
+
+        /** */
+        FOUND,
+
+        /** */
+        NOT_FOUND,
+
+        /** */
+        RETRY,
+
+        /** */
+        RETRY_ROOT
     }
 
     /**
      * Four state boolean.
      */
     enum Bool {
-        FALSE, TRUE, READY, DONE
+        /** */
+        FALSE,
+
+        /** */
+        TRUE,
+
+        /** */
+        READY,
+
+        /** */
+        DONE
     }
 }