You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by se...@apache.org on 2017/04/06 17:02:48 UTC

[02/12] ignite git commit: GC pressure

http://git-wip-us.apache.org/repos/asf/ignite/blob/37eed342/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 9597f87..5a8c49c 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
@@ -28,7 +28,6 @@ import java.util.concurrent.atomic.AtomicLong;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.internal.IgniteInterruptedCheckedException;
-import org.apache.ignite.internal.pagemem.Page;
 import org.apache.ignite.internal.pagemem.PageIdUtils;
 import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager;
@@ -40,7 +39,6 @@ import org.apache.ignite.internal.pagemem.wal.record.delta.MetaPageAddRootRecord
 import org.apache.ignite.internal.pagemem.wal.record.delta.MetaPageCutRootRecord;
 import org.apache.ignite.internal.pagemem.wal.record.delta.MetaPageInitRootInlineRecord;
 import org.apache.ignite.internal.pagemem.wal.record.delta.NewRootInitRecord;
-import org.apache.ignite.internal.pagemem.wal.record.delta.RecycleRecord;
 import org.apache.ignite.internal.pagemem.wal.record.delta.RemoveRecord;
 import org.apache.ignite.internal.pagemem.wal.record.delta.ReplaceRecord;
 import org.apache.ignite.internal.pagemem.wal.record.delta.SplitExistingPageRecord;
@@ -75,10 +73,6 @@ import static org.apache.ignite.internal.processors.cache.database.tree.BPlusTre
 import static org.apache.ignite.internal.processors.cache.database.tree.BPlusTree.Result.NOT_FOUND;
 import static org.apache.ignite.internal.processors.cache.database.tree.BPlusTree.Result.RETRY;
 import static org.apache.ignite.internal.processors.cache.database.tree.BPlusTree.Result.RETRY_ROOT;
-import static org.apache.ignite.internal.processors.cache.database.tree.util.PageHandler.initPage;
-import static org.apache.ignite.internal.processors.cache.database.tree.util.PageHandler.isWalDeltaRecordNeeded;
-import static org.apache.ignite.internal.processors.cache.database.tree.util.PageHandler.readPage;
-import static org.apache.ignite.internal.processors.cache.database.tree.util.PageHandler.writePage;
 
 /**
  * Abstract B+Tree.
@@ -126,71 +120,86 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         /** */
         private boolean keys = true;
 
-        @Override protected List<Long> getChildren(Long pageId) {
+        @Override protected List<Long> getChildren(final Long pageId) {
             if (pageId == null || pageId == 0L)
                 return null;
 
-            try (Page page = page(pageId)) {
-                long pageAddr = readLock(page); // No correctness guaranties.
+            try {
+                long page = acquirePage(pageId);
 
                 try {
-                    BPlusIO io = io(pageAddr);
+                    long pageAddr = readLock(pageId, page); // No correctness guaranties.
 
-                    if (io.isLeaf())
-                        return null;
+                    try {
+                        BPlusIO io = io(pageAddr);
 
-                    int cnt = io.getCount(pageAddr);
+                        if (io.isLeaf())
+                            return null;
 
-                    assert cnt >= 0 : cnt;
+                        int cnt = io.getCount(pageAddr);
 
-                    List<Long> res;
+                        assert cnt >= 0 : cnt;
 
-                    if (cnt > 0) {
-                        res = new ArrayList<>(cnt + 1);
+                        List<Long> res;
 
-                        for (int i = 0; i < cnt; i++)
-                            res.add(inner(io).getLeft(pageAddr, i));
+                        if (cnt > 0) {
+                            res = new ArrayList<>(cnt + 1);
 
-                        res.add(inner(io).getRight(pageAddr, cnt - 1));
-                    }
-                    else {
-                        long left = inner(io).getLeft(pageAddr, 0);
+                            for (int i = 0; i < cnt; i++)
+                                res.add(inner(io).getLeft(pageAddr, i));
 
-                        res = left == 0 ? Collections.<Long>emptyList() : Collections.singletonList(left);
-                    }
+                            res.add(inner(io).getRight(pageAddr, cnt - 1));
+                        }
+                        else {
+                            long left = inner(io).getLeft(pageAddr, 0);
+
+                            res = left == 0 ? Collections.<Long>emptyList() : Collections.singletonList(left);
+                        }
 
-                    return res;
+                        return res;
+                    }
+                    finally {
+                        readUnlock(pageId, page, pageAddr);
+                    }
                 }
                 finally {
-                    readUnlock(page, pageAddr);
+                    releasePage(pageId, page);
                 }
             }
-            catch (IgniteCheckedException e) {
-                throw new IllegalStateException(e);
+            catch (IgniteCheckedException ignored) {
+                throw new AssertionError("Can not acquire page.");
             }
         }
 
-        @Override protected String formatTreeNode(Long pageId) {
+        @Override protected String formatTreeNode(final Long pageId) {
             if (pageId == null)
                 return ">NPE<";
 
             if (pageId == 0L)
                 return "<Zero>";
 
-            try (Page page = page(pageId)) {
-                long pageAddr = readLock(page); // No correctness guaranties.
-
+            try {
+                long page = acquirePage(pageId);
                 try {
-                    BPlusIO<L> io = io(pageAddr);
+                    long pageAddr = readLock(pageId, page); // No correctness guaranties.
+                    try {
+                        BPlusIO<L> io = io(pageAddr);
 
-                    return printPage(io, pageAddr, keys);
+                        return printPage(io, pageAddr, keys);
+                    }
+                    catch (IgniteCheckedException e) {
+                        throw new IllegalStateException(e);
+                    }
+                    finally {
+                        readUnlock(pageId, page, pageAddr);
+                    }
                 }
                 finally {
-                    readUnlock(page, pageAddr);
+                    releasePage(pageId, page);
                 }
             }
-            catch (IgniteCheckedException e) {
-                throw new IllegalStateException(e);
+            catch (IgniteCheckedException ignored) {
+                throw new AssertionError("Can not acquire page.");
             }
         }
     };
@@ -203,7 +212,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
      */
     private class AskNeighbor extends GetPageHandler<Get> {
         /** {@inheritDoc} */
-        @Override public Result run0(Page page, long pageAddr, BPlusIO<L> io, Get g, int isBack) {
+        @Override public Result run0(long pageId, long page, long pageAddr, BPlusIO<L> io, Get g, int isBack) {
             assert !io.isLeaf(); // Inner page.
 
             boolean back = isBack == TRUE.ordinal();
@@ -234,7 +243,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
      */
     private class Search extends GetPageHandler<Get> {
         /** {@inheritDoc} */
-        @Override public Result run0(Page page, long pageAddr, BPlusIO<L> io, Get g, int lvl)
+        @Override public Result run0(long pageId, long page, long pageAddr, BPlusIO<L> io, Get g, int lvl)
             throws IgniteCheckedException {
             // Check the triangle invariant.
             if (io.getForward(pageAddr) != g.fwdId)
@@ -317,7 +326,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
     private class Replace extends GetPageHandler<Put> {
         /** {@inheritDoc} */
         @SuppressWarnings("unchecked")
-        @Override public Result run0(Page page, long pageAddr, BPlusIO<L> io, Put p, int lvl)
+        @Override public Result run0(long pageId, long page, long pageAddr, BPlusIO<L> io, Put p, int lvl)
             throws IgniteCheckedException  {
             // Check the triangle invariant.
             if (io.getForward(pageAddr) != p.fwdId)
@@ -359,12 +368,12 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                 p.finish();
             }
 
-            boolean needWal = needWalDeltaRecord(page);
+            boolean needWal = needWalDeltaRecord(pageId, page, null);
 
             byte[] newRowBytes = io.store(pageAddr, idx, newRow, null, needWal);
 
             if (needWal)
-                wal.log(new ReplaceRecord<>(cacheId, page.id(), io, newRowBytes, idx));
+                wal.log(new ReplaceRecord<>(cacheId, pageId, io, newRowBytes, idx));
 
             return FOUND;
         }
@@ -378,7 +387,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
      */
     private class Insert extends GetPageHandler<Put> {
         /** {@inheritDoc} */
-        @Override public Result run0(Page page, long pageAddr, BPlusIO<L> io, Put p, int lvl)
+        @Override public Result run0(long pageId, long 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;
 
@@ -395,7 +404,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             idx = fix(idx);
 
             // Do insert.
-            L moveUpRow = p.insert(page, io, pageAddr, idx, lvl);
+            L moveUpRow = p.insert(pageId, page, pageAddr, io, idx, lvl);
 
             // Check if split happened.
             if (moveUpRow != null) {
@@ -408,7 +417,7 @@ 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(pageAddr);
-                p.tail(page, pageAddr);
+                p.tail(pageId, page, pageAddr);
 
                 assert p.rightId != 0;
             }
@@ -427,15 +436,15 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
      */
     private class RemoveFromLeaf extends GetPageHandler<Remove> {
         /** {@inheritDoc} */
-        @Override public Result run0(Page leaf, long pageAddr, BPlusIO<L> io, Remove r, int lvl)
+        @Override public Result run0(long leafId, long leafPage, long leafAddr, BPlusIO<L> io, Remove r, int lvl)
             throws IgniteCheckedException {
             assert lvl == 0 : lvl; // Leaf.
 
-            final int cnt = io.getCount(pageAddr);
+            final int cnt = io.getCount(leafAddr);
 
             assert cnt <= Short.MAX_VALUE: cnt;
 
-            int idx = findInsertionPoint(io, pageAddr, 0, cnt, r.row, 0);
+            int idx = findInsertionPoint(io, leafAddr, 0, cnt, r.row, 0);
 
             if (idx < 0)
                 return RETRY; // We've found exact match on search but now it's gone.
@@ -444,14 +453,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(pageAddr) != 0;
+            boolean needReplaceInner = canGetRowFromInner && idx == cnt - 1 && io.getForward(leafAddr) != 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(pageAddr, pageSize())))) {
+                ((r.fwdId != 0 || r.backId != 0) && mayMerge(cnt - 1, io.getMaxCount(leafAddr, 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);
@@ -475,7 +484,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                 if (needReplaceInner)
                     r.needReplaceInner = TRUE;
 
-                Tail<L> t = r.addTail(leaf, pageAddr, io, 0, Tail.EXACT);
+                Tail<L> t = r.addTail(leafId, leafPage, leafAddr, io, 0, Tail.EXACT);
 
                 t.idx = (short)idx;
 
@@ -484,7 +493,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                 return FOUND;
             }
 
-            r.removeDataRowFromLeaf(leaf, io, pageAddr, cnt, idx);
+            r.removeDataRowFromLeaf(leafId, leafPage, leafAddr, null, io, cnt, idx);
 
             return FOUND;
         }
@@ -498,10 +507,10 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
      */
     private class LockBackAndRmvFromLeaf extends GetPageHandler<Remove> {
         /** {@inheritDoc} */
-        @Override protected Result run0(Page back, long pageAddr, BPlusIO<L> io, Remove r, int lvl)
+        @Override protected Result run0(long backId, long backPage, long backAddr, BPlusIO<L> io, Remove r, int lvl)
             throws IgniteCheckedException {
             // Check that we have consistent view of the world.
-            if (io.getForward(pageAddr) != r.pageId)
+            if (io.getForward(backAddr) != r.pageId)
                 return RETRY;
 
             // Correct locking order: from back to forward.
@@ -509,7 +518,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, pageAddr, io, lvl, Tail.BACK);
+                r.addTail(backId, backPage, backAddr, io, lvl, Tail.BACK);
 
             return res;
         }
@@ -523,17 +532,17 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
      */
     private class LockBackAndTail extends GetPageHandler<Remove> {
         /** {@inheritDoc} */
-        @Override public Result run0(Page back, long pageAddr, BPlusIO<L> io, Remove r, int lvl)
+        @Override public Result run0(long backId, long backPage, long backAddr, BPlusIO<L> io, Remove r, int lvl)
             throws IgniteCheckedException {
             // Check that we have consistent view of the world.
-            if (io.getForward(pageAddr) != r.pageId)
+            if (io.getForward(backAddr) != r.pageId)
                 return RETRY;
 
             // Correct locking order: from back to forward.
             Result res = r.doLockTail(lvl);
 
             if (res == FOUND)
-                r.addTail(back, pageAddr, io, lvl, Tail.BACK);
+                r.addTail(backId, backPage, backAddr, io, lvl, Tail.BACK);
 
             return res;
         }
@@ -547,9 +556,9 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
      */
     private class LockTailForward extends GetPageHandler<Remove> {
         /** {@inheritDoc} */
-        @Override protected Result run0(Page page, long pageAddr, BPlusIO<L> io, Remove r, int lvl)
+        @Override protected Result run0(long pageId, long page, long pageAddr, BPlusIO<L> io, Remove r, int lvl)
             throws IgniteCheckedException {
-            r.addTail(page, pageAddr, io, lvl, Tail.FORWARD);
+            r.addTail(pageId, page, pageAddr, io, lvl, Tail.FORWARD);
 
             return FOUND;
         }
@@ -563,7 +572,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
      */
     private class LockTail extends GetPageHandler<Remove> {
         /** {@inheritDoc} */
-        @Override public Result run0(Page page, long pageAddr, BPlusIO<L> io, Remove r, int lvl)
+        @Override public Result run0(long pageId, long page, long pageAddr, BPlusIO<L> io, Remove r, int lvl)
             throws IgniteCheckedException {
             assert lvl > 0 : lvl; // We are not at the bottom.
 
@@ -579,7 +588,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                     return res; // Retry.
             }
 
-            r.addTail(page, pageAddr, io, lvl, Tail.EXACT);
+            r.addTail(pageId, page, pageAddr, io, lvl, Tail.EXACT);
 
             return FOUND;
         }
@@ -593,23 +602,23 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
      */
     private class CutRoot extends PageHandler<Void, Bool> {
         /** {@inheritDoc} */
-        @Override public Bool run(Page meta, PageIO iox, long pageAddr, Void ignore, int lvl)
+        @Override public Bool run(int cacheId, long metaId, long metaPage, long metaAddr, PageIO iox, Boolean walPlc, 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(pageAddr); // Can drop only root.
+            assert lvl == io.getRootLevel(metaAddr); // Can drop only root.
 
-            io.cutRoot(pageAddr, pageSize());
+            io.cutRoot(metaAddr, pageSize());
 
-            if (needWalDeltaRecord(meta))
-                wal.log(new MetaPageCutRootRecord(cacheId, meta.id()));
+            if (needWalDeltaRecord(metaId, metaPage, walPlc))
+                wal.log(new MetaPageCutRootRecord(cacheId, metaId));
 
             int newLvl = lvl - 1;
 
-            assert io.getRootLevel(pageAddr) == newLvl;
+            assert io.getRootLevel(metaAddr) == newLvl;
 
-            treeMeta = new TreeMetaData(newLvl, io.getFirstPageId(pageAddr, newLvl));
+            treeMeta = new TreeMetaData(newLvl, io.getFirstPageId(metaAddr, newLvl));
 
             return TRUE;
         }
@@ -623,7 +632,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
      */
     private class AddRoot extends PageHandler<Long, Bool> {
         /** {@inheritDoc} */
-        @Override public Bool run(Page meta, PageIO iox, long pageAddr, Long rootPageId, int lvl)
+        @Override public Bool run(int cacheId, long metaId, long metaPage, long pageAddr, PageIO iox, Boolean walPlc, Long rootPageId, int lvl)
             throws IgniteCheckedException {
             assert rootPageId != null;
 
@@ -634,8 +643,8 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
             io.addRoot(pageAddr, rootPageId, pageSize());
 
-            if (needWalDeltaRecord(meta))
-                wal.log(new MetaPageAddRootRecord(cacheId, meta.id(), rootPageId));
+            if (needWalDeltaRecord(metaId, metaPage, walPlc))
+                wal.log(new MetaPageAddRootRecord(cacheId, metaId, rootPageId));
 
             assert io.getRootLevel(pageAddr) == lvl;
             assert io.getFirstPageId(pageAddr, lvl) == rootPageId;
@@ -654,7 +663,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
      */
     private class InitRoot extends PageHandler<Long, Bool> {
         /** {@inheritDoc} */
-        @Override public Bool run(Page meta, PageIO iox, long pageAddr, Long rootId, int inlineSize)
+        @Override public Bool run(int cacheId, long metaId, long metaPage, long pageAddr, PageIO iox, Boolean walPlc, Long rootId, int inlineSize)
             throws IgniteCheckedException {
             assert rootId != null;
 
@@ -664,8 +673,8 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             io.initRoot(pageAddr, rootId, pageSize());
             io.setInlineSize(pageAddr, inlineSize);
 
-            if (needWalDeltaRecord(meta))
-                wal.log(new MetaPageInitRootInlineRecord(cacheId, meta.id(), rootId, inlineSize));
+            if (needWalDeltaRecord(metaId, metaPage, walPlc))
+                wal.log(new MetaPageInitRootInlineRecord(cacheId, metaId, rootId, inlineSize));
 
             assert io.getRootLevel(pageAddr) == 0;
             assert io.getFirstPageId(pageAddr, 0) == rootId;
@@ -760,14 +769,6 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
     }
 
     /**
-     * @param page Updated page.
-     * @return {@code true} If we need to make a delta WAL record for the change in this page.
-     */
-    private boolean needWalDeltaRecord(Page page) {
-        return isWalDeltaRecordNeeded(wal, page);
-    }
-
-    /**
      * Initialize new tree.
      *
      * @param initNew {@code True} if new tree should be created.
@@ -789,16 +790,12 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             // Allocate the first leaf page, it will be our root.
             long rootId = allocatePage(null);
 
-            try (Page root = page(rootId)) {
-                initPage(pageMem, root, this, latestLeafIO(), wal);
-            }
+            init(rootId, latestLeafIO());
 
             // Initialize meta page with new root page.
-            try (Page meta = page(metaPageId)) {
-                Bool res = writePage(pageMem, meta, this, initRoot, BPlusMetaIO.VERSIONS.latest(), wal, rootId, inlineSize, FALSE);
+            Bool res = write(metaPageId, initRoot, BPlusMetaIO.VERSIONS.latest(), rootId, inlineSize, FALSE);
 
-                assert res == TRUE: res;
-            }
+            assert res == TRUE: res;
 
             assert treeMeta != null;
         }
@@ -814,10 +811,11 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         if (meta0 != null)
             return meta0;
 
-        try (Page meta = page(metaPageId)) {
-            long pageAddr = readLock(meta); // Meta can't be removed.
+        final long metaPage = acquirePage(metaPageId);
+        try {
+            long pageAddr = readLock(metaPageId, metaPage); // Meta can't be removed.
 
-            assert pageAddr != 0 : "Failed to read lock meta page [page=" + meta + ", metaPageId=" +
+            assert pageAddr != 0 : "Failed to read lock meta page [metaPageId=" +
                 U.hexLong(metaPageId) + ']';
 
             try {
@@ -829,9 +827,12 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                 treeMeta = meta0 = new TreeMetaData(rootLvl, rootId);
             }
             finally {
-                readUnlock(meta, pageAddr);
+                readUnlock(metaPageId, metaPage, pageAddr);
             }
         }
+        finally {
+            releasePage(metaPageId, metaPage);
+        }
 
         return meta0;
     }
@@ -849,12 +850,13 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
     }
 
     /**
-     * @param meta Meta page.
+     * @param metaId Meta page ID.
+     * @param metaPage Meta page pointer.
      * @param lvl Level, if {@code 0} then it is a bottom level, if negative then root.
      * @return Page ID.
      */
-    private long getFirstPageId(Page meta, int lvl) {
-        long pageAddr = readLock(meta); // Meta can't be removed.
+    private long getFirstPageId(long metaId, long metaPage, int lvl) {
+        long pageAddr = readLock(metaId, metaPage); // Meta can't be removed.
 
         try {
             BPlusMetaIO io = BPlusMetaIO.VERSIONS.forPage(pageAddr);
@@ -868,7 +870,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             return io.getFirstPageId(pageAddr, lvl);
         }
         finally {
-            readUnlock(meta, pageAddr);
+            readUnlock(metaId, metaPage, pageAddr);
         }
     }
 
@@ -882,20 +884,29 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
         long firstPageId;
 
-        try (Page meta = page(metaPageId)) {
-            firstPageId = getFirstPageId(meta, 0); // Level 0 is always at the bottom.
+        long metaPage = acquirePage(metaPageId);
+        try  {
+            firstPageId = getFirstPageId(metaPageId, metaPage, 0); // Level 0 is always at the bottom.
+        }
+        finally {
+            releasePage(metaPageId, metaPage);
         }
 
-        try (Page first = page(firstPageId)) {
-            long pageAddr = readLock(first); // We always merge pages backwards, the first page is never removed.
+        long firstPage = acquirePage(firstPageId);
+
+        try {
+            long pageAddr = readLock(firstPageId, firstPage); // We always merge pages backwards, the first page is never removed.
 
             try {
                 cursor.init(pageAddr, io(pageAddr), 0);
             }
             finally {
-                readUnlock(first, pageAddr);
+                readUnlock(firstPageId, firstPage, pageAddr);
             }
         }
+        finally {
+            releasePage(firstPageId, firstPage);
+        }
 
         return cursor;
     }
@@ -1007,7 +1018,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
      */
     private Result findDown(final Get g, final long pageId, final long fwdId, final int lvl)
         throws IgniteCheckedException {
-        Page page = page(pageId);
+        long page = acquirePage(pageId);
 
         try {
             for (;;) {
@@ -1015,7 +1026,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                 g.pageId = pageId;
                 g.fwdId = fwdId;
 
-                Result res = readPage(page, this, search, g, lvl, RETRY);
+                Result res = read(pageId, page, search, g, lvl, RETRY);
 
                 switch (res) {
                     case GO_DOWN:
@@ -1049,8 +1060,8 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             }
         }
         finally {
-            if (g.canRelease(page, lvl))
-                page.close();
+            if (g.canRelease(pageId, lvl))
+                releasePage(pageId, page);
         }
     }
 
@@ -1067,16 +1078,18 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
      * For debug.
      *
      * @return Tree as {@link String}.
+     * @throws IgniteCheckedException If failed.
      */
     @SuppressWarnings("unused")
-    public final String printTree() {
+    public final String printTree() throws IgniteCheckedException {
         long rootPageId;
 
-        try (Page meta = page(metaPageId)) {
-            rootPageId = getFirstPageId(meta, -1);
+        long metaPage = acquirePage(metaPageId);
+        try {
+            rootPageId = getFirstPageId(metaPageId, metaPage, -1);
         }
-        catch (IgniteCheckedException e) {
-            throw new IllegalStateException(e);
+        finally {
+            releasePage(metaPageId, metaPage);
         }
 
         return treePrinter.print(rootPageId);
@@ -1089,20 +1102,24 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         long rootPageId;
         int rootLvl;
 
-        try (Page meta = page(metaPageId)) {
+        long metaPage = acquirePage(metaPageId);
+        try  {
             rootLvl = getRootLevel();
 
             if (rootLvl < 0)
                 fail("Root level: " + rootLvl);
 
-            validateFirstPages(meta, rootLvl);
+            validateFirstPages(metaPageId, metaPage, rootLvl);
 
-            rootPageId = getFirstPageId(meta, rootLvl);
+            rootPageId = getFirstPageId(metaPageId, metaPage, rootLvl);
 
             validateDownPages(rootPageId, 0L, rootLvl);
 
             validateDownKeys(rootPageId, null);
         }
+        finally {
+            releasePage(metaPageId, metaPage);
+        }
     }
 
     /**
@@ -1111,8 +1128,9 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
      * @throws IgniteCheckedException If failed.
      */
     private void validateDownKeys(long pageId, L minRow) throws IgniteCheckedException {
-        try (Page page = page(pageId)) {
-            long pageAddr = readLock(page); // No correctness guaranties.
+        long page = acquirePage(pageId);
+        try {
+            long pageAddr = readLock(pageId, page); // No correctness guaranties.
 
             try {
                 BPlusIO<L> io = io(pageAddr);
@@ -1161,9 +1179,12 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                 validateDownKeys(rightId, minRow);
             }
             finally {
-                readUnlock(page, pageAddr);
+                readUnlock(pageId, page, pageAddr);
             }
         }
+        finally {
+            releasePage(pageId, page);
+        }
     }
 
     /**
@@ -1172,8 +1193,9 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
      * @throws IgniteCheckedException If failed.
      */
     private L getGreatestRowInSubTree(long pageId) throws IgniteCheckedException {
-        try (Page page = page(pageId)) {
-            long pageAddr = readLock(page); // No correctness guaranties.
+        long page = acquirePage(pageId);
+        try {
+            long pageAddr = readLock(pageId, page); // No correctness guaranties.
 
             try {
                 BPlusIO<L> io = io(pageAddr);
@@ -1192,19 +1214,23 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                 return getGreatestRowInSubTree(rightId);
             }
             finally {
-                readUnlock(page, pageAddr);
+                readUnlock(pageId, page, pageAddr);
             }
         }
+        finally {
+            releasePage(pageId, page);
+        }
     }
 
     /**
-     * @param meta Meta page.
+     * @param metaId Meta page ID.
+     * @param metaPage Meta page pointer.
      * @param rootLvl Root level.
      * @throws IgniteCheckedException If failed.
      */
-    private void validateFirstPages(Page meta, int rootLvl) throws IgniteCheckedException {
+    private void validateFirstPages(long metaId, long metaPage, int rootLvl) throws IgniteCheckedException {
         for (int lvl = rootLvl; lvl > 0; lvl--)
-            validateFirstPage(meta, lvl);
+            validateFirstPage(metaId, metaPage, lvl);
     }
 
     /**
@@ -1215,20 +1241,22 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
     }
 
     /**
-     * @param meta Meta page.
+     * @param metaId Meta page ID.
+     * @param metaPage Meta page pointer.
      * @param lvl Level.
      * @throws IgniteCheckedException If failed.
      */
-    private void validateFirstPage(Page meta, int lvl) throws IgniteCheckedException {
+    private void validateFirstPage(long metaId, long metaPage, int lvl) throws IgniteCheckedException {
         if (lvl == 0)
             fail("Leaf level: " + lvl);
 
-        long pageId = getFirstPageId(meta, lvl);
+        long pageId = getFirstPageId(metaId, metaPage, lvl);
 
         long leftmostChildId;
 
-        try (Page page = page(pageId)) {
-            long pageAddr = readLock(page); // No correctness guaranties.
+        long page = acquirePage(pageId);
+        try {
+            long pageAddr = readLock(pageId, page); // No correctness guaranties.
 
             try {
                 BPlusIO<L> io = io(pageAddr);
@@ -1239,11 +1267,14 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                 leftmostChildId = inner(io).getLeft(pageAddr, 0);
             }
             finally {
-                readUnlock(page, pageAddr);
+                readUnlock(pageId, page, pageAddr);
             }
         }
+        finally {
+            releasePage(pageId, page);
+        }
 
-        long firstDownPageId = getFirstPageId(meta, lvl - 1);
+        long firstDownPageId = getFirstPageId(metaId, metaPage, lvl - 1);
 
         if (firstDownPageId != leftmostChildId)
             fail(new SB("First: meta ").appendHex(firstDownPageId).a(", child ").appendHex(leftmostChildId));
@@ -1255,9 +1286,10 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
      * @param lvl Level.
      * @throws IgniteCheckedException If failed.
      */
-    private void validateDownPages(long pageId, long fwdId, final int lvl) throws IgniteCheckedException {
-        try (Page page = page(pageId)) {
-            long pageAddr = readLock(page); // No correctness guaranties.
+    private void validateDownPages(long pageId, long fwdId, int lvl) throws IgniteCheckedException {
+        long page = acquirePage(pageId);
+        try {
+            long pageAddr = readLock(pageId, page); // No correctness guaranties.
 
             try {
                 long realPageId = BPlusIO.getPageId(pageAddr);
@@ -1291,8 +1323,10 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
                     if (fwdId != 0) {
                         // For the rightmost child ask neighbor.
-                        try (Page fwd = page(fwdId)) {
-                            long fwdPageAddr = readLock(fwd); // No correctness guaranties.
+                        long fwdId0 = fwdId;
+                        long fwdPage = acquirePage(fwdId0);
+                        try {
+                            long fwdPageAddr = readLock(fwdId0, fwdPage); // No correctness guaranties.
 
                             try {
                                 if (io(fwdPageAddr) != io)
@@ -1301,20 +1335,26 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                                 fwdId = inner(io).getLeft(fwdPageAddr, 0);
                             }
                             finally {
-                                readUnlock(fwd, fwdPageAddr);
+                                readUnlock(fwdId0, fwdPage, fwdPageAddr);
                             }
                         }
+                        finally {
+                            releasePage(fwdId0, fwdPage);
+                        }
                     }
 
-                    pageId = inner(io).getLeft(pageAddr, cnt); // The same as io.getRight(cnt - 1) but works for routing pages.
+                    long leftId = inner(io).getLeft(pageAddr, cnt); // The same as io.getRight(cnt - 1) but works for routing pages.
 
-                    validateDownPages(pageId, fwdId, lvl - 1);
+                    validateDownPages(leftId, fwdId, lvl - 1);
                 }
             }
             finally {
-                readUnlock(page, pageAddr);
+                readUnlock(pageId, page, pageAddr);
             }
         }
+        finally {
+            releasePage(pageId, page);
+        }
     }
 
     /**
@@ -1512,7 +1552,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         if (x.isTail(pageId, lvl))
             return FOUND; // We've already locked this page, so return that we are ok.
 
-        final Page page = page(pageId);
+        long page = acquirePage(pageId);
 
         try {
             for (;;) {
@@ -1521,7 +1561,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                 x.fwdId(fwdId);
                 x.backId(backId);
 
-                Result res = readPage(page, this, search, x, lvl, RETRY);
+                Result res = read(pageId, page, search, x, lvl, RETRY);
 
                 switch (res) {
                     case GO_DOWN_X:
@@ -1540,7 +1580,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
                         // Intentional fallthrough.
                     case GO_DOWN:
-                        res = x.tryReplaceInner(page, pageId, fwdId, lvl);
+                        res = x.tryReplaceInner(pageId, page, fwdId, lvl);
 
                         if (res != RETRY)
                             res = invokeDown(x, x.pageId, x.backId, x.fwdId, lvl - 1);
@@ -1560,7 +1600,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
                         assert x.isRemove(); // Guarded by isFinished.
 
-                        res = x.finishOrLockTail(page, pageId, backId, fwdId, lvl);
+                        res = x.finishOrLockTail(pageId, page, backId, fwdId, lvl);
 
                         return res;
 
@@ -1568,13 +1608,13 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                         if (lvl == 0)
                             x.invokeClosure();
 
-                        return x.onNotFound(page, pageId, fwdId, lvl);
+                        return x.onNotFound(pageId, page, fwdId, lvl);
 
                     case FOUND:
                         if (lvl == 0)
                             x.invokeClosure();
 
-                        return x.onFound(page, pageId, backId, fwdId, lvl);
+                        return x.onFound(pageId, page, backId, fwdId, lvl);
 
                     default:
                         return res;
@@ -1584,8 +1624,8 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         finally {
             x.levelExit();
 
-            if (x.canRelease(page, lvl))
-                page.close();
+            if (x.canRelease(pageId, lvl))
+                releasePage(pageId, page);
         }
     }
 
@@ -1666,7 +1706,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         if (r.isTail(pageId, lvl))
             return FOUND; // We've already locked this page, so return that we are ok.
 
-        final Page page = page(pageId);
+        long page = acquirePage(pageId);
 
         try {
             for (;;) {
@@ -1675,7 +1715,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                 r.fwdId = fwdId;
                 r.backId = backId;
 
-                Result res = readPage(page, this, search, r, lvl, RETRY);
+                Result res = read(pageId, page, search, r, lvl, RETRY);
 
                 switch (res) {
                     case GO_DOWN_X:
@@ -1705,7 +1745,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                         if (res == RETRY_ROOT || r.isFinished())
                             return res;
 
-                        res = r.finishOrLockTail(page, pageId, backId, fwdId, lvl);
+                        res = r.finishOrLockTail(pageId, page, backId, fwdId, lvl);
 
                         return res;
 
@@ -1718,7 +1758,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                         return res;
 
                     case FOUND:
-                        return r.tryRemoveFromLeaf(page, pageId, backId, fwdId, lvl);
+                        return r.tryRemoveFromLeaf(pageId, page, backId, fwdId, lvl);
 
                     default:
                         return res;
@@ -1726,10 +1766,10 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             }
         }
         finally {
-            r.page = null;
+            r.page = 0L;
 
-            if (r.canRelease(page, lvl))
-                page.close();
+            if (r.canRelease(pageId, lvl))
+                releasePage(pageId, page);
         }
     }
 
@@ -1782,8 +1822,12 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
         long pageId;
 
-        try (Page meta = page(metaPageId)) {
-            pageId = getFirstPageId(meta, 0); // Level 0 is always at the bottom.
+        long metaPage = acquirePage(metaPageId);
+        try {
+            pageId = getFirstPageId(metaPageId, metaPage, 0); // Level 0 is always at the bottom.
+        }
+        finally {
+            releasePage(metaPageId, metaPage);
         }
 
         BPlusIO<L> io = null;
@@ -1791,24 +1835,29 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         long cnt = 0;
 
         while (pageId != 0) {
-            try (Page page = page(pageId)) {
-                long pageAddr = readLock(page); // No correctness guaranties.
+            long pageId0 = pageId;
+            long page = acquirePage(pageId0);
+            try {
+                long curAddr = readLock(pageId0, page); // No correctness guaranties.
 
                 try {
                     if (io == null) {
-                        io = io(pageAddr);
+                        io = io(curAddr);
 
                         assert io.isLeaf();
                     }
 
-                    cnt += io.getCount(pageAddr);
+                    cnt += io.getCount(curAddr);
 
-                    pageId = io.getForward(pageAddr);
+                    pageId = io.getForward(curAddr);
                 }
                 finally {
-                    readUnlock(page, pageAddr);
+                    readUnlock(pageId0, page, curAddr);
                 }
             }
+            finally {
+                releasePage(pageId0, page);
+            }
         }
 
         return cnt;
@@ -1921,16 +1970,18 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
         long pagesCnt = 0;
 
-        try (Page meta = page(metaPageId)) {
-            long metaPageAddr = writeLock(meta); // No checks, we must be out of use.
+        long metaPage = acquirePage(metaPageId);
+        try {
+            long metaPageAddr = writeLock(metaPageId, metaPage); // No checks, we must be out of use.
 
             try {
                 for (long pageId : getFirstPageIds(metaPageAddr)) {
                     assert pageId != 0;
 
                     do {
-                        try (Page page = page(pageId)) {
-                            long pageAddr = writeLock(page); // No checks, we must be out of use.
+                        long page = acquirePage(pageId);
+                        try {
+                            long pageAddr = writeLock(pageId, page); // No checks, we must be out of use.
 
                             try {
                                 BPlusIO<L> io = io(pageAddr);
@@ -1940,15 +1991,18 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
                                 long fwdPageId = io.getForward(pageAddr);
 
-                                bag.addFreePage(recyclePage(pageId, page, pageAddr));
+                                bag.addFreePage(recyclePage(pageId, page, pageAddr, null));
                                 pagesCnt++;
 
                                 pageId = fwdPageId;
                             }
                             finally {
-                                writeUnlock(page, pageAddr, true);
+                                writeUnlock(pageId, page, pageAddr, true);
                             }
                         }
+                        finally {
+                            releasePage(pageId, page);
+                        }
 
                         if (bag.size() == 128) {
                             reuseList.addForRecycle(bag);
@@ -1959,13 +2013,16 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                     while (pageId != 0);
                 }
 
-                bag.addFreePage(recyclePage(metaPageId, meta, metaPageAddr));
+                bag.addFreePage(recyclePage(metaPageId, metaPage, metaPageAddr, null));
                 pagesCnt++;
             }
             finally {
-                writeUnlock(meta, metaPageAddr, true);
+                writeUnlock(metaPageId, metaPage, metaPageAddr, true);
             }
         }
+        finally {
+            releasePage(metaPageId, metaPage);
+        }
 
         reuseList.addForRecycle(bag);
 
@@ -1998,45 +2055,19 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
     /**
      * @param pageId Page ID.
-     * @param page Page.
-     * @param pageAddr Page address.
-     * @return Recycled page ID.
-     * @throws IgniteCheckedException If failed.
-     */
-    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(pageAddr, pageId);
-
-        if (needWalDeltaRecord(page))
-            wal.log(new RecycleRecord(cacheId, page.id(), pageId));
-
-        return pageId;
-    }
-
-    /**
+     * @param page Page pointer.
+     * @param pageAddr Page address
      * @param io IO.
-     * @param page Page to split.
-     * @param buf Splitting buffer.
      * @param fwdId Forward page ID.
-     * @param fwd Forward page.
      * @param fwdBuf Forward buffer.
      * @param idx Insertion index.
      * @return {@code true} The middle index was shifted to the right.
      * @throws IgniteCheckedException If failed.
      */
     private boolean splitPage(
-        BPlusIO io,
-        Page page,
-        long buf,
-        long fwdId,
-        Page fwd,
-        long fwdBuf,
-        int idx
+        long pageId, long page, long pageAddr, BPlusIO io, long fwdId, long fwdBuf, int idx
     ) throws IgniteCheckedException {
-        int cnt = io.getCount(buf);
+        int cnt = io.getCount(pageAddr);
         int mid = cnt >>> 1;
 
         boolean res = false;
@@ -2048,43 +2079,41 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         }
 
         // Update forward page.
-        io.splitForwardPage(buf, fwdId, fwdBuf, mid, cnt, pageSize());
-
-        // TODO GG-11640 log a correct forward page record.
-        fwd.fullPageWalRecordPolicy(Boolean.TRUE);
+        io.splitForwardPage(pageAddr, fwdId, fwdBuf, mid, cnt, pageSize());
 
         // Update existing page.
-        io.splitExistingPage(buf, mid, fwdId);
+        io.splitExistingPage(pageAddr, mid, fwdId);
 
-        if (needWalDeltaRecord(page))
-            wal.log(new SplitExistingPageRecord(cacheId, page.id(), mid, fwdId));
+        if (needWalDeltaRecord(pageId, page, null))
+            wal.log(new SplitExistingPageRecord(cacheId, pageId, mid, fwdId));
 
         return res;
     }
 
     /**
-     * @param page Page.
-     * @param pageAddr Page address.
+     * @param pageId Page ID.
+     * @param page Page pointer.
+     * @param pageAddr Page address
+     * @param walPlc Full page WAL record policy.
      */
-    private void writeUnlockAndClose(Page page, long pageAddr) {
+    private void writeUnlockAndClose(long pageId, long page, long pageAddr, Boolean walPlc) {
         try {
-            writeUnlock(page, pageAddr, true);
+            writeUnlock(pageId, page, pageAddr, walPlc, true);
         }
         finally {
-            page.close();
+            releasePage(pageId, page);
         }
     }
 
     /**
      * @param pageId Inner page ID.
+     * @param g Get.
      * @param back Get back (if {@code true}) or forward page (if {@code false}).
      * @return Operation result.
+     * @throws IgniteCheckedException If failed.
      */
     private Result askNeighbor(long pageId, Get g, boolean back) throws IgniteCheckedException {
-        try (Page page = page(pageId)) {
-            return readPage(page, this, askNeighbor, g,
-                back ? TRUE.ordinal() : FALSE.ordinal(), RETRY);
-        }
+        return read(pageId, askNeighbor, g, back ? TRUE.ordinal() : FALSE.ordinal(), RETRY);
     }
 
     /**
@@ -2099,7 +2128,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         throws IgniteCheckedException {
         assert lvl >= 0 : lvl;
 
-        final Page page = page(pageId);
+        final long page = acquirePage(pageId);
 
         try {
             for (;;) {
@@ -2107,7 +2136,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                 p.pageId = pageId;
                 p.fwdId = fwdId;
 
-                Result res = readPage(page, this, search, p, lvl, RETRY);
+                Result res = read(pageId, page, search, p, lvl, RETRY);
 
                 switch (res) {
                     case GO_DOWN:
@@ -2116,7 +2145,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                         assert p.pageId != pageId;
                         assert p.fwdId != fwdId || fwdId == 0;
 
-                        res = p.tryReplaceInner(page, pageId, fwdId, lvl);
+                        res = p.tryReplaceInner(pageId, page, fwdId, lvl);
 
                         if (res != RETRY) // Go down recursively.
                             res = putDown(p, p.pageId, p.fwdId, lvl - 1);
@@ -2132,13 +2161,13 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                     case FOUND: // Do replace.
                         assert lvl == 0 : "This replace can happen only at the bottom level.";
 
-                        return p.tryReplace(page, pageId, fwdId, lvl);
+                        return p.tryReplace(pageId, page, fwdId, lvl);
 
                     case NOT_FOUND: // Do insert.
                         assert lvl == p.btmLvl : "must insert at the bottom level";
                         assert p.needReplaceInner == FALSE : p.needReplaceInner + " " + lvl;
 
-                        return p.tryInsert(page, pageId, fwdId, lvl);
+                        return p.tryInsert(pageId, page, fwdId, lvl);
 
                     default:
                         return res;
@@ -2146,8 +2175,8 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             }
         }
         finally {
-            if (p.canRelease(page, lvl))
-                page.close();
+            if (p.canRelease(pageId, lvl))
+                releasePage(pageId, page);
         }
     }
 
@@ -2289,12 +2318,12 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         }
 
         /**
-         * @param page Page.
+         * @param pageId Page.
          * @param lvl Level.
          * @return {@code true} If we can release the given page.
          */
-        boolean canRelease(Page page, int lvl) {
-            return page != null;
+        boolean canRelease(long pageId, int lvl) {
+            return pageId != 0L;
         }
 
         /**
@@ -2408,10 +2437,13 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
          * It is needed because split row will be "in flight" and if we'll release tail, remove on
          * split row may fail.
          */
-        Page tail;
+        long tailId;
+
+        /** */
+        long tailPage;
 
         /** */
-        long tailPageAddr;
+        long tailAddr;
 
         /**
          * Bottom level for insertion (insert can't go deeper). Will be incremented on split on each level.
@@ -2459,22 +2491,25 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         }
 
         /**
-         * @param tail Tail page.
-         * @param tailPageAddr Tail page address.
+         * @param tailId Tail page ID.
+         * @param tailPage Tail page pointer.
+         * @param tailPageAddr Tail page address
          */
-        private void tail(Page tail, long tailPageAddr) {
-            assert (tail == null) == (tailPageAddr == 0L);
+        private void tail(long tailId, long tailPage, long tailPageAddr) {
+            assert (tailId == 0L) == (tailPage == 0L);
+            assert (tailPage == 0L) == (tailPageAddr == 0L);
 
-            if (this.tail != null)
-                writeUnlockAndClose(this.tail, this.tailPageAddr);
+            if (this.tailAddr != 0L)
+                writeUnlockAndClose(this.tailId, this.tailPage, this.tailAddr, null);
 
-            this.tail = tail;
-            this.tailPageAddr = tailPageAddr;
+            this.tailId = tailId;
+            this.tailPage = tailPage;
+            this.tailAddr = tailPageAddr;
         }
 
         /** {@inheritDoc} */
-        @Override boolean canRelease(Page page, int lvl) {
-            return page != null && tail != page;
+        @Override boolean canRelease(long pageId, int lvl) {
+            return pageId != 0L && tailId != pageId;
         }
 
         /**
@@ -2484,7 +2519,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             row = null;
             rightId = 0;
 
-            tail(null, 0L);
+            tail(0L, 0L, 0L);
         }
 
         /** {@inheritDoc} */
@@ -2493,87 +2528,92 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         }
 
         /**
-         * @param page Page.
-         * @param io IO.
+         * @param pageId Page ID.
+         * @param page Page pointer.
          * @param pageAddr Page address.
+         * @param io IO.
          * @param idx Index.
          * @param lvl Level.
          * @return Move up row.
          * @throws IgniteCheckedException If failed.
          */
-        private L insert(Page page, BPlusIO<L> io, long pageAddr, int idx, int lvl)
+        private L insert(long pageId, long page, long pageAddr, BPlusIO<L> io, int idx, int lvl)
             throws IgniteCheckedException {
             int maxCnt = io.getMaxCount(pageAddr, pageSize());
             int cnt = io.getCount(pageAddr);
 
             if (cnt == maxCnt) // Need to split page.
-                return insertWithSplit(page, io, pageAddr, idx, lvl);
+                return insertWithSplit(pageId, page, pageAddr, io, idx, lvl);
 
-            insertSimple(page, io, pageAddr, idx);
+            insertSimple(pageId, page, pageAddr, io, idx, null);
 
             return null;
         }
 
         /**
-         * @param page Page.
-         * @param io IO.
+         * @param pageId Page ID.
+         * @param page Page pointer.
          * @param pageAddr Page address.
+         * @param io IO.
          * @param idx Index.
+         * @param walPlc Full page WAL record policy.
          * @throws IgniteCheckedException If failed.
          */
-        private void insertSimple(Page page, BPlusIO<L> io, long pageAddr, int idx)
+        private void insertSimple(long pageId, long page, long pageAddr, BPlusIO<L> io, int idx, Boolean walPlc)
             throws IgniteCheckedException {
-            boolean needWal = needWalDeltaRecord(page);
+            boolean needWal = needWalDeltaRecord(pageId, page, walPlc);
 
             byte[] rowBytes = io.insert(pageAddr, idx, row, null, rightId, needWal);
 
             if (needWal)
-                wal.log(new InsertRecord<>(cacheId, page.id(), io, idx, rowBytes, rightId));
+                wal.log(new InsertRecord<>(cacheId, pageId, io, idx, rowBytes, rightId));
         }
 
         /**
-         * @param page Page.
-         * @param io IO.
+         * @param pageId Page ID.
+         * @param page Page pointer.
          * @param pageAddr Page address.
+         * @param io IO.
          * @param idx Index.
          * @param lvl Level.
          * @return Move up row.
          * @throws IgniteCheckedException If failed.
          */
-        private L insertWithSplit(Page page, BPlusIO<L> io, final long pageAddr, int idx, int lvl)
+        private L insertWithSplit(long pageId, long page, long pageAddr, BPlusIO<L> io, int idx, int lvl)
             throws IgniteCheckedException {
             long fwdId = allocatePage(null);
+            long fwdPage = acquirePage(fwdId);
 
-            try (Page fwd = page(fwdId)) {
+            try {
                 // Need to check this before the actual split, because after the split we will have new forward page here.
                 boolean hadFwd = io.getForward(pageAddr) != 0;
 
-                long fwdPageAddr = writeLock(fwd); // Initial write, no need to check for concurrent modification.
+                long fwdPageAddr = writeLock(fwdId, fwdPage); // Initial write, no need to check for concurrent modification.
 
                 assert fwdPageAddr != 0L;
 
-                try {
-                    // Never write full forward page, because it is known to be new.
-                    fwd.fullPageWalRecordPolicy(Boolean.FALSE);
+                // TODO GG-11640 log a correct forward page record.
+                Boolean fwdPageWalPlc = Boolean.TRUE;
 
-                    boolean midShift = splitPage(io, page, pageAddr, fwdId, fwd, fwdPageAddr, idx);
+                try {
+                    boolean midShift = splitPage(pageId, page, pageAddr, io, fwdId, fwdPageAddr, idx);
 
                     // Do insert.
                     int cnt = io.getCount(pageAddr);
 
                     if (idx < cnt || (idx == cnt && !midShift)) { // Insert into back page.
-                        insertSimple(page, io, pageAddr, idx);
+                        insertSimple(pageId, page, pageAddr, io, idx, null);
 
                         // Fix leftmost child of forward page, because newly inserted row will go up.
                         if (idx == cnt && !io.isLeaf()) {
                             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));
+                            if (needWalDeltaRecord(fwdId, fwdPage, fwdPageWalPlc)) // Rare case, we can afford separate WAL record to avoid complexity.
+                                wal.log(new FixLeftmostChildRecord(cacheId, fwdId, rightId));
                         }
                     }
                     else // Insert into newly allocated forward page.
-                        insertSimple(fwd, io, fwdPageAddr, idx - cnt);
+                        insertSimple(fwdId, fwdPage, fwdPageAddr, io, idx - cnt, fwdPageWalPlc);
 
                     // Do move up.
                     cnt = io.getCount(pageAddr);
@@ -2584,30 +2624,29 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                     if (!io.isLeaf()) { // Leaf pages must contain all the links, inner pages remove moveUpLink.
                         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));
+                        if (needWalDeltaRecord(pageId, page, null)) // Rare case, we can afford separate WAL record to avoid complexity.
+                            wal.log(new FixCountRecord(cacheId, pageId, cnt - 1));
                     }
 
                     if (!hadFwd && lvl == getRootLevel()) { // We are splitting root.
                         long newRootId = allocatePage(null);
+                        long newRootPage = acquirePage(newRootId);
 
-                        try (Page newRoot = page(newRootId)) {
+                        try {
                             if (io.isLeaf())
                                 io = latestInnerIO();
 
-                            long newRootPageAddr = writeLock(newRoot); // Initial write.
+                            long newRootAddr = writeLock(newRootId, newRootPage); // Initial write.
 
-                            assert newRootPageAddr != 0L;
+                            assert newRootAddr != 0L;
 
-                            try {
-                                // Never write full new root page, because it is known to be new.
-                                newRoot.fullPageWalRecordPolicy(Boolean.FALSE);
-
-                                long pageId = PageIO.getPageId(pageAddr);
+                            // Never write full new root page, because it is known to be new.
+                            Boolean newRootPageWalPlc = Boolean.FALSE;
 
-                                boolean needWal = needWalDeltaRecord(newRoot);
+                            try {
+                                boolean needWal = needWalDeltaRecord(newRootId, newRootPage, newRootPageWalPlc);
 
-                                byte[] moveUpRowBytes = inner(io).initNewRoot(newRootPageAddr,
+                                byte[] moveUpRowBytes = inner(io).initNewRoot(newRootAddr,
                                     newRootId,
                                     pageId,
                                     moveUpRow,
@@ -2617,19 +2656,20 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                                     needWal);
 
                                 if (needWal)
-                                    wal.log(new NewRootInitRecord<>(cacheId, newRoot.id(), newRootId,
+                                    wal.log(new NewRootInitRecord<>(cacheId, newRootId, newRootId,
                                         inner(io), pageId, moveUpRowBytes, fwdId));
                             }
                             finally {
-                                writeUnlock(newRoot, newRootPageAddr, true);
+                                writeUnlock(newRootId, newRootPage, newRootAddr, newRootPageWalPlc, true);
                             }
                         }
+                        finally {
+                            releasePage(newRootId, newRootPage);
+                        }
 
-                        try (Page meta = page(metaPageId)) {
-                            Bool res = writePage(pageMem, meta, BPlusTree.this, addRoot, newRootId, lvl + 1, FALSE);
+                        Bool res = write(metaPageId, addRoot, newRootId, lvl + 1, FALSE);
 
-                            assert res == TRUE : res;
-                        }
+                        assert res == TRUE : res;
 
                         return null; // We've just moved link up to root, nothing to return here.
                     }
@@ -2638,20 +2678,23 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                     return moveUpRow;
                 }
                 finally {
-                    writeUnlock(fwd, fwdPageAddr, true);
+                    writeUnlock(fwdId, fwdPage, fwdPageAddr, fwdPageWalPlc, true);
                 }
             }
+            finally {
+                releasePage(fwdId, fwdPage);
+            }
         }
 
         /**
-         * @param page Page.
          * @param pageId Page ID.
+         * @param page Page pointer.
          * @param fwdId Forward ID.
          * @param lvl Level.
          * @return Result.
          * @throws IgniteCheckedException If failed.
          */
-        private Result tryReplaceInner(Page page, long pageId, long fwdId, int lvl)
+        private Result tryReplaceInner(long pageId, long page, long fwdId, int lvl)
             throws IgniteCheckedException {
             // Need to replace key in inner page. There is no race because we keep tail lock after split.
             if (needReplaceInner == TRUE) {
@@ -2664,7 +2707,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                 this.fwdId = fwdId;
                 this.pageId = pageId;
 
-                Result res = writePage(pageMem, page, BPlusTree.this, replace, this, lvl, RETRY);
+                Result res = write(pageId, page, replace, this, lvl, RETRY);
 
                 // Restore args.
                 this.pageId = oldPageId;
@@ -2682,35 +2725,35 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         }
 
         /**
-         * @param page Page.
          * @param pageId Page ID.
+         * @param page Page pointer.
          * @param fwdId Forward ID.
          * @param lvl Level.
          * @return Result.
          * @throws IgniteCheckedException If failed.
          */
-        private Result tryInsert(Page page, long pageId, long fwdId, int lvl) throws IgniteCheckedException {
+        private Result tryInsert(long pageId, long page, long fwdId, int lvl) throws IgniteCheckedException {
             // Init args.
             this.pageId = pageId;
             this.fwdId = fwdId;
 
-            return writePage(pageMem, page, BPlusTree.this, insert, this, lvl, RETRY);
+            return write(pageId, page, insert, this, lvl, RETRY);
         }
 
         /**
-         * @param page Page.
          * @param pageId Page ID.
+         * @param page Page pointer.
          * @param fwdId Forward ID.
          * @param lvl Level.
          * @return Result.
          * @throws IgniteCheckedException If failed.
          */
-        public Result tryReplace(Page page, long pageId, long fwdId, int lvl) throws IgniteCheckedException {
+        public Result tryReplace(long pageId, long page, long fwdId, int lvl) throws IgniteCheckedException {
             // Init args.
             this.pageId = pageId;
             this.fwdId = fwdId;
 
-            return writePage(pageMem, page, BPlusTree.this, replace, this, lvl, RETRY);
+            return write(pageId, page, replace, this, lvl, RETRY);
         }
     }
 
@@ -2855,14 +2898,14 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         }
 
         /** {@inheritDoc} */
-        @Override boolean canRelease(Page page, int lvl) {
-            if (page == null)
+        @Override boolean canRelease(long pageId, int lvl) {
+            if (pageId == 0L)
                 return false;
 
             if (op == null)
                 return true;
 
-            return op.canRelease(page, lvl);
+            return op.canRelease(pageId, lvl);
         }
 
         /**
@@ -2892,11 +2935,12 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
          */
         private void levelExit() {
             if (isRemove())
-                ((Remove)op).page = null;
+                ((Remove)op).page = 0L;
         }
 
         /**
          * Release all the resources by the end of operation.
+         * @throws IgniteCheckedException if failed.
          */
         private void releaseAll() throws IgniteCheckedException {
             if (isRemove())
@@ -2904,14 +2948,14 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         }
 
         /**
-         * @param page Page.
          * @param pageId Page ID.
+         * @param page Page pointer.
          * @param fwdId Forward ID.
          * @param lvl Level.
          * @return Result.
          * @throws IgniteCheckedException If failed.
          */
-        private Result onNotFound(Page page, long pageId, long fwdId, int lvl)
+        private Result onNotFound(long pageId, long page, long fwdId, int lvl)
             throws IgniteCheckedException {
             if (op == null)
                 return NOT_FOUND;
@@ -2924,27 +2968,27 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                 return NOT_FOUND;
             }
 
-            return ((Put)op).tryInsert(page, pageId, fwdId, lvl);
+            return ((Put)op).tryInsert(pageId, page, fwdId, lvl);
         }
 
         /**
-         * @param page Page.
          * @param pageId Page ID.
+         * @param page Page pointer.
          * @param backId Back page ID.
          * @param fwdId Forward ID.
          * @param lvl Level.
          * @return Result.
          * @throws IgniteCheckedException If failed.
          */
-        private Result onFound(Page page, long pageId, long backId, long fwdId, int lvl)
+        private Result onFound(long pageId, long page, long backId, long fwdId, int lvl)
             throws IgniteCheckedException {
             if (op == null)
                 return FOUND;
 
             if (isRemove())
-                return ((Remove)op).tryRemoveFromLeaf(page, pageId, backId, fwdId, lvl);
+                return ((Remove)op).tryRemoveFromLeaf(pageId, page, backId, fwdId, lvl);
 
-            return  ((Put)op).tryReplace(page, pageId, fwdId, lvl);
+            return  ((Put)op).tryReplace(pageId, page, fwdId, lvl);
         }
 
         /**
@@ -2979,32 +3023,32 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         }
 
         /**
-         * @param page Page.
          * @param pageId Page ID.
+         * @param page Page pointer.
          * @param fwdId Forward ID.
          * @param lvl Level.
          * @return Result.
          * @throws IgniteCheckedException If failed.
          */
-        Result tryReplaceInner(Page page, long pageId, long fwdId, int lvl) throws IgniteCheckedException {
+        Result tryReplaceInner(long pageId, long page, long fwdId, int lvl) throws IgniteCheckedException {
             if (!isPut())
                 return NOT_FOUND;
 
-            return ((Put)op).tryReplaceInner(page, pageId, fwdId, lvl);
+            return ((Put)op).tryReplaceInner(pageId, page, fwdId, lvl);
         }
 
         /**
-         * @param page Page.
          * @param pageId Page ID.
+         * @param page Page pointer.
          * @param backId Back page ID.
          * @param fwdId Forward ID.
          * @param lvl Level.
          * @return Result.
          * @throws IgniteCheckedException If failed.
          */
-        public Result finishOrLockTail(Page page, long pageId, long backId, long fwdId, int lvl)
+        public Result finishOrLockTail(long pageId, long page, long backId, long fwdId, int lvl)
             throws IgniteCheckedException {
-            return ((Remove)op).finishOrLockTail(page, pageId, backId, fwdId, lvl);
+            return ((Remove)op).finishOrLockTail(pageId, page, backId, fwdId, lvl);
         }
     }
 
@@ -3024,8 +3068,8 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         /** Removed row. */
         T rmvd;
 
-        /** Current page. */
-        Page page;
+        /** Current page absolute pointer. */
+        long page;
 
         /** */
         Object freePages;
@@ -3107,6 +3151,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
         /**
          * @throws IgniteCheckedException If failed.
+         * @return Tail to release if an empty branch was not merged.
          */
         private Tail<L> mergeEmptyBranch() throws IgniteCheckedException {
             assert needMergeEmptyBranch == TRUE: needMergeEmptyBranch;
@@ -3301,7 +3346,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                     if (tail.getCount() == 0 && tail.lvl != 0 && getRootLevel() == tail.lvl) {
                         // Free root if it became empty after merge.
                         cutRoot(tail.lvl);
-                        freePage(tail.page, tail.buf, false);
+                        freePage(tail.id, tail.page, tail.buf, tail.walPlc, false);
 
                         // Exit: we are done.
                     }
@@ -3336,21 +3381,21 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
             Tail<L> leaf = getTail(t, 0);
 
-            removeDataRowFromLeaf(leaf.page, leaf.io, leaf.buf, leaf.getCount(), insertionPoint(leaf));
+            removeDataRowFromLeaf(leaf.id, leaf.page, leaf.buf, leaf.walPlc, leaf.io, leaf.getCount(), insertionPoint(leaf));
         }
 
         /**
          * @param leafId Leaf page ID.
-         * @param leaf Leaf page.
+         * @param leafPage Leaf page pointer.
          * @param backId Back page ID.
          * @param fwdId Forward ID.
          * @return Result code.
          * @throws IgniteCheckedException If failed.
          */
-        private Result removeFromLeaf(long leafId, Page leaf, long backId, long fwdId) throws IgniteCheckedException {
+        private Result removeFromLeaf(long leafId, long leafPage, long backId, long fwdId) throws IgniteCheckedException {
             // Init parameters.
             pageId = leafId;
-            page = leaf;
+            page = leafPage;
             this.backId = backId;
             this.fwdId = fwdId;
 
@@ -3359,14 +3404,14 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                 return doRemoveFromLeaf();
 
             // Lock back page before the remove, we'll need it for merges.
-            Page back = page(backId);
+            long backPage = acquirePage(backId);
 
             try {
-                return writePage(pageMem, back, BPlusTree.this, lockBackAndRmvFromLeaf, this, 0, RETRY);
+                return write(backId, backPage, lockBackAndRmvFromLeaf, this, 0, RETRY);
             }
             finally {
-                if (canRelease(back, 0))
-                    back.close();
+                if (canRelease(backId, 0))
+                    releasePage(backId, backPage);
             }
         }
 
@@ -3375,9 +3420,9 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
          * @throws IgniteCheckedException If failed.
          */
         private Result doRemoveFromLeaf() throws IgniteCheckedException {
-            assert page != null;
+            assert page != 0L;
 
-            return writePage(pageMem, page, BPlusTree.this, rmvFromLeaf, this, 0, RETRY);
+            return write(pageId, page, rmvFromLeaf, this, 0, RETRY);
         }
 
         /**
@@ -3386,21 +3431,21 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
          * @throws IgniteCheckedException If failed.
          */
         private Result doLockTail(int lvl) throws IgniteCheckedException {
-            assert page != null;
+            assert page != 0L;
 
-            return writePage(pageMem, page, BPlusTree.this, lockTail, this, lvl, RETRY);
+            return write(pageId, page, lockTail, this, lvl, RETRY);
         }
 
         /**
          * @param pageId Page ID.
-         * @param page Page.
+         * @param page Page pointer.
          * @param backId Back page ID.
          * @param fwdId Expected forward page ID.
          * @param lvl Level.
          * @return Result code.
          * @throws IgniteCheckedException If failed.
          */
-        private Result lockTail(long pageId, Page page, long backId, long fwdId, int lvl)
+        private Result lockTail(long pageId, long page, long backId, long fwdId, int lvl)
             throws IgniteCheckedException {
             assert tail != null;
 
@@ -3413,14 +3458,14 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             if (backId == 0) // Back page ID is provided only when the last move was to the right.
                 return doLockTail(lvl);
 
-            Page back = page(backId);
+            long backPage = acquirePage(backId);
 
             try {
-                return writePage(pageMem, back, BPlusTree.this, lockBackAndTail, this, lvl, RETRY);
+                return write(backId, backPage, lockBackAndTail, this, lvl, RETRY);
             }
             finally {
-                if (canRelease(back, lvl))
-                    back.close();
+                if (canRelease(backId, lvl))
+                    releasePage(backId, backPage);
             }
         }
 
@@ -3433,28 +3478,32 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             assert fwdId != 0: fwdId;
             assert backId == 0: backId;
 
-            Page fwd = page(fwdId);
+            long fwdId = this.fwdId;
+            long fwdPage = acquirePage(fwdId);
 
             try {
-                return writePage(pageMem, fwd, BPlusTree.this, lockTailForward, this, lvl, RETRY);
+                return write(fwdId, fwdPage, lockTailForward, this, lvl, RETRY);
             }
             finally {
                 // If we were not able to lock forward page as tail, release the page.
-                if (canRelease(fwd, lvl))
-                    fwd.close();
+                if (canRelease(fwdId, lvl))
+                    releasePage(fwdId, fwdPage);
             }
         }
 
         /**
-         * @param page Page.
-         * @param io IO.
+         * @param pageId Page ID.
+         * @param page Page pointer.
          * @param pageAddr Page address.
+         * @param walPlc Full page WAL record policy.
+         * @param io IO.
          * @param cnt Count.
          * @param idx Index to remove.
          * @throws IgniteCheckedException If failed.
          */
         @SuppressWarnings("unchecked")
-        private void removeDataRowFromLeaf(Page page, BPlusIO<L> io, long pageAddr, int cnt, int idx)
+        private void removeDataRowFromLeaf(long pageId, long page, long pageAddr, Boolean walPlc, BPlusIO<L> io, int cnt,
+            int idx)
             throws IgniteCheckedException {
             assert idx >= 0 && idx < cnt: idx;
             assert io.isLeaf(): "inner";
@@ -3463,28 +3512,32 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             // Detach the row.
             rmvd = needOld ? getRow(io, pageAddr, idx) : (T)Boolean.TRUE;
 
-            doRemove(page, io, pageAddr, cnt, idx);
+            doRemove(pageId, page, pageAddr, walPlc, io, cnt, idx);
 
             assert isRemoved();
         }
 
         /**
-         * @param page Page.
-         * @param io IO.
+         *
+         * @param pageId Page ID.
+         * @param page Page pointer.
          * @param pageAddr Page address.
+         * @param walPlc Full page WAL record policy.
+         * @param io IO.
          * @param cnt Count.
          * @param idx Index to remove.
          * @throws IgniteCheckedException If failed.
          */
-        private void doRemove(Page page, BPlusIO<L> io, long pageAddr, int cnt, int idx)
+        private void doRemove(long pageId, long page, long pageAddr, Boolean walPlc, BPlusIO<L> io, int cnt,
+            int idx)
             throws IgniteCheckedException {
             assert cnt > 0 : cnt;
             assert idx >= 0 && idx < cnt : idx + " " + cnt;
 
             io.remove(pageAddr, idx, cnt);
 
-            if (needWalDeltaRecord(page))
-                wal.log(new RemoveRecord(cacheId, page.id(), idx, cnt));
+            if (needWalDeltaRecord(pageId, page, walPlc))
+                wal.log(new RemoveRecord(cacheId, pageId, idx, cnt));
         }
 
         /**
@@ -3524,7 +3577,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             Tail<L> left = t.getLeftChild();
             Tail<L> right = t.getRightChild();
 
-            assert left.page.id() != right.page.id();
+            assert left.id != right.id;
 
             int cnt = t.getCount();
 
@@ -3575,7 +3628,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             if (right && cnt != 0)
                 idx++;
 
-            return inner(prnt.io).getLeft(prnt.buf, idx) == child.page.id();
+            return inner(prnt.io).getLeft(prnt.buf, idx) == child.id;
         }
 
         /**
@@ -3588,8 +3641,8 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         private boolean checkChildren(Tail<L> prnt, Tail<L> left, Tail<L> right, int idx) {
             assert idx >= 0 && idx < prnt.getCount(): idx;
 
-            return inner(prnt.io).getLeft(prnt.buf, idx) == left.page.id() &&
-                inner(prnt.io).getRight(prnt.buf, idx) == right.page.id();
+            return inner(prnt.io).getLeft(prnt.buf, idx) == left.id &&
+                inner(prnt.io).getRight(prnt.buf, idx) == right.id;
         }
 
         /**
@@ -3602,7 +3655,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         private boolean doMerge(Tail<L> prnt, Tail<L> left, Tail<L> right)
             throws IgniteCheckedException {
             assert right.io == left.io; // Otherwise incompatible.
-            assert left.io.getForward(left.buf) == right.page.id();
+            assert left.io.getForward(left.buf) == right.id;
 
             int prntCnt = prnt.getCount();
             int prntIdx = fix(insertionPoint(prnt));
@@ -3630,39 +3683,40 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             left.idx = Short.MIN_VALUE;
 
             // TODO GG-11640 log a correct merge record.
-            left.page.fullPageWalRecordPolicy(Boolean.TRUE);
+            left.walPlc = Boolean.TRUE;
 
             // Remove split key from parent. If we are merging empty branch then remove only on the top iteration.
             if (needMergeEmptyBranch != READY)
-                doRemove(prnt.page, prnt.io, prnt.buf, prntCnt, prntIdx);
+                doRemove(prnt.id, prnt.page, prnt.buf, prnt.walPlc, prnt.io, prntCnt, prntIdx);
 
             // Forward page is now empty and has no links, can free and release it right away.
-            freePage(right.page, right.buf, true);
+            freePage(right.id, right.page, right.buf, right.walPlc, true);
 
             return true;
         }
 
         /**
-         * @param page Page.
+         * @param pageId Page ID.
+         * @param page Page pointer.
          * @param pageAddr Page address.
+         * @param walPlc Full page WAL record policy.
          * @param release Release write lock and release page.
          * @throws IgniteCheckedException If failed.
          */
-        private void freePage(Page page, long pageAddr, boolean release)
+        private void freePage(long pageId, long page, long pageAddr, Boolean walPlc, boolean release)
             throws IgniteCheckedException {
-            long pageId = page.id();
 
             long effectivePageId = PageIdUtils.effectivePageId(pageId);
 
-            pageId = recyclePage(pageId, page, pageAddr);
+            long recycled = recyclePage(pageId, page, pageAddr, walPlc);
 
             if (effectivePageId != PageIdUtils.effectivePageId(pageId))
                 throw new IllegalStateException("Effective page ID must stay the same.");
 
             if (release)
-                writeUnlockAndClose(page, pageAddr);
+                writeUnlockAndClose(pageId, page, pageAddr, walPlc);
 
-            addFreePage(pageId);
+            addFreePage(recycled);
         }
 
         /**
@@ -3670,11 +3724,9 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
          * @throws IgniteCheckedException If failed.
          */
         private void cutRoot(int lvl) throws IgniteCheckedException {
-            try (Page meta = page(metaPageId)) {
-                Bool res = writePage(pageMem, meta, BPlusTree.this, cutRoot, null, lvl, FALSE);
+                Bool res = write(metaPageId, cutRoot, lvl, FALSE);
 
                 assert res == TRUE : res;
-            }
         }
 
         /**
@@ -3732,13 +3784,13 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             inner.io.setRemoveId(inner.buf, rmvId);
 
             // TODO GG-11640 log a correct inner replace record.
-            inner.page.fullPageWalRecordPolicy(Boolean.TRUE);
+            inner.walPlc = Boolean.TRUE;
 
             // Update remove ID for the leaf page.
             leaf.io.setRemoveId(leaf.buf, rmvId);
 
-            if (needWalDeltaRecord(leaf.page))
-                wal.log(new FixRemoveId(cacheId, leaf.page.id(), rmvId));
+            if (needWalDeltaRecord(leaf.id, leaf.page, leaf.walPlc))
+                wal.log(new FixRemoveId(cacheId, leaf.id, rmvId));
         }
 
         /**
@@ -3796,20 +3848,20 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
          */
         private void doReleaseTail(Tail<L> t) {
             while (t != null) {
-                writeUnlockAndClose(t.page, t.buf);
+                writeUnlockAndClose(t.id, t.page, t.buf, t.walPlc);
 
                 Tail<L> s = t.sibling;
 
                 if (s != null)
-                    writeUnlockAndClose(s.page, s.buf);
+                    writeUnlockAndClose(s.id, s.page, s.buf, s.walPlc);
 
                 t = t.down;
             }
         }
 
         /** {@inheritDoc} */
-        @Override boolean canRelease(Page page, int lvl) {
-            return page != null && !isTail(page.id(), lvl);
+        @Override boolean canRelease(long pageId, int lvl) {
+            return pageId != 0L && !isTail(pageId, lvl);
         }
 
         /**
@@ -3825,12 +3877,12 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                     return false;
 
                 if (t.lvl == lvl) {
-                    if (t.page.id() == pageId)
+                    if (t.id == pageId)
                         return true;
 
                     t = t.sibling;
 
-                    return t != null && t.page.id() == pageId;
+                    return t != null && t.id == pageId;
                 }
 
                 t = t.down;
@@ -3840,15 +3892,16 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         }
 
         /**
-         * @param page Page.
+         * @param pageId Page ID.
+         * @param page Page pointer.
          * @param pageAddr Page address.
          * @param io IO.
          * @param lvl Level.
          * @param type Type.
          * @return Added tail.
          */
-        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);
+        private Tail<L> addTail(long pageId, long page, long pageAddr, BPlusIO<L> io, int lvl, byte type) {
+            final Tail<L> t = new Tail<>(pageId, page, pageAddr, io, type, lvl);
 
             if (tail == null)
                 tail = t;
@@ -3946,15 +3999,15 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         }
 
         /**
-         * @param page Page.
          * @param pageId Page ID.
+         * @param page Page pointer.
          * @param backId Back page ID.
          * @param fwdId Forward ID.
          * @param lvl Level.
          * @return Result.
          * @throws IgniteCheckedException If failed.
          */
-        private Result finishOrLockTail(Page page, long pageId, long backId, long fwdId, int lvl)
+        private Result finishOrLockTail(long pageId, long page, long backId, long fwdId, int lvl)
             throws IgniteCheckedException {
             Result res = finishTail();
 
@@ -3965,15 +4018,15 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         }
 
         /**
-         * @param page Page.
          * @param pageId Page ID.
+         * @param page Page pointer.
          * @param backId Back page ID.
          * @param fwdId Forward ID.
          * @param lvl Level.
          * @return Result.
          * @throws IgniteCheckedException If failed.
          */
-        private Result tryRemoveFromLeaf(Page page, long pageId, long backId, long fwdId, int lvl)
+        private Result tryRemoveFromLeaf(long pageId, long page, long backId, long fwdId, int lvl)
             throws IgniteCheckedException {
             // We must be at the bottom here, just need to remove row from the current page.
             assert lvl == 0 : lvl;
@@ -4001,12 +4054,18 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         static final byte FORWARD = 2;
 
         /** */
-        private final Page page;
+        private final long id;
+
+        /** */
+        private final long page;
 
         /** */
         private final long buf;
 
         /** */
+        private Boolean walPlc;
+
+        /** */
         private final BPlusIO<L> io;
 
         /** */
@@ -4025,17 +4084,21 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         private Tail<L> down;
 
         /**
-         * @param page Write locked page.
+         * @param pageId Page ID.
+         * @param page Page absolute pointer.
          * @param buf Buffer.
          * @param io IO.
          * @param type Type.
          *

<TRUNCATED>