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:08 UTC

[1/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

Repository: ignite
Updated Branches:
  refs/heads/ignite-3477 cb60e3865 -> 7db65ddd1


http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteDistributedJoinTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteDistributedJoinTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteDistributedJoinTestSuite.java
index 5371856..cf67041 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteDistributedJoinTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteDistributedJoinTestSuite.java
@@ -34,7 +34,7 @@ import org.apache.ignite.internal.processors.query.h2.sql.H2CompareBigQueryDistr
  */
 public class IgniteDistributedJoinTestSuite extends TestSuite {
     /**
-     *
+     * @return Suite.
      */
     public static TestSuite suite() {
         TestSuite suite = new TestSuite("Distributed Joins Test Suite.");


[7/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

Posted by sb...@apache.org.
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


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

Branch: refs/heads/ignite-3477
Commit: 7db65ddd16beae5f5bf95f67c638d5ae1c58653c
Parents: cb60e38
Author: sboikov <sb...@gridgain.com>
Authored: Tue Jan 17 14:45:51 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Jan 17 14:45:53 2017 +0300

----------------------------------------------------------------------
 .../internal/binary/BinaryEnumObjectImpl.java   |   7 +
 .../internal/binary/BinaryObjectImpl.java       |   5 +
 .../binary/BinaryObjectOffheapImpl.java         |   5 +
 .../apache/ignite/internal/pagemem/Page.java    |  11 +-
 .../ignite/internal/pagemem/PageMemory.java     |   9 +-
 .../ignite/internal/pagemem/PageUtils.java      | 185 ++++++
 .../pagemem/impl/PageMemoryNoStoreImpl.java     | 134 ++--
 .../internal/pagemem/impl/PageNoStoreImpl.java  |  91 +--
 .../delta/DataPageInsertFragmentRecord.java     |   8 +-
 .../wal/record/delta/DataPageInsertRecord.java  |   8 +-
 .../wal/record/delta/DataPageRemoveRecord.java  |   8 +-
 .../delta/DataPageSetFreeListPageRecord.java    |   8 +-
 .../wal/record/delta/FixCountRecord.java        |   8 +-
 .../record/delta/FixLeftmostChildRecord.java    |   8 +-
 .../pagemem/wal/record/delta/FixRemoveId.java   |   8 +-
 .../wal/record/delta/InitNewPageRecord.java     |   8 +-
 .../wal/record/delta/InnerReplaceRecord.java    |   4 +-
 .../pagemem/wal/record/delta/InsertRecord.java  |   5 +-
 .../pagemem/wal/record/delta/MergeRecord.java   |   3 +-
 .../wal/record/delta/MetaPageAddRootRecord.java |   8 +-
 .../wal/record/delta/MetaPageCutRootRecord.java |   7 +-
 .../wal/record/delta/MetaPageInitRecord.java    |  14 +-
 .../record/delta/MetaPageInitRootRecord.java    |   9 +-
 .../delta/MetaPageUpdateLastAllocatedIndex.java |  10 +-
 ...aPageUpdateLastSuccessfulFullSnapshotId.java |   8 +-
 .../MetaPageUpdateLastSuccessfulSnapshotId.java |   8 +-
 .../delta/MetaPageUpdateNextSnapshotId.java     |   8 +-
 .../MetaPageUpdatePartitionDataRecord.java      |  12 +-
 .../wal/record/delta/NewRootInitRecord.java     |   6 +-
 .../wal/record/delta/PageDeltaRecord.java       |   7 +-
 .../delta/PageListMetaResetCountRecord.java     |   8 +-
 .../record/delta/PagesListAddPageRecord.java    |   8 +-
 .../delta/PagesListInitNewPageRecord.java       |  10 +-
 .../record/delta/PagesListRemovePageRecord.java |   8 +-
 .../record/delta/PagesListSetNextRecord.java    |   8 +-
 .../delta/PagesListSetPreviousRecord.java       |   8 +-
 .../pagemem/wal/record/delta/RecycleRecord.java |   6 +-
 .../pagemem/wal/record/delta/RemoveRecord.java  |  12 +-
 .../pagemem/wal/record/delta/ReplaceRecord.java |   7 +-
 .../record/delta/SplitExistingPageRecord.java   |   8 +-
 .../record/delta/SplitForwardPageRecord.java    |   3 +-
 .../record/delta/TrackingPageDeltaRecord.java   |  10 +-
 .../internal/processors/cache/CacheObject.java  |   7 +
 .../processors/cache/CacheObjectAdapter.java    |  40 +-
 .../cache/CacheObjectByteArrayImpl.java         |   5 +
 .../cache/IgniteCacheOffheapManagerImpl.java    | 372 +++++++----
 .../processors/cache/database/CacheDataRow.java |  13 +-
 .../cache/database/CacheDataRowAdapter.java     |  72 ++-
 .../cache/database/CacheSearchRow.java          |  40 ++
 .../cache/database/DataStructure.java           |  36 +-
 .../IgniteCacheDatabaseSharedManager.java       |   2 +-
 .../cache/database/MetadataStorage.java         | 149 +++--
 .../cache/database/freelist/FreeListImpl.java   |  61 +-
 .../cache/database/freelist/PagesList.java      | 212 +++----
 .../database/freelist/io/PagesListMetaIO.java   |  71 +--
 .../database/freelist/io/PagesListNodeIO.java   | 117 ++--
 .../cache/database/tree/BPlusTree.java          | 616 ++++++++++---------
 .../cache/database/tree/io/BPlusIO.java         | 201 +++---
 .../cache/database/tree/io/BPlusInnerIO.java    |  71 ++-
 .../cache/database/tree/io/BPlusLeafIO.java     |  11 +-
 .../cache/database/tree/io/BPlusMetaIO.java     |  79 +--
 .../cache/database/tree/io/CacheVersionIO.java  |  46 ++
 .../cache/database/tree/io/DataPageIO.java      | 545 ++++++++--------
 .../cache/database/tree/io/DataPagePayload.java |  64 ++
 .../cache/database/tree/io/IOVersions.java      |  12 +-
 .../cache/database/tree/io/PageIO.java          | 110 +++-
 .../cache/database/tree/io/PageMetaIO.java      | 118 ++--
 .../database/tree/io/PagePartitionMetaIO.java   |  67 +-
 .../database/tree/reuse/ReuseListImpl.java      |   2 +-
 .../cache/database/tree/util/PageHandler.java   | 135 ++--
 .../database/tree/util/PageLockListener.java    |  17 +-
 .../atomic/GridDhtAtomicSingleUpdateFuture.java |   3 +-
 .../atomic/GridNearAtomicUpdateResponse.java    |   3 +
 .../apache/ignite/internal/util/GridUnsafe.java |  12 +
 .../apache/ignite/internal/util/IgniteTree.java |  31 +-
 .../internal/GridAffinityNoCacheSelfTest.java   |   5 +
 .../pagemem/impl/PageMemoryNoLoadSelfTest.java  |  29 +-
 .../IgniteIncompleteCacheObjectSelfTest.java    |   5 +
 .../database/tree/io/TrackingPageIOTest.java    |  38 +-
 .../database/BPlusTreeReuseSelfTest.java        |  22 +-
 .../processors/database/BPlusTreeSelfTest.java  |  85 +--
 .../database/FreeListImplSelfTest.java          |  15 +-
 .../database/MetadataStorageSelfTest.java       |   2 +-
 .../internal/processors/query/h2/H2Cursor.java  |   9 +-
 .../processors/query/h2/database/H2Tree.java    |   5 +-
 .../query/h2/database/H2TreeIndex.java          |   5 +-
 .../query/h2/database/io/H2InnerIO.java         |  24 +-
 .../query/h2/database/io/H2LeafIO.java          |  22 +-
 .../query/h2/database/io/H2RowLinkIO.java       |   6 +-
 .../processors/query/h2/opt/GridH2Row.java      |   5 +
 .../query/IgniteSqlDistributedJoinSelfTest.java |  46 +-
 .../h2/GridIndexingSpiAbstractSelfTest.java     |   5 +
 .../IgniteDistributedJoinTestSuite.java         |   2 +-
 93 files changed, 2579 insertions(+), 1807 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryEnumObjectImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryEnumObjectImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryEnumObjectImpl.java
index a17f755..d6b6168 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryEnumObjectImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryEnumObjectImpl.java
@@ -276,6 +276,13 @@ public class BinaryEnumObjectImpl implements BinaryObjectEx, Externalizable, Cac
     }
 
     /** {@inheritDoc} */
+    @Override public int putValue(long addr) throws IgniteCheckedException {
+        assert valBytes != null : "Value bytes must be initialized before object is stored";
+
+        return CacheObjectAdapter.putValue(addr, cacheObjectType(), valBytes, 0);
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean putValue(final ByteBuffer buf, int off, int len) throws IgniteCheckedException {
         return CacheObjectAdapter.putValue(cacheObjectType(), buf, off, len, valBytes, 0);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectImpl.java
index ff14b63..0442a95 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectImpl.java
@@ -165,6 +165,11 @@ public final class BinaryObjectImpl extends BinaryObjectExImpl implements Extern
     }
 
     /** {@inheritDoc} */
+    @Override public int putValue(long addr) throws IgniteCheckedException {
+        return CacheObjectAdapter.putValue(addr, cacheObjectType(), arr, start);
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean putValue(final ByteBuffer buf, int off, int len) throws IgniteCheckedException {
         return CacheObjectAdapter.putValue(cacheObjectType(), buf, off, len, arr, start);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectOffheapImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectOffheapImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectOffheapImpl.java
index 4cf2bdf..d45815a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectOffheapImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectOffheapImpl.java
@@ -152,6 +152,11 @@ public class BinaryObjectOffheapImpl extends BinaryObjectExImpl implements Exter
     }
 
     /** {@inheritDoc} */
+    @Override public int putValue(long addr) throws IgniteCheckedException {
+        throw new UnsupportedOperationException("TODO implement");
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean putValue(final ByteBuffer buf, final int off, final int len)
         throws IgniteCheckedException {
         throw new UnsupportedOperationException("TODO implement");

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/Page.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/Page.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/Page.java
index e08fad6..2667e44 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/Page.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/Page.java
@@ -17,9 +17,6 @@
 
 package org.apache.ignite.internal.pagemem;
 
-import java.nio.ByteBuffer;
-import org.jetbrains.annotations.Nullable;
-
 /**
  *
  */
@@ -39,9 +36,9 @@ public interface Page extends AutoCloseable {
     public FullPageId fullId();
 
     /**
-     * @return ByteBuffer for modifying the page.
+     * @return Pointer for modifying the page.
      */
-    public ByteBuffer getForRead();
+    public long getForReadPointer();
 
     /**
      * Releases reserved page. Released page can be evicted from RAM after flushing modifications to disk.
@@ -51,12 +48,12 @@ public interface Page extends AutoCloseable {
     /**
      * @return ByteBuffer for modifying the page.
      */
-    public ByteBuffer getForWrite();
+    public long getForWritePointer();
 
     /**
      * @return ByteBuffer for modifying the page of {@code null} if failed to get write lock.
      */
-    @Nullable public ByteBuffer tryGetForWrite();
+    public long tryGetForWritePointer();
 
     /**
      * Releases reserved page. Released page can be evicted from RAM after flushing modifications to disk.

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageMemory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageMemory.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageMemory.java
index 53b37f6..cfee19f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageMemory.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageMemory.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.pagemem;
 
+import java.nio.ByteBuffer;
 import java.nio.ByteOrder;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.lifecycle.LifecycleAware;
@@ -25,7 +26,7 @@ import org.apache.ignite.lifecycle.LifecycleAware;
  */
 public interface PageMemory extends LifecycleAware, PageIdAllocator {
     /** */
-    ByteOrder NATIVE_BYTE_ORDER = ByteOrder.nativeOrder();
+    public static final ByteOrder NATIVE_BYTE_ORDER = ByteOrder.nativeOrder();
 
     /**
      * Gets the page associated with the given page ID. Each page obtained with this method must be released by
@@ -62,4 +63,10 @@ public interface PageMemory extends LifecycleAware, PageIdAllocator {
      * @return Page size with system overhead, in bytes.
      */
     public int systemPageSize();
+
+    /**
+     * @param pageAddr Page address.
+     * @return Page byte buffer.
+     */
+    public ByteBuffer pageBuffer(long pageAddr);
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageUtils.java
new file mode 100644
index 0000000..f824368
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageUtils.java
@@ -0,0 +1,185 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.pagemem;
+
+import org.apache.ignite.internal.util.GridUnsafe;
+
+/**
+ *
+ */
+@SuppressWarnings("deprecation")
+public class PageUtils {
+    /**
+     * @param addr Start address. 
+     * @param off Offset.
+     * @return Byte value from given address.
+     */
+    public static byte getByte(long addr, int off) {
+        assert addr > 0 : addr;
+        assert off >= 0;
+
+        return GridUnsafe.getByte(addr + off);
+    }
+
+    /**
+     * @param addr Start address.
+     * @param off Offset.
+     * @param len Bytes length.
+     * @return Bytes from given address.
+     */
+    public static byte[] getBytes(long addr, int off, int len) {
+        assert addr > 0 : addr;
+        assert off >= 0;
+        assert len >= 0;
+
+        byte[] bytes = new byte[len];
+
+        GridUnsafe.copyMemory(null, addr + off, bytes, GridUnsafe.BYTE_ARR_OFF, len);
+
+        return bytes;
+    }
+
+    /**
+     * @param srcAddr Source address.
+     * @param srcOff Source offset.
+     * @param dst Destination array.
+     * @param dstOff Destination offset.
+     * @param len Length.
+     */
+    public static void getBytes(long srcAddr, int srcOff, byte[] dst, int dstOff, int len) {
+        assert srcAddr > 0;
+        assert srcOff > 0;
+        assert dst != null;
+        assert dstOff >= 0;
+        assert len >= 0;
+
+        GridUnsafe.copyMemory(null, srcAddr + srcOff, dst, GridUnsafe.BYTE_ARR_OFF + dstOff, len);
+    }
+
+    /**
+     * @param addr Address.
+     * @param off Offset.
+     * @return Value.
+     */
+    public static short getShort(long addr, int off) {
+        assert addr > 0 : addr;
+        assert off >= 0;
+
+        return GridUnsafe.getShort(addr + off);
+    }
+
+    /**
+     * @param addr Address.
+     * @param off Offset.
+     * @return Value.
+     */
+    public static int getInt(long addr, int off) {
+        assert addr > 0 : addr;
+        assert off >= 0;
+
+        return GridUnsafe.getInt(addr + off);
+    }
+
+    /**
+     * @param addr Address.
+     * @param off Offset.
+     * @return Value.
+     */
+    public static long getLong(long addr, int off) {
+        assert addr > 0 : addr;
+        assert off >= 0;
+
+        return GridUnsafe.getLong(addr + off);
+    }
+
+    /**
+     * @param addr Address/
+     * @param off Offset.
+     * @param bytes Bytes.
+     */
+    public static void putBytes(long addr, int off, byte[] bytes) {
+        assert addr > 0 : addr;
+        assert off >= 0;
+        assert bytes != null;
+
+        GridUnsafe.copyMemory(bytes, GridUnsafe.BYTE_ARR_OFF, null, addr + off, bytes.length);
+    }
+
+    /**
+     * @param addr Address.
+     * @param off Offset.
+     * @param bytes Bytes array.
+     * @param bytesOff Bytes array offset.
+     */
+    public static void putBytes(long addr, int off, byte[] bytes, int bytesOff) {
+        assert addr > 0 : addr;
+        assert off >= 0;
+        assert bytes != null;
+        assert bytesOff >= 0 && (bytesOff < bytes.length || bytes.length == 0) : bytesOff;
+
+        GridUnsafe.copyMemory(bytes, GridUnsafe.BYTE_ARR_OFF + bytesOff, null, addr + off, bytes.length - bytesOff);
+    }
+
+    /**
+     * @param addr Address.
+     * @param off Offset.
+     * @param v Value.
+     */
+    public static void putByte(long addr, int off, byte v) {
+        assert addr > 0 : addr;
+        assert off >= 0;
+
+        GridUnsafe.putByte(addr + off, v);
+    }
+
+    /**
+     * @param addr Address.
+     * @param off Offset.
+     * @param v Value.
+     */
+    public static void putShort(long addr, int off, short v) {
+        assert addr > 0 : addr;
+        assert off >= 0;
+
+        GridUnsafe.putShort(addr + off, v);
+    }
+
+    /**
+     * @param addr Address.
+     * @param off Offset.
+     * @param v Value.
+     */
+    public static void putInt(long addr, int off, int v) {
+        assert addr > 0 : addr;
+        assert off >= 0;
+
+        GridUnsafe.putInt(addr + off, v);
+    }
+
+    /**
+     * @param addr Address.
+     * @param off Offset.
+     * @param v Value.
+     */
+    public static void putLong(long addr, int off, long v) {
+        assert addr > 0 : addr;
+        assert off >= 0;
+
+        GridUnsafe.putLong(addr + off, v);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/impl/PageMemoryNoStoreImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/impl/PageMemoryNoStoreImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/impl/PageMemoryNoStoreImpl.java
index 1d29df8..41e401d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/impl/PageMemoryNoStoreImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/impl/PageMemoryNoStoreImpl.java
@@ -20,15 +20,14 @@ package org.apache.ignite.internal.pagemem.impl;
 import java.io.Closeable;
 import java.io.IOException;
 import java.nio.ByteBuffer;
-import java.nio.ByteOrder;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.internal.mem.DirectMemory;
-import org.apache.ignite.internal.mem.DirectMemoryRegion;
 import org.apache.ignite.internal.mem.DirectMemoryProvider;
+import org.apache.ignite.internal.mem.DirectMemoryRegion;
 import org.apache.ignite.internal.mem.OutOfMemoryException;
 import org.apache.ignite.internal.pagemem.Page;
 import org.apache.ignite.internal.pagemem.PageIdUtils;
@@ -37,7 +36,6 @@ import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
 import org.apache.ignite.internal.util.GridUnsafe;
 import org.apache.ignite.internal.util.OffheapReadWriteLock;
 import org.apache.ignite.internal.util.offheap.GridOffHeapOutOfMemoryException;
-import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lifecycle.LifecycleAware;
 import sun.misc.JavaNioAccess;
 import sun.misc.SharedSecrets;
@@ -90,10 +88,7 @@ public class PageMemoryNoStoreImpl implements PageMemory {
     public static final int PAGE_ID_OFFSET = 8;
 
     /** Page pin counter offset. */
-    public static final int PIN_CNT_OFFSET = 16;
-
-    /** Page pin counter offset. */
-    public static final int LOCK_OFFSET = 24;
+    public static final int LOCK_OFFSET = 16;
 
     /**
      * Need a 8-byte pointer for linked list, 8 bytes for internal needs (flags),
@@ -134,21 +129,28 @@ public class PageMemoryNoStoreImpl implements PageMemory {
     /** */
     private OffheapReadWriteLock rwLock;
 
+    /** */
+    private final boolean trackAcquiredPages;
+
     /**
+     * @param log Logger.
      * @param directMemoryProvider Memory allocator to use.
      * @param sharedCtx Cache shared context.
      * @param pageSize Page size.
+     * @param trackAcquiredPages If {@code true} tracks number of allocated pages (for tests purpose only).
      */
     public PageMemoryNoStoreImpl(
         IgniteLogger log,
         DirectMemoryProvider directMemoryProvider,
         GridCacheSharedContext<?, ?> sharedCtx,
-        int pageSize
+        int pageSize,
+        boolean trackAcquiredPages
     ) {
         assert log != null || sharedCtx != null;
 
         this.log = sharedCtx != null ? sharedCtx.logger(PageMemoryNoStoreImpl.class) : log;
         this.directMemoryProvider = directMemoryProvider;
+        this.trackAcquiredPages = trackAcquiredPages;
 
         sysPageSize = pageSize + PAGE_OVERHEAD;
 
@@ -205,6 +207,11 @@ public class PageMemoryNoStoreImpl implements PageMemory {
     }
 
     /** {@inheritDoc} */
+    @Override public ByteBuffer pageBuffer(long pageAddr) {
+        return wrapPointer(pageAddr, pageSize());
+    }
+
+    /** {@inheritDoc} */
     @Override public long allocatePage(int cacheId, int partId, byte flags) {
         long relPtr = INVALID_REL_PTR;
         long absPtr = 0;
@@ -213,13 +220,13 @@ public class PageMemoryNoStoreImpl implements PageMemory {
             relPtr = seg.borrowFreePage();
 
             if (relPtr != INVALID_REL_PTR) {
-                absPtr = seg.absolute(relPtr);
+                absPtr = seg.absolute(PageIdUtils.pageIndex(relPtr));
 
                 break;
             }
         }
 
-        // No segments conatined a free page.
+        // No segments contained a free page.
         if (relPtr == INVALID_REL_PTR) {
             int segAllocIdx = nextRoundRobinIndex();
 
@@ -231,7 +238,7 @@ public class PageMemoryNoStoreImpl implements PageMemory {
                 relPtr = seg.allocateFreePage(flags);
 
                 if (relPtr != INVALID_REL_PTR) {
-                    absPtr = seg.absolute(relPtr);
+                    absPtr = seg.absolute(PageIdUtils.pageIndex(relPtr));
 
                     break;
                 }
@@ -248,9 +255,6 @@ public class PageMemoryNoStoreImpl implements PageMemory {
 
         writePageId(absPtr, pageId);
 
-        // Clear pin counter.
-        GridUnsafe.putLong(absPtr + PIN_CNT_OFFSET, 0);
-
         // TODO pass an argument to decide whether the page should be cleaned.
         GridUnsafe.setMemory(absPtr + PAGE_OVERHEAD, sysPageSize - PAGE_OVERHEAD, (byte)0);
 
@@ -259,7 +263,7 @@ public class PageMemoryNoStoreImpl implements PageMemory {
 
     /** {@inheritDoc} */
     @Override public boolean freePage(int cacheId, long pageId) {
-        Segment seg = segment(pageId);
+        Segment seg = segment(PageIdUtils.pageIndex(pageId));
 
         seg.releaseFreePage(pageId);
 
@@ -268,25 +272,25 @@ public class PageMemoryNoStoreImpl implements PageMemory {
 
     /** {@inheritDoc} */
     @Override public Page page(int cacheId, long pageId) throws IgniteCheckedException {
-        Segment seg = segment(pageId);
+        int pageIdx = PageIdUtils.pageIndex(pageId);
 
-        return seg.acquirePage(cacheId, pageId, false);
+        Segment seg = segment(pageIdx);
+
+        return seg.acquirePage(pageIdx, pageId);
     }
 
     /** {@inheritDoc} */
     @Override public Page page(int cacheId, long pageId, boolean restore) throws IgniteCheckedException {
-        Segment seg = segment(pageId);
-
-        return seg.acquirePage(cacheId, pageId, restore);
+        throw new UnsupportedOperationException();
     }
 
     /** {@inheritDoc} */
     @Override public void releasePage(Page p) {
-        PageNoStoreImpl page = (PageNoStoreImpl)p;
-
-        Segment seg = segments[page.segmentIndex()];
+        if (trackAcquiredPages) {
+            Segment seg = segment(PageIdUtils.pageIndex(p.id()));
 
-        seg.releasePage(page);
+            seg.onPageRelease();
+        }
     }
 
     /** {@inheritDoc} */
@@ -435,17 +439,15 @@ public class PageMemoryNoStoreImpl implements PageMemory {
      * @param absPtr Absolute memory pointer to the page header.
      * @param pageId Page ID to write.
      */
-    void writePageId(long absPtr, long pageId) {
+    private void writePageId(long absPtr, long pageId) {
         GridUnsafe.putLong(absPtr + PAGE_ID_OFFSET, pageId);
     }
 
     /**
-     * @param pageId Page ID.
+     * @param pageIdx Page index.
      * @return Segment.
      */
-    private Segment segment(long pageId) {
-        long pageIdx = PageIdUtils.pageIndex(pageId);
-
+    private Segment segment(int pageIdx) {
         int segIdx = segmentIndex(pageIdx);
 
         return segments[segIdx];
@@ -539,63 +541,31 @@ public class PageMemoryNoStoreImpl implements PageMemory {
          * @return Pinned page impl.
          */
         @SuppressWarnings("TypeMayBeWeakened")
-        private PageNoStoreImpl acquirePage(int cacheId, long pageId, boolean restore) {
-            long absPtr = absolute(pageId);
-
-            long marker = GridUnsafe.getLong(absPtr);
-
-            if (marker != PAGE_MARKER)
-                throw new IllegalStateException("Page was not allocated [absPtr=" + U.hexLong(absPtr) +
-                    ", cacheId=" + cacheId + ", pageId=" + U.hexLong(pageId) +
-                    ", marker=" + U.hexLong(marker) + ']');
-
-            while (true) {
-                long pinCnt = GridUnsafe.getLong(absPtr + PIN_CNT_OFFSET);
+        private PageNoStoreImpl acquirePage(int pageIdx, long pageId) {
+            long absPtr = absolute(pageIdx);
 
-                if (pinCnt < 0)
-                    throw new IllegalStateException("Page has been deallocated [absPtr=" + U.hexLong(absPtr) +
-                        ", cacheId=" + cacheId + ", pageId=" + U.hexLong(pageId) + ", pinCnt=" + pinCnt + ']');
+            if (trackAcquiredPages)
+                acquiredPages.incrementAndGet();
 
-                if (GridUnsafe.compareAndSwapLong(null, absPtr + PIN_CNT_OFFSET, pinCnt, pinCnt + 1))
-                    break;
-            }
-
-            acquiredPages.incrementAndGet();
-
-            return new PageNoStoreImpl(PageMemoryNoStoreImpl.this, idx, absPtr, cacheId, pageId, restore);
+            return new PageNoStoreImpl(PageMemoryNoStoreImpl.this, absPtr, pageId);
         }
 
         /**
-         * @param pinnedPage Page to unpin.
          */
-        private void releasePage(PageNoStoreImpl pinnedPage) {
-            long absPtr = pinnedPage.absolutePointer();
-
-            while (true) {
-                long pinCnt = GridUnsafe.getLong(absPtr + PIN_CNT_OFFSET);
-
-                assert pinCnt > 0 : "Releasing a page that was not pinned [page=" + pinnedPage +
-                    ", pinCnt=" + pinCnt + ']';
-
-                if (GridUnsafe.compareAndSwapLong(null, absPtr + PIN_CNT_OFFSET, pinCnt, pinCnt - 1))
-                    break;
-            }
-
+        private void onPageRelease() {
             acquiredPages.decrementAndGet();
         }
 
         /**
-         * @param relativePtr Relative pointer.
+         * @param pageIdx Page index.
          * @return Absolute pointer.
          */
-        private long absolute(long relativePtr) {
-            int pageIdx = PageIdUtils.pageIndex(relativePtr);
-
+        private long absolute(int pageIdx) {
             pageIdx &= idxMask;
 
-            long offset = ((long)pageIdx) * sysPageSize;
+            long off = ((long)pageIdx) * sysPageSize;
 
-            return pagesBase + offset;
+            return pagesBase + off;
         }
 
         /**
@@ -616,24 +586,12 @@ public class PageMemoryNoStoreImpl implements PageMemory {
          * @param pageId Page ID to release.
          */
         private void releaseFreePage(long pageId) {
-            // Clear out flags and file ID.
-            long relPtr = PageIdUtils.pageId(0, (byte)0, PageIdUtils.pageIndex(pageId));
-
-            long absPtr = absolute(relPtr);
-
-            // Prepare page to free.
-            // First, swap pin counter down to -1.
-            while (true) {
-                long pinCnt = GridUnsafe.getLong(absPtr + PIN_CNT_OFFSET);
-
-                assert pinCnt >= 0 : "pinCnt=" + pinCnt + ", relPtr=" + U.hexLong(relPtr);
+            int pageIdx = PageIdUtils.pageIndex(pageId);
 
-                if (pinCnt > 0)
-                    throw new IllegalStateException("Releasing a page being in use: " + U.hexLong(relPtr));
+            // Clear out flags and file ID.
+            long relPtr = PageIdUtils.pageId(0, (byte)0, pageIdx);
 
-                if (GridUnsafe.compareAndSwapLong(null, absPtr + PIN_CNT_OFFSET, 0, -1))
-                    break;
-            }
+            long absPtr = absolute(pageIdx);
 
             // Second, write clean relative pointer instead of page ID.
             writePageId(absPtr, relPtr);
@@ -665,7 +623,7 @@ public class PageMemoryNoStoreImpl implements PageMemory {
                 long cnt = ((freePageRelPtrMasked & COUNTER_MASK) + COUNTER_INC) & COUNTER_MASK;
 
                 if (freePageRelPtr != INVALID_REL_PTR) {
-                    long freePageAbsPtr = absolute(freePageRelPtr);
+                    long freePageAbsPtr = absolute(PageIdUtils.pageIndex(freePageRelPtr));
 
                     long nextFreePageRelPtr = GridUnsafe.getLong(freePageAbsPtr) & ADDRESS_MASK;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/impl/PageNoStoreImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/impl/PageNoStoreImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/impl/PageNoStoreImpl.java
index 404c0b2..b52df55 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/impl/PageNoStoreImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/impl/PageNoStoreImpl.java
@@ -20,54 +20,39 @@ package org.apache.ignite.internal.pagemem.impl;
 import org.apache.ignite.internal.pagemem.FullPageId;
 import org.apache.ignite.internal.pagemem.Page;
 import org.apache.ignite.internal.pagemem.PageIdUtils;
-import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.processors.cache.database.tree.io.PageIO;
-import org.apache.ignite.internal.util.OffheapReadWriteLock;
 import org.apache.ignite.internal.util.typedef.internal.SB;
 
-import java.nio.ByteBuffer;
-
 /**
  *
  */
 public class PageNoStoreImpl implements Page {
     /** */
-    private int segIdx;
-
-    /** */
     private long absPtr;
 
     /** */
     private long pageId;
 
     /** */
-    private int cacheId;
-
-    /** */
     private PageMemoryNoStoreImpl pageMem;
 
-    /** */
-    private final ByteBuffer buf;
-
-    /** Page for memory restore */
-    private final boolean noTagCheck;
-
     /**
-     * @param segIdx Segment index.
+     * @param pageMem Page memory.
      * @param absPtr Absolute pointer.
+     * @param pageId Page ID.
      */
-    public PageNoStoreImpl(
-        PageMemoryNoStoreImpl pageMem, int segIdx, long absPtr, int cacheId, long pageId, boolean noTagCheck
-    ) {
+    PageNoStoreImpl(PageMemoryNoStoreImpl pageMem, long absPtr, long pageId) {
         this.pageMem = pageMem;
-        this.segIdx = segIdx;
         this.absPtr = absPtr;
 
-        this.cacheId = cacheId;
         this.pageId = pageId;
-        this.noTagCheck = noTagCheck;
+    }
 
-        buf = pageMem.wrapPointer(absPtr + PageMemoryNoStoreImpl.PAGE_OVERHEAD, pageMem.pageSize());
+    /**
+     * @return Data pointer.
+     */
+    private long pointer() {
+        return absPtr + PageMemoryNoStoreImpl.PAGE_OVERHEAD;
     }
 
     /** {@inheritDoc} */
@@ -77,15 +62,15 @@ public class PageNoStoreImpl implements Page {
 
     /** {@inheritDoc} */
     @Override public FullPageId fullId() {
-        return new FullPageId(pageId, cacheId);
+        throw new UnsupportedOperationException();
     }
 
     /** {@inheritDoc} */
-    @Override public ByteBuffer getForRead() {
+    @Override public long getForReadPointer() {
         if (pageMem.readLockPage(absPtr, PageIdUtils.tag(pageId)))
-            return reset(buf.asReadOnlyBuffer());
+            return pointer();
 
-        return null;
+        return 0L;
     }
 
     /** {@inheritDoc} */
@@ -94,31 +79,29 @@ public class PageNoStoreImpl implements Page {
     }
 
     /** {@inheritDoc} */
-    @Override public ByteBuffer getForWrite() {
-        int tag =  noTagCheck ? OffheapReadWriteLock.TAG_LOCK_ALWAYS :  PageIdUtils.tag(pageId);
+    @Override public long getForWritePointer() {
+        int tag = PageIdUtils.tag(pageId);
         boolean locked = pageMem.writeLockPage(absPtr, tag);
 
-        if (!locked && !noTagCheck)
-            return null;
-
-        assert locked;
+        if (!locked)
+            return 0L;
 
-        return reset(buf);
+        return pointer();
     }
 
     /** {@inheritDoc} */
-    @Override public ByteBuffer tryGetForWrite() {
-        int tag =  noTagCheck ? OffheapReadWriteLock.TAG_LOCK_ALWAYS :  PageIdUtils.tag(pageId);
+    @Override public long tryGetForWritePointer() {
+        int tag = PageIdUtils.tag(pageId);
 
         if (pageMem.tryWriteLockPage(absPtr, tag))
-            return reset(buf);
+            return pointer();
 
-        return null;
+        return 0L;
     }
 
     /** {@inheritDoc} */
     @Override public void releaseWrite(boolean markDirty) {
-        long updatedPageId = PageIO.getPageId(buf);
+        long updatedPageId = PageIO.getPageId(pointer());
 
         pageMem.writeUnlockPage(absPtr, PageIdUtils.tag(updatedPageId));
     }
@@ -143,39 +126,11 @@ public class PageNoStoreImpl implements Page {
         pageMem.releasePage(this);
     }
 
-    /**
-     * @return Segment index.
-     */
-    int segmentIndex() {
-        return segIdx;
-    }
-
-    /**
-     * @return Absolute pointer to the system page start.
-     */
-    long absolutePointer() {
-        return absPtr;
-    }
-
-    /**
-     * @param buf Byte buffer.
-     * @return The given buffer back.
-     */
-    private ByteBuffer reset(ByteBuffer buf) {
-        buf.order(PageMemory.NATIVE_BYTE_ORDER);
-
-        buf.rewind();
-
-        return buf;
-    }
-
     /** {@inheritDoc} */
     @Override public String toString() {
         SB sb = new SB("PageNoStoreImpl [absPtr=0x");
 
         sb.appendHex(absPtr);
-        sb.a(", segIdx=").a(segIdx);
-        sb.a(", cacheId=").a(cacheId);
         sb.a(", pageId=0x").appendHex(pageId);
         sb.a("]");
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageInsertFragmentRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageInsertFragmentRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageInsertFragmentRecord.java
index 49b2626..eeaabd1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageInsertFragmentRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageInsertFragmentRecord.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.processors.cache.database.tree.io.DataPageIO;
 
 /**
@@ -50,10 +50,10 @@ public class DataPageInsertFragmentRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
-        DataPageIO io = DataPageIO.VERSIONS.forPage(buf);
+    @Override public void applyDelta(PageMemory pageMem, long pageAddr) throws IgniteCheckedException {
+        DataPageIO io = DataPageIO.VERSIONS.forPage(pageAddr);
 
-        io.addRowFragment(buf, payload, lastLink);
+        io.addRowFragment(pageAddr, payload, lastLink, pageMem.pageSize());
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageInsertRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageInsertRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageInsertRecord.java
index ceb06c7..f23d57a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageInsertRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageInsertRecord.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.processors.cache.database.tree.io.DataPageIO;
 
 /**
@@ -51,12 +51,12 @@ public class DataPageInsertRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
+    @Override public void applyDelta(PageMemory pageMem, long pageAddr) throws IgniteCheckedException {
         assert payload != null;
 
-        DataPageIO io = DataPageIO.VERSIONS.forPage(buf);
+        DataPageIO io = DataPageIO.VERSIONS.forPage(pageAddr);
 
-        io.addRow(buf, payload);
+        io.addRow(pageAddr, payload, pageMem.pageSize());
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageRemoveRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageRemoveRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageRemoveRecord.java
index 511094f..17c7fe8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageRemoveRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageRemoveRecord.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.processors.cache.database.tree.io.DataPageIO;
 import org.apache.ignite.internal.util.typedef.internal.S;
 
@@ -48,11 +48,11 @@ public class DataPageRemoveRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf)
+    @Override public void applyDelta(PageMemory pageMem, long pageAddr)
         throws IgniteCheckedException {
-        DataPageIO io = DataPageIO.VERSIONS.forPage(buf);
+        DataPageIO io = DataPageIO.VERSIONS.forPage(pageAddr);
 
-        io.removeRow(buf, itemId);
+        io.removeRow(pageAddr, itemId, pageMem.pageSize());
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageSetFreeListPageRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageSetFreeListPageRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageSetFreeListPageRecord.java
index 6463989..c835052 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageSetFreeListPageRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageSetFreeListPageRecord.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.processors.cache.database.tree.io.DataPageIO;
 
 import static org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType.DATA_PAGE_SET_FREE_LIST_PAGE;
@@ -49,10 +49,10 @@ public class DataPageSetFreeListPageRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
-        DataPageIO io = DataPageIO.VERSIONS.forPage(buf);
+    @Override public void applyDelta(PageMemory pageMem, long pageAddr) throws IgniteCheckedException {
+        DataPageIO io = DataPageIO.VERSIONS.forPage(pageAddr);
 
-        io.setFreeListPageId(buf, freeListPage);
+        io.setFreeListPageId(pageAddr, freeListPage);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixCountRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixCountRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixCountRecord.java
index 5089c76..c727710 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixCountRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixCountRecord.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.processors.cache.database.tree.io.BPlusIO;
 import org.apache.ignite.internal.processors.cache.database.tree.io.PageIO;
 
@@ -40,10 +40,10 @@ public class FixCountRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
-        BPlusIO<?> io = PageIO.getBPlusIO(buf);
+    @Override public void applyDelta(PageMemory pageMem, long pageAddr) throws IgniteCheckedException {
+        BPlusIO<?> io = PageIO.getBPlusIO(pageAddr);
 
-        io.setCount(buf, cnt);
+        io.setCount(pageAddr, cnt);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixLeftmostChildRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixLeftmostChildRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixLeftmostChildRecord.java
index fe09dd1..94155c9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixLeftmostChildRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixLeftmostChildRecord.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.processors.cache.database.tree.io.BPlusInnerIO;
 import org.apache.ignite.internal.processors.cache.database.tree.io.PageIO;
 
@@ -41,10 +41,10 @@ public class FixLeftmostChildRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
-        BPlusInnerIO<?> io = PageIO.getBPlusIO(buf);
+    @Override public void applyDelta(PageMemory pageMem, long pageAddr) throws IgniteCheckedException {
+        BPlusInnerIO<?> io = PageIO.getBPlusIO(pageAddr);
 
-        io.setLeft(buf, 0, rightId);
+        io.setLeft(pageAddr, 0, rightId);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixRemoveId.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixRemoveId.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixRemoveId.java
index 5147854..b9900e1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixRemoveId.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixRemoveId.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.processors.cache.database.tree.io.BPlusIO;
 import org.apache.ignite.internal.processors.cache.database.tree.io.PageIO;
 
@@ -41,11 +41,11 @@ public class FixRemoveId extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf)
+    @Override public void applyDelta(PageMemory pageMem, long pageAddr)
         throws IgniteCheckedException {
-        BPlusIO<?> io = PageIO.getBPlusIO(buf);
+        BPlusIO<?> io = PageIO.getBPlusIO(pageAddr);
 
-        io.setRemoveId(buf, rmvId);
+        io.setRemoveId(pageAddr, rmvId);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InitNewPageRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InitNewPageRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InitNewPageRecord.java
index 5ffc193..f5607dd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InitNewPageRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InitNewPageRecord.java
@@ -17,15 +17,15 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.processors.cache.database.tree.io.PageIO;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 
 /**
- * Initializes new page by calling {@link PageIO#initNewPage(ByteBuffer, long)}.
+ * Initializes new page by calling {@link PageIO#initNewPage(long, long, int)}.
  */
 public class InitNewPageRecord extends PageDeltaRecord {
     /** */
@@ -54,10 +54,10 @@ public class InitNewPageRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
+    @Override public void applyDelta(PageMemory pageMem, long pageAddr) throws IgniteCheckedException {
         PageIO io = PageIO.getPageIO(ioType, ioVer);
 
-        io.initNewPage(buf, newPageId);
+        io.initNewPage(pageAddr, newPageId, pageMem.pageSize());
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InnerReplaceRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InnerReplaceRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InnerReplaceRecord.java
index ccc3449..35d23c1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InnerReplaceRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InnerReplaceRecord.java
@@ -17,9 +17,9 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.pagemem.PageMemory;
 
 /**
  * Inner replace on remove.
@@ -57,7 +57,7 @@ public class InnerReplaceRecord<L> extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer dstBuf) throws IgniteCheckedException {
+    @Override public void applyDelta(PageMemory pageMem, long pageAddr) throws IgniteCheckedException {
         throw new IgniteCheckedException("Inner replace record should not be logged.");
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InsertRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InsertRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InsertRecord.java
index f439bf3..fa598b2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InsertRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InsertRecord.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.pagemem.wal.record.delta;
 
 import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.processors.cache.database.tree.io.BPlusIO;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.typedef.internal.S;
@@ -72,8 +73,8 @@ public class InsertRecord<L> extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
-        io.insert(buf, idx, row, rowBytes, rightId);
+    @Override public void applyDelta(PageMemory pageMem, long pageAddr) throws IgniteCheckedException {
+        io.insert(pageAddr, idx, row, rowBytes, rightId);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MergeRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MergeRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MergeRecord.java
index d8113fe..84770d5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MergeRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MergeRecord.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.pagemem.wal.record.delta;
 import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
@@ -63,7 +64,7 @@ public class MergeRecord<L> extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer leftBuf) throws IgniteCheckedException {
+    @Override public void applyDelta(PageMemory pageMem, long pageAddr) throws IgniteCheckedException {
         throw new IgniteCheckedException("Merge record should not be logged.");
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageAddRootRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageAddRootRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageAddRootRecord.java
index 75aaca4..176df33 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageAddRootRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageAddRootRecord.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.processors.cache.database.tree.io.BPlusMetaIO;
 
 /**
@@ -40,10 +40,10 @@ public class MetaPageAddRootRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
-        BPlusMetaIO io = BPlusMetaIO.VERSIONS.forPage(buf);
+    @Override public void applyDelta(PageMemory pageMem, long pageAddr) throws IgniteCheckedException {
+        BPlusMetaIO io = BPlusMetaIO.VERSIONS.forPage(pageAddr);
 
-        io.addRoot(buf, rootId);
+        io.addRoot(pageAddr, rootId, pageMem.pageSize());
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageCutRootRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageCutRootRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageCutRootRecord.java
index ba1d04d..50ccddc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageCutRootRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageCutRootRecord.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.pagemem.wal.record.delta;
 
 import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.processors.cache.database.tree.io.BPlusMetaIO;
 
 /**
@@ -34,10 +35,10 @@ public class MetaPageCutRootRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
-        BPlusMetaIO io = BPlusMetaIO.VERSIONS.forPage(buf);
+    @Override public void applyDelta(PageMemory pageMem, long pageAddr) throws IgniteCheckedException {
+        BPlusMetaIO io = BPlusMetaIO.VERSIONS.forPage(pageAddr);
 
-        io.cutRoot(buf);
+        io.cutRoot(pageAddr, pageMem.pageSize());
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageInitRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageInitRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageInitRecord.java
index df671f3..3d351a2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageInitRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageInitRecord.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.processors.cache.database.tree.io.PageIO;
 import org.apache.ignite.internal.processors.cache.database.tree.io.PageMetaIO;
 import org.apache.ignite.internal.processors.cache.database.tree.io.PagePartitionMetaIO;
@@ -75,15 +75,15 @@ public class MetaPageInitRecord extends InitNewPageRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
+    @Override public void applyDelta(PageMemory pageMem, long pageAddr) throws IgniteCheckedException {
         PageMetaIO io = ioType == PageIO.T_META ?
-            PageMetaIO.VERSIONS.forPage(buf) :
-            PagePartitionMetaIO.VERSIONS.forPage(buf);
+            PageMetaIO.VERSIONS.forPage(pageAddr) :
+            PagePartitionMetaIO.VERSIONS.forPage(pageAddr);
 
-        io.initNewPage(buf, newPageId);
+        io.initNewPage(pageAddr, newPageId, pageMem.pageSize());
 
-        io.setTreeRoot(buf, treeRoot);
-        io.setReuseListRoot(buf, reuseListRoot);
+        io.setTreeRoot(pageAddr, treeRoot);
+        io.setReuseListRoot(pageAddr, reuseListRoot);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageInitRootRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageInitRootRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageInitRootRecord.java
index a9d2677..4d56db0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageInitRootRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageInitRootRecord.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.processors.cache.database.tree.io.BPlusMetaIO;
 
 /**
@@ -40,11 +40,10 @@ public class MetaPageInitRootRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf)
-        throws IgniteCheckedException {
-        BPlusMetaIO io = BPlusMetaIO.VERSIONS.forPage(buf);
+    @Override public void applyDelta(PageMemory pageMem, long pageAddr) throws IgniteCheckedException {
+        BPlusMetaIO io = BPlusMetaIO.VERSIONS.forPage(pageAddr);
 
-        io.initRoot(buf, rootId);
+        io.initRoot(pageAddr, rootId, pageMem.pageSize());
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastAllocatedIndex.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastAllocatedIndex.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastAllocatedIndex.java
index 836a4eb..bd9b100 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastAllocatedIndex.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastAllocatedIndex.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.processors.cache.database.tree.io.PageIO;
 import org.apache.ignite.internal.processors.cache.database.tree.io.PageMetaIO;
 
@@ -39,12 +39,12 @@ public class MetaPageUpdateLastAllocatedIndex extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
-        assert PageIO.getType(buf) == PageIO.T_META || PageIO.getType(buf) == PageIO.T_PART_META;
+    @Override public void applyDelta(PageMemory pageMem, long pageAddr) throws IgniteCheckedException {
+        assert PageIO.getType(pageAddr) == PageIO.T_META || PageIO.getType(pageAddr) == PageIO.T_PART_META;
 
-        PageMetaIO io = PageMetaIO.VERSIONS.forVersion(PageIO.getVersion(buf));
+        PageMetaIO io = PageMetaIO.VERSIONS.forVersion(PageIO.getVersion(pageAddr));
 
-        io.setLastAllocatedIndex(buf, lastAllocatedIdx);
+        io.setLastAllocatedIndex(pageAddr, lastAllocatedIdx);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastSuccessfulFullSnapshotId.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastSuccessfulFullSnapshotId.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastSuccessfulFullSnapshotId.java
index 5633354..e322b4f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastSuccessfulFullSnapshotId.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastSuccessfulFullSnapshotId.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.processors.cache.database.tree.io.PageMetaIO;
 
 /**
@@ -38,10 +38,10 @@ public class MetaPageUpdateLastSuccessfulFullSnapshotId extends PageDeltaRecord
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
-        PageMetaIO io = PageMetaIO.VERSIONS.forPage(buf);
+    @Override public void applyDelta(PageMemory pageMem, long pageAddr) throws IgniteCheckedException {
+        PageMetaIO io = PageMetaIO.VERSIONS.forPage(pageAddr);
 
-        io.setLastSuccessfulFullSnapshotId(buf, lastSuccessfulFullSnapshotId);
+        io.setLastSuccessfulFullSnapshotId(pageAddr, lastSuccessfulFullSnapshotId);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastSuccessfulSnapshotId.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastSuccessfulSnapshotId.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastSuccessfulSnapshotId.java
index 20e136b..df9d778 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastSuccessfulSnapshotId.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastSuccessfulSnapshotId.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.processors.cache.database.tree.io.PageMetaIO;
 
 /**
@@ -42,10 +42,10 @@ public class MetaPageUpdateLastSuccessfulSnapshotId extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
-        PageMetaIO io = PageMetaIO.VERSIONS.forPage(buf);
+    @Override public void applyDelta(PageMemory pageMem, long pageAddr) throws IgniteCheckedException {
+        PageMetaIO io = PageMetaIO.VERSIONS.forPage(pageAddr);
 
-        io.setLastSuccessfulSnapshotId(buf, lastSuccessfulSnapshotId);
+        io.setLastSuccessfulSnapshotId(pageAddr, lastSuccessfulSnapshotId);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateNextSnapshotId.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateNextSnapshotId.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateNextSnapshotId.java
index 0a92aef..1403bd7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateNextSnapshotId.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateNextSnapshotId.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.processors.cache.database.tree.io.PageMetaIO;
 
 /**
@@ -38,10 +38,10 @@ public class MetaPageUpdateNextSnapshotId extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
-        PageMetaIO io = PageMetaIO.VERSIONS.forPage(buf);
+    @Override public void applyDelta(PageMemory pageMem, long pageAddr) throws IgniteCheckedException {
+        PageMetaIO io = PageMetaIO.VERSIONS.forPage(pageAddr);
 
-        io.setNextSnapshotTag(buf, nextSnapshotId);
+        io.setNextSnapshotTag(pageAddr, nextSnapshotId);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdatePartitionDataRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdatePartitionDataRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdatePartitionDataRecord.java
index ae6210a..66efc6f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdatePartitionDataRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdatePartitionDataRecord.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.processors.cache.database.tree.io.PagePartitionMetaIO;
 
 /**
@@ -85,12 +85,12 @@ public class MetaPageUpdatePartitionDataRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
-        PagePartitionMetaIO io = PagePartitionMetaIO.VERSIONS.forPage(buf);
+    @Override public void applyDelta(PageMemory pageMem, long pageAddr) throws IgniteCheckedException {
+        PagePartitionMetaIO io = PagePartitionMetaIO.VERSIONS.forPage(pageAddr);
 
-        io.setUpdateCounter(buf, updateCntr);
-        io.setGlobalRemoveId(buf, globalRmvId);
-        io.setSize(buf, partSize);
+        io.setUpdateCounter(pageAddr, updateCntr);
+        io.setGlobalRemoveId(pageAddr, globalRmvId);
+        io.setSize(pageAddr, partSize);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/NewRootInitRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/NewRootInitRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/NewRootInitRecord.java
index e0284f7..26ee364 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/NewRootInitRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/NewRootInitRecord.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.processors.cache.database.tree.io.BPlusInnerIO;
 
 /**
@@ -74,8 +74,8 @@ public class NewRootInitRecord<L> extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
-        io.initNewRoot(buf, newRootId, leftChildId, row, rowBytes, rightChildId);
+    @Override public void applyDelta(PageMemory pageMem, long pageAddr) throws IgniteCheckedException {
+        io.initNewRoot(pageAddr, newRootId, leftChildId, row, rowBytes, rightChildId, pageMem.pageSize());
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PageDeltaRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PageDeltaRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PageDeltaRecord.java
index 9acdd08..ca52bd2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PageDeltaRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PageDeltaRecord.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.pagemem.wal.record.WALRecord;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.typedef.internal.S;
@@ -62,10 +62,11 @@ public abstract class PageDeltaRecord extends WALRecord {
      * Apply changes from this delta to the given page.
      * It is assumed that the given buffer represents page state right before this update.
      *
-     * @param buf Page buffer.
+     * @param pageMem Page memory.
+     * @param pageAddr Page address.
      * @throws IgniteCheckedException If failed.
      */
-    public abstract void applyDelta(ByteBuffer buf) throws IgniteCheckedException;
+    public abstract void applyDelta(PageMemory pageMem, long pageAddr) throws IgniteCheckedException;
 
     /** {@inheritDoc} */
     @Override public String toString() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PageListMetaResetCountRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PageListMetaResetCountRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PageListMetaResetCountRecord.java
index 23ebcee..16587d3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PageListMetaResetCountRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PageListMetaResetCountRecord.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.processors.cache.database.freelist.io.PagesListMetaIO;
 
 /**
@@ -34,10 +34,10 @@ public class PageListMetaResetCountRecord extends PageDeltaRecord  {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
-        PagesListMetaIO io = PagesListMetaIO.VERSIONS.forPage(buf);
+    @Override public void applyDelta(PageMemory pageMem, long pageAddr) throws IgniteCheckedException {
+        PagesListMetaIO io = PagesListMetaIO.VERSIONS.forPage(pageAddr);
 
-        io.resetCount(buf);
+        io.resetCount(pageAddr);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListAddPageRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListAddPageRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListAddPageRecord.java
index 7ea16b8..a503b46 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListAddPageRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListAddPageRecord.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.processors.cache.database.freelist.io.PagesListNodeIO;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.typedef.internal.S;
@@ -51,10 +51,10 @@ public class PagesListAddPageRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
-        PagesListNodeIO io = PagesListNodeIO.VERSIONS.forPage(buf);
+    @Override public void applyDelta(PageMemory pageMem, long pageAddr) throws IgniteCheckedException {
+        PagesListNodeIO io = PagesListNodeIO.VERSIONS.forPage(pageAddr);
 
-        int cnt = io.addPage(buf, dataPageId);
+        int cnt = io.addPage(pageAddr, dataPageId, pageMem.pageSize());
 
         assert cnt >= 0 : cnt;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListInitNewPageRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListInitNewPageRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListInitNewPageRecord.java
index 922f212..b035323 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListInitNewPageRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListInitNewPageRecord.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.processors.cache.database.freelist.io.PagesListNodeIO;
 import org.apache.ignite.internal.processors.cache.database.tree.io.PageIO;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
@@ -73,14 +73,14 @@ public class PagesListInitNewPageRecord extends InitNewPageRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
+    @Override public void applyDelta(PageMemory pageMem, long pageAddr) throws IgniteCheckedException {
         PagesListNodeIO io = PageIO.getPageIO(PageIO.T_PAGE_LIST_NODE, ioVer);
 
-        io.initNewPage(buf, pageId());
-        io.setPreviousId(buf, prevPageId);
+        io.initNewPage(pageAddr, pageId(), pageMem.pageSize());
+        io.setPreviousId(pageAddr, prevPageId);
 
         if (addDataPageId != 0L) {
-            int cnt = io.addPage(buf, addDataPageId);
+            int cnt = io.addPage(pageAddr, addDataPageId, pageMem.pageSize());
 
             assert cnt == 0 : cnt;
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListRemovePageRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListRemovePageRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListRemovePageRecord.java
index 128e396..26f832c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListRemovePageRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListRemovePageRecord.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.processors.cache.database.freelist.io.PagesListNodeIO;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.typedef.internal.S;
@@ -51,10 +51,10 @@ public class PagesListRemovePageRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
-        PagesListNodeIO io = PagesListNodeIO.VERSIONS.forPage(buf);
+    @Override public void applyDelta(PageMemory pageMem, long pageAddr) throws IgniteCheckedException {
+        PagesListNodeIO io = PagesListNodeIO.VERSIONS.forPage(pageAddr);
 
-        boolean rmvd = io.removePage(buf, rmvdPageId);
+        boolean rmvd = io.removePage(pageAddr, rmvdPageId);
 
         assert rmvd;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListSetNextRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListSetNextRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListSetNextRecord.java
index 07ce6d2..c0bed60 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListSetNextRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListSetNextRecord.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.processors.cache.database.freelist.io.PagesListNodeIO;
 
 /**
@@ -47,10 +47,10 @@ public class PagesListSetNextRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
-        PagesListNodeIO io = PagesListNodeIO.VERSIONS.forPage(buf);
+    @Override public void applyDelta(PageMemory pageMem, long pageAddr) throws IgniteCheckedException {
+        PagesListNodeIO io = PagesListNodeIO.VERSIONS.forPage(pageAddr);
 
-        io.setNextId(buf, nextPageId);
+        io.setNextId(pageAddr, nextPageId);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListSetPreviousRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListSetPreviousRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListSetPreviousRecord.java
index bb3a877..21c3ef2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListSetPreviousRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListSetPreviousRecord.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.processors.cache.database.freelist.io.PagesListNodeIO;
 
 /**
@@ -47,10 +47,10 @@ public class PagesListSetPreviousRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
-        PagesListNodeIO io = PagesListNodeIO.VERSIONS.forPage(buf);
+    @Override public void applyDelta(PageMemory pageMem, long pageAddr) throws IgniteCheckedException {
+        PagesListNodeIO io = PagesListNodeIO.VERSIONS.forPage(pageAddr);
 
-        io.setPreviousId(buf, prevPageId);
+        io.setPreviousId(pageAddr, prevPageId);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/RecycleRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/RecycleRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/RecycleRecord.java
index 470f441..1737e12 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/RecycleRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/RecycleRecord.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.processors.cache.database.tree.io.PageIO;
 
 /**
@@ -40,8 +40,8 @@ public class RecycleRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
-        PageIO.setPageId(buf, newPageId);
+    @Override public void applyDelta(PageMemory pageMem, long pageAddr) throws IgniteCheckedException {
+        PageIO.setPageId(pageAddr, newPageId);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/RemoveRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/RemoveRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/RemoveRecord.java
index 453525c..6d278e9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/RemoveRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/RemoveRecord.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.processors.cache.database.tree.io.BPlusIO;
 import org.apache.ignite.internal.processors.cache.database.tree.io.PageIO;
 import org.apache.ignite.internal.util.typedef.internal.S;
@@ -47,13 +47,13 @@ public class RemoveRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
-        BPlusIO<?> io = PageIO.getBPlusIO(buf);
+    @Override public void applyDelta(PageMemory pageMem, long pageAddr) throws IgniteCheckedException {
+        BPlusIO<?> io = PageIO.getBPlusIO(pageAddr);
 
-        if (io.getCount(buf) != cnt)
-            throw new DeltaApplicationException("Count is wrong [expCnt=" + cnt + ", actual=" + io.getCount(buf) + ']');
+        if (io.getCount(pageAddr) != cnt)
+            throw new DeltaApplicationException("Count is wrong [expCnt=" + cnt + ", actual=" + io.getCount(pageAddr) + ']');
 
-        io.remove(buf, idx, cnt);
+        io.remove(pageAddr, idx, cnt);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/ReplaceRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/ReplaceRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/ReplaceRecord.java
index 8a9e4e3..757afe1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/ReplaceRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/ReplaceRecord.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.pagemem.wal.record.delta;
 
 import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.processors.cache.database.tree.io.BPlusIO;
 
 /**
@@ -55,12 +56,12 @@ public class ReplaceRecord<L> extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf)
+    @Override public void applyDelta(PageMemory pageMem, long pageAddr)
         throws IgniteCheckedException {
-        if (io.getCount(buf) < idx)
+        if (io.getCount(pageAddr) < idx)
             throw new DeltaApplicationException("Index is greater than count: " + idx);
 
-        io.store(buf, idx, row, rowBytes);
+        io.store(pageAddr, idx, row, rowBytes);
     }
 
     /** {@inheritDoc} */


[5/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

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/PagesList.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/PagesList.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/PagesList.java
index 4da44bc..e44838d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/PagesList.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/PagesList.java
@@ -86,17 +86,17 @@ public abstract class PagesList extends DataStructure {
 
     /** */
     private final PageHandler<Void, Boolean> cutTail = new PageHandler<Void, Boolean>() {
-        @Override public Boolean run(Page page, PageIO pageIo, ByteBuffer buf, Void ignore, int bucket)
+        @Override public Boolean run(Page page, PageIO pageIo, long pageAddr, Void ignore, int bucket)
             throws IgniteCheckedException {
-            assert getPageId(buf) == page.id();
+            assert getPageId(pageAddr) == page.id();
 
             PagesListNodeIO io = (PagesListNodeIO)pageIo;
 
-            long tailId = io.getNextId(buf);
+            long tailId = io.getNextId(pageAddr);
 
             assert tailId != 0;
 
-            io.setNextId(buf, 0L);
+            io.setNextId(pageAddr, 0L);
 
             if (isWalDeltaRecordNeeded(wal, page))
                 wal.log(new PagesListSetNextRecord(cacheId, page.id(), 0L));
@@ -140,7 +140,7 @@ public abstract class PagesList extends DataStructure {
         if (metaPageId != 0L) {
             if (initNew) {
                 try (Page page = page(metaPageId)) {
-                    initPage(page, this, PagesListMetaIO.VERSIONS.latest(), wal);
+                    initPage(pageMem, page, this, PagesListMetaIO.VERSIONS.latest(), wal);
                 }
             }
             else {
@@ -150,16 +150,16 @@ public abstract class PagesList extends DataStructure {
 
                 while (nextPageId != 0) {
                     try (Page page = page(nextPageId)) {
-                        ByteBuffer buf = readLock(page); // No concurrent recycling on init.
+                        long pageAddr = readLock(page); // No concurrent recycling on init.
 
-                        assert buf != null;
+                        assert pageAddr != 0L;
 
                         try {
-                            PagesListMetaIO io = PagesListMetaIO.VERSIONS.forPage(buf);
+                            PagesListMetaIO io = PagesListMetaIO.VERSIONS.forPage(pageAddr);
 
-                            io.getBucketsData(buf, bucketsData);
+                            io.getBucketsData(pageAddr, bucketsData);
 
-                            long next0 = io.getNextMetaPageId(buf);
+                            long next0 = io.getNextMetaPageId(pageAddr);
 
                             assert next0 != nextPageId :
                                 "Loop detected [next=" + U.hexLong(next0) + ", cur=" + U.hexLong(nextPageId) + ']';
@@ -167,7 +167,7 @@ public abstract class PagesList extends DataStructure {
                             nextPageId = next0;
                         }
                         finally {
-                            readUnlock(page, buf);
+                            readUnlock(page, pageAddr);
                         }
                     }
                 }
@@ -199,7 +199,7 @@ public abstract class PagesList extends DataStructure {
         assert metaPageId != 0;
 
         Page curPage = null;
-        ByteBuffer curBuf = null;
+        long curPageAddr = 0L;
         PagesListMetaIO curIo = null;
 
         long nextPageId = metaPageId;
@@ -212,39 +212,39 @@ public abstract class PagesList extends DataStructure {
                     int tailIdx = 0;
 
                     while (tailIdx < tails.length) {
-                        int written = curPage != null ? curIo.addTails(curBuf, bucket, tails, tailIdx) : 0;
+                        int written = curPage != null ? curIo.addTails(pageMem.pageSize(), curPageAddr, bucket, tails, tailIdx) : 0;
 
                         if (written == 0) {
                             if (nextPageId == 0L) {
                                 nextPageId = allocatePageNoReuse();
 
                                 if (curPage != null) {
-                                    curIo.setNextMetaPageId(curBuf, nextPageId);
+                                    curIo.setNextMetaPageId(curPageAddr, nextPageId);
 
-                                    releaseAndClose(curPage, curBuf);
+                                    releaseAndClose(curPage, curPageAddr);
                                     curPage = null;
                                 }
 
                                 curPage = page(nextPageId);
-                                curBuf = writeLock(curPage);
+                                curPageAddr = writeLock(curPage);
 
                                 curIo = PagesListMetaIO.VERSIONS.latest();
 
-                                curIo.initNewPage(curBuf, nextPageId);
+                                curIo.initNewPage(curPageAddr, nextPageId, pageSize());
                             }
                             else {
-                                releaseAndClose(curPage, curBuf);
+                                releaseAndClose(curPage, curPageAddr);
                                 curPage = null;
 
                                 curPage = page(nextPageId);
-                                curBuf = writeLock(curPage);
+                                curPageAddr = writeLock(curPage);
 
-                                curIo = PagesListMetaIO.VERSIONS.forPage(curBuf);
+                                curIo = PagesListMetaIO.VERSIONS.forPage(curPageAddr);
 
-                                curIo.resetCount(curBuf);
+                                curIo.resetCount(curPageAddr);
                             }
 
-                            nextPageId = curIo.getNextMetaPageId(curBuf);
+                            nextPageId = curIo.getNextMetaPageId(curPageAddr);
                         }
                         else
                             tailIdx += written;
@@ -253,25 +253,25 @@ public abstract class PagesList extends DataStructure {
             }
         }
         finally {
-            releaseAndClose(curPage, curBuf);
+            releaseAndClose(curPage, curPageAddr);
         }
 
         while (nextPageId != 0L) {
             try (Page page = page(nextPageId)) {
-                ByteBuffer buf = writeLock(page);
+                long pageAddr = writeLock(page);
 
                 try {
-                    PagesListMetaIO io = PagesListMetaIO.VERSIONS.forPage(buf);
+                    PagesListMetaIO io = PagesListMetaIO.VERSIONS.forPage(pageAddr);
 
-                    io.resetCount(buf);
+                    io.resetCount(pageAddr);
 
                     if (PageHandler.isWalDeltaRecordNeeded(wal, page))
                         wal.log(new PageListMetaResetCountRecord(cacheId, nextPageId));
 
-                    nextPageId = io.getNextMetaPageId(buf);
+                    nextPageId = io.getNextMetaPageId(pageAddr);
                 }
                 finally {
-                    writeUnlock(page, buf, true);
+                    writeUnlock(page, pageAddr, true);
                 }
             }
         }
@@ -281,7 +281,7 @@ public abstract class PagesList extends DataStructure {
      * @param page Page.
      * @param buf Buffer.
      */
-    private void releaseAndClose(Page page, ByteBuffer buf) {
+    private void releaseAndClose(Page page, long buf) {
         if (page != null) {
             try {
                 // No special WAL record because we most likely changed the whole page.
@@ -322,10 +322,10 @@ public abstract class PagesList extends DataStructure {
      * @param nextId Next page ID.
      * @param next Next page buffer.
      */
-    private void setupNextPage(PagesListNodeIO io, long prevId, ByteBuffer prev, long nextId, ByteBuffer next) {
+    private void setupNextPage(PagesListNodeIO io, long prevId, long prev, long nextId, long next) {
         assert io.getNextId(prev) == 0L;
 
-        io.initNewPage(next, nextId);
+        io.initNewPage(next, nextId, pageSize());
         io.setPreviousId(next, prevId);
 
         io.setNextId(prev, nextId);
@@ -342,7 +342,7 @@ public abstract class PagesList extends DataStructure {
         long pageId = reuse ? allocatePage(null) : allocatePageNoReuse();
 
         try (Page page = page(pageId)) {
-            initPage(page, this, PagesListNodeIO.VERSIONS.latest(), wal);
+            initPage(pageMem, page, this, PagesListNodeIO.VERSIONS.latest(), wal);
         }
 
         Stripe stripe = new Stripe(pageId);
@@ -466,19 +466,19 @@ public abstract class PagesList extends DataStructure {
                 long pageId = tail.tailId;
 
                 try (Page page = page(pageId)) {
-                    ByteBuffer buf = readLock(page); // No correctness guaranties.
+                    long pageAddr = readLock(page); // No correctness guaranties.
 
                     try {
-                        PagesListNodeIO io = PagesListNodeIO.VERSIONS.forPage(buf);
+                        PagesListNodeIO io = PagesListNodeIO.VERSIONS.forPage(pageAddr);
 
-                        int cnt = io.getCount(buf);
+                        int cnt = io.getCount(pageAddr);
 
                         assert cnt >= 0;
 
                         res += cnt;
                     }
                     finally {
-                        readUnlock(page, buf);
+                        readUnlock(page, pageAddr);
                     }
                 }
             }
@@ -493,9 +493,9 @@ public abstract class PagesList extends DataStructure {
      * @param bucket Bucket.
      * @throws IgniteCheckedException If failed.
      */
-    protected final void put(ReuseBag bag, Page dataPage, ByteBuffer dataPageBuf, int bucket)
+    protected final void put(ReuseBag bag, Page dataPage, long dataPageBuf, int bucket)
         throws IgniteCheckedException {
-        assert bag == null ^ dataPageBuf == null;
+        assert bag == null ^ dataPageBuf == 0L;
 
         for (int lockAttempt = 0; ;) {
             Stripe stripe = getPageForPut(bucket);
@@ -503,9 +503,9 @@ public abstract class PagesList extends DataStructure {
             long tailId = stripe.tailId;
 
             try (Page tail = page(tailId)) {
-                ByteBuffer buf = writeLockPage(tail, bucket, lockAttempt++); // Explicit check.
+                long buf = writeLockPage(tail, bucket, lockAttempt++); // Explicit check.
 
-                if (buf == null)
+                if (buf == 0L)
                     continue;
 
                 assert PageIO.getPageId(buf) == tailId : "bufPageId = " + PageIO.getPageId(buf) + ", tailId = " + tailId;
@@ -551,10 +551,10 @@ public abstract class PagesList extends DataStructure {
     private boolean putDataPage(
         long pageId,
         Page page,
-        ByteBuffer buf,
+        long buf,
         PagesListNodeIO io,
         Page dataPage,
-        ByteBuffer dataPageBuf,
+        long dataPageBuf,
         int bucket
     ) throws IgniteCheckedException {
         if (io.getNextId(buf) != 0L)
@@ -562,7 +562,7 @@ public abstract class PagesList extends DataStructure {
 
         long dataPageId = dataPage.id();
 
-        int idx = io.addPage(buf, dataPageId);
+        int idx = io.addPage(buf, dataPageId, pageSize());
 
         if (idx == -1)
             handlePageFull(pageId, page, buf, io, dataPage, dataPageBuf, bucket);
@@ -593,10 +593,10 @@ public abstract class PagesList extends DataStructure {
     private void handlePageFull(
         long pageId,
         Page page,
-        ByteBuffer buf,
+        long buf,
         PagesListNodeIO io,
         Page dataPage,
-        ByteBuffer dataPageBuf,
+        long dataPageBuf,
         int bucket
     ) throws IgniteCheckedException {
         long dataPageId = dataPage.id();
@@ -631,17 +631,17 @@ public abstract class PagesList extends DataStructure {
             long nextId = allocatePage(null);
 
             try (Page next = page(nextId)) {
-                ByteBuffer nextBuf = writeLock(next); // Newly allocated page.
+                long nextPageAddr = writeLock(next); // Newly allocated page.
 
-                assert nextBuf != null;
+                assert nextPageAddr != 0L;
 
                 try {
-                    setupNextPage(io, pageId, buf, nextId, nextBuf);
+                    setupNextPage(io, pageId, buf, nextId, nextPageAddr);
 
                     if (isWalDeltaRecordNeeded(wal, page))
                         wal.log(new PagesListSetNextRecord(cacheId, pageId, nextId));
 
-                    int idx = io.addPage(nextBuf, dataPageId);
+                    int idx = io.addPage(nextPageAddr, dataPageId, pageSize());
 
                     // Here we should never write full page, because it is known to be new.
                     next.fullPageWalRecordPolicy(FALSE);
@@ -667,7 +667,7 @@ public abstract class PagesList extends DataStructure {
                     updateTail(bucket, pageId, nextId);
                 }
                 finally {
-                    writeUnlock(next, nextBuf, true);
+                    writeUnlock(next, nextPageAddr, true);
                 }
             }
         }
@@ -687,7 +687,7 @@ public abstract class PagesList extends DataStructure {
     private boolean putReuseBag(
         final long pageId,
         Page page,
-        final ByteBuffer buf,
+        final long buf,
         PagesListNodeIO io,
         ReuseBag bag,
         int bucket
@@ -696,21 +696,21 @@ public abstract class PagesList extends DataStructure {
             return false; // Splitted.
 
         long nextId;
-        ByteBuffer prevBuf = buf;
+        long prevBuf = buf;
         long prevId = pageId;
 
         List<Page> locked = null; // TODO may be unlock right away and do not keep all these pages locked?
-        List<ByteBuffer> lockedBufs = null;
+        List<Long> lockedBufs = null;
 
         try {
             while ((nextId = bag.pollFreePage()) != 0L) {
-                int idx = io.addPage(prevBuf, nextId);
+                int idx = io.addPage(prevBuf, nextId, pageSize());
 
                 if (idx == -1) { // Attempt to add page failed: the node page is full.
                     try (Page next = page(nextId)) {
-                        ByteBuffer nextBuf = writeLock(next); // Page from reuse bag can't be concurrently recycled.
+                        long nextPageAddr = writeLock(next); // Page from reuse bag can't be concurrently recycled.
 
-                        assert nextBuf != null;
+                        assert nextPageAddr != 0L;
 
                         if (locked == null) {
                             lockedBufs = new ArrayList<>(2);
@@ -718,9 +718,9 @@ public abstract class PagesList extends DataStructure {
                         }
 
                         locked.add(next);
-                        lockedBufs.add(nextBuf);
+                        lockedBufs.add(nextPageAddr);
 
-                        setupNextPage(io, prevId, prevBuf, nextId, nextBuf);
+                        setupNextPage(io, prevId, prevBuf, nextId, nextPageAddr);
 
                         if (isWalDeltaRecordNeeded(wal, page))
                             wal.log(new PagesListSetNextRecord(cacheId, prevId, nextId));
@@ -741,7 +741,7 @@ public abstract class PagesList extends DataStructure {
 
                         // Switch to this new page, which is now a part of our list
                         // to add the rest of the bag to the new page.
-                        prevBuf = nextBuf;
+                        prevBuf = nextPageAddr;
                         prevId = nextId;
                         page = next;
                     }
@@ -784,15 +784,15 @@ public abstract class PagesList extends DataStructure {
      * @param page Page.
      * @param bucket Bucket.
      * @param lockAttempt Lock attempts counter.
-     * @return Buffer if page is locket of {@code null} if can retry lock.
+     * @return Page address if page is locked of {@code null} if can retry lock.
      * @throws IgniteCheckedException If failed.
      */
-    @Nullable private ByteBuffer writeLockPage(Page page, int bucket, int lockAttempt)
+    private long writeLockPage(Page page, int bucket, int lockAttempt)
         throws IgniteCheckedException {
-        ByteBuffer buf = tryWriteLock(page);
+        long pageAddr = tryWriteLock(page);
 
-        if (buf != null)
-            return buf;
+        if (pageAddr != 0L)
+            return pageAddr;
 
         if (lockAttempt == TRY_LOCK_ATTEMPTS) {
             Stripe[] stripes = getBucket(bucket);
@@ -800,11 +800,11 @@ public abstract class PagesList extends DataStructure {
             if (stripes == null || stripes.length < MAX_STRIPES_PER_BUCKET) {
                 addStripe(bucket, false);
 
-                return null;
+                return 0L;
             }
         }
 
-        return lockAttempt < TRY_LOCK_ATTEMPTS ? null : writeLock(page); // Must be explicitly checked further.
+        return lockAttempt < TRY_LOCK_ATTEMPTS ? 0L : writeLock(page); // Must be explicitly checked further.
     }
 
     /**
@@ -823,9 +823,9 @@ public abstract class PagesList extends DataStructure {
             long tailId = stripe.tailId;
 
             try (Page tail = page(tailId)) {
-                ByteBuffer tailBuf = writeLockPage(tail, bucket, lockAttempt++); // Explicit check.
+                long tailBuf = writeLockPage(tail, bucket, lockAttempt++); // Explicit check.
 
-                if (tailBuf == null)
+                if (tailBuf == 0L)
                     continue;
 
                 assert PageIO.getPageId(tailBuf) == tailId : "tailId = " + tailId + ", tailBufId = " + PageIO.getPageId(tailBuf);
@@ -859,7 +859,7 @@ public abstract class PagesList extends DataStructure {
                             if (prevId != 0L) {
                                 try (Page prev = page(prevId)) {
                                     // Lock pages from next to previous.
-                                    Boolean ok = writePage(prev, this, cutTail, null, bucket, FALSE);
+                                    Boolean ok = writePage(pageMem, prev, this, cutTail, null, bucket, FALSE);
 
                                     assert ok == TRUE : ok;
                                 }
@@ -878,7 +878,7 @@ public abstract class PagesList extends DataStructure {
 
                             try (Page prev = page(prevId)) {
                                 // Lock pages from next to previous.
-                                Boolean ok = writePage(prev, this, cutTail, null, bucket, FALSE);
+                                Boolean ok = writePage(pageMem, prev, this, cutTail, null, bucket, FALSE);
 
                                 assert ok == TRUE : ok;
                             }
@@ -888,7 +888,7 @@ public abstract class PagesList extends DataStructure {
 
                                 PageIO initIo = initIoVers.latest();
 
-                                initIo.initNewPage(tailBuf, tailId);
+                                initIo.initNewPage(tailBuf, tailId, pageSize());
 
                                 if (isWalDeltaRecordNeeded(wal, tail)) {
                                     wal.log(new InitNewPageRecord(cacheId, tail.id(), initIo.getType(),
@@ -933,7 +933,7 @@ public abstract class PagesList extends DataStructure {
      * @throws IgniteCheckedException If failed.
      * @return {@code True} if page was removed.
      */
-    protected final boolean removeDataPage(Page dataPage, ByteBuffer dataPageBuf, DataPageIO dataIO, int bucket)
+    protected final boolean removeDataPage(Page dataPage, long dataPageBuf, DataPageIO dataIO, int bucket)
         throws IgniteCheckedException {
         long dataPageId = dataPage.id();
 
@@ -947,17 +947,17 @@ public abstract class PagesList extends DataStructure {
 
             long recycleId = 0L;
 
-            ByteBuffer buf = writeLock(page); // Explicit check.
+            long pageAddr = writeLock(page); // Explicit check.
 
-            if (buf == null)
+            if (pageAddr == 0L)
                 return false;
 
             boolean rmvd = false;
 
             try {
-                PagesListNodeIO io = PagesListNodeIO.VERSIONS.forPage(buf);
+                PagesListNodeIO io = PagesListNodeIO.VERSIONS.forPage(pageAddr);
 
-                rmvd = io.removePage(buf, dataPageId);
+                rmvd = io.removePage(pageAddr, dataPageId);
 
                 if (!rmvd)
                     return false;
@@ -971,20 +971,20 @@ public abstract class PagesList extends DataStructure {
                 if (isWalDeltaRecordNeeded(wal, dataPage))
                     wal.log(new DataPageSetFreeListPageRecord(cacheId, dataPageId, 0L));
 
-                if (!io.isEmpty(buf))
+                if (!io.isEmpty(pageAddr))
                     return true; // In optimistic case we still have something in the page and can leave it as is.
 
                 // If the page is empty, we have to try to drop it and link next and previous with each other.
-                nextId = io.getNextId(buf);
-                prevId = io.getPreviousId(buf);
+                nextId = io.getNextId(pageAddr);
+                prevId = io.getPreviousId(pageAddr);
 
                 // If there are no next page, then we can try to merge without releasing current write lock,
                 // because if we will need to lock previous page, the locking order will be already correct.
                 if (nextId == 0L)
-                    recycleId = mergeNoNext(pageId, page, buf, prevId, bucket);
+                    recycleId = mergeNoNext(pageId, page, pageAddr, prevId, bucket);
             }
             finally {
-                writeUnlock(page, buf, rmvd);
+                writeUnlock(page, pageAddr, rmvd);
             }
 
             // Perform a fair merge after lock release (to have a correct locking order).
@@ -1007,7 +1007,7 @@ public abstract class PagesList extends DataStructure {
      * @return Page ID to recycle.
      * @throws IgniteCheckedException If failed.
      */
-    private long mergeNoNext(long pageId, Page page, ByteBuffer buf, long prevId, int bucket)
+    private long mergeNoNext(long pageId, Page page, long buf, long prevId, int bucket)
         throws IgniteCheckedException {
         // If we do not have a next page (we are tail) and we are on reuse bucket,
         // then we can leave as is as well, because it is normal to have an empty tail page here.
@@ -1016,7 +1016,7 @@ public abstract class PagesList extends DataStructure {
 
         if (prevId != 0L) { // Cut tail if we have a previous page.
             try (Page prev = page(prevId)) {
-                Boolean ok = writePage(prev, this, cutTail, null, bucket, FALSE);
+                Boolean ok = writePage(pageMem, prev, this, cutTail, null, bucket, FALSE);
 
                 assert ok == TRUE: ok; // Because we keep lock on current tail and do a world consistency check.
             }
@@ -1044,25 +1044,25 @@ public abstract class PagesList extends DataStructure {
             try (Page next = nextId == 0L ? null : page(nextId)) {
                 boolean write = false;
 
-                ByteBuffer nextBuf = next == null ? null : writeLock(next); // Explicit check.
-                ByteBuffer buf = writeLock(page); // Explicit check.
+                long nextPageAddr = next == null ? 0L : writeLock(next); // Explicit check.
+                long pageAddr = writeLock(page); // Explicit check.
 
-                if (buf == null) {
-                    if (nextBuf != null) // Unlock next page if needed.
-                        writeUnlock(next, nextBuf, false);
+                if (pageAddr == 0L) {
+                    if (nextPageAddr != 0L) // Unlock next page if needed.
+                        writeUnlock(next, nextPageAddr, false);
 
                     return 0L; // Someone has merged or taken our empty page concurrently. Nothing to do here.
                 }
 
                 try {
-                    PagesListNodeIO io = PagesListNodeIO.VERSIONS.forPage(buf);
+                    PagesListNodeIO io = PagesListNodeIO.VERSIONS.forPage(pageAddr);
 
-                    if (!io.isEmpty(buf))
+                    if (!io.isEmpty(pageAddr))
                         return 0L; // No need to merge anymore.
 
                     // Check if we see a consistent state of the world.
-                    if (io.getNextId(buf) == nextId && (nextId == 0L) == (nextBuf == null)) {
-                        long recycleId = doMerge(pageId, page, buf, io, next, nextId, nextBuf, bucket);
+                    if (io.getNextId(pageAddr) == nextId && (nextId == 0L) == (nextPageAddr == 0L)) {
+                        long recycleId = doMerge(pageId, page, pageAddr, io, next, nextId, nextPageAddr, bucket);
 
                         write = true;
 
@@ -1070,13 +1070,13 @@ public abstract class PagesList extends DataStructure {
                     }
 
                     // Reread next page ID and go for retry.
-                    nextId = io.getNextId(buf);
+                    nextId = io.getNextId(pageAddr);
                 }
                 finally {
-                    if (nextBuf != null)
-                        writeUnlock(next, nextBuf, write);
+                    if (nextPageAddr != 0L)
+                        writeUnlock(next, nextPageAddr, write);
 
-                    writeUnlock(page, buf, write);
+                    writeUnlock(page, pageAddr, write);
                 }
             }
         }
@@ -1097,11 +1097,11 @@ public abstract class PagesList extends DataStructure {
     private long doMerge(
         long pageId,
         Page page,
-        ByteBuffer buf,
+        long buf,
         PagesListNodeIO io,
         Page next,
         long nextId,
-        ByteBuffer nextBuf,
+        long nextBuf,
         int bucket
     ) throws IgniteCheckedException {
         long prevId = io.getPreviousId(buf);
@@ -1143,22 +1143,22 @@ public abstract class PagesList extends DataStructure {
         long pageId,
         long nextId,
         Page next,
-        ByteBuffer nextBuf)
+        long nextBuf)
         throws IgniteCheckedException {
         try (Page prev = page(prevId)) {
-            ByteBuffer prevBuf = writeLock(prev); // No check, we keep a reference.
+            long prevPageAddr = writeLock(prev); // No check, we keep a reference.
 
-            assert prevBuf != null;
+            assert prevPageAddr != 0L;
 
             try {
-                PagesListNodeIO prevIO = PagesListNodeIO.VERSIONS.forPage(prevBuf);
+                PagesListNodeIO prevIO = PagesListNodeIO.VERSIONS.forPage(prevPageAddr);
                 PagesListNodeIO nextIO = PagesListNodeIO.VERSIONS.forPage(nextBuf);
 
                 // These references must be updated at the same time in write locks.
-                assert prevIO.getNextId(prevBuf) == pageId;
+                assert prevIO.getNextId(prevPageAddr) == pageId;
                 assert nextIO.getPreviousId(nextBuf) == pageId;
 
-                prevIO.setNextId(prevBuf, nextId);
+                prevIO.setNextId(prevPageAddr, nextId);
 
                 if (isWalDeltaRecordNeeded(wal, prev))
                     wal.log(new PagesListSetNextRecord(cacheId, prevId, nextId));
@@ -1169,7 +1169,7 @@ public abstract class PagesList extends DataStructure {
                     wal.log(new PagesListSetPreviousRecord(cacheId, nextId, prevId));
             }
             finally {
-                writeUnlock(prev, prevBuf, true);
+                writeUnlock(prev, prevPageAddr, true);
             }
         }
     }
@@ -1181,7 +1181,7 @@ public abstract class PagesList extends DataStructure {
      * @return Rotated page ID.
      * @throws IgniteCheckedException If failed.
      */
-    private long recyclePage(long pageId, Page page, ByteBuffer buf) throws IgniteCheckedException {
+    private long recyclePage(long pageId, Page page, long buf) throws IgniteCheckedException {
         pageId = PageIdUtils.rotatePageId(pageId);
 
         PageIO.setPageId(buf, pageId);

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/io/PagesListMetaIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/io/PagesListMetaIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/io/PagesListMetaIO.java
index b525a01..6bd50a3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/io/PagesListMetaIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/io/PagesListMetaIO.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.processors.cache.database.freelist.io;
 
-import java.nio.ByteBuffer;
 import java.util.Map;
+import org.apache.ignite.internal.pagemem.PageUtils;
 import org.apache.ignite.internal.processors.cache.database.freelist.PagesList;
 import org.apache.ignite.internal.processors.cache.database.tree.io.IOVersions;
 import org.apache.ignite.internal.processors.cache.database.tree.io.PageIO;
@@ -53,66 +53,67 @@ public class PagesListMetaIO extends PageIO {
     }
 
     /** {@inheritDoc} */
-    @Override public void initNewPage(ByteBuffer buf, long pageId) {
-        super.initNewPage(buf, pageId);
+    @Override public void initNewPage(long pageAddr, long pageId, int pageSize) {
+        super.initNewPage(pageAddr, pageId, pageSize);
 
-        setCount(buf, 0);
-        setNextMetaPageId(buf, 0L);
+        setCount(pageAddr, 0);
+        setNextMetaPageId(pageAddr, 0L);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return Stored items count.
      */
-    private int getCount(ByteBuffer buf) {
-        return buf.getShort(CNT_OFF);
+    private int getCount(long pageAddr) {
+        return PageUtils.getShort(pageAddr, CNT_OFF);
     }
 
     /**
-     * @param buf Buffer,
+     * @param pageAddr Page address.
      * @param cnt Stored items count.
      */
-    private void setCount(ByteBuffer buf, int cnt) {
+    private void setCount(long pageAddr, int cnt) {
         assert cnt >= 0 && cnt <= Short.MAX_VALUE : cnt;
 
-        buf.putShort(CNT_OFF, (short)cnt);
+        PageUtils.putShort(pageAddr, CNT_OFF, (short)cnt);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return Next meta page ID.
      */
-    public long getNextMetaPageId(ByteBuffer buf) {
-        return buf.getLong(NEXT_META_PAGE_OFF);
+    public long getNextMetaPageId(long pageAddr) {
+        return PageUtils.getLong(pageAddr, NEXT_META_PAGE_OFF);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param metaPageId Next meta page ID.
      */
-    public void setNextMetaPageId(ByteBuffer buf, long metaPageId) {
-        buf.putLong(NEXT_META_PAGE_OFF, metaPageId);
+    public void setNextMetaPageId(long pageAddr, long metaPageId) {
+        PageUtils.putLong(pageAddr, NEXT_META_PAGE_OFF, metaPageId);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      */
-    public void resetCount(ByteBuffer buf) {
-        setCount(buf, 0);
+    public void resetCount(long pageAddr) {
+        setCount(pageAddr, 0);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageSize Page size.
+     * @param pageAddr Page address.
      * @param bucket Bucket number.
      * @param tails Tails.
      * @param tailsOff Tails offset.
      * @return Number of items written.
      */
-    public int addTails(ByteBuffer buf, int bucket, PagesList.Stripe[] tails, int tailsOff) {
+    public int addTails(int pageSize, long pageAddr, int bucket, PagesList.Stripe[] tails, int tailsOff) {
         assert bucket >= 0 && bucket <= Short.MAX_VALUE : bucket;
 
-        int cnt = getCount(buf);
-        int cap = getCapacity(buf);
+        int cnt = getCount(pageAddr);
+        int cap = getCapacity(pageSize, pageAddr);
 
         if (cnt == cap)
             return 0;
@@ -122,25 +123,25 @@ public class PagesListMetaIO extends PageIO {
         int write = Math.min(cap - cnt, tails.length - tailsOff);
 
         for (int i = 0; i < write; i++) {
-            buf.putShort(off, (short)bucket);
-            buf.putLong(off + 2, tails[tailsOff].tailId);
+            PageUtils.putShort(pageAddr, off, (short)bucket);
+            PageUtils.putLong(pageAddr, off + 2, tails[tailsOff].tailId);
 
             tailsOff++;
 
             off += ITEM_SIZE;
         }
 
-        setCount(buf, cnt + write);
+        setCount(pageAddr, cnt + write);
 
         return write;
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param res Results map.
      */
-    public void getBucketsData(ByteBuffer buf, Map<Integer, GridLongList> res) {
-        int cnt = getCount(buf);
+    public void getBucketsData(long pageAddr, Map<Integer, GridLongList> res) {
+        int cnt = getCount(pageAddr);
 
         assert cnt >= 0 && cnt <= Short.MAX_VALUE : cnt;
 
@@ -150,10 +151,10 @@ public class PagesListMetaIO extends PageIO {
         int off = offset(0);
 
         for (int i = 0; i < cnt; i++) {
-            Integer bucket = (int)buf.getShort(off);
+            Integer bucket = (int)PageUtils.getShort(pageAddr, off);
             assert bucket >= 0 && bucket <= Short.MAX_VALUE : bucket;
 
-            long tailId = buf.getLong(off + 2);
+            long tailId = PageUtils.getLong(pageAddr, off + 2);
             assert tailId != 0;
 
             GridLongList list = res.get(bucket);
@@ -168,11 +169,11 @@ public class PagesListMetaIO extends PageIO {
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return Maximum number of items which can be stored in buffer.
      */
-    private int getCapacity(ByteBuffer buf) {
-        return (buf.capacity() - ITEMS_OFF) / ITEM_SIZE;
+    private int getCapacity(int pageSize, long pageAddr) {
+        return (pageSize - ITEMS_OFF) / ITEM_SIZE;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/io/PagesListNodeIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/io/PagesListNodeIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/io/PagesListNodeIO.java
index 2349fa1..6bd0532 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/io/PagesListNodeIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/io/PagesListNodeIO.java
@@ -17,14 +17,14 @@
 
 package org.apache.ignite.internal.processors.cache.database.freelist.io;
 
-import java.nio.ByteBuffer;
+import org.apache.ignite.internal.pagemem.PageUtils;
 import org.apache.ignite.internal.processors.cache.database.tree.io.IOVersions;
 import org.apache.ignite.internal.processors.cache.database.tree.io.PageIO;
 
 import static org.apache.ignite.internal.processors.cache.database.tree.util.PageHandler.copyMemory;
 
 /**
- * TODO optimize: now we have slow {@link #removePage(ByteBuffer, long)}
+ * TODO optimize: now we have slow {@link #removePage(long, long)}
  */
 public class PagesListNodeIO extends PageIO {
     /** */
@@ -52,84 +52,84 @@ public class PagesListNodeIO extends PageIO {
     }
 
     /** {@inheritDoc} */
-    @Override public void initNewPage(ByteBuffer buf, long pageId) {
-        super.initNewPage(buf, pageId);
+    @Override public void initNewPage(long pageAddr, long pageId, int pageSize) {
+        super.initNewPage(pageAddr, pageId, pageSize);
 
-        setEmpty(buf);
+        setEmpty(pageAddr);
 
-        setPreviousId(buf, 0L);
-        setNextId(buf, 0L);
+        setPreviousId(pageAddr, 0L);
+        setNextId(pageAddr, 0L);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      */
-    private void setEmpty(ByteBuffer buf) {
-        setCount(buf, 0);
+    private void setEmpty(long pageAddr) {
+        setCount(pageAddr, 0);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return Next page ID.
      */
-    public long getNextId(ByteBuffer buf) {
-        return buf.getLong(NEXT_PAGE_ID_OFF);
+    public long getNextId(long pageAddr) {
+        return PageUtils.getLong(pageAddr, NEXT_PAGE_ID_OFF);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param nextId Next page ID.
      */
-    public void setNextId(ByteBuffer buf, long nextId) {
-        buf.putLong(NEXT_PAGE_ID_OFF, nextId);
+    public void setNextId(long pageAddr, long nextId) {
+        PageUtils.putLong(pageAddr, NEXT_PAGE_ID_OFF, nextId);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return Previous page ID.
      */
-    public long getPreviousId(ByteBuffer buf) {
-        return buf.getLong(PREV_PAGE_ID_OFF);
+    public long getPreviousId(long pageAddr) {
+        return PageUtils.getLong(pageAddr, PREV_PAGE_ID_OFF);
     }
 
     /**
-     * @param buf Page buffer.
+     * @param pageAddr Page address.
      * @param prevId Previous  page ID.
      */
-    public void setPreviousId(ByteBuffer buf, long prevId) {
-        buf.putLong(PREV_PAGE_ID_OFF, prevId);
+    public void setPreviousId(long pageAddr, long prevId) {
+        PageUtils.putLong(pageAddr, PREV_PAGE_ID_OFF, prevId);
     }
 
     /**
      * Gets total count of entries in this page. Does not change the buffer state.
      *
-     * @param buf Page buffer to get count from.
+     * @param pageAddr Page address to get count from.
      * @return Total number of entries.
      */
-    public int getCount(ByteBuffer buf) {
-        return buf.getShort(CNT_OFF);
+    public int getCount(long pageAddr) {
+        return PageUtils.getShort(pageAddr, CNT_OFF);
     }
 
     /**
      * Sets total count of entries in this page. Does not change the buffer state.
      *
-     * @param buf Page buffer to write to.
+     * @param pageAddr Page address to write to.
      * @param cnt Count.
      */
-    private void setCount(ByteBuffer buf, int cnt) {
+    private void setCount(long pageAddr, int cnt) {
         assert cnt >= 0 && cnt <= Short.MAX_VALUE : cnt;
 
-        buf.putShort(CNT_OFF, (short)cnt);
+        PageUtils.putShort(pageAddr, CNT_OFF, (short)cnt);
     }
 
     /**
      * Gets capacity of this page in items.
      *
-     * @param buf Page buffer to get capacity.
+     * @param pageSize Page size.
      * @return Capacity of this page in items.
      */
-    private int getCapacity(ByteBuffer buf) {
-        return (buf.capacity() - PAGE_IDS_OFF) >>> 3; // /8
+    private int getCapacity(int pageSize) {
+        return (pageSize - PAGE_IDS_OFF) >>> 3; // /8
     }
 
     /**
@@ -141,38 +141,39 @@ public class PagesListNodeIO extends PageIO {
     }
 
     /**
-     * @param buf Page buffer.
+     * @param pageAddr Page address.
      * @param idx Item index.
      * @return Item at the given index.
      */
-    private long getAt(ByteBuffer buf, int idx) {
-        return buf.getLong(offset(idx));
+    private long getAt(long pageAddr, int idx) {
+        return PageUtils.getLong(pageAddr, offset(idx));
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param idx Item index.
      * @param pageId Item value to write.
      */
-    private void setAt(ByteBuffer buf, int idx, long pageId) {
-        buf.putLong(offset(idx), pageId);
+    private void setAt(long pageAddr, int idx, long pageId) {
+        PageUtils.putLong(pageAddr, offset(idx), pageId);
     }
 
     /**
      * Adds page to the end of pages list.
      *
-     * @param buf Page buffer.
+     * @param pageAddr Page address.
      * @param pageId Page ID.
+     * @param pageSize Page size.
      * @return Total number of items in this page.
      */
-    public int addPage(ByteBuffer buf, long pageId) {
-        int cnt = getCount(buf);
+    public int addPage(long pageAddr, long pageId, int pageSize) {
+        int cnt = getCount(pageAddr);
 
-        if (cnt == getCapacity(buf))
+        if (cnt == getCapacity(pageSize))
             return -1;
 
-        setAt(buf, cnt, pageId);
-        setCount(buf, cnt + 1);
+        setAt(pageAddr, cnt, pageId);
+        setCount(pageAddr, cnt + 1);
 
         return cnt;
     }
@@ -180,38 +181,38 @@ public class PagesListNodeIO extends PageIO {
     /**
      * Removes any page from the pages list.
      *
-     * @param buf Page buffer.
+     * @param pageAddr Page address.
      * @return Removed page ID.
      */
-    public long takeAnyPage(ByteBuffer buf) {
-        int cnt = getCount(buf);
+    public long takeAnyPage(long pageAddr) {
+        int cnt = getCount(pageAddr);
 
         if (cnt == 0)
             return 0L;
 
-        setCount(buf, --cnt);
+        setCount(pageAddr, --cnt);
 
-        return getAt(buf, cnt);
+        return getAt(pageAddr, cnt);
     }
 
     /**
      * Removes the given page ID from the pages list.
      *
-     * @param buf Page buffer.
+     * @param pageAddr Page address.
      * @param dataPageId Page ID to remove.
      * @return {@code true} if page was in the list and was removed, {@code false} otherwise.
      */
-    public boolean removePage(ByteBuffer buf, long dataPageId) {
+    public boolean removePage(long pageAddr, long dataPageId) {
         assert dataPageId != 0;
 
-        int cnt = getCount(buf);
+        int cnt = getCount(pageAddr);
 
         for (int i = 0; i < cnt; i++) {
-            if (getAt(buf, i) == dataPageId) {
+            if (getAt(pageAddr, i) == dataPageId) {
                 if (i != cnt - 1)
-                    copyMemory(buf, buf, offset(i + 1), offset(i), 8 * (cnt - i - 1));
+                    copyMemory(pageAddr, pageAddr, offset(i + 1), offset(i), 8 * (cnt - i - 1));
 
-                setCount(buf, cnt - 1);
+                setCount(pageAddr, cnt - 1);
 
                 return true;
             }
@@ -221,10 +222,10 @@ public class PagesListNodeIO extends PageIO {
     }
 
     /**
-     * @param buf Page buffer.
-     * @return {@code true} if there are no items in this page.
+     * @param pageAddr Page address.
+     * @return {@code True} if there are no items in this page.
      */
-    public boolean isEmpty(ByteBuffer buf) {
-        return getCount(buf) == 0;
+    public boolean isEmpty(long pageAddr) {
+        return getCount(pageAddr) == 0;
     }
 }


[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

Posted by sb...@apache.org.
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
     }
 }


[6/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

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/SplitExistingPageRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/SplitExistingPageRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/SplitExistingPageRecord.java
index 8ec5f8f..418d28b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/SplitExistingPageRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/SplitExistingPageRecord.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.processors.cache.database.tree.io.BPlusIO;
 import org.apache.ignite.internal.processors.cache.database.tree.io.PageIO;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
@@ -50,10 +50,10 @@ public class SplitExistingPageRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
-        BPlusIO<?> io = PageIO.getBPlusIO(buf);
+    @Override public void applyDelta(PageMemory pageMem, long pageAddr) throws IgniteCheckedException {
+        BPlusIO<?> io = PageIO.getBPlusIO(pageAddr);
 
-        io.splitExistingPage(buf, mid, fwdId);
+        io.splitExistingPage(pageAddr, mid, fwdId);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/SplitForwardPageRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/SplitForwardPageRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/SplitForwardPageRecord.java
index b4487fa..39f2669 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/SplitForwardPageRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/SplitForwardPageRecord.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.pagemem.wal.record.delta;
 import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.pagemem.PageMemory;
 
 /**
  * Split forward page record.
@@ -79,7 +80,7 @@ public class SplitForwardPageRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer fwdBuf) throws IgniteCheckedException {
+    @Override public void applyDelta(PageMemory pageMem, long pageAddr) throws IgniteCheckedException {
         throw new IgniteCheckedException("Split forward page record should not be logged.");
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/TrackingPageDeltaRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/TrackingPageDeltaRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/TrackingPageDeltaRecord.java
index 9d00d77..7cd0948 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/TrackingPageDeltaRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/TrackingPageDeltaRecord.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.processors.cache.database.tree.io.TrackingPageIO;
 
 /**
@@ -70,8 +70,12 @@ public class TrackingPageDeltaRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
-        TrackingPageIO.VERSIONS.forPage(buf).markChanged(buf, pageIdToMark, nextSnapshotId, lastSuccessfulSnapshotId, buf.capacity());
+    @Override public void applyDelta(PageMemory pageMem, long pageAddr) throws IgniteCheckedException {
+        TrackingPageIO.VERSIONS.forPage(pageAddr).markChanged(pageMem.pageBuffer(pageAddr),
+            pageIdToMark,
+            nextSnapshotId,
+            lastSuccessfulSnapshotId,
+            pageMem.pageSize());
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObject.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObject.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObject.java
index 92b72ce..c226ba2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObject.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObject.java
@@ -68,6 +68,13 @@ public interface CacheObject extends Message {
     public boolean putValue(ByteBuffer buf) throws IgniteCheckedException;
 
     /**
+     * @param addr Address tp write value to.
+     * @return Number of bytes written.
+     * @throws IgniteCheckedException If failed.
+     */
+    public int putValue(long addr) throws IgniteCheckedException;
+
+    /**
      * @param buf Buffer to write value to.
      * @param off Offset in source binary data.
      * @param len Length of the data to write.

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectAdapter.java
index 1f13c6f..688b92f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectAdapter.java
@@ -24,6 +24,7 @@ import java.io.ObjectOutput;
 import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.GridDirectTransient;
+import org.apache.ignite.internal.pagemem.PageUtils;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
@@ -78,6 +79,35 @@ public abstract class CacheObjectAdapter implements CacheObject, Externalizable
     }
 
     /** {@inheritDoc} */
+    @Override public int putValue(long addr) throws IgniteCheckedException {
+        assert valBytes != null : "Value bytes must be initialized before object is stored";
+
+        return putValue(addr, cacheObjectType(), valBytes, 0);
+    }
+
+    /**
+     * @param addr Write address.
+     * @param type Object type.
+     * @param valBytes Value bytes array.
+     * @param valOff Value bytes array offset.
+     * @return
+     */
+    public static int putValue(long addr, byte type, byte[] valBytes, int valOff) {
+        int off = 0;
+
+        PageUtils.putInt(addr, off, valBytes.length);
+        off += 4;
+
+        PageUtils.putByte(addr, off, type);
+        off++;
+
+        PageUtils.putBytes(addr, off, valBytes, valOff);
+        off += valBytes.length - valOff;
+
+        return off;
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean putValue(final ByteBuffer buf, int off, int len) throws IgniteCheckedException {
         assert valBytes != null : "Value bytes must be initialized before object is stored";
 
@@ -167,8 +197,14 @@ public abstract class CacheObjectAdapter implements CacheObject, Externalizable
      * @return {@code True} if data were successfully written.
      * @throws IgniteCheckedException If failed.
      */
-    public static boolean putValue(byte cacheObjType, final ByteBuffer buf, int off, int len,
-        byte[] valBytes, final int start) throws IgniteCheckedException {
+    public static boolean putValue(byte cacheObjType,
+        final ByteBuffer buf,
+        int off,
+        int len,
+        byte[] valBytes,
+        final int start)
+        throws IgniteCheckedException
+    {
         int dataLen = valBytes.length;
 
         if (buf.remaining() < len)

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectByteArrayImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectByteArrayImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectByteArrayImpl.java
index b3a4117..eee6fcc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectByteArrayImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectByteArrayImpl.java
@@ -82,6 +82,11 @@ public class CacheObjectByteArrayImpl implements CacheObject, Externalizable {
     }
 
     /** {@inheritDoc} */
+    @Override public int putValue(long addr) throws IgniteCheckedException {
+        return CacheObjectAdapter.putValue(addr, cacheObjectType(), val, 0);
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean putValue(final ByteBuffer buf, int off, int len) throws IgniteCheckedException {
         assert val != null : "Value is not initialized";
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
index 76450fb..f7e46d8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
@@ -23,7 +23,6 @@ import java.util.Iterator;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicLong;
 import javax.cache.Cache;
 import org.apache.ignite.IgniteCheckedException;
@@ -31,11 +30,14 @@ import org.apache.ignite.IgniteException;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.internal.NodeStoppingException;
 import org.apache.ignite.internal.pagemem.FullPageId;
+import org.apache.ignite.internal.pagemem.Page;
 import org.apache.ignite.internal.pagemem.PageIdUtils;
 import org.apache.ignite.internal.pagemem.PageMemory;
+import org.apache.ignite.internal.pagemem.PageUtils;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.database.CacheDataRow;
 import org.apache.ignite.internal.processors.cache.database.CacheDataRowAdapter;
+import org.apache.ignite.internal.processors.cache.database.CacheSearchRow;
 import org.apache.ignite.internal.processors.cache.database.IgniteCacheDatabaseSharedManager;
 import org.apache.ignite.internal.processors.cache.database.RootPage;
 import org.apache.ignite.internal.processors.cache.database.RowStore;
@@ -44,6 +46,8 @@ import org.apache.ignite.internal.processors.cache.database.tree.BPlusTree;
 import org.apache.ignite.internal.processors.cache.database.tree.io.BPlusIO;
 import org.apache.ignite.internal.processors.cache.database.tree.io.BPlusInnerIO;
 import org.apache.ignite.internal.processors.cache.database.tree.io.BPlusLeafIO;
+import org.apache.ignite.internal.processors.cache.database.tree.io.DataPageIO;
+import org.apache.ignite.internal.processors.cache.database.tree.io.DataPagePayload;
 import org.apache.ignite.internal.processors.cache.database.tree.io.IOVersions;
 import org.apache.ignite.internal.processors.cache.database.tree.reuse.ReuseList;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtInvalidPartitionException;
@@ -69,6 +73,8 @@ import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_IDX;
 import static org.apache.ignite.internal.pagemem.PageIdAllocator.INDEX_PARTITION;
+import static org.apache.ignite.internal.pagemem.PageIdUtils.itemId;
+import static org.apache.ignite.internal.pagemem.PageIdUtils.pageId;
 
 /**
  *
@@ -880,7 +886,7 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
             CacheObject val,
             GridCacheVersion ver,
             long expireTime) throws IgniteCheckedException {
-            DataRow dataRow = new DataRow(key.hashCode(), key, val, ver, p, expireTime);
+            DataRow dataRow = new DataRow(key, val, ver, p, expireTime);
 
             // Make sure value bytes initialized.
             key.valueBytes(cctx.cacheObjectContext());
@@ -894,7 +900,7 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
 
                 assert dataRow.link() != 0 : dataRow;
 
-                DataRow old = dataTree.put(dataRow);
+                CacheDataRow old = dataTree.put(dataRow);
 
                 if (old == null)
                     storageSize.incrementAndGet();
@@ -933,7 +939,7 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
                 throw new NodeStoppingException("Operation has been cancelled (node is stopping).");
 
             try {
-                DataRow dataRow = dataTree.remove(new KeySearchRow(key.hashCode(), key, 0));
+                CacheDataRow dataRow = dataTree.remove(new SearchRow(key));
 
                 CacheObject val = null;
                 GridCacheVersion ver = null;
@@ -970,7 +976,7 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
         /** {@inheritDoc} */
         @Override public CacheDataRow find(KeyCacheObject key)
             throws IgniteCheckedException {
-            return dataTree.findOne(new KeySearchRow(key.hashCode(), key, 0));
+            return dataTree.findOne(new SearchRow(key));
         }
 
         /** {@inheritDoc} */
@@ -981,14 +987,14 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
         /** {@inheritDoc} */
         @Override public GridCursor<? extends CacheDataRow> cursor(KeyCacheObject lower,
             KeyCacheObject upper) throws IgniteCheckedException {
-            KeySearchRow lowerRow = null;
-            KeySearchRow upperRow = null;
+            SearchRow lowerRow = null;
+            SearchRow upperRow = null;
 
             if (lower != null)
-                lowerRow = new KeySearchRow(lower.hashCode(), lower, 0);
+                lowerRow = new SearchRow(lower);
 
             if (upper != null)
-                upperRow = new KeySearchRow(upper.hashCode(), upper, 0);
+                upperRow = new SearchRow(upper);
 
             return dataTree.find(lowerRow, upperRow);
         }
@@ -1034,82 +1040,81 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
     /**
      *
      */
-    private class KeySearchRow extends CacheDataRowAdapter {
+    private static class SearchRow implements CacheSearchRow {
         /** */
-        protected int hash;
+        private final KeyCacheObject key;
+
+        /** */
+        private final int hash;
 
         /**
-         * @param hash Hash code.
          * @param key Key.
-         * @param link Link.
          */
-        KeySearchRow(int hash, KeyCacheObject key, long link) {
-            super(link);
-
+        SearchRow(KeyCacheObject key) {
             this.key = key;
-            this.hash = hash;
-        }
 
-        /**
-         * Init data.
-         *
-         * @param keyOnly Initialize only key.
-         */
-        protected final void initData(boolean keyOnly) {
-            if (key != null)
-                return;
-
-            assert link() != 0;
-
-            try {
-                initFromLink(cctx, keyOnly);
-            }
-            catch (IgniteCheckedException e) {
-                throw new IgniteException(e.getMessage(), e);
-            }
+            hash = key.hashCode();
         }
 
-        /**
-         * @return Key.
-         */
+        /** {@inheritDoc} */
         @Override public KeyCacheObject key() {
-            initData(true);
-
             return key;
         }
+
+        /** {@inheritDoc} */
+        @Override public long link() {
+            throw new UnsupportedOperationException();
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hash() {
+            return hash;
+        }
     }
 
     /**
      *
      */
-    private class DataRow extends KeySearchRow {
+    private class DataRow extends CacheDataRowAdapter {
         /** */
         protected int part = -1;
 
+        /** */
+        protected int hash;
+
         /**
          * @param hash Hash code.
          * @param link Link.
+         * @param keyOnly If {@code true} initializes only key.
          */
-        DataRow(int hash, long link) {
-            super(hash, null, link);
+        DataRow(int hash, long link, boolean keyOnly) {
+            super(link);
+
+            this.hash = hash;
 
             part = PageIdUtils.partId(link);
 
-            // We can not init data row lazily because underlying buffer can be concurrently cleared.
-            initData(false);
+            try {
+                // We can not init data row lazily because underlying buffer can be concurrently cleared.
+                initFromLink(cctx, keyOnly);
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException(e);
+            }
         }
 
         /**
-         * @param hash Hash code.
          * @param key Key.
          * @param val Value.
          * @param ver Version.
          * @param part Partition.
          * @param expireTime Expire time.
          */
-        DataRow(int hash, KeyCacheObject key, CacheObject val, GridCacheVersion ver, int part, long expireTime) {
-            super(hash, key, 0);
+        DataRow(KeyCacheObject key, CacheObject val, GridCacheVersion ver, int part, long expireTime) {
+            super(0);
 
+            this.hash = key.hashCode();
+            this.key = key;
             this.val = val;
             this.ver = ver;
             this.part = part;
@@ -1122,6 +1127,11 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
         }
 
         /** {@inheritDoc} */
+        @Override public int hash() {
+            return hash;
+        }
+
+        /** {@inheritDoc} */
         @Override public void link(long link) {
             this.link = link;
         }
@@ -1130,7 +1140,7 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
     /**
      *
      */
-    protected static class CacheDataTree extends BPlusTree<KeySearchRow, DataRow> {
+    protected static class CacheDataTree extends BPlusTree<CacheSearchRow, CacheDataRow> {
         /** */
         private final CacheDataRowStore rowStore;
 
@@ -1169,38 +1179,76 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
         }
 
         /** {@inheritDoc} */
-        @Override protected int compare(BPlusIO<KeySearchRow> io, ByteBuffer buf, int idx, KeySearchRow row)
+        @Override protected int compare(BPlusIO<CacheSearchRow> io, long pageAddr, int idx, CacheSearchRow row)
             throws IgniteCheckedException {
-            int hash = ((RowLinkIO)io).getHash(buf, idx);
+            int hash = ((RowLinkIO)io).getHash(pageAddr, idx);
 
-            int cmp = Integer.compare(hash, row.hash);
+            int cmp = Integer.compare(hash, row.hash());
 
             if (cmp != 0)
                 return cmp;
 
-            KeySearchRow row0 = io.getLookupRow(this, buf, idx);
+            long link = ((RowLinkIO)io).getLink(pageAddr, idx);
 
-            return compareKeys(row0.key(), row.key());
-        }
+            assert row.key() != null : row;
 
-        /** {@inheritDoc} */
-        @Override protected DataRow getRow(BPlusIO<KeySearchRow> io, ByteBuffer buf, int idx)
-            throws IgniteCheckedException {
-            int hash = ((RowLinkIO)io).getHash(buf, idx);
-            long link = ((RowLinkIO)io).getLink(buf, idx);
-
-            return rowStore.dataRow(hash, link);
+            return compareKeys(row.key(), link);
         }
 
         /**
-         * @param key1 First key.
-         * @param key2 Second key.
+         * @param key Key.
+         * @param link Link.
          * @return Compare result.
          * @throws IgniteCheckedException If failed.
          */
-        private int compareKeys(CacheObject key1, CacheObject key2) throws IgniteCheckedException {
-            byte[] bytes1 = key1.valueBytes(cctx.cacheObjectContext());
-            byte[] bytes2 = key2.valueBytes(cctx.cacheObjectContext());
+        private int compareKeys(KeyCacheObject key, final long link) throws IgniteCheckedException {
+            byte[] bytes = key.valueBytes(cctx.cacheObjectContext());
+
+            PageMemory pageMem = cctx.shared().database().pageMemory();
+
+            try (Page page = page(pageId(link))) {
+                long pageAddr = page.getForReadPointer(); // Non-empty data page must not be recycled.
+
+                assert pageAddr != 0L : link;
+
+                try {
+                    DataPageIO io = DataPageIO.VERSIONS.forPage(pageAddr);
+
+                    DataPagePayload data = io.readPayload(pageAddr,
+                        itemId(link),
+                        pageMem.pageSize());
+
+                    if (data.nextLink() == 0) {
+                        long addr = pageAddr + data.offset();
+
+                        int len = PageUtils.getInt(addr, 0);
+
+                        int size = Math.min(bytes.length, len);
+
+                        addr += 5; // Skip length and type byte.
+
+                        for (int i = 0; i < size; i++) {
+                            byte b1 = PageUtils.getByte(addr, i);
+                            byte b2 = bytes[i];
+
+                            if (b1 != b2)
+                                return b1 > b2 ? 1 : -1;
+                        }
+
+                        return Integer.compare(len, bytes.length);
+                    }
+                }
+                finally {
+                    page.releaseRead();
+                }
+            }
+
+            // TODO GG-11768.
+            CacheDataRowAdapter other = new CacheDataRowAdapter(link);
+            other.initFromLink(cctx, true);
+
+            byte[] bytes1 = other.key().valueBytes(cctx.cacheObjectContext());
+            byte[] bytes2 = key.valueBytes(cctx.cacheObjectContext());
 
             int len = Math.min(bytes1.length, bytes2.length);
 
@@ -1214,6 +1262,15 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
 
             return Integer.compare(bytes1.length, bytes2.length);
         }
+
+        /** {@inheritDoc} */
+        @Override protected CacheDataRow getRow(BPlusIO<CacheSearchRow> io, long pageAddr, int idx)
+            throws IgniteCheckedException {
+            int hash = ((RowLinkIO)io).getHash(pageAddr, idx);
+            long link = ((RowLinkIO)io).getLink(pageAddr, idx);
+
+            return rowStore.dataRow(hash, link);
+        }
     }
 
     /**
@@ -1233,8 +1290,8 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
          * @param link Link.
          * @return Search row.
          */
-        private KeySearchRow keySearchRow(int hash, long link) {
-            return new KeySearchRow(hash, null, link);
+        private CacheSearchRow keySearchRow(int hash, long link) {
+            return new DataRow(hash, link, true);
         }
 
         /**
@@ -1242,8 +1299,8 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
          * @param link Link.
          * @return Data row.
          */
-        private DataRow dataRow(int hash, long link) {
-            return new DataRow(hash, link);
+        private CacheDataRow dataRow(int hash, long link) {
+            return new DataRow(hash, link, false);
         }
     }
 
@@ -1259,28 +1316,39 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
     }
 
     /**
+     * @param pageAddr Page address.
+     * @param off Offset.
+     * @param link Link.
+     * @param hash Hash.
+     */
+    private static void store0(long pageAddr, int off, long link, int hash) {
+        PageUtils.putLong(pageAddr, off, link);
+        PageUtils.putInt(pageAddr, off + 8, hash);
+    }
+
+    /**
      *
      */
     private interface RowLinkIO {
         /**
-         * @param buf Buffer.
+         * @param pageAddr Page address.
          * @param idx Index.
          * @return Row link.
          */
-        public long getLink(ByteBuffer buf, int idx);
+        public long getLink(long pageAddr, int idx);
 
         /**
-         * @param buf Buffer.
+         * @param pageAddr Page address.
          * @param idx Index.
          * @return Key hash code.
          */
-        public int getHash(ByteBuffer buf, int idx);
+        public int getHash(long pageAddr, int idx);
     }
 
     /**
      *
      */
-    public static final class DataInnerIO extends BPlusInnerIO<KeySearchRow> implements RowLinkIO {
+    public static final class DataInnerIO extends BPlusInnerIO<CacheSearchRow> implements RowLinkIO {
         /** */
         public static final IOVersions<DataInnerIO> VERSIONS = new IOVersions<>(
             new DataInnerIO(1)
@@ -1294,46 +1362,53 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
         }
 
         /** {@inheritDoc} */
-        @Override public void storeByOffset(ByteBuffer buf, int off, KeySearchRow row) {
+        @Override public void storeByOffset(ByteBuffer buf, int off, CacheSearchRow row) throws IgniteCheckedException {
             assert row.link() != 0;
 
-            store0(buf, off, row.link(), row.hash);
+            store0(buf, off, row.link(), row.hash());
         }
 
         /** {@inheritDoc} */
-        @Override public KeySearchRow getLookupRow(BPlusTree<KeySearchRow, ?> tree, ByteBuffer buf, int idx) {
-            int hash = getHash(buf, idx);
-            long link = getLink(buf, idx);
+        @Override public void storeByOffset(long pageAddr, int off, CacheSearchRow row) {
+            assert row.link() != 0;
+
+            store0(pageAddr, off, row.link(), row.hash());
+        }
+
+        /** {@inheritDoc} */
+        @Override public CacheSearchRow getLookupRow(BPlusTree<CacheSearchRow, ?> tree, long pageAddr, int idx) {
+            int hash = getHash(pageAddr, idx);
+            long link = getLink(pageAddr, idx);
 
             return ((CacheDataTree)tree).rowStore.keySearchRow(hash, link);
         }
 
         /** {@inheritDoc} */
-        @Override public void store(ByteBuffer dst, int dstIdx, BPlusIO<KeySearchRow> srcIo, ByteBuffer src,
+        @Override public void store(long dstPageAddr, int dstIdx, BPlusIO<CacheSearchRow> srcIo, long srcPageAddr,
             int srcIdx) {
-            int hash = ((RowLinkIO)srcIo).getHash(src, srcIdx);
-            long link = ((RowLinkIO)srcIo).getLink(src, srcIdx);
+            int hash = ((RowLinkIO)srcIo).getHash(srcPageAddr, srcIdx);
+            long link = ((RowLinkIO)srcIo).getLink(srcPageAddr, srcIdx);
 
-            store0(dst, offset(dstIdx), link, hash);
+            store0(dstPageAddr, offset(dstIdx), link, hash);
         }
 
         /** {@inheritDoc} */
-        @Override public long getLink(ByteBuffer buf, int idx) {
-            assert idx < getCount(buf) : idx;
+        @Override public long getLink(long pageAddr, int idx) {
+            assert idx < getCount(pageAddr) : idx;
 
-            return buf.getLong(offset(idx));
+            return PageUtils.getLong(pageAddr, offset(idx));
         }
 
         /** {@inheritDoc} */
-        @Override public int getHash(ByteBuffer buf, int idx) {
-            return buf.getInt(offset(idx) + 8);
+        @Override public int getHash(long pageAddr, int idx) {
+            return PageUtils.getInt(pageAddr, offset(idx) + 8);
         }
     }
 
     /**
      *
      */
-    public static final class DataLeafIO extends BPlusLeafIO<KeySearchRow> implements RowLinkIO {
+    public static final class DataLeafIO extends BPlusLeafIO<CacheSearchRow> implements RowLinkIO {
         /** */
         public static final IOVersions<DataLeafIO> VERSIONS = new IOVersions<>(
             new DataLeafIO(1)
@@ -1347,20 +1422,27 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
         }
 
         /** {@inheritDoc} */
-        @Override public void storeByOffset(ByteBuffer buf, int off, KeySearchRow row) {
+        @Override public void storeByOffset(ByteBuffer buf, int off, CacheSearchRow row) throws IgniteCheckedException {
+            assert row.link() != 0;
+
+            store0(buf, off, row.link(), row.hash());
+        }
+
+        /** {@inheritDoc} */
+        @Override public void storeByOffset(long pageAddr, int off, CacheSearchRow row) {
             assert row.link() != 0;
 
-            store0(buf, off, row.link(), row.hash);
+            store0(pageAddr, off, row.link(), row.hash());
         }
 
         /** {@inheritDoc} */
-        @Override public void store(ByteBuffer dst, int dstIdx, BPlusIO<KeySearchRow> srcIo, ByteBuffer src,
+        @Override public void store(long dstPageAddr, int dstIdx, BPlusIO<CacheSearchRow> srcIo, long srcPageAddr,
             int srcIdx) {
-            store0(dst, offset(dstIdx), getLink(src, srcIdx), getHash(src, srcIdx));
+            store0(dstPageAddr, offset(dstIdx), getLink(srcPageAddr, srcIdx), getHash(srcPageAddr, srcIdx));
         }
 
         /** {@inheritDoc} */
-        @Override public KeySearchRow getLookupRow(BPlusTree<KeySearchRow, ?> tree, ByteBuffer buf, int idx) {
+        @Override public CacheSearchRow getLookupRow(BPlusTree<CacheSearchRow, ?> tree, long buf, int idx) {
             int hash = getHash(buf, idx);
             long link = getLink(buf, idx);
 
@@ -1368,15 +1450,15 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
         }
 
         /** {@inheritDoc} */
-        @Override public long getLink(ByteBuffer buf, int idx) {
-            assert idx < getCount(buf) : idx;
+        @Override public long getLink(long pageAddr, int idx) {
+            assert idx < getCount(pageAddr) : idx;
 
-            return buf.getLong(offset(idx));
+            return PageUtils.getLong(pageAddr, offset(idx));
         }
 
         /** {@inheritDoc} */
-        @Override public int getHash(ByteBuffer buf, int idx) {
-            return buf.getInt(offset(idx) + 8);
+        @Override public int getHash(long pageAddr, int idx) {
+            return PageUtils.getInt(pageAddr, offset(idx) + 8);
         }
     }
 
@@ -1471,9 +1553,9 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
         }
 
         /** {@inheritDoc} */
-        @Override protected int compare(BPlusIO<PendingRow> io, ByteBuffer buf, int idx, PendingRow row)
+        @Override protected int compare(BPlusIO<PendingRow> io, long pageAddr, int idx, PendingRow row)
             throws IgniteCheckedException {
-            long expireTime = ((PendingRowIO)io).getExpireTime(buf, idx);
+            long expireTime = ((PendingRowIO)io).getExpireTime(pageAddr, idx);
 
             int cmp = Long.compare(expireTime, row.expireTime);
 
@@ -1483,15 +1565,15 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
             if (row.link == 0L)
                 return 0;
 
-            long link = ((PendingRowIO)io).getLink(buf, idx);
+            long link = ((PendingRowIO)io).getLink(pageAddr, idx);
 
             return Long.compare(link, row.link);
         }
 
         /** {@inheritDoc} */
-        @Override protected PendingRow getRow(BPlusIO<PendingRow> io, ByteBuffer buf, int idx)
+        @Override protected PendingRow getRow(BPlusIO<PendingRow> io, long pageAddr, int idx)
             throws IgniteCheckedException {
-            return io.getLookupRow(this, buf, idx);
+            return io.getLookupRow(this, pageAddr, idx);
         }
     }
 
@@ -1500,18 +1582,18 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
      */
     private interface PendingRowIO {
         /**
-         * @param buf Buffer.
+         * @param pageAddr Page address.
          * @param idx Index.
          * @return Expire time.
          */
-        long getExpireTime(ByteBuffer buf, int idx);
+        long getExpireTime(long pageAddr, int idx);
 
         /**
-         * @param buf Buffer.
+         * @param pageAddr Page address.
          * @param idx Index.
          * @return Link.
          */
-        long getLink(ByteBuffer buf, int idx);
+        long getLink(long pageAddr, int idx);
     }
 
     /**
@@ -1540,34 +1622,45 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
         }
 
         /** {@inheritDoc} */
-        @Override public void store(ByteBuffer dst,
+        @Override public void storeByOffset(long pageAddr, int off, PendingRow row) throws IgniteCheckedException {
+            assert row.link != 0;
+            assert row.expireTime != 0;
+
+            PageUtils.putLong(pageAddr, off, row.expireTime);
+            PageUtils.putLong(pageAddr, off + 8, row.link);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void store(long dstPageAddr,
             int dstIdx,
             BPlusIO<PendingRow> srcIo,
-            ByteBuffer src,
+            long srcPageAddr,
             int srcIdx) throws IgniteCheckedException {
             int dstOff = offset(dstIdx);
 
-            long link = ((PendingRowIO)srcIo).getLink(src, srcIdx);
-            long expireTime = ((PendingRowIO)srcIo).getExpireTime(src, srcIdx);
+            long link = ((PendingRowIO)srcIo).getLink(srcPageAddr, srcIdx);
+            long expireTime = ((PendingRowIO)srcIo).getExpireTime(srcPageAddr, srcIdx);
 
-            dst.putLong(dstOff, expireTime);
-            dst.putLong(dstOff + 8, link);
+            PageUtils.putLong(dstPageAddr, dstOff, expireTime);
+            PageUtils.putLong(dstPageAddr, dstOff + 8, link);
         }
 
         /** {@inheritDoc} */
-        @Override public PendingRow getLookupRow(BPlusTree<PendingRow, ?> tree, ByteBuffer buf, int idx)
+        @Override public PendingRow getLookupRow(BPlusTree<PendingRow, ?> tree, long pageAddr, int idx)
             throws IgniteCheckedException {
-            return PendingRow.createRowWithKey(((PendingEntriesTree)tree).cctx, getExpireTime(buf, idx), getLink(buf, idx));
+            return PendingRow.createRowWithKey(((PendingEntriesTree)tree).cctx,
+                getExpireTime(pageAddr, idx),
+                getLink(pageAddr, idx));
         }
 
         /** {@inheritDoc} */
-        @Override public long getExpireTime(ByteBuffer buf, int idx) {
-            return buf.getLong(offset(idx));
+        @Override public long getExpireTime(long pageAddr, int idx) {
+            return PageUtils.getLong(pageAddr, offset(idx));
         }
 
         /** {@inheritDoc} */
-        @Override public long getLink(ByteBuffer buf, int idx) {
-            return buf.getLong(offset(idx) + 8);
+        @Override public long getLink(long pageAddr, int idx) {
+            return PageUtils.getLong(pageAddr, offset(idx) + 8);
         }
     }
 
@@ -1597,34 +1690,45 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
         }
 
         /** {@inheritDoc} */
-        @Override public void store(ByteBuffer dst,
+        @Override public void storeByOffset(long pageAddr, int off, PendingRow row) throws IgniteCheckedException {
+            assert row.link != 0;
+            assert row.expireTime != 0;
+
+            PageUtils.putLong(pageAddr, off, row.expireTime);
+            PageUtils.putLong(pageAddr, off + 8, row.link);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void store(long dstPageAddr,
             int dstIdx,
             BPlusIO<PendingRow> srcIo,
-            ByteBuffer src,
+            long srcPageAddr,
             int srcIdx) throws IgniteCheckedException {
             int dstOff = offset(dstIdx);
 
-            long link = ((PendingRowIO)srcIo).getLink(src, srcIdx);
-            long expireTime = ((PendingRowIO)srcIo).getExpireTime(src, srcIdx);
+            long link = ((PendingRowIO)srcIo).getLink(srcPageAddr, srcIdx);
+            long expireTime = ((PendingRowIO)srcIo).getExpireTime(srcPageAddr, srcIdx);
 
-            dst.putLong(dstOff, expireTime);
-            dst.putLong(dstOff + 8, link);
+            PageUtils.putLong(dstPageAddr, dstOff, expireTime);
+            PageUtils.putLong(dstPageAddr, dstOff + 8, link);
         }
 
         /** {@inheritDoc} */
-        @Override public PendingRow getLookupRow(BPlusTree<PendingRow, ?> tree, ByteBuffer buf, int idx)
+        @Override public PendingRow getLookupRow(BPlusTree<PendingRow, ?> tree, long pageAddr, int idx)
             throws IgniteCheckedException {
-            return PendingRow.createRowWithKey(((PendingEntriesTree)tree).cctx, getExpireTime(buf, idx), getLink(buf, idx));
+            return PendingRow.createRowWithKey(((PendingEntriesTree)tree).cctx,
+                getExpireTime(pageAddr, idx),
+                getLink(pageAddr, idx));
         }
 
         /** {@inheritDoc} */
-        @Override public long getExpireTime(ByteBuffer buf, int idx) {
-            return buf.getLong(offset(idx));
+        @Override public long getExpireTime(long pageAddr, int idx) {
+            return PageUtils.getLong(pageAddr, offset(idx));
         }
 
         /** {@inheritDoc} */
-        @Override public long getLink(ByteBuffer buf, int idx) {
-            return buf.getLong(offset(idx) + 8);
+        @Override public long getLink(long pageAddr, int idx) {
+            return PageUtils.getLong(pageAddr, offset(idx) + 8);
         }
     }
 }

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/CacheDataRowAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/CacheDataRowAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/CacheDataRowAdapter.java
index b5babc4..5288aad 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/CacheDataRowAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/CacheDataRowAdapter.java
@@ -21,14 +21,17 @@ import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.pagemem.Page;
 import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.PageMemory;
+import org.apache.ignite.internal.pagemem.PageUtils;
 import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.CacheObjectContext;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.IncompleteCacheObject;
 import org.apache.ignite.internal.processors.cache.IncompleteObject;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
-import org.apache.ignite.internal.processors.cache.database.tree.io.DataPageIO;
 import org.apache.ignite.internal.processors.cache.database.tree.io.CacheVersionIO;
+import org.apache.ignite.internal.processors.cache.database.tree.io.DataPageIO;
+import org.apache.ignite.internal.processors.cache.database.tree.io.DataPagePayload;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
@@ -88,20 +91,26 @@ public class CacheDataRowAdapter implements CacheDataRow {
         boolean first = true;
 
         do {
+            PageMemory pageMem = cctx.shared().database().pageMemory();
+
             try (Page page = page(pageId(nextLink), cctx)) {
-                ByteBuffer buf = page.getForRead(); // Non-empty data page must not be recycled.
+                long pageAddr = page.getForReadPointer(); // Non-empty data page must not be recycled.
 
-                assert buf != null: nextLink;
+                assert pageAddr != 0L : nextLink;
 
                 try {
-                    DataPageIO io = DataPageIO.VERSIONS.forPage(buf);
+                    DataPageIO io = DataPageIO.VERSIONS.forPage(pageAddr);
 
-                    nextLink = io.setPositionAndLimitOnPayload(buf, itemId(nextLink));
+                    DataPagePayload data = io.readPayload(pageAddr,
+                        itemId(nextLink),
+                        pageMem.pageSize());
+
+                    nextLink = data.nextLink();
 
                     if (first) {
                         if (nextLink == 0) {
                             // Fast path for a single page row.
-                            readFullRow(coctx, buf, keyOnly);
+                            readFullRow(coctx, pageAddr + data.offset(), keyOnly);
 
                             return;
                         }
@@ -109,6 +118,11 @@ public class CacheDataRowAdapter implements CacheDataRow {
                         first = false;
                     }
 
+                    ByteBuffer buf = pageMem.pageBuffer(pageAddr);
+
+                    buf.position(data.offset());
+                    buf.limit(data.offset() + data.payloadSize());
+
                     incomplete = readFragment(coctx, buf, keyOnly, incomplete);
 
                     if (keyOnly && key != null)
@@ -121,7 +135,7 @@ public class CacheDataRowAdapter implements CacheDataRow {
         }
         while(nextLink != 0);
 
-        assert isReady(): "ready";
+        assert isReady() : "ready";
     }
 
     /**
@@ -130,6 +144,7 @@ public class CacheDataRowAdapter implements CacheDataRow {
      * @param keyOnly {@code true} If need to read only key object.
      * @param incomplete Incomplete object.
      * @throws IgniteCheckedException If failed.
+     * @return Read object.
      */
     private IncompleteObject<?> readFragment(
         CacheObjectContext coctx,
@@ -175,12 +190,23 @@ public class CacheDataRowAdapter implements CacheDataRow {
 
     /**
      * @param coctx Cache object context.
-     * @param buf Buffer.
+     * @param addr Address.
      * @param keyOnly {@code true} If need to read only key object.
      * @throws IgniteCheckedException If failed.
      */
-    private void readFullRow(CacheObjectContext coctx, ByteBuffer buf, boolean keyOnly) throws IgniteCheckedException {
-        key = coctx.processor().toKeyCacheObject(coctx, buf);
+    private void readFullRow(CacheObjectContext coctx, long addr, boolean keyOnly) throws IgniteCheckedException {
+        int off = 0;
+
+        int len = PageUtils.getInt(addr, off);
+        off += 4;
+
+        byte type = PageUtils.getByte(addr, off);
+        off++;
+
+        byte[] bytes = PageUtils.getBytes(addr, off, len);
+        off += len;
+
+        key = coctx.processor().toKeyCacheObject(coctx, type, bytes);
 
         if (keyOnly) {
             assert key != null: "key";
@@ -188,9 +214,22 @@ public class CacheDataRowAdapter implements CacheDataRow {
             return;
         }
 
-        val = coctx.processor().toCacheObject(coctx, buf);
-        ver = CacheVersionIO.read(buf, false);
-        expireTime = buf.getLong();
+        len = PageUtils.getInt(addr, off);
+        off += 4;
+
+        type = PageUtils.getByte(addr, off);
+        off++;
+
+        bytes = PageUtils.getBytes(addr, off, len);
+        off += len;
+
+        val = coctx.processor().toCacheObject(coctx, type, bytes);
+
+        ver = CacheVersionIO.read(addr + off, false);
+
+        off += CacheVersionIO.size(ver, false);
+
+        expireTime = PageUtils.getLong(addr, off);
 
         assert isReady(): "ready";
     }
@@ -249,6 +288,7 @@ public class CacheDataRowAdapter implements CacheDataRow {
      * @param buf Buffer.
      * @param incomplete Incomplete object.
      * @return Incomplete object.
+     * @throws IgniteCheckedException If failed.
      */
     private IncompleteObject<?> readIncompleteExpireTime(
         ByteBuffer buf,
@@ -292,6 +332,7 @@ public class CacheDataRowAdapter implements CacheDataRow {
      * @param buf Buffer.
      * @param incomplete Incomplete object.
      * @return Incomplete object.
+     * @throws IgniteCheckedException If failed.
      */
     private IncompleteObject<?> readIncompleteVersion(
         ByteBuffer buf,
@@ -385,6 +426,11 @@ public class CacheDataRowAdapter implements CacheDataRow {
     }
 
     /** {@inheritDoc} */
+    @Override public int hash() {
+        throw new UnsupportedOperationException();
+    }
+
+    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(CacheDataRowAdapter.class, this, "link", U.hexLong(link));
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/CacheSearchRow.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/CacheSearchRow.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/CacheSearchRow.java
new file mode 100644
index 0000000..d51cf0e
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/CacheSearchRow.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.database;
+
+import org.apache.ignite.internal.processors.cache.KeyCacheObject;
+
+/**
+ *
+ */
+public interface CacheSearchRow {
+    /**
+     * @return Cache key.
+     */
+    public KeyCacheObject key();
+
+    /**
+     * @return Link for this row.
+     */
+    public long link();
+
+    /**
+     * @return Key hash code.
+     */
+    public int hash();
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/DataStructure.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/DataStructure.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/DataStructure.java
index 5fd64b0..f47a697 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/DataStructure.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/DataStructure.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.internal.processors.cache.database;
 
-import java.nio.ByteBuffer;
 import java.util.Random;
 import java.util.concurrent.ThreadLocalRandom;
 import org.apache.ignite.IgniteCheckedException;
@@ -131,35 +130,35 @@ public abstract class DataStructure implements PageLockListener {
 
     /**
      * @param page Page.
-     * @return Buffer.
+     * @return Page address.
      */
-    protected final ByteBuffer tryWriteLock(Page page) {
+    protected final long tryWriteLock(Page page) {
         return PageHandler.writeLock(page, this, true);
     }
 
 
     /**
      * @param page Page.
-     * @return Buffer.
+     * @return Page address.
      */
-    protected final ByteBuffer writeLock(Page page) {
+    protected final long writeLock(Page page) {
         return PageHandler.writeLock(page, this, false);
     }
 
     /**
      * @param page Page.
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param dirty Dirty page.
      */
-    protected final void writeUnlock(Page page, ByteBuffer buf, boolean dirty) {
-        PageHandler.writeUnlock(page, buf, this, dirty);
+    protected final void writeUnlock(Page page, long pageAddr, boolean dirty) {
+        PageHandler.writeUnlock(page, pageAddr, this, dirty);
     }
 
     /**
      * @param page Page.
-     * @return Buffer.
+     * @return Page address.
      */
-    protected final ByteBuffer readLock(Page page) {
+    protected final long readLock(Page page) {
         return PageHandler.readLock(page, this);
     }
 
@@ -167,22 +166,29 @@ public abstract class DataStructure implements PageLockListener {
      * @param page Page.
      * @param buf Buffer.
      */
-    protected final void readUnlock(Page page, ByteBuffer buf) {
+    protected final void readUnlock(Page page, long buf) {
         PageHandler.readUnlock(page, buf, this);
     }
 
+    /**
+     * @return Page size.
+     */
+    protected final int pageSize() {
+        return pageMem.pageSize();
+    }
+
     /** {@inheritDoc} */
     @Override public void onBeforeWriteLock(Page page) {
         // No-op.
     }
 
     /** {@inheritDoc} */
-    @Override public void onWriteLock(Page page, ByteBuffer buf) {
+    @Override public void onWriteLock(Page page, long pageAddr) {
         // No-op.
     }
 
     /** {@inheritDoc} */
-    @Override public void onWriteUnlock(Page page, ByteBuffer buf) {
+    @Override public void onWriteUnlock(Page page, long pageAddr) {
         // No-op.
     }
 
@@ -192,12 +198,12 @@ public abstract class DataStructure implements PageLockListener {
     }
 
     /** {@inheritDoc} */
-    @Override public void onReadLock(Page page, ByteBuffer buf) {
+    @Override public void onReadLock(Page page, long pageAddr) {
         // No-op.
     }
 
     /** {@inheritDoc} */
-    @Override public void onReadUnlock(Page page, ByteBuffer buf) {
+    @Override public void onReadUnlock(Page page, long pageAddr) {
         // No-op.
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/IgniteCacheDatabaseSharedManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/IgniteCacheDatabaseSharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/IgniteCacheDatabaseSharedManager.java
index 18b3a1f..9c10057 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/IgniteCacheDatabaseSharedManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/IgniteCacheDatabaseSharedManager.java
@@ -256,7 +256,7 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
                 true,
                 sizes);
 
-        return new PageMemoryNoStoreImpl(log, memProvider, cctx, dbCfg.getPageSize());
+        return new PageMemoryNoStoreImpl(log, memProvider, cctx, dbCfg.getPageSize(), false);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/MetadataStorage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/MetadataStorage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/MetadataStorage.java
index 26151ac..cf6decb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/MetadataStorage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/MetadataStorage.java
@@ -24,6 +24,7 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.internal.pagemem.FullPageId;
 import org.apache.ignite.internal.pagemem.PageMemory;
+import org.apache.ignite.internal.pagemem.PageUtils;
 import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager;
 import org.apache.ignite.internal.processors.cache.database.tree.BPlusTree;
 import org.apache.ignite.internal.processors.cache.database.tree.io.BPlusIO;
@@ -31,6 +32,7 @@ import org.apache.ignite.internal.processors.cache.database.tree.io.BPlusInnerIO
 import org.apache.ignite.internal.processors.cache.database.tree.io.BPlusLeafIO;
 import org.apache.ignite.internal.processors.cache.database.tree.io.IOVersions;
 import org.apache.ignite.internal.processors.cache.database.tree.reuse.ReuseList;
+import org.apache.ignite.internal.processors.cache.database.tree.util.PageHandler;
 import org.apache.ignite.internal.util.typedef.internal.U;
 
 /**
@@ -190,19 +192,19 @@ public class MetadataStorage implements MetaStore {
         }
 
         /** {@inheritDoc} */
-        @Override protected int compare(final BPlusIO<IndexItem> io, final ByteBuffer buf, final int idx,
+        @Override protected int compare(final BPlusIO<IndexItem> io, final long pageAddr, final int idx,
             final IndexItem row) throws IgniteCheckedException {
             final int off = ((IndexIO)io).getOffset(idx);
 
             int shift = 0;
 
             // Compare index names.
-            final byte len = buf.get(off + shift);
+            final byte len = PageUtils.getByte(pageAddr, off + shift);
 
             shift += BYTE_LEN;
 
             for (int i = 0; i < len && i < row.idxName.length; i++) {
-                final int cmp = Byte.compare(buf.get(off + i + shift), row.idxName[i]);
+                final int cmp = Byte.compare(PageUtils.getByte(pageAddr, off + i + shift), row.idxName[i]);
 
                 if (cmp != 0)
                     return cmp;
@@ -212,9 +214,9 @@ public class MetadataStorage implements MetaStore {
         }
 
         /** {@inheritDoc} */
-        @Override protected IndexItem getRow(final BPlusIO<IndexItem> io, final ByteBuffer buf,
+        @Override protected IndexItem getRow(final BPlusIO<IndexItem> io, final long pageAddr,
             final int idx) throws IgniteCheckedException {
-            return readRow(buf, ((IndexIO)io).getOffset(idx));
+            return readRow(pageAddr, ((IndexIO)io).getOffset(idx));
         }
     }
 
@@ -275,78 +277,78 @@ public class MetadataStorage implements MetaStore {
     }
 
     /**
-     * Copy row data.
+     * Store row to buffer.
      *
-     * @param dst Destination buffer.
-     * @param dstOff Destination buf offset.
-     * @param src Source buffer.
-     * @param srcOff Src buf offset.
+     * @param pageAddr Page address.
+     * @param off Offset in buf.
+     * @param row Row to store.
      */
     private static void storeRow(
-        final ByteBuffer dst,
-        final int dstOff,
-        final ByteBuffer src,
-        final int srcOff
+        final long pageAddr,
+        int off,
+        final IndexItem row
     ) {
-        int srcOrigPos = src.position();
-        int dstOrigPos = dst.position();
-
-        try {
-            src.position(srcOff);
-            dst.position(dstOff);
-
-            // Index name length.
-            final byte len = src.get();
+        // Index name length.
+        PageUtils.putByte(pageAddr, off, (byte)row.idxName.length);
+        off++;
 
-            dst.put(len);
+        // Index name.
+        PageUtils.putBytes(pageAddr, off, row.idxName);
+        off += row.idxName.length;
 
-            int lim = src.limit();
+        // Page ID.
+        PageUtils.putLong(pageAddr, off, row.pageId);
+    }
 
-            src.limit(src.position() + len);
+    /**
+     * Copy row data.
+     *
+     * @param dstPageAddr Destination page address.
+     * @param dstOff Destination buf offset.
+     * @param srcPageAddr Source page address.
+     * @param srcOff Src buf offset.
+     */
+    private static void storeRow(
+        final long dstPageAddr,
+        int dstOff,
+        final long srcPageAddr,
+        int srcOff
+    ) {
+        // Index name length.
+        final byte len = PageUtils.getByte(srcPageAddr, srcOff);
+        srcOff++;
 
-            // Index name.
-            dst.put(src);
+        PageUtils.putByte(dstPageAddr, dstOff, len);
+        dstOff++;
 
-            src.limit(lim);
+        PageHandler.copyMemory(srcPageAddr, srcOff, dstPageAddr, dstOff, len);
+        srcOff += len;
+        dstOff += len;
 
-            // Page ID.
-            dst.putLong(src.getLong());
-        }
-        finally {
-            src.position(srcOrigPos);
-            dst.position(dstOrigPos);
-        }
+        // Page ID.
+        PageUtils.putLong(dstPageAddr, dstOff, PageUtils.getLong(srcPageAddr, srcOff));
     }
 
     /**
      * Read row from buffer.
      *
-     * @param buf Buffer to read.
-     * @param off Offset in buf.
+     * @param pageAddr Page address.
+     * @param off Offset.
      * @return Read row.
      */
-    private static IndexItem readRow(final ByteBuffer buf, final int off) {
-        int origOff = buf.position();
-
-        try {
-            buf.position(off);
-
-            // Index name length.
-            final int len = buf.get() & 0xFF;
-
-            // Index name.
-            final byte[] idxName = new byte[len];
+    private static IndexItem readRow(final long pageAddr, int off) {
+        // Index name length.
+        final int len = PageUtils.getByte(pageAddr, off) & 0xFF;
+        off++;
 
-            buf.get(idxName);
+        // Index name.
+        final byte[] idxName = PageUtils.getBytes(pageAddr, off, len);
+        off += len;
 
-            // Page ID.
-            final long pageId = buf.getLong();
+        // Page ID.
+        final long pageId = PageUtils.getLong(pageAddr, off);
 
-            return new IndexItem(idxName, pageId);
-        }
-        finally {
-            buf.position(origOff);
-        }
+        return new IndexItem(idxName, pageId);
     }
 
     /**
@@ -383,16 +385,21 @@ public class MetadataStorage implements MetaStore {
         }
 
         /** {@inheritDoc} */
-        @Override public void store(final ByteBuffer dst, final int dstIdx, final BPlusIO<IndexItem> srcIo,
-            final ByteBuffer src,
+        @Override public void storeByOffset(long pageAddr, int off, IndexItem row) throws IgniteCheckedException {
+            storeRow(pageAddr, off, row);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void store(final long dstPageAddr, final int dstIdx, final BPlusIO<IndexItem> srcIo,
+            final long srcPageAddr,
             final int srcIdx) throws IgniteCheckedException {
-            storeRow(dst, offset(dstIdx), src, ((IndexIO)srcIo).getOffset(srcIdx));
+            storeRow(dstPageAddr, offset(dstIdx), srcPageAddr, ((IndexIO)srcIo).getOffset(srcIdx));
         }
 
         /** {@inheritDoc} */
-        @Override public IndexItem getLookupRow(final BPlusTree<IndexItem, ?> tree, final ByteBuffer buf,
+        @Override public IndexItem getLookupRow(final BPlusTree<IndexItem, ?> tree, final long pageAddr,
             final int idx) throws IgniteCheckedException {
-            return readRow(buf, offset(idx));
+            return readRow(pageAddr, offset(idx));
         }
 
         /** {@inheritDoc} */
@@ -424,16 +431,24 @@ public class MetadataStorage implements MetaStore {
         }
 
         /** {@inheritDoc} */
-        @Override public void store(final ByteBuffer dst, final int dstIdx, final BPlusIO<IndexItem> srcIo,
-            final ByteBuffer src,
+        @Override public void storeByOffset(long buf, int off, IndexItem row) throws IgniteCheckedException {
+            storeRow(buf, off, row);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void store(final long dstPageAddr,
+            final int dstIdx,
+            final BPlusIO<IndexItem> srcIo,
+            final long srcPageAddr,
             final int srcIdx) throws IgniteCheckedException {
-            storeRow(dst, offset(dstIdx), src, ((IndexIO)srcIo).getOffset(srcIdx));
+            storeRow(dstPageAddr, offset(dstIdx), srcPageAddr, ((IndexIO)srcIo).getOffset(srcIdx));
         }
 
         /** {@inheritDoc} */
-        @Override public IndexItem getLookupRow(final BPlusTree<IndexItem, ?> tree, final ByteBuffer buf,
+        @Override public IndexItem getLookupRow(final BPlusTree<IndexItem, ?> tree,
+            final long pageAddr,
             final int idx) throws IgniteCheckedException {
-            return readRow(buf, offset(idx));
+            return readRow(pageAddr, offset(idx));
         }
 
         /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/FreeListImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/FreeListImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/FreeListImpl.java
index 6a29027..6c1b21b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/FreeListImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/FreeListImpl.java
@@ -17,13 +17,13 @@
 
 package org.apache.ignite.internal.processors.cache.database.freelist;
 
-import java.nio.ByteBuffer;
 import java.util.concurrent.atomic.AtomicReferenceArray;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.pagemem.Page;
 import org.apache.ignite.internal.pagemem.PageIdAllocator;
 import org.apache.ignite.internal.pagemem.PageIdUtils;
 import org.apache.ignite.internal.pagemem.PageMemory;
+import org.apache.ignite.internal.pagemem.PageUtils;
 import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager;
 import org.apache.ignite.internal.pagemem.wal.record.delta.DataPageInsertFragmentRecord;
 import org.apache.ignite.internal.pagemem.wal.record.delta.DataPageInsertRecord;
@@ -31,6 +31,7 @@ import org.apache.ignite.internal.pagemem.wal.record.delta.DataPageRemoveRecord;
 import org.apache.ignite.internal.processors.cache.database.CacheDataRow;
 import org.apache.ignite.internal.processors.cache.database.tree.io.CacheVersionIO;
 import org.apache.ignite.internal.processors.cache.database.tree.io.DataPageIO;
+import org.apache.ignite.internal.processors.cache.database.tree.io.DataPagePayload;
 import org.apache.ignite.internal.processors.cache.database.tree.io.PageIO;
 import org.apache.ignite.internal.processors.cache.database.tree.reuse.ReuseBag;
 import org.apache.ignite.internal.processors.cache.database.tree.reuse.ReuseList;
@@ -72,26 +73,26 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
     /** */
     private final PageHandler<CacheDataRow, Integer> writeRow =
         new PageHandler<CacheDataRow, Integer>() {
-            @Override public Integer run(Page page, PageIO iox, ByteBuffer buf, CacheDataRow row, int written)
+            @Override public Integer run(Page page, PageIO iox, long pageAddr, CacheDataRow row, int written)
                 throws IgniteCheckedException {
                 DataPageIO io = (DataPageIO)iox;
 
                 int rowSize = getRowSize(row);
-                int oldFreeSpace = io.getFreeSpace(buf);
+                int oldFreeSpace = io.getFreeSpace(pageAddr);
 
                 assert oldFreeSpace > 0 : oldFreeSpace;
 
                 // If the full row does not fit into this page write only a fragment.
-                written = (written == 0 && oldFreeSpace >= rowSize) ? addRow(page, buf, io, row, rowSize):
-                    addRowFragment(page, buf, io, row, written, rowSize);
+                written = (written == 0 && oldFreeSpace >= rowSize) ? addRow(page, pageAddr, io, row, rowSize):
+                    addRowFragment(page, pageAddr, io, row, written, rowSize);
 
                 // Reread free space after update.
-                int newFreeSpace = io.getFreeSpace(buf);
+                int newFreeSpace = io.getFreeSpace(pageAddr);
 
                 if (newFreeSpace > MIN_PAGE_FREE_SPACE) {
                     int bucket = bucket(newFreeSpace, false);
 
-                    put(null, page, buf, bucket);
+                    put(null, page, pageAddr, bucket);
                 }
 
                 // Avoid boxing with garbage generation for usual case.
@@ -109,24 +110,22 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
              */
             private int addRow(
                 Page page,
-                ByteBuffer buf,
+                long buf,
                 DataPageIO io,
                 CacheDataRow row,
                 int rowSize
             ) throws IgniteCheckedException {
-                // TODO: context parameter.
-                io.addRow(buf, row, rowSize);
+                io.addRow(buf, row, rowSize, pageSize());
 
                 if (isWalDeltaRecordNeeded(wal, page)) {
                     // TODO This record must contain only a reference to a logical WAL record with the actual data.
                     byte[] payload = new byte[rowSize];
 
-                    io.setPositionAndLimitOnPayload(buf, PageIdUtils.itemId(row.link()));
+                    DataPagePayload data = io.readPayload(buf, PageIdUtils.itemId(row.link()), pageSize());
 
-                    assert buf.remaining() == rowSize;
+                    assert data.payloadSize() == rowSize;
 
-                    buf.get(payload);
-                    buf.position(0);
+                    PageUtils.getBytes(buf, data.offset(), payload, 0, rowSize);
 
                     wal.log(new DataPageInsertRecord(
                         cacheId,
@@ -149,7 +148,7 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
              */
             private int addRowFragment(
                 Page page,
-                ByteBuffer buf,
+                long buf,
                 DataPageIO io,
                 CacheDataRow row,
                 int written,
@@ -158,17 +157,17 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
                 // Read last link before the fragment write, because it will be updated there.
                 long lastLink = row.link();
 
-                int payloadSize = io.addRowFragment(buf, row, written, rowSize);
+                int payloadSize = io.addRowFragment(pageMem, buf, row, written, rowSize, pageSize());
 
-                assert payloadSize > 0: payloadSize;
+                assert payloadSize > 0 : payloadSize;
 
                 if (isWalDeltaRecordNeeded(wal, page)) {
                     // TODO This record must contain only a reference to a logical WAL record with the actual data.
                     byte[] payload = new byte[payloadSize];
 
-                    io.setPositionAndLimitOnPayload(buf, PageIdUtils.itemId(row.link()));
-                    buf.get(payload);
-                    buf.position(0);
+                    DataPagePayload data = io.readPayload(buf, PageIdUtils.itemId(row.link()), pageSize());
+
+                    PageUtils.getBytes(buf, data.offset(), payload, 0, payloadSize);
 
                     wal.log(new DataPageInsertFragmentRecord(cacheId, page.id(), payload, lastLink));
                 }
@@ -179,15 +178,15 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
 
     /** */
     private final PageHandler<Void, Long> rmvRow = new PageHandler<Void, Long>() {
-        @Override public Long run(Page page, PageIO iox, ByteBuffer buf, Void arg, int itemId)
+        @Override public Long run(Page page, PageIO iox, long pageAddr, Void arg, int itemId)
             throws IgniteCheckedException {
             DataPageIO io = (DataPageIO)iox;
 
-            int oldFreeSpace = io.getFreeSpace(buf);
+            int oldFreeSpace = io.getFreeSpace(pageAddr);
 
             assert oldFreeSpace >= 0: oldFreeSpace;
 
-            long nextLink = io.removeRow(buf, itemId);
+            long nextLink = io.removeRow(pageAddr, itemId, pageSize());
 
             if (isWalDeltaRecordNeeded(wal, page))
                 wal.log(new DataPageRemoveRecord(cacheId, page.id(), itemId));
@@ -200,7 +199,7 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
 //                    put(null, page, buf, REUSE_BUCKET);
 //            }
 
-            int newFreeSpace = io.getFreeSpace(buf);
+            int newFreeSpace = io.getFreeSpace(pageAddr);
 
             if (newFreeSpace > MIN_PAGE_FREE_SPACE) {
                 int newBucket = bucket(newFreeSpace, false);
@@ -210,12 +209,12 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
 
                     if (oldBucket != newBucket) {
                         // It is possible that page was concurrently taken for put, in this case put will handle bucket change.
-                        if (removeDataPage(page, buf, io, oldBucket))
-                            put(null, page, buf, newBucket);
+                        if (removeDataPage(page, pageAddr, io, oldBucket))
+                            put(null, page, pageAddr, newBucket);
                     }
                 }
                 else
-                    put(null, page, buf, newBucket);
+                    put(null, page, pageAddr, newBucket);
             }
 
             // For common case boxed 0L will be cached inside of Long, so no garbage will be produced.
@@ -326,7 +325,7 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
                 // If it is an existing page, we do not need to initialize it.
                 DataPageIO init = reuseBucket || pageId == 0L ? DataPageIO.VERSIONS.latest() : null;
 
-                written = writePage(page, this, writeRow, init, wal, row, written, FAIL_I);
+                written = writePage(pageMem, page, this, writeRow, init, wal, row, written, FAIL_I);
 
                 assert written != FAIL_I; // We can't fail here.
             }
@@ -344,7 +343,7 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
         long nextLink;
 
         try (Page page = pageMem.page(cacheId, pageId)) {
-            nextLink = writePage(page, this, rmvRow, null, itemId, FAIL_L);
+            nextLink = writePage(pageMem, page, this, rmvRow, null, itemId, FAIL_L);
 
             assert nextLink != FAIL_L; // Can't fail here.
         }
@@ -354,7 +353,7 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
             pageId = PageIdUtils.pageId(nextLink);
 
             try (Page page = pageMem.page(cacheId, pageId)) {
-                nextLink = writePage(page, this, rmvRow, null, itemId, FAIL_L);
+                nextLink = writePage(pageMem, page, this, rmvRow, null, itemId, FAIL_L);
 
                 assert nextLink != FAIL_L; // Can't fail here.
             }
@@ -380,7 +379,7 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
     @Override public void addForRecycle(ReuseBag bag) throws IgniteCheckedException {
         assert reuseList == this: "not allowed to be a reuse list";
 
-        put(bag, null, null, REUSE_BUCKET);
+        put(bag, null, 0L, REUSE_BUCKET);
     }
 
     /** {@inheritDoc} */


[2/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

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PageIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PageIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PageIO.java
index aa2d368..f6ac905 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PageIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PageIO.java
@@ -20,6 +20,8 @@ package org.apache.ignite.internal.processors.cache.database.tree.io;
 import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.pagemem.Page;
+import org.apache.ignite.internal.pagemem.PageMemory;
+import org.apache.ignite.internal.pagemem.PageUtils;
 import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager;
 import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManagerImpl;
 import org.apache.ignite.internal.processors.cache.database.MetadataStorage;
@@ -36,10 +38,10 @@ import org.apache.ignite.internal.processors.cache.database.tree.util.PageLockLi
  * 1. IO should not have any `public static` methods.
  *    We have versioned IOs and any static method will mean that it have to always work in backward
  *    compatible way between all the IO versions. The base class {@link PageIO} has
- *    static methods (like {@code {@link #getPageId(ByteBuffer)}}) intentionally:
+ *    static methods (like {@code {@link #getPageId(long)}}) intentionally:
  *    this base format can not be changed between versions.
  *
- * 2. IO must correctly override {@link #initNewPage(ByteBuffer, long)} method and call super.
+ * 2. IO must correctly override {@link #initNewPage(long, long, int)} method and call super.
  *    We have logic that relies on this behavior.
  *
  * 3. Page IO type ID constant must be declared in this class to have a list of all the
@@ -58,7 +60,7 @@ import org.apache.ignite.internal.processors.cache.database.tree.util.PageLockLi
  *
  * 7. It is almost always preferable to read or write (especially write) page contents using
  *    static methods on {@link PageHandler}. To just initialize new page use
- *    {@link PageHandler#initPage(Page, PageLockListener, PageIO, IgniteWriteAheadLogManager)}
+ *    {@link PageHandler#initPage(PageMemory, Page, PageLockListener, PageIO, IgniteWriteAheadLogManager)}
  *    method with needed IO instance.
  */
 public abstract class PageIO {
@@ -164,6 +166,7 @@ public abstract class PageIO {
     }
 
     /**
+     * @param buf Buffer.
      * @return Page type.
      */
     public static int getType(ByteBuffer buf) {
@@ -171,13 +174,21 @@ public abstract class PageIO {
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page addres.
+     * @return Page type.
+     */
+    public static int getType(long pageAddr) {
+        return PageUtils.getShort(pageAddr, TYPE_OFF) & 0xFFFF;
+    }
+
+    /**
+     * @param pageAddr Page address.
      * @param type Type.
      */
-    public static void setType(ByteBuffer buf, int type) {
-        buf.putShort(TYPE_OFF, (short)type);
+    public static void setType(long pageAddr, int type) {
+        PageUtils.putShort(pageAddr, TYPE_OFF, (short)type);
 
-        assert getType(buf) == type;
+        assert getType(pageAddr) == type;
     }
 
     /**
@@ -189,13 +200,21 @@ public abstract class PageIO {
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
+     * @return Version.
+     */
+    public static int getVersion(long pageAddr) {
+        return PageUtils.getShort(pageAddr, VER_OFF) & 0xFFFF;
+    }
+
+    /**
+     * @param pageAddr Page address.
      * @param ver Version.
      */
-    public static void setVersion(ByteBuffer buf, int ver) {
-        buf.putShort(VER_OFF, (short)ver);
+    private static void setVersion(long pageAddr, int ver) {
+        PageUtils.putShort(pageAddr, VER_OFF, (short)ver);
 
-        assert getVersion(buf) == ver;
+        assert getVersion(pageAddr) == ver;
     }
 
     /**
@@ -207,13 +226,37 @@ public abstract class PageIO {
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
+     * @return Page ID.
+     */
+    public static long getPageId(long pageAddr) {
+        return PageUtils.getLong(pageAddr, PAGE_ID_OFF);
+    }
+
+    /**
+     * @param pageAddr Page address.
      * @param pageId Page ID.
      */
-    public static void setPageId(ByteBuffer buf, long pageId) {
-        buf.putLong(PAGE_ID_OFF, pageId);
+    public static void setPageId(long pageAddr, long pageId) {
+        PageUtils.putLong(pageAddr, PAGE_ID_OFF, pageId);
+
+        assert getPageId(pageAddr) == pageId;
+    }
+
+    /**
+     * @param pageAddr Page address.
+     * @return Checksum.
+     */
+    public static int getCrc(long pageAddr) {
+        return PageUtils.getInt(pageAddr, CRC_OFF);
+    }
 
-        assert getPageId(buf) == pageId;
+    /**
+     * @param pageAddr Page address.
+     * @param crc Checksum.
+     */
+    public static void setCrc(long pageAddr, int crc) {
+        PageUtils.putInt(pageAddr, CRC_OFF, crc);
     }
 
     /**
@@ -272,18 +315,19 @@ public abstract class PageIO {
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param pageId Page ID.
+     * @param pageSize Page size.
      */
-    public void initNewPage(ByteBuffer buf, long pageId) {
-        setType(buf, getType());
-        setVersion(buf, getVersion());
-        setPageId(buf, pageId);
-        setCrc(buf, 0);
-
-        buf.putLong(RESERVED_1_OFF, 0L);
-        buf.putLong(RESERVED_2_OFF, 0L);
-        buf.putLong(RESERVED_3_OFF, 0L);
+    public void initNewPage(long pageAddr, long pageId, int pageSize) {
+        setType(pageAddr, getType());
+        setVersion(pageAddr, getVersion());
+        setPageId(pageAddr, pageId);
+        setCrc(pageAddr, 0);
+
+        PageUtils.putLong(pageAddr, RESERVED_1_OFF, 0L);
+        PageUtils.putLong(pageAddr, RESERVED_2_OFF, 0L);
+        PageUtils.putLong(pageAddr, RESERVED_3_OFF, 0L);
     }
 
     /** {@inheritDoc} */
@@ -292,13 +336,13 @@ public abstract class PageIO {
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return IO.
      * @throws IgniteCheckedException If failed.
      */
-    public static <Q extends PageIO> Q getPageIO(ByteBuffer buf) throws IgniteCheckedException {
-        int type = getType(buf);
-        int ver = getVersion(buf);
+    public static <Q extends PageIO> Q getPageIO(long pageAddr) throws IgniteCheckedException {
+        int type = getType(pageAddr);
+        int ver = getVersion(pageAddr);
 
         return getPageIO(type, ver);
     }
@@ -339,13 +383,13 @@ public abstract class PageIO {
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return IO for either inner or leaf B+Tree page.
      * @throws IgniteCheckedException If failed.
      */
-    public static <Q extends BPlusIO<?>> Q getBPlusIO(ByteBuffer buf) throws IgniteCheckedException {
-        int type = getType(buf);
-        int ver = getVersion(buf);
+    public static <Q extends BPlusIO<?>> Q getBPlusIO(long pageAddr) throws IgniteCheckedException {
+        int type = getType(pageAddr);
+        int ver = getVersion(pageAddr);
 
         return getBPlusIO(type, ver);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PageMetaIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PageMetaIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PageMetaIO.java
index 8d12f7c..e768f47 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PageMetaIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PageMetaIO.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.processors.cache.database.tree.io;
 
 import java.nio.ByteBuffer;
+import org.apache.ignite.internal.pagemem.PageUtils;
 import org.jetbrains.annotations.NotNull;
 
 /**
@@ -72,117 +73,117 @@ public class PageMetaIO extends PageIO {
     }
 
     /** {@inheritDoc} */
-    @Override public void initNewPage(ByteBuffer buf, long pageId) {
-        super.initNewPage(buf, pageId);
+    @Override public void initNewPage(long pageAddr, long pageId, int pageSize) {
+        super.initNewPage(pageAddr, pageId, pageSize);
 
-        setTreeRoot(buf, 0);
-        setReuseListRoot(buf, 0);
-        setLastSuccessfulFullSnapshotId(buf, 0);
-        setLastSuccessfulSnapshotId(buf, 0);
-        setNextSnapshotTag(buf, 1);
-        setLastSuccessfulSnapshotTag(buf, 0);
-        setLastAllocatedIndex(buf, 0);
-        setCandidateAllocatedIndex(buf, 0);
+        setTreeRoot(pageAddr, 0);
+        setReuseListRoot(pageAddr, 0);
+        setLastSuccessfulFullSnapshotId(pageAddr, 0);
+        setLastSuccessfulSnapshotId(pageAddr, 0);
+        setNextSnapshotTag(pageAddr, 1);
+        setLastSuccessfulSnapshotTag(pageAddr, 0);
+        setLastAllocatedIndex(pageAddr, 0);
+        setCandidateAllocatedIndex(pageAddr, 0);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return Tree root page.
      */
-    public long getTreeRoot(ByteBuffer buf) {
-        return buf.getLong(TREE_ROOT_OFF);
+    public long getTreeRoot(long pageAddr) {
+        return PageUtils.getLong(pageAddr, TREE_ROOT_OFF);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param treeRoot Tree root
      */
-    public void setTreeRoot(@NotNull ByteBuffer buf, long treeRoot) {
-        buf.putLong(TREE_ROOT_OFF, treeRoot);
+    public void setTreeRoot(long pageAddr, long treeRoot) {
+        PageUtils.putLong(pageAddr, TREE_ROOT_OFF, treeRoot);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return Reuse list root page.
      */
-    public long getReuseListRoot(ByteBuffer buf) {
-        return buf.getLong(REUSE_LIST_ROOT_OFF);
+    public long getReuseListRoot(long pageAddr) {
+        return PageUtils.getLong(pageAddr, REUSE_LIST_ROOT_OFF);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param pageId Root page ID.
      */
-    public void setReuseListRoot(@NotNull ByteBuffer buf, long pageId) {
-        buf.putLong(REUSE_LIST_ROOT_OFF, pageId);
+    public void setReuseListRoot(long pageAddr, long pageId) {
+        PageUtils.putLong(pageAddr, REUSE_LIST_ROOT_OFF, pageId);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param lastSuccessfulSnapshotId Last successful snapshot id.
      */
-    public void setLastSuccessfulSnapshotId(@NotNull ByteBuffer buf, long lastSuccessfulSnapshotId) {
-        buf.putLong(LAST_SUCCESSFUL_SNAPSHOT_ID_OFF, lastSuccessfulSnapshotId);
+    public void setLastSuccessfulSnapshotId(long pageAddr, long lastSuccessfulSnapshotId) {
+        PageUtils.putLong(pageAddr, LAST_SUCCESSFUL_SNAPSHOT_ID_OFF, lastSuccessfulSnapshotId);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      */
-    public long getLastSuccessfulSnapshotId(@NotNull ByteBuffer buf) {
-        return buf.getLong(LAST_SUCCESSFUL_SNAPSHOT_ID_OFF);
+    public long getLastSuccessfulSnapshotId(long pageAddr) {
+        return PageUtils.getLong(pageAddr, LAST_SUCCESSFUL_SNAPSHOT_ID_OFF);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param lastSuccessfulFullSnapshotId Last successful full snapshot id.
      */
-    public void setLastSuccessfulFullSnapshotId(@NotNull ByteBuffer buf, long lastSuccessfulFullSnapshotId) {
-        buf.putLong(LAST_SUCCESSFUL_FULL_SNAPSHOT_ID_OFF, lastSuccessfulFullSnapshotId);
+    public void setLastSuccessfulFullSnapshotId(long pageAddr, long lastSuccessfulFullSnapshotId) {
+        PageUtils.putLong(pageAddr, LAST_SUCCESSFUL_FULL_SNAPSHOT_ID_OFF, lastSuccessfulFullSnapshotId);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      */
-    public long getLastSuccessfulFullSnapshotId(@NotNull ByteBuffer buf) {
-        return buf.getLong(LAST_SUCCESSFUL_FULL_SNAPSHOT_ID_OFF);
+    public long getLastSuccessfulFullSnapshotId(long pageAddr) {
+        return PageUtils.getLong(pageAddr, LAST_SUCCESSFUL_FULL_SNAPSHOT_ID_OFF);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param nextSnapshotId Next snapshot id.
      */
-    public void setNextSnapshotTag(@NotNull ByteBuffer buf, long nextSnapshotId) {
-        buf.putLong(NEXT_SNAPSHOT_TAG_OFF, nextSnapshotId);
+    public void setNextSnapshotTag(long pageAddr, long nextSnapshotId) {
+        PageUtils.putLong(pageAddr, NEXT_SNAPSHOT_TAG_OFF, nextSnapshotId);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      */
-    public long getLastSuccessfulSnapshotTag(@NotNull ByteBuffer buf) {
-        return buf.getLong(LAST_SUCCESSFUL_FULL_SNAPSHOT_TAG_OFF);
+    public long getLastSuccessfulSnapshotTag(long pageAddr) {
+        return PageUtils.getLong(pageAddr, LAST_SUCCESSFUL_FULL_SNAPSHOT_TAG_OFF);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param lastSuccessfulSnapshotTag Last successful snapshot tag.
      */
-    public void setLastSuccessfulSnapshotTag(@NotNull ByteBuffer buf, long lastSuccessfulSnapshotTag) {
-        buf.putLong(LAST_SUCCESSFUL_FULL_SNAPSHOT_TAG_OFF, lastSuccessfulSnapshotTag);
+    public void setLastSuccessfulSnapshotTag(long pageAddr, long lastSuccessfulSnapshotTag) {
+        PageUtils.putLong(pageAddr, LAST_SUCCESSFUL_FULL_SNAPSHOT_TAG_OFF, lastSuccessfulSnapshotTag);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      */
-    public long getNextSnapshotTag(@NotNull ByteBuffer buf) {
-        return buf.getLong(NEXT_SNAPSHOT_TAG_OFF);
+    public long getNextSnapshotTag(long pageAddr) {
+        return PageUtils.getLong(pageAddr, NEXT_SNAPSHOT_TAG_OFF);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param lastAllocatedIdx Last allocated index.
      */
-    public void setLastAllocatedIndex(@NotNull ByteBuffer buf, int lastAllocatedIdx) {
-        buf.putInt(LAST_ALLOCATED_INDEX_OFF, lastAllocatedIdx);
+    public void setLastAllocatedIndex(long pageAddr, int lastAllocatedIdx) {
+        PageUtils.putInt(pageAddr, LAST_ALLOCATED_INDEX_OFF, lastAllocatedIdx);
     }
 
     /**
@@ -193,17 +194,24 @@ public class PageMetaIO extends PageIO {
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
+     */
+    public int getLastAllocatedIndex(long pageAddr) {
+        return PageUtils.getInt(pageAddr, LAST_ALLOCATED_INDEX_OFF);
+    }
+
+    /**
+     * @param pageAddr Page address.
      * @param previousAllocatedIdx Last allocated index.
      */
-    public void setCandidateAllocatedIndex(@NotNull ByteBuffer buf, int previousAllocatedIdx) {
-        buf.putInt(CANDIDATE_ALLOCATED_INDEX_OFF, previousAllocatedIdx);
+    public void setCandidateAllocatedIndex(long pageAddr, int previousAllocatedIdx) {
+        PageUtils.putInt(pageAddr, CANDIDATE_ALLOCATED_INDEX_OFF, previousAllocatedIdx);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      */
-    public int getCandidateAllocatedIndex(@NotNull ByteBuffer buf) {
-        return buf.getInt(CANDIDATE_ALLOCATED_INDEX_OFF);
+    public int getCandidateAllocatedIndex(long pageAddr) {
+        return PageUtils.getInt(pageAddr, CANDIDATE_ALLOCATED_INDEX_OFF);
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PagePartitionMetaIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PagePartitionMetaIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PagePartitionMetaIO.java
index c01d774..aca0725 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PagePartitionMetaIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PagePartitionMetaIO.java
@@ -18,8 +18,11 @@
 
 package org.apache.ignite.internal.processors.cache.database.tree.io;
 
-import java.nio.ByteBuffer;
+import org.apache.ignite.internal.pagemem.PageUtils;
 
+/**
+ *
+ */
 public class PagePartitionMetaIO extends PageMetaIO {
     /** */
     private static final int SIZE_OFF = PageMetaIO.END_OF_PAGE_META;
@@ -39,13 +42,13 @@ public class PagePartitionMetaIO extends PageMetaIO {
     );
 
     /** {@inheritDoc} */
-    @Override public void initNewPage(ByteBuffer buf, long pageId) {
-        super.initNewPage(buf, pageId);
+    @Override public void initNewPage(long pageAddr, long pageId, int pageSize) {
+        super.initNewPage(pageAddr, pageId, pageSize);
 
-        setSize(buf, 0);
-        setUpdateCounter(buf, 0);
-        setGlobalRemoveId(buf, 0);
-        setPartitionState(buf, (byte)-1);
+        setSize(pageAddr, 0);
+        setUpdateCounter(pageAddr, 0);
+        setGlobalRemoveId(pageAddr, 0);
+        setPartitionState(pageAddr, (byte)-1);
     }
 
     /**
@@ -56,65 +59,65 @@ public class PagePartitionMetaIO extends PageMetaIO {
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return Partition size.
      */
-    public long getSize(ByteBuffer buf) {
-        return buf.getLong(SIZE_OFF);
+    public long getSize(long pageAddr) {
+        return PageUtils.getLong(pageAddr, SIZE_OFF);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param size Partition size.
      */
-    public void setSize(ByteBuffer buf, long size) {
-        buf.putLong(SIZE_OFF, size);
+    public void setSize(long pageAddr, long size) {
+        PageUtils.putLong(pageAddr, SIZE_OFF, size);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return Partition update counter.
      */
-    public long getUpdateCounter(ByteBuffer buf) {
-        return buf.getLong(UPDATE_CNTR_OFF);
+    public long getUpdateCounter(long pageAddr) {
+        return PageUtils.getLong(pageAddr, UPDATE_CNTR_OFF);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param cntr Partition update counter.
      */
-    public void setUpdateCounter(ByteBuffer buf, long cntr) {
-        buf.putLong(UPDATE_CNTR_OFF, cntr);
+    public void setUpdateCounter(long pageAddr, long cntr) {
+        PageUtils.putLong(pageAddr, UPDATE_CNTR_OFF, cntr);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return Global remove ID.
      */
-    public long getGlobalRemoveId(ByteBuffer buf) {
-        return buf.getLong(GLOBAL_RMV_ID_OFF);
+    public long getGlobalRemoveId(long pageAddr) {
+        return PageUtils.getLong(pageAddr, GLOBAL_RMV_ID_OFF);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param rmvId Global remove ID.
      */
-    public void setGlobalRemoveId(ByteBuffer buf, long rmvId) {
-        buf.putLong(GLOBAL_RMV_ID_OFF, rmvId);
+    public void setGlobalRemoveId(long pageAddr, long rmvId) {
+        PageUtils.putLong(pageAddr, GLOBAL_RMV_ID_OFF, rmvId);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      */
-    public byte getPartitionState(ByteBuffer buf) {
-        return buf.get(PARTITION_STATE_OFF);
+    public byte getPartitionState(long pageAddr) {
+        return PageUtils.getByte(pageAddr, PARTITION_STATE_OFF);
     }
 
     /**
-     * @param buf
-     * @param state
+     * @param pageAddr Page address
+     * @param state State.
      */
-    public void setPartitionState(ByteBuffer buf, byte state) {
-        buf.put(PARTITION_STATE_OFF, state);
+    public void setPartitionState(long pageAddr, byte state) {
+        PageUtils.putByte(pageAddr, PARTITION_STATE_OFF, state);
     }
 }

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/util/PageHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/util/PageHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/util/PageHandler.java
index fadf74d..97b5a04 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/util/PageHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/util/PageHandler.java
@@ -20,11 +20,11 @@ package org.apache.ignite.internal.processors.cache.database.tree.util;
 import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.pagemem.Page;
+import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager;
 import org.apache.ignite.internal.pagemem.wal.record.delta.InitNewPageRecord;
 import org.apache.ignite.internal.processors.cache.database.tree.io.PageIO;
 import org.apache.ignite.internal.util.GridUnsafe;
-import sun.nio.ch.DirectBuffer;
 
 import static java.lang.Boolean.FALSE;
 import static java.lang.Boolean.TRUE;
@@ -35,7 +35,7 @@ import static java.lang.Boolean.TRUE;
 public abstract class PageHandler<X, R> {
     /** */
     private static final PageHandler<Void, Boolean> NOOP = new PageHandler<Void, Boolean>() {
-        @Override public Boolean run(Page page, PageIO io, ByteBuffer buf, Void arg, int intArg)
+        @Override public Boolean run(Page page, PageIO io, long pageAddr, Void arg, int intArg)
             throws IgniteCheckedException {
             return TRUE;
         }
@@ -44,13 +44,13 @@ public abstract class PageHandler<X, R> {
     /**
      * @param page Page.
      * @param io IO.
-     * @param buf Page buffer.
+     * @param pageAddr Page address.
      * @param arg Argument.
      * @param intArg Argument of type {@code int}.
      * @return Result.
      * @throws IgniteCheckedException If failed.
      */
-    public abstract R run(Page page, PageIO io, ByteBuffer buf, X arg, int intArg)
+    public abstract R run(Page page, PageIO io, long pageAddr, X arg, int intArg)
         throws IgniteCheckedException;
 
     /**
@@ -65,6 +65,7 @@ public abstract class PageHandler<X, R> {
 
     /**
      * @param page Page.
+     * @param lockLsnr Lock listener.
      * @param h Handler.
      * @param arg Argument.
      * @param intArg Argument of type {@code int}.
@@ -74,29 +75,31 @@ public abstract class PageHandler<X, R> {
      */
     public static <X, R> R readPage(
         Page page,
-        PageLockListener lockListener,
+        PageLockListener lockLsnr,
         PageHandler<X, R> h,
         X arg,
         int intArg,
         R lockFailed
     ) throws IgniteCheckedException {
-        ByteBuffer buf = readLock(page, lockListener);
+        long pageAddr = readLock(page, lockLsnr);
 
-        if (buf == null)
+        if (pageAddr == 0L)
             return lockFailed;
 
         try {
-            PageIO io = PageIO.getPageIO(buf);
+            PageIO io = PageIO.getPageIO(pageAddr);
 
-            return h.run(page, io, buf, arg, intArg);
+            return h.run(page, io, pageAddr, arg, intArg);
         }
         finally {
-            readUnlock(page, buf, lockListener);
+            readUnlock(page, pageAddr, lockLsnr);
         }
     }
 
     /**
+     * @param pageMem Page memory.
      * @param page Page.
+     * @param lockLsnr Lock listener.
      * @param h Handler.
      * @param arg Argument.
      * @param intArg Argument of type {@code int}.
@@ -105,92 +108,97 @@ public abstract class PageHandler<X, R> {
      * @throws IgniteCheckedException If failed.
      */
     public static <X, R> R writePage(
+        PageMemory pageMem,
         Page page,
-        PageLockListener lockListener,
+        PageLockListener lockLsnr,
         PageHandler<X, R> h,
         X arg,
         int intArg,
         R lockFailed
     ) throws IgniteCheckedException {
-        return writePage(page, lockListener, h, null, null, arg, intArg, lockFailed);
+        return writePage(pageMem, page, lockLsnr, h, null, null, arg, intArg, lockFailed);
     }
 
     /**
+     * @param pageMem Page memory.
      * @param page Page.
-     * @param lockListener Lock listener.
+     * @param lockLsnr Lock listener.
      * @param init IO for new page initialization or {@code null} if it is an existing page.
+     * @param wal WAL manager.
      * @throws IgniteCheckedException If failed.
      */
     public static void initPage(
+        PageMemory pageMem,
         Page page,
-        PageLockListener lockListener,
+        PageLockListener lockLsnr,
         PageIO init,
         IgniteWriteAheadLogManager wal
     ) throws IgniteCheckedException {
-        Boolean res = writePage(page, lockListener, NOOP, init, wal, null, 0, FALSE);
+        Boolean res = writePage(pageMem, page, lockLsnr, NOOP, init, wal, null, 0, FALSE);
 
         assert res == TRUE : res; // It must be newly allocated page, can't be recycled.
     }
 
     /**
      * @param page Page.
-     * @param lockListener Lock listener.
-     * @return Byte buffer or {@code null} if failed to lock due to recycling.
+     * @param lockLsnr Lock listener.
+     * @return Page address or {@code 0} if failed to lock due to recycling.
      */
-    public static ByteBuffer readLock(Page page, PageLockListener lockListener) {
-        lockListener.onBeforeReadLock(page);
+    public static long readLock(Page page, PageLockListener lockLsnr) {
+        lockLsnr.onBeforeReadLock(page);
 
-        ByteBuffer buf = page.getForRead();
+        long pageAddr = page.getForReadPointer();
 
-        lockListener.onReadLock(page, buf);
+        lockLsnr.onReadLock(page, pageAddr);
 
-        return buf;
+        return pageAddr;
     }
 
     /**
      * @param page Page.
-     * @param buf Page buffer.
-     * @param lockListener Lock listener.
+     * @param pageAddr Page address.
+     * @param lockLsnr Lock listener.
      */
-    public static void readUnlock(Page page, ByteBuffer buf, PageLockListener lockListener) {
-        lockListener.onReadUnlock(page, buf);
+    public static void readUnlock(Page page, long pageAddr, PageLockListener lockLsnr) {
+        lockLsnr.onReadUnlock(page, pageAddr);
 
         page.releaseRead();
     }
 
     /**
      * @param page Page.
-     * @param lockListener Lock listener.
+     * @param lockLsnr Lock listener.
      * @param tryLock Only try to lock without waiting.
-     * @return Byte buffer or {@code null} if failed to lock due to recycling.
+     * @return Page address or {@code 0} if failed to lock due to recycling.
      */
-    public static ByteBuffer writeLock(Page page, PageLockListener lockListener, boolean tryLock) {
-        lockListener.onBeforeWriteLock(page);
+    public static long writeLock(Page page, PageLockListener lockLsnr, boolean tryLock) {
+        lockLsnr.onBeforeWriteLock(page);
 
-        ByteBuffer buf = tryLock ? page.tryGetForWrite() : page.getForWrite();
+        long pageAddr = tryLock ? page.tryGetForWritePointer() : page.getForWritePointer();
 
-        lockListener.onWriteLock(page, buf);
+        lockLsnr.onWriteLock(page, pageAddr);
 
-        return buf;
+        return pageAddr;
     }
 
     /**
      * @param page Page.
      * @param buf Page buffer.
-     * @param lockListener Lock listener.
+     * @param lockLsnr Lock listener.
      * @param dirty Page is dirty.
      */
-    public static void writeUnlock(Page page, ByteBuffer buf, PageLockListener lockListener, boolean dirty) {
-        lockListener.onWriteUnlock(page, buf);
+    public static void writeUnlock(Page page, long buf, PageLockListener lockLsnr, boolean dirty) {
+        lockLsnr.onWriteUnlock(page, buf);
 
         page.releaseWrite(dirty);
     }
 
     /**
      * @param page Page.
-     * @param lockListener Lock listener.
+     * @param lockLsnr Lock listener.
      * @param h Handler.
      * @param init IO for new page initialization or {@code null} if it is an existing page.
+     * @param wal WAL manager.
      * @param arg Argument.
      * @param intArg Argument of type {@code int}.
      * @param lockFailed Result in case of lock failure due to page recycling.
@@ -198,8 +206,9 @@ public abstract class PageHandler<X, R> {
      * @throws IgniteCheckedException If failed.
      */
     public static <X, R> R writePage(
+        PageMemory pageMem,
         Page page,
-        PageLockListener lockListener,
+        PageLockListener lockLsnr,
         PageHandler<X, R> h,
         PageIO init,
         IgniteWriteAheadLogManager wal,
@@ -207,9 +216,9 @@ public abstract class PageHandler<X, R> {
         int intArg,
         R lockFailed
     ) throws IgniteCheckedException {
-        ByteBuffer buf = writeLock(page, lockListener, false);
+        long pageAddr = writeLock(page, lockLsnr, false);
 
-        if (buf == null)
+        if (pageAddr == 0L)
             return lockFailed;
 
         R res;
@@ -218,42 +227,44 @@ public abstract class PageHandler<X, R> {
 
         try {
             if (init != null) // It is a new page and we have to initialize it.
-                doInitPage(page, buf, init, wal);
+                doInitPage(pageMem, page, pageAddr, init, wal);
             else
-                init = PageIO.getPageIO(buf);
+                init = PageIO.getPageIO(pageAddr);
 
-            res = h.run(page, init, buf, arg, intArg);
+            res = h.run(page, init, pageAddr, arg, intArg);
 
             ok = true;
         }
         finally {
-            assert PageIO.getCrc(buf) == 0; //TODO GG-11480
+            assert PageIO.getCrc(pageAddr) == 0; //TODO GG-11480
 
             if (h.releaseAfterWrite(page, arg, intArg))
-                writeUnlock(page, buf, lockListener, ok);
+                writeUnlock(page, pageAddr, lockLsnr, ok);
         }
 
         return res;
     }
 
     /**
+     * @param pageMem Page memory.
      * @param page Page.
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param init Initial IO.
      * @param wal Write ahead log.
      * @throws IgniteCheckedException If failed.
      */
     private static void doInitPage(
+        PageMemory pageMem,
         Page page,
-        ByteBuffer buf,
+        long pageAddr,
         PageIO init,
         IgniteWriteAheadLogManager wal
     ) throws IgniteCheckedException {
-        assert PageIO.getCrc(buf) == 0; //TODO GG-11480
+        assert PageIO.getCrc(pageAddr) == 0; //TODO GG-11480
 
         long pageId = page.id();
 
-        init.initNewPage(buf, pageId);
+        init.initNewPage(pageAddr, pageId, pageMem.pageSize());
 
         // Here we should never write full page, because it is known to be new.
         page.fullPageWalRecordPolicy(FALSE);
@@ -288,8 +299,8 @@ public abstract class PageHandler<X, R> {
         long srcArrOff = src.hasArray() ? src.arrayOffset() + GridUnsafe.BYTE_ARR_OFF : 0;
         long dstArrOff = dst.hasArray() ? dst.arrayOffset() + GridUnsafe.BYTE_ARR_OFF : 0;
 
-        long srcPtr = src.isDirect() ? ((DirectBuffer)src).address() : 0;
-        long dstPtr = dst.isDirect() ? ((DirectBuffer)dst).address() : 0;
+        long srcPtr = src.isDirect() ? GridUnsafe.bufferAddress(src) : 0;
+        long dstPtr = dst.isDirect() ? GridUnsafe.bufferAddress(dst) : 0;
 
         GridUnsafe.copyMemory(srcArr, srcPtr + srcArrOff + srcOff, dstArr, dstPtr + dstArrOff + dstOff, cnt);
     }
@@ -302,11 +313,31 @@ public abstract class PageHandler<X, R> {
      */
     public static void zeroMemory(ByteBuffer buf, int off, int len) {
         if (buf.isDirect())
-            GridUnsafe.setMemory(((DirectBuffer)buf).address() + off, len, (byte)0);
+            GridUnsafe.setMemory(GridUnsafe.bufferAddress(buf) + off, len, (byte)0);
 
         else {
             for (int i = off; i < off + len; i++)
                 buf.put(i, (byte)0); //TODO Optimize!
         }
     }
+
+    /**
+     * @param srcAddr Source.
+     * @param dstAddr Destination.
+     * @param srcOff Source offset in bytes.
+     * @param dstOff Destination offset in bytes.
+     * @param cnt Bytes count to copy.
+     */
+    public static void copyMemory(long srcAddr, long dstAddr, long srcOff, long dstOff, long cnt) {
+        GridUnsafe.copyMemory(null, srcAddr + srcOff, null, dstAddr + dstOff, cnt);
+    }
+
+    /**
+     * @param addr Address.
+     * @param off Offset.
+     * @param len Length.
+     */
+    public static void zeroMemory(long addr, int off, int len) {
+        GridUnsafe.setMemory(addr + off, len, (byte)0);
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/util/PageLockListener.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/util/PageLockListener.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/util/PageLockListener.java
index 216bdd3..07cdd43 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/util/PageLockListener.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/util/PageLockListener.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.internal.processors.cache.database.tree.util;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.internal.pagemem.Page;
 
 /**
@@ -31,15 +30,15 @@ public interface PageLockListener {
 
     /**
      * @param page Page.
-     * @param buf Buffer or {@code null} if attempt to lock failed.
+     * @param pageAddr Page address or {@code 0} if attempt to lock failed.
      */
-    public void onWriteLock(Page page, ByteBuffer buf);
+    public void onWriteLock(Page page, long pageAddr);
 
     /**
      * @param page Page.
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      */
-    public void onWriteUnlock(Page page, ByteBuffer buf);
+    public void onWriteUnlock(Page page, long pageAddr);
 
     /**
      * @param page Page.
@@ -48,13 +47,13 @@ public interface PageLockListener {
 
     /**
      * @param page Page.
-     * @param buf Buffer or {@code null} if attempt to lock failed.
+     * @param pageAddr Page address or {@code 0} if attempt to lock failed.
      */
-    public void onReadLock(Page page, ByteBuffer buf);
+    public void onReadLock(Page page, long pageAddr);
 
     /**
      * @param page Page.
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      */
-    public void onReadUnlock(Page page, ByteBuffer buf);
+    public void onReadUnlock(Page page, long pageAddr);
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicSingleUpdateFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicSingleUpdateFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicSingleUpdateFuture.java
index 20d6e90..5fa1daa 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicSingleUpdateFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicSingleUpdateFuture.java
@@ -163,7 +163,8 @@ class GridDhtAtomicSingleUpdateFuture extends GridDhtAtomicAbstractUpdateFuture
 
     /** {@inheritDoc} */
     @Override protected void addFailedKeys(GridNearAtomicUpdateResponse updateRes, Throwable err) {
-        updateRes.addFailedKey(key, err);
+        if (key != null)
+            updateRes.addFailedKey(key, err);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateResponse.java
index 2e38733..b5b11db 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateResponse.java
@@ -341,6 +341,9 @@ public class GridNearAtomicUpdateResponse extends GridCacheMessage implements Gr
      * @param e Error cause.
      */
     public synchronized void addFailedKey(KeyCacheObject key, Throwable e) {
+        assert key != null;
+        assert e != null;
+
         if (failedKeys == null)
             failedKeys = new ConcurrentLinkedQueue<>();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/util/GridUnsafe.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/GridUnsafe.java b/modules/core/src/main/java/org/apache/ignite/internal/util/GridUnsafe.java
index 783ab96..1926f01 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/GridUnsafe.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/GridUnsafe.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.util;
 
 import java.lang.reflect.Field;
+import java.nio.ByteBuffer;
 import java.nio.ByteOrder;
 import java.security.AccessController;
 import java.security.PrivilegedActionException;
@@ -25,6 +26,7 @@ import java.security.PrivilegedExceptionAction;
 
 import org.apache.ignite.IgniteSystemProperties;
 import sun.misc.Unsafe;
+import sun.nio.ch.DirectBuffer;
 
 /**
  * <p>Wrapper for {@link sun.misc.Unsafe} class.</p>
@@ -1602,4 +1604,14 @@ public abstract class GridUnsafe {
             UNSAFE.putByte(addr, (byte)(val));
         }
     }
+
+    /**
+     * @param buf Direct buffer.
+     * @return Buffer memory address.
+     */
+    public static long bufferAddress(ByteBuffer buf) {
+        assert buf instanceof DirectBuffer : buf;
+
+        return ((DirectBuffer)buf).address();
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteTree.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteTree.java b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteTree.java
index 0c08cd9..8dcd205 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteTree.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteTree.java
@@ -17,10 +17,8 @@
 
 package org.apache.ignite.internal.util;
 
-import org.apache.ignite.*;
-import org.apache.ignite.internal.util.lang.*;
-
-import java.util.*;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.util.lang.GridCursor;
 
 /**
  * Interface for ignite internal tree.
@@ -29,10 +27,11 @@ public interface IgniteTree<L, T> {
     /**
      * Put value in this tree.
      *
-     * @param value value to be associated with the specified key
-     * @return the previous value associated with key
+     * @param val Value to be associated with the specified key.
+     * @return The previous value associated with key.
+     * @throws IgniteCheckedException If failed.
      */
-    T put(T value) throws IgniteCheckedException;
+    public T put(T val) throws IgniteCheckedException;
 
     /**
      * Returns the value to which the specified key is mapped, or {@code null} if this tree contains no mapping for the
@@ -40,9 +39,10 @@ public interface IgniteTree<L, T> {
      *
      * @param key the key whose associated value is to be returned
      * @return the value to which the specified key is mapped, or {@code null} if this tree contains no mapping for the
-     * key
+     *  key.
+     * @throws IgniteCheckedException If failed.
      */
-    T findOne(L key) throws IgniteCheckedException;
+    public T findOne(L key) throws IgniteCheckedException;
 
     /**
      * Returns a cursor from lower to upper bounds inclusive.
@@ -50,21 +50,24 @@ public interface IgniteTree<L, T> {
      * @param lower Lower bound or {@code null} if unbounded.
      * @param upper Upper bound or {@code null} if unbounded.
      * @return Cursor.
+     * @throws IgniteCheckedException If failed.
      */
-    GridCursor<T> find(L lower, L upper) throws IgniteCheckedException;
+    public GridCursor<T> find(L lower, L upper) throws IgniteCheckedException;
 
     /**
      * Removes the mapping for a key from this tree if it is present.
      *
-     * @param key key whose mapping is to be removed from the tree
-     * @return the previous value associated with key, or null if there was no mapping for key.
+     * @param key Key whose mapping is to be removed from the tree.
+     * @return The previous value associated with key, or null if there was no mapping for key.
+     * @throws IgniteCheckedException If failed.
      */
-    T remove(L key) throws IgniteCheckedException;
+    public T remove(L key) throws IgniteCheckedException;
 
     /**
      * Returns the number of elements in this tree.
      *
      * @return the number of elements in this tree
+     * @throws IgniteCheckedException If failed.
      */
-    long size() throws IgniteCheckedException;
+    public long size() throws IgniteCheckedException;
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/test/java/org/apache/ignite/internal/GridAffinityNoCacheSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridAffinityNoCacheSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridAffinityNoCacheSelfTest.java
index ef8b4b4..0052de9 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/GridAffinityNoCacheSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/GridAffinityNoCacheSelfTest.java
@@ -267,6 +267,11 @@ public class GridAffinityNoCacheSelfTest extends GridCommonAbstractTest {
         }
 
         /** {@inheritDoc} */
+        @Override public int putValue(long addr) throws IgniteCheckedException {
+            return 0;
+        }
+
+        /** {@inheritDoc} */
         @Override public boolean putValue(final ByteBuffer buf, final int off, final int len)
             throws IgniteCheckedException {
             return false;

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/test/java/org/apache/ignite/internal/pagemem/impl/PageMemoryNoLoadSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/pagemem/impl/PageMemoryNoLoadSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/pagemem/impl/PageMemoryNoLoadSelfTest.java
index ae81616..ec62c9b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/pagemem/impl/PageMemoryNoLoadSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/pagemem/impl/PageMemoryNoLoadSelfTest.java
@@ -18,12 +18,10 @@
 package org.apache.ignite.internal.pagemem.impl;
 
 import java.io.File;
-import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashSet;
 import java.util.List;
-
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.mem.DirectMemoryProvider;
 import org.apache.ignite.internal.mem.file.MappedFileMemoryProvider;
@@ -32,6 +30,7 @@ import org.apache.ignite.internal.pagemem.Page;
 import org.apache.ignite.internal.pagemem.PageIdAllocator;
 import org.apache.ignite.internal.pagemem.PageIdUtils;
 import org.apache.ignite.internal.pagemem.PageMemory;
+import org.apache.ignite.internal.pagemem.PageUtils;
 import org.apache.ignite.internal.processors.cache.database.tree.io.PageIO;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
@@ -190,7 +189,7 @@ public class PageMemoryNoLoadSelfTest extends GridCommonAbstractTest {
             // Check that initial pages are accessible.
             for (FullPageId id : old) {
                 try (Page page = mem.page(id.cacheId(), id.pageId())) {
-                    ByteBuffer buf = page.getForWrite();
+                    long buf = page.getForWritePointer();
 
                     assertNotNull(buf);
 
@@ -210,17 +209,17 @@ public class PageMemoryNoLoadSelfTest extends GridCommonAbstractTest {
             // Check that updated pages are inaccessible using old IDs.
             for (FullPageId id : old) {
                 try (Page page = mem.page(id.cacheId(), id.pageId())) {
-                    ByteBuffer buf = page.getForWrite();
+                    long pageAddr = page.getForWritePointer();
 
-                    if (buf != null) {
+                    if (pageAddr != 0L) {
                         page.releaseWrite(false);
 
                         fail("Was able to acquire page write lock.");
                     }
 
-                    buf = page.getForRead();
+                    pageAddr = page.getForReadPointer();
 
-                    if (buf != null) {
+                    if (pageAddr != 0) {
                         page.releaseRead();
 
                         fail("Was able to acquire page read lock.");
@@ -231,7 +230,7 @@ public class PageMemoryNoLoadSelfTest extends GridCommonAbstractTest {
             // Check that updated pages are accessible using new IDs.
             for (FullPageId id : updated) {
                 try (Page page = mem.page(id.cacheId(), id.pageId())) {
-                    ByteBuffer buf = page.getForWrite();
+                    long buf = page.getForWritePointer();
 
                     assertNotNull(buf);
 
@@ -242,7 +241,7 @@ public class PageMemoryNoLoadSelfTest extends GridCommonAbstractTest {
                         page.releaseWrite(false);
                     }
 
-                    buf = page.getForRead();
+                    buf = page.getForReadPointer();
 
                     assertNotNull(buf);
 
@@ -274,7 +273,7 @@ public class PageMemoryNoLoadSelfTest extends GridCommonAbstractTest {
         DirectMemoryProvider provider = new MappedFileMemoryProvider(log(), memDir, true,
             sizes);
 
-        return new PageMemoryNoStoreImpl(log(), provider, null, PAGE_SIZE);
+        return new PageMemoryNoStoreImpl(log(), provider, null, PAGE_SIZE, true);
     }
 
     /**
@@ -282,13 +281,13 @@ public class PageMemoryNoLoadSelfTest extends GridCommonAbstractTest {
      * @param val Value to write.
      */
     private void writePage(Page page, int val) {
-        ByteBuffer bytes = page.getForWrite();
+        long bytes = page.getForWritePointer();
 
         try {
             PageIO.setPageId(bytes, page.id());
 
             for (int i = PageIO.COMMON_HEADER_END; i < PAGE_SIZE; i++)
-                bytes.put(i, (byte)val);
+                PageUtils.putByte(bytes, i, (byte)val);
         }
         finally {
             page.releaseWrite(true);
@@ -302,13 +301,13 @@ public class PageMemoryNoLoadSelfTest extends GridCommonAbstractTest {
     private void readPage(Page page, int expVal) {
         expVal &= 0xFF;
 
-        ByteBuffer bytes = page.getForRead();
+        long pageAddr = page.getForReadPointer();
 
-        assertNotNull(bytes);
+        assert(pageAddr != 0);
 
         try {
             for (int i = PageIO.COMMON_HEADER_END; i < PAGE_SIZE; i++) {
-                int val = bytes.get(i) & 0xFF;
+                int val = PageUtils.getByte(pageAddr, i) & 0xFF;
 
                 assertEquals("Unexpected value at position: " + i, expVal, val);
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteIncompleteCacheObjectSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteIncompleteCacheObjectSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteIncompleteCacheObjectSelfTest.java
index 0224a88..32503d2 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteIncompleteCacheObjectSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteIncompleteCacheObjectSelfTest.java
@@ -122,6 +122,11 @@ public class IgniteIncompleteCacheObjectSelfTest extends GridCommonAbstractTest
         }
 
         /** {@inheritDoc} */
+        @Override public int putValue(long addr) throws IgniteCheckedException {
+            throw new UnsupportedOperationException();
+        }
+
+        /** {@inheritDoc} */
         @Override public boolean putValue(final ByteBuffer buf, final int off, final int len)
             throws IgniteCheckedException {
             return false;

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/database/tree/io/TrackingPageIOTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/database/tree/io/TrackingPageIOTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/database/tree/io/TrackingPageIOTest.java
index cb8ba1f..e2767bb 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/database/tree/io/TrackingPageIOTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/database/tree/io/TrackingPageIOTest.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.processors.cache.database.tree.io;
 
 import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.NavigableSet;
@@ -25,6 +26,7 @@ import java.util.Set;
 import java.util.TreeSet;
 import java.util.concurrent.ThreadLocalRandom;
 import junit.framework.TestCase;
+import org.apache.ignite.internal.util.GridUnsafe;
 
 /**
  *
@@ -33,13 +35,15 @@ public class TrackingPageIOTest extends TestCase {
     /** Page size. */
     public static final int PAGE_SIZE = 2048;
 
+    /** */
     private final TrackingPageIO io = TrackingPageIO.VERSIONS.latest();
 
     /**
      *
      */
     public void testBasics() {
-        ByteBuffer buf = ByteBuffer.allocate(PAGE_SIZE);
+        ByteBuffer buf = ByteBuffer.allocateDirect(PAGE_SIZE);
+        buf.order(ByteOrder.nativeOrder());
 
         io.markChanged(buf, 2, 0, -1, PAGE_SIZE);
 
@@ -54,7 +58,8 @@ public class TrackingPageIOTest extends TestCase {
      *
      */
     public void testMarkingRandomly() {
-        ByteBuffer buf = ByteBuffer.allocate(PAGE_SIZE);
+        ByteBuffer buf = ByteBuffer.allocateDirect(PAGE_SIZE);
+        buf.order(ByteOrder.nativeOrder());
 
         int cntOfPageToTrack = io.countOfPageToTrack(PAGE_SIZE);
 
@@ -66,7 +71,8 @@ public class TrackingPageIOTest extends TestCase {
      *
      */
     public void testZeroingRandomly() {
-        ByteBuffer buf = ByteBuffer.allocate(PAGE_SIZE);
+        ByteBuffer buf = ByteBuffer.allocateDirect(PAGE_SIZE);
+        buf.order(ByteOrder.nativeOrder());
 
         for (int i = 0; i < 1001; i++)
             checkMarkingRandomly(buf, i, true);
@@ -87,7 +93,7 @@ public class TrackingPageIOTest extends TestCase {
 
         assert basePageId >= 0;
 
-        PageIO.setPageId(buf, basePageId);
+        PageIO.setPageId(GridUnsafe.bufferAddress(buf), basePageId);
 
         Map<Long, Boolean> map = new HashMap<>();
 
@@ -122,8 +128,12 @@ public class TrackingPageIOTest extends TestCase {
         }
     }
 
+    /**
+     * @throws Exception If failed.
+     */
     public void testFindNextChangedPage() throws Exception {
-        ByteBuffer buf = ByteBuffer.allocate(PAGE_SIZE);
+        ByteBuffer buf = ByteBuffer.allocateDirect(PAGE_SIZE);
+        buf.order(ByteOrder.nativeOrder());
 
         for (int i = 0; i < 101; i++)
             checkFindingRandomly(buf, i);
@@ -144,7 +154,7 @@ public class TrackingPageIOTest extends TestCase {
 
         assert basePageId >= 0;
 
-        PageIO.setPageId(buf, basePageId);
+        PageIO.setPageId(GridUnsafe.bufferAddress(buf), basePageId);
 
         try {
             TreeSet<Long> setIdx = new TreeSet<>();
@@ -174,8 +184,12 @@ public class TrackingPageIOTest extends TestCase {
         }
     }
 
+    /**
+     *
+     */
     public void testMerging() {
-        ByteBuffer buf = ByteBuffer.allocate(PAGE_SIZE);
+        ByteBuffer buf = ByteBuffer.allocateDirect(PAGE_SIZE);
+        buf.order(ByteOrder.nativeOrder());
 
         ThreadLocalRandom rand = ThreadLocalRandom.current();
 
@@ -185,7 +199,7 @@ public class TrackingPageIOTest extends TestCase {
 
         assert basePageId >= 0;
 
-        PageIO.setPageId(buf, basePageId);
+        PageIO.setPageId(GridUnsafe.bufferAddress(buf), basePageId);
 
         TreeSet<Long> setIdx = new TreeSet<>();
 
@@ -209,8 +223,12 @@ public class TrackingPageIOTest extends TestCase {
             assertFalse(io.wasChanged(buf, i, 5, 4, PAGE_SIZE));
     }
 
+    /**
+     *
+     */
     public void testMerging_MarksShouldBeDropForSuccessfulBackup() {
-        ByteBuffer buf = ByteBuffer.allocate(PAGE_SIZE);
+        ByteBuffer buf = ByteBuffer.allocateDirect(PAGE_SIZE);
+        buf.order(ByteOrder.nativeOrder());
 
         ThreadLocalRandom rand = ThreadLocalRandom.current();
 
@@ -220,7 +238,7 @@ public class TrackingPageIOTest extends TestCase {
 
         assert basePageId >= 0;
 
-        PageIO.setPageId(buf, basePageId);
+        PageIO.setPageId(GridUnsafe.bufferAddress(buf), basePageId);
 
         TreeSet<Long> setIdx = new TreeSet<>();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/test/java/org/apache/ignite/internal/processors/database/BPlusTreeReuseSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/BPlusTreeReuseSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/BPlusTreeReuseSelfTest.java
index cfc58b0..5f38bb0 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/BPlusTreeReuseSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/BPlusTreeReuseSelfTest.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.internal.processors.database;
 
-import java.nio.ByteBuffer;
 import java.util.HashSet;
 import java.util.Set;
 import org.apache.ignite.IgniteCheckedException;
@@ -47,6 +46,9 @@ public class BPlusTreeReuseSelfTest extends BPlusTreeSelfTest {
         assertTrue(TestReuseList.checkNoLocks());
     }
 
+    /**
+     *
+     */
     private static class TestReuseList extends ReuseListImpl {
         /** */
         private static ThreadLocal<Set<Long>> readLocks = new ThreadLocal<Set<Long>>() {
@@ -88,15 +90,15 @@ public class BPlusTreeReuseSelfTest extends BPlusTreeSelfTest {
         }
 
         /** {@inheritDoc} */
-        @Override public void onReadLock(Page page, ByteBuffer buf) {
-            checkPageId(page, buf);
+        @Override public void onReadLock(Page page, long pageAddr) {
+            checkPageId(page, pageAddr);
 
             assertTrue(readLocks.get().add(page.id()));
         }
 
         /** {@inheritDoc} */
-        @Override public void onReadUnlock(Page page, ByteBuffer buf) {
-            checkPageId(page, buf);
+        @Override public void onReadUnlock(Page page, long pageAddr) {
+            checkPageId(page, pageAddr);
 
             assertTrue(readLocks.get().remove(page.id()));
         }
@@ -107,18 +109,18 @@ public class BPlusTreeReuseSelfTest extends BPlusTreeSelfTest {
         }
 
         /** {@inheritDoc} */
-        @Override public void onWriteLock(Page page, ByteBuffer buf) {
-            if (buf == null)
+        @Override public void onWriteLock(Page page, long pageAddr) {
+            if (pageAddr == 0L)
                 return; // Failed to lock.
 
-            checkPageId(page, buf);
+            checkPageId(page, pageAddr);
 
             assertTrue(writeLocks.get().add(page.id()));
         }
 
         /** {@inheritDoc} */
-        @Override public void onWriteUnlock(Page page, ByteBuffer buf) {
-            assertEquals(effectivePageId(page.id()), effectivePageId(PageIO.getPageId(buf)));
+        @Override public void onWriteUnlock(Page page, long pageAddr) {
+            assertEquals(effectivePageId(page.id()), effectivePageId(PageIO.getPageId(pageAddr)));
 
             assertTrue(writeLocks.get().remove(page.id()));
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/test/java/org/apache/ignite/internal/processors/database/BPlusTreeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/BPlusTreeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/BPlusTreeSelfTest.java
index 4dcb7a9..928c86d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/BPlusTreeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/BPlusTreeSelfTest.java
@@ -38,6 +38,7 @@ import org.apache.ignite.internal.pagemem.FullPageId;
 import org.apache.ignite.internal.pagemem.Page;
 import org.apache.ignite.internal.pagemem.PageIdAllocator;
 import org.apache.ignite.internal.pagemem.PageMemory;
+import org.apache.ignite.internal.pagemem.PageUtils;
 import org.apache.ignite.internal.pagemem.impl.PageMemoryNoStoreImpl;
 import org.apache.ignite.internal.processors.cache.database.DataStructure;
 import org.apache.ignite.internal.processors.cache.database.tree.BPlusTree;
@@ -1165,10 +1166,10 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
 
     /**
      * @param page Page.
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      */
-    public static void checkPageId(Page page, ByteBuffer buf) {
-        long pageId = PageIO.getPageId(buf);
+    public static void checkPageId(Page page, long pageAddr) {
+        long pageId = PageIO.getPageId(pageAddr);
 
         // Page ID must be 0L for newly allocated page, for reused page effective ID must remain the same.
         if (pageId != 0L && page.id() != pageId)
@@ -1232,18 +1233,18 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
         }
 
         /** {@inheritDoc} */
-        @Override protected int compare(BPlusIO<Long> io, ByteBuffer buf, int idx, Long n2)
+        @Override protected int compare(BPlusIO<Long> io, long pageAddr, int idx, Long n2)
             throws IgniteCheckedException {
-            Long n1 = io.getLookupRow(this, buf, idx);
+            Long n1 = io.getLookupRow(this, pageAddr, idx);
 
             return Long.compare(n1, n2);
         }
 
         /** {@inheritDoc} */
-        @Override protected Long getRow(BPlusIO<Long> io, ByteBuffer buf, int idx) throws IgniteCheckedException {
+        @Override protected Long getRow(BPlusIO<Long> io, long pageAddr, int idx) throws IgniteCheckedException {
             assert io.canGetRow() : io;
 
-            return io.getLookupRow(this, buf, idx);
+            return io.getLookupRow(this, pageAddr, idx);
         }
 
         /**
@@ -1280,11 +1281,11 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
         }
 
         /** {@inheritDoc} */
-        @Override public void onReadLock(Page page, ByteBuffer buf) {
-            if (buf != null) {
-                long pageId = PageIO.getPageId(buf);
+        @Override public void onReadLock(Page page, long pageAddr) {
+            if (pageAddr != 0L) {
+                long pageId = PageIO.getPageId(pageAddr);
 
-                checkPageId(page, buf);
+                checkPageId(page, pageAddr);
 
                 assertNull(locks(true).put(page.id(), pageId));
             }
@@ -1293,10 +1294,10 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
         }
 
         /** {@inheritDoc} */
-        @Override public void onReadUnlock(Page page, ByteBuffer buf) {
-            checkPageId(page, buf);
+        @Override public void onReadUnlock(Page page, long pageAddr) {
+            checkPageId(page, pageAddr);
 
-            long pageId = PageIO.getPageId(buf);
+            long pageId = PageIO.getPageId(pageAddr);
 
             assertEquals(Long.valueOf(pageId), locks(true).remove(page.id()));
         }
@@ -1307,11 +1308,11 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
         }
 
         /** {@inheritDoc} */
-        @Override public void onWriteLock(Page page, ByteBuffer buf) {
-            if (buf != null) {
-                checkPageId(page, buf);
+        @Override public void onWriteLock(Page page, long pageAddr) {
+            if (pageAddr != 0L) {
+                checkPageId(page, pageAddr);
 
-                long pageId = PageIO.getPageId(buf);
+                long pageId = PageIO.getPageId(pageAddr);
 
                 if (pageId == 0L)
                     pageId = page.id(); // It is a newly allocated page.
@@ -1323,8 +1324,8 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
         }
 
         /** {@inheritDoc} */
-        @Override public void onWriteUnlock(Page page, ByteBuffer buf) {
-            assertEquals(effectivePageId(page.id()), effectivePageId(PageIO.getPageId(buf)));
+        @Override public void onWriteUnlock(Page page, long pageAddr) {
+            assertEquals(effectivePageId(page.id()), effectivePageId(PageIO.getPageId(pageAddr)));
 
             assertEquals(Long.valueOf(page.id()), locks(false).remove(page.id()));
         }
@@ -1391,15 +1392,15 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
         }
 
         /** {@inheritDoc} */
-        @Override public int getMaxCount(ByteBuffer buf) {
+        @Override public int getMaxCount(long buf, int pageSize) {
             if (MAX_PER_PAGE != 0)
                 return MAX_PER_PAGE;
 
-            return super.getMaxCount(buf);
+            return super.getMaxCount(buf, pageSize);
         }
 
         /** {@inheritDoc} */
-        @Override public void store(ByteBuffer dst, int dstIdx, BPlusIO<Long> srcIo, ByteBuffer src, int srcIdx)
+        @Override public void store(long dst, int dstIdx, BPlusIO<Long> srcIo, long src, int srcIdx)
             throws IgniteCheckedException {
             Long row = srcIo.getLookupRow(null, src, srcIdx);
 
@@ -1415,16 +1416,21 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
         }
 
         /** {@inheritDoc} */
-        @Override public void storeByOffset(ByteBuffer buf, int off, Long row) {
+        @Override public void storeByOffset(ByteBuffer buf, int off, Long row) throws IgniteCheckedException {
+            throw new UnsupportedOperationException();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void storeByOffset(long pageAddr, int off, Long row) {
             checkNotRemoved(row);
 
-            buf.putLong(off, row);
+            PageUtils.putLong(pageAddr, off, row);
         }
 
         /** {@inheritDoc} */
-        @Override public Long getLookupRow(BPlusTree<Long,?> tree, ByteBuffer buf, int idx)
+        @Override public Long getLookupRow(BPlusTree<Long,?> tree, long pageAddr, int idx)
             throws IgniteCheckedException {
-            Long row = buf.getLong(offset(idx));
+            Long row = PageUtils.getLong(pageAddr, offset(idx));
 
             checkNotRemoved(row);
 
@@ -1441,7 +1447,7 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
         for (int i = 0; i < sizes.length; i++)
             sizes[i] = 1024 * MB / CPUS;
 
-        PageMemory pageMem = new PageMemoryNoStoreImpl(log, new UnsafeMemoryProvider(sizes), null, PAGE_SIZE);
+        PageMemory pageMem = new PageMemoryNoStoreImpl(log, new UnsafeMemoryProvider(sizes), null, PAGE_SIZE, true);
 
         pageMem.start();
 
@@ -1461,34 +1467,39 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
     private static final class LongLeafIO extends BPlusLeafIO<Long> {
         /**
          */
-        protected LongLeafIO() {
+        LongLeafIO() {
             super(LONG_LEAF_IO, 1, 8);
         }
 
         /** {@inheritDoc} */
-        @Override public int getMaxCount(ByteBuffer buf) {
+        @Override public int getMaxCount(long pageAddr, int pageSize) {
             if (MAX_PER_PAGE != 0)
                 return MAX_PER_PAGE;
 
-            return super.getMaxCount(buf);
+            return super.getMaxCount(pageAddr, pageSize);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void storeByOffset(ByteBuffer buf, int off, Long row) throws IgniteCheckedException {
+            throw new UnsupportedOperationException();
         }
 
         /** {@inheritDoc} */
-        @Override public void storeByOffset(ByteBuffer buf, int off, Long row) {
-            buf.putLong(off, row);
+        @Override public void storeByOffset(long pageAddr, int off, Long row) {
+            PageUtils.putLong(pageAddr, off, row);
         }
 
         /** {@inheritDoc} */
-        @Override public void store(ByteBuffer dst, int dstIdx, BPlusIO<Long> srcIo, ByteBuffer src, int srcIdx) {
+        @Override public void store(long dst, int dstIdx, BPlusIO<Long> srcIo, long src, int srcIdx) {
             assert srcIo == this;
 
-            dst.putLong(offset(dstIdx), src.getLong(offset(srcIdx)));
+            PageUtils.putLong(dst, offset(dstIdx), PageUtils.getLong(src, offset(srcIdx)));
         }
 
         /** {@inheritDoc} */
-        @Override public Long getLookupRow(BPlusTree<Long,?> tree, ByteBuffer buf, int idx)
+        @Override public Long getLookupRow(BPlusTree<Long,?> tree, long pageAddr, int idx)
             throws IgniteCheckedException {
-            return buf.getLong(offset(idx));
+            return PageUtils.getLong(pageAddr, offset(idx));
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/test/java/org/apache/ignite/internal/processors/database/FreeListImplSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/FreeListImplSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/FreeListImplSelfTest.java
index 2b93d48..d0d495e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/FreeListImplSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/FreeListImplSelfTest.java
@@ -32,6 +32,7 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.mem.unsafe.UnsafeMemoryProvider;
 import org.apache.ignite.internal.pagemem.PageIdAllocator;
 import org.apache.ignite.internal.pagemem.PageMemory;
+import org.apache.ignite.internal.pagemem.PageUtils;
 import org.apache.ignite.internal.pagemem.impl.PageMemoryNoStoreImpl;
 import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.CacheObjectContext;
@@ -315,7 +316,7 @@ public class FreeListImplSelfTest extends GridCommonAbstractTest {
         for (int i = 0; i < sizes.length; i++)
             sizes[i] = 1024 * MB / CPUS;
 
-        PageMemory pageMem = new PageMemoryNoStoreImpl(log, new UnsafeMemoryProvider(sizes), null, pageSize);
+        PageMemory pageMem = new PageMemoryNoStoreImpl(log, new UnsafeMemoryProvider(sizes), null, pageSize, true);
 
         pageMem.start();
 
@@ -395,6 +396,11 @@ public class FreeListImplSelfTest extends GridCommonAbstractTest {
         @Override public void link(long link) {
             this.link = link;
         }
+
+        /** {@inheritDoc} */
+        @Override public int hash() {
+            throw new UnsupportedOperationException();
+        }
     }
 
     /**
@@ -458,6 +464,13 @@ public class FreeListImplSelfTest extends GridCommonAbstractTest {
         }
 
         /** {@inheritDoc} */
+        @Override public int putValue(long addr) throws IgniteCheckedException {
+            PageUtils.putBytes(addr, 0, data);
+
+            return data.length;
+        }
+
+        /** {@inheritDoc} */
         @Override public boolean putValue(ByteBuffer buf, int off, int len) throws IgniteCheckedException {
             buf.put(data, off, len);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/test/java/org/apache/ignite/internal/processors/database/MetadataStorageSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/MetadataStorageSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/MetadataStorageSelfTest.java
index 252ea82..7eeda4b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/MetadataStorageSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/MetadataStorageSelfTest.java
@@ -160,6 +160,6 @@ public class MetadataStorageSelfTest extends GridCommonAbstractTest {
 
         DirectMemoryProvider provider = new MappedFileMemoryProvider(log(), allocationPath, clean, sizes);
 
-        return new PageMemoryNoStoreImpl(log, provider, null, PAGE_SIZE);
+        return new PageMemoryNoStoreImpl(log, provider, null, PAGE_SIZE, true);
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2Cursor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2Cursor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2Cursor.java
index cc71813..de3111d 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2Cursor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2Cursor.java
@@ -31,13 +31,13 @@ import org.h2.result.*;
  */
 public class H2Cursor implements Cursor {
     /** */
-    final GridCursor<GridH2Row> cursor;
+    private final GridCursor<GridH2Row> cursor;
 
     /** */
-    final IgniteBiPredicate<Object,Object> filter;
+    private final IgniteBiPredicate<Object,Object> filter;
 
     /** */
-    final long time = U.currentTimeMillis();
+    private final long time = U.currentTimeMillis();
 
     /**
      * @param cursor Cursor.
@@ -50,6 +50,9 @@ public class H2Cursor implements Cursor {
         this.filter = filter;
     }
 
+    /**
+     * @param cursor Cursor.
+     */
     public H2Cursor(GridCursor<GridH2Row> cursor) {
         this(cursor, null);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2Tree.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2Tree.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2Tree.java
index 0593c60..5c60b16 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2Tree.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2Tree.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.internal.processors.query.h2.database;
 
-import java.nio.ByteBuffer;
 import java.util.concurrent.atomic.AtomicLong;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.pagemem.PageMemory;
@@ -76,9 +75,9 @@ public abstract class H2Tree extends BPlusTree<SearchRow, GridH2Row> {
     }
 
     /** {@inheritDoc} */
-    @Override protected GridH2Row getRow(BPlusIO<SearchRow> io, ByteBuffer buf, int idx)
+    @Override protected GridH2Row getRow(BPlusIO<SearchRow> io, long pageAddr, int idx)
         throws IgniteCheckedException {
-        return (GridH2Row)io.getLookupRow(this, buf, idx);
+        return (GridH2Row)io.getLookupRow(this, pageAddr, idx);
     }
 }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2TreeIndex.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2TreeIndex.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2TreeIndex.java
index 19cbbf9..6cabd77 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2TreeIndex.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2TreeIndex.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.internal.processors.query.h2.database;
 
-import java.nio.ByteBuffer;
 import java.util.List;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
@@ -87,9 +86,9 @@ public class H2TreeIndex extends GridH2IndexBase {
             tree = new H2Tree(name, cctx.offheap().reuseListForIndex(name), cctx.cacheId(),
                 dbMgr.pageMemory(), cctx.shared().wal(), cctx.offheap().globalRemoveId(),
                 tbl.rowFactory(), page.pageId().pageId(), page.isAllocated()) {
-                @Override protected int compare(BPlusIO<SearchRow> io, ByteBuffer buf, int idx, SearchRow row)
+                @Override protected int compare(BPlusIO<SearchRow> io, long pageAddr, int idx, SearchRow row)
                     throws IgniteCheckedException {
-                    return compareRows(getRow(io, buf, idx), row);
+                    return compareRows(getRow(io, pageAddr, idx), row);
                 }
             };
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2InnerIO.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2InnerIO.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2InnerIO.java
index 6868205..f697121 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2InnerIO.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2InnerIO.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.query.h2.database.io;
 
 import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageUtils;
 import org.apache.ignite.internal.processors.cache.database.tree.BPlusTree;
 import org.apache.ignite.internal.processors.cache.database.tree.io.BPlusIO;
 import org.apache.ignite.internal.processors.cache.database.tree.io.BPlusInnerIO;
@@ -53,22 +54,31 @@ public class H2InnerIO extends BPlusInnerIO<SearchRow> implements H2RowLinkIO {
     }
 
     /** {@inheritDoc} */
-    @Override public SearchRow getLookupRow(BPlusTree<SearchRow,?> tree, ByteBuffer buf, int idx)
+    @Override public void storeByOffset(long pageAddr, int off, SearchRow row) {
+        GridH2Row row0 = (GridH2Row)row;
+
+        assert row0.link != 0;
+
+        PageUtils.putLong(pageAddr, off, row0.link);
+    }
+
+    /** {@inheritDoc} */
+    @Override public SearchRow getLookupRow(BPlusTree<SearchRow,?> tree, long pageAddr, int idx)
         throws IgniteCheckedException {
-        long link = getLink(buf, idx);
+        long link = getLink(pageAddr, idx);
 
         return ((H2Tree)tree).getRowFactory().getRow(link);
     }
 
     /** {@inheritDoc} */
-    @Override public void store(ByteBuffer dst, int dstIdx, BPlusIO<SearchRow> srcIo, ByteBuffer src, int srcIdx) {
-        long link = ((H2RowLinkIO)srcIo).getLink(src, srcIdx);
+    @Override public void store(long dstPageAddr, int dstIdx, BPlusIO<SearchRow> srcIo, long srcPageAddr, int srcIdx) {
+        long link = ((H2RowLinkIO)srcIo).getLink(srcPageAddr, srcIdx);
 
-        dst.putLong(offset(dstIdx), link);
+        PageUtils.putLong(dstPageAddr, offset(dstIdx), link);
     }
 
     /** {@inheritDoc} */
-    @Override public long getLink(ByteBuffer buf, int idx) {
-        return buf.getLong(offset(idx));
+    @Override public long getLink(long pageAddr, int idx) {
+        return PageUtils.getLong(pageAddr, offset(idx));
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2LeafIO.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2LeafIO.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2LeafIO.java
index ce517a8..26cbdc5 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2LeafIO.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2LeafIO.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.query.h2.database.io;
 
 import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageUtils;
 import org.apache.ignite.internal.processors.cache.database.tree.BPlusTree;
 import org.apache.ignite.internal.processors.cache.database.tree.io.BPlusIO;
 import org.apache.ignite.internal.processors.cache.database.tree.io.BPlusLeafIO;
@@ -53,22 +54,31 @@ public class H2LeafIO extends BPlusLeafIO<SearchRow> implements H2RowLinkIO {
     }
 
     /** {@inheritDoc} */
-    @Override public void store(ByteBuffer dst, int dstIdx, BPlusIO<SearchRow> srcIo, ByteBuffer src, int srcIdx) {
+    @Override public void storeByOffset(long pageAddr, int off, SearchRow row) {
+        GridH2Row row0 = (GridH2Row)row;
+
+        assert row0.link != 0;
+
+        PageUtils.putLong(pageAddr, off, row0.link);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void store(long dstPageAddr, int dstIdx, BPlusIO<SearchRow> srcIo, long srcPageAddr, int srcIdx) {
         assert srcIo == this;
 
-        dst.putLong(offset(dstIdx), getLink(src, srcIdx));
+        PageUtils.putLong(dstPageAddr, offset(dstIdx), getLink(srcPageAddr, srcIdx));
     }
 
     /** {@inheritDoc} */
-    @Override public SearchRow getLookupRow(BPlusTree<SearchRow,?> tree, ByteBuffer buf, int idx)
+    @Override public SearchRow getLookupRow(BPlusTree<SearchRow,?> tree, long pageAddr, int idx)
         throws IgniteCheckedException {
-        long link = getLink(buf, idx);
+        long link = getLink(pageAddr, idx);
 
         return ((H2Tree)tree).getRowFactory().getRow(link);
     }
 
     /** {@inheritDoc} */
-    @Override public long getLink(ByteBuffer buf, int idx) {
-        return buf.getLong(offset(idx));
+    @Override public long getLink(long pageAddr, int idx) {
+        return PageUtils.getLong(pageAddr, offset(idx));
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2RowLinkIO.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2RowLinkIO.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2RowLinkIO.java
index 9c4737d..ce69197 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2RowLinkIO.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2RowLinkIO.java
@@ -17,16 +17,14 @@
 
 package org.apache.ignite.internal.processors.query.h2.database.io;
 
-import java.nio.ByteBuffer;
-
 /**
  * Row link IO.
  */
 public interface H2RowLinkIO {
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param idx Index.
      * @return Row link.
      */
-    public long getLink(ByteBuffer buf, int idx);
+    public long getLink(long pageAddr, int idx);
 }

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlDistributedJoinSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlDistributedJoinSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlDistributedJoinSelfTest.java
index e9f41c8..81201c8 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlDistributedJoinSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlDistributedJoinSelfTest.java
@@ -17,18 +17,18 @@
 
 package org.apache.ignite.internal.processors.query;
 
-import org.apache.ignite.*;
-import org.apache.ignite.cache.*;
-import org.apache.ignite.cache.query.*;
-import org.apache.ignite.cache.query.annotations.*;
-import org.apache.ignite.configuration.*;
-import org.apache.ignite.plugin.*;
-import org.apache.ignite.spi.discovery.tcp.*;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
-import org.apache.ignite.testframework.junits.common.*;
-
-import java.util.*;
+import java.util.List;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.query.SqlQuery;
+import org.apache.ignite.cache.query.annotations.QuerySqlField;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 /**
  * Tests for correct distributed sql joins.
@@ -36,8 +36,14 @@ import java.util.*;
 public class IgniteSqlDistributedJoinSelfTest extends GridCommonAbstractTest {
     /** */
     private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
     private static final int NODES_COUNT = 2;
+
+    /** */
     private static final int ORG_COUNT = NODES_COUNT;
+
+    /** */
     private static final int PERSON_PER_ORG_COUNT = 50;
 
     /** {@inheritDoc} */
@@ -81,7 +87,7 @@ public class IgniteSqlDistributedJoinSelfTest extends GridCommonAbstractTest {
     }
 
     /**
-     *
+     * @throws Exception If failed.
      */
     public void testNonCollocatedDistributedJoin() throws Exception {
         CacheConfiguration ccfg1 = cacheConfig("pers", true, String.class, Person.class);
@@ -141,11 +147,19 @@ public class IgniteSqlDistributedJoinSelfTest extends GridCommonAbstractTest {
         }
     }
 
+    /**
+     *
+     */
     private static class Person {
+        /** */
         @QuerySqlField(index = true)
         private String id;
+
+        /** */
         @QuerySqlField(index = true)
         private String orgId;
+
+        /** */
         @QuerySqlField(index = true)
         private String name;
 
@@ -162,9 +176,15 @@ public class IgniteSqlDistributedJoinSelfTest extends GridCommonAbstractTest {
         public void setName(String name) { this.name = name; }
     }
 
+    /**
+     *
+     */
     private static class Organization {
+        /** */
         @QuerySqlField(index = true)
         private String id;
+
+        /** */
         @QuerySqlField(index = true)
         private String name;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
index 593ba95..b9152fa 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
@@ -694,6 +694,11 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract
         }
 
         /** {@inheritDoc} */
+        @Override public int putValue(long addr) throws IgniteCheckedException {
+            return 0;
+        }
+
+        /** {@inheritDoc} */
         @Override public boolean putValue(final ByteBuffer buf, final int off, final int len)
             throws IgniteCheckedException {
             return false;


[3/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

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusIO.java
index 5fc3d25..c34296a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusIO.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.cache.database.tree.io;
 
 import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageUtils;
 import org.apache.ignite.internal.processors.cache.database.tree.BPlusTree;
 
 /**
@@ -72,56 +73,56 @@ public abstract class BPlusIO<L> extends PageIO {
     }
 
     /** {@inheritDoc} */
-    @Override public void initNewPage(ByteBuffer buf, long pageId) {
-        super.initNewPage(buf, pageId);
+    @Override public void initNewPage(long pageAddr, long pageId, int pageSize) {
+        super.initNewPage(pageAddr, pageId, pageSize);
 
-        setCount(buf, 0);
-        setForward(buf, 0);
-        setRemoveId(buf, 0);
+        setCount(pageAddr, 0);
+        setForward(pageAddr, 0);
+        setRemoveId(pageAddr, 0);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return Forward page ID.
      */
-    public final long getForward(ByteBuffer buf) {
-        return buf.getLong(FORWARD_OFF);
+    public final long getForward(long pageAddr) {
+        return PageUtils.getLong(pageAddr, FORWARD_OFF);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param pageId Forward page ID.
      */
-    public final void setForward(ByteBuffer buf, long pageId) {
-        buf.putLong(FORWARD_OFF, pageId);
+    public final void setForward(long pageAddr, long pageId) {
+        PageUtils.putLong(pageAddr, FORWARD_OFF, pageId);
 
-        assert getForward(buf) == pageId;
+        assert getForward(pageAddr) == pageId;
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return Remove ID.
      */
-    public final long getRemoveId(ByteBuffer buf) {
-        return buf.getLong(REMOVE_ID_OFF);
+    public final long getRemoveId(long pageAddr) {
+        return PageUtils.getLong(pageAddr, REMOVE_ID_OFF);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param rmvId Remove ID.
      */
-    public final void setRemoveId(ByteBuffer buf, long rmvId) {
-        buf.putLong(REMOVE_ID_OFF, rmvId);
+    public final void setRemoveId(long pageAddr, long rmvId) {
+        PageUtils.putLong(pageAddr, REMOVE_ID_OFF, rmvId);
 
-        assert getRemoveId(buf) == rmvId;
+        assert getRemoveId(pageAddr) == rmvId;
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return Items count in the page.
      */
-    public final int getCount(ByteBuffer buf) {
-        int cnt = buf.getShort(CNT_OFF) & 0xFFFF;
+    public final int getCount(long pageAddr) {
+        int cnt = PageUtils.getShort(pageAddr, CNT_OFF) & 0xFFFF;
 
         assert cnt >= 0: cnt;
 
@@ -129,20 +130,20 @@ public abstract class BPlusIO<L> extends PageIO {
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param cnt Count.
      */
-    public final void setCount(ByteBuffer buf, int cnt) {
+    public final void setCount(long pageAddr, int cnt) {
         assert cnt >= 0: cnt;
 
-        buf.putShort(CNT_OFF, (short)cnt);
+        PageUtils.putShort(pageAddr, CNT_OFF, (short)cnt);
 
-        assert getCount(buf) == cnt;
+        assert getCount(pageAddr) == cnt;
     }
 
     /**
      * @return {@code true} If we can get the full row from this page using
-     * method {@link BPlusTree#getRow(BPlusIO, ByteBuffer, int)}.
+     * method {@link BPlusTree#getRow(BPlusIO, long, int)}.
      * Must always be {@code true} for leaf pages.
      */
     public final boolean canGetRow() {
@@ -157,27 +158,28 @@ public abstract class BPlusIO<L> extends PageIO {
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
+     * @param pageSize Page size.
      * @return Max items count.
      */
-    public abstract int getMaxCount(ByteBuffer buf);
+    public abstract int getMaxCount(long pageAddr, int pageSize);
 
     /**
      * Store the needed info about the row in the page. Leaf and inner pages can store different info.
      *
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param idx Index.
      * @param row Lookup or full row.
      * @param rowBytes Row bytes.
      * @throws IgniteCheckedException If failed.
      */
-    public final void store(ByteBuffer buf, int idx, L row, byte[] rowBytes) throws IgniteCheckedException {
+    public final void store(long pageAddr, int idx, L row, byte[] rowBytes) throws IgniteCheckedException {
         int off = offset(idx);
 
         if (rowBytes == null)
-            storeByOffset(buf, off, row);
+            storeByOffset(pageAddr, off, row);
         else
-            putBytes(buf, off, rowBytes);
+            putBytes(pageAddr, off, rowBytes);
     }
 
     /**
@@ -189,6 +191,17 @@ public abstract class BPlusIO<L> extends PageIO {
     /**
      * Store the needed info about the row in the page. Leaf and inner pages can store different info.
      *
+     * @param pageAddr Page address.
+     * @param off Offset in bytes.
+     * @param row Lookup or full row.
+     * @throws IgniteCheckedException If failed.
+     */
+    public abstract void storeByOffset(long pageAddr, int off, L row) throws IgniteCheckedException;
+
+
+    /**
+     * Store the needed info about the row in the page. Leaf and inner pages can store different info.
+     *
      * @param buf Buffer.
      * @param off Offset in bytes.
      * @param row Lookup or full row.
@@ -199,137 +212,141 @@ public abstract class BPlusIO<L> extends PageIO {
     /**
      * Store row info from the given source.
      *
-     * @param dst Destination buffer
+     * @param dstPageAddr Destination page address.
      * @param dstIdx Destination index.
      * @param srcIo Source IO.
-     * @param src Source buffer.
+     * @param srcPageAddr Source page address.
      * @param srcIdx Source index.
      * @throws IgniteCheckedException If failed.
      */
-    public abstract void store(ByteBuffer dst, int dstIdx, BPlusIO<L> srcIo, ByteBuffer src, int srcIdx)
+    public abstract void store(long dstPageAddr, int dstIdx, BPlusIO<L> srcIo, long srcPageAddr, int srcIdx)
         throws IgniteCheckedException;
 
     /**
      * Get lookup row.
      *
      * @param tree Tree.
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param idx Index.
      * @return Lookup row.
      * @throws IgniteCheckedException If failed.
      */
-    public abstract L getLookupRow(BPlusTree<L, ?> tree, ByteBuffer buf, int idx) throws IgniteCheckedException;
+    public abstract L getLookupRow(BPlusTree<L, ?> tree, long pageAddr, int idx) throws IgniteCheckedException;
 
     /**
-     * Copy items from source buffer to destination buffer.
+     * Copy items from source page to destination page.
      * Both pages must be of the same type and the same version.
      *
-     * @param src Source buffer.
-     * @param dst Destination buffer.
+     * @param srcPageAddr Source page address.
+     * @param dstPageAddr Destination page address.
      * @param srcIdx Source begin index.
      * @param dstIdx Destination begin index.
      * @param cnt Items count.
      * @param cpLeft Copy leftmost link (makes sense only for inner pages).
      * @throws IgniteCheckedException If failed.
      */
-    public abstract void copyItems(ByteBuffer src, ByteBuffer dst, int srcIdx, int dstIdx, int cnt, boolean cpLeft)
+    public abstract void copyItems(long srcPageAddr, long dstPageAddr, int srcIdx, int dstIdx, int cnt, boolean cpLeft)
         throws IgniteCheckedException;
 
     // Methods for B+Tree logic.
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param idx Index.
      * @param row Row to insert.
      * @param rowBytes Row bytes.
      * @param rightId Page ID which will be to the right child for the inserted item.
      * @throws IgniteCheckedException If failed.
      */
-    public void insert(ByteBuffer buf, int idx, L row, byte[] rowBytes, long rightId)
+    public void insert(long pageAddr, int idx, L row, byte[] rowBytes, long rightId)
         throws IgniteCheckedException {
-        int cnt = getCount(buf);
+        int cnt = getCount(pageAddr);
 
         // Move right all the greater elements to make a free slot for a new row link.
-        copyItems(buf, buf, idx, idx + 1, cnt - idx, false);
+        copyItems(pageAddr, pageAddr, idx, idx + 1, cnt - idx, false);
 
-        setCount(buf, cnt + 1);
+        setCount(pageAddr, cnt + 1);
 
-        store(buf, idx, row, rowBytes);
+        store(pageAddr, idx, row, rowBytes);
     }
 
     /**
-     * @param buf Splitting buffer.
+     * @param pageAddr Splitting page address.
      * @param fwdId Forward page ID.
-     * @param fwdBuf Forward buffer.
+     * @param fwdPageAddr Forward page address.
      * @param mid Bisection index.
      * @param cnt Initial elements count in the page being split.
+     * @param pageSize Page size.
      * @throws IgniteCheckedException If failed.
      */
     public void splitForwardPage(
-        ByteBuffer buf,
+        long pageAddr,
         long fwdId,
-        ByteBuffer fwdBuf,
+        long fwdPageAddr,
         int mid,
-        int cnt
+        int cnt,
+        int pageSize
     ) throws IgniteCheckedException {
-        initNewPage(fwdBuf, fwdId);
+        initNewPage(fwdPageAddr, fwdId, pageSize);
 
         cnt -= mid;
 
-        copyItems(buf, fwdBuf, mid, 0, cnt, true);
+        copyItems(pageAddr, fwdPageAddr, mid, 0, cnt, true);
 
-        setCount(fwdBuf, cnt);
-        setForward(fwdBuf, getForward(buf));
+        setCount(fwdPageAddr, cnt);
+        setForward(fwdPageAddr, getForward(pageAddr));
 
         // Copy remove ID to make sure that if inner remove touched this page, then retry
         // will happen even for newly allocated forward page.
-        setRemoveId(fwdBuf, getRemoveId(buf));
+        setRemoveId(fwdPageAddr, getRemoveId(pageAddr));
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param mid Bisection index.
      * @param fwdId New forward page ID.
      */
-    public void splitExistingPage(ByteBuffer buf, int mid, long fwdId) {
-        setCount(buf, mid);
-        setForward(buf, fwdId);
+    public void splitExistingPage(long pageAddr, int mid, long fwdId) {
+        setCount(pageAddr, mid);
+        setForward(pageAddr, fwdId);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param idx Index.
      * @param cnt Count.
      * @throws IgniteCheckedException If failed.
      */
-    public void remove(ByteBuffer buf, int idx, int cnt) throws IgniteCheckedException {
+    public void remove(long pageAddr, int idx, int cnt) throws IgniteCheckedException {
         cnt--;
 
-        copyItems(buf, buf, idx + 1, idx, cnt - idx, false);
-        setCount(buf, cnt);
+        copyItems(pageAddr, pageAddr, idx + 1, idx, cnt - idx, false);
+        setCount(pageAddr, cnt);
     }
 
     /**
      * @param prntIo Parent IO.
-     * @param prnt Parent buffer.
+     * @param prntPageAddr Parent page address.
      * @param prntIdx Split key index in parent.
-     * @param left Left buffer.
-     * @param right Right buffer.
+     * @param leftPageAddr Left page address.
+     * @param rightPageAddr Right page address.
      * @param emptyBranch We are merging an empty branch.
+     * @param pageSize Page size.
      * @return {@code false} If we were not able to merge.
      * @throws IgniteCheckedException If failed.
      */
     public boolean merge(
         BPlusIO<L> prntIo,
-        ByteBuffer prnt,
+        long prntPageAddr,
         int prntIdx,
-        ByteBuffer left,
-        ByteBuffer right,
-        boolean emptyBranch
+        long leftPageAddr,
+        long rightPageAddr,
+        boolean emptyBranch,
+        int pageSize
     ) throws IgniteCheckedException {
-        int prntCnt = prntIo.getCount(prnt);
-        int leftCnt = getCount(left);
-        int rightCnt = getCount(right);
+        int prntCnt = prntIo.getCount(prntPageAddr);
+        int leftCnt = getCount(leftPageAddr);
+        int rightCnt = getCount(rightPageAddr);
 
         int newCnt = leftCnt + rightCnt;
 
@@ -337,13 +354,13 @@ public abstract class BPlusIO<L> extends PageIO {
         if (!isLeaf() && !emptyBranch)
             newCnt++;
 
-        if (newCnt > getMaxCount(left)) {
+        if (newCnt > getMaxCount(leftPageAddr, pageSize)) {
             assert !emptyBranch;
 
             return false;
         }
 
-        setCount(left, newCnt);
+        setCount(leftPageAddr, newCnt);
 
         // Move down split key in inner pages.
         if (!isLeaf() && !emptyBranch) {
@@ -351,37 +368,29 @@ public abstract class BPlusIO<L> extends PageIO {
 
             // We can be sure that we have enough free space to store split key here,
             // because we've done remove already and did not release child locks.
-            store(left, leftCnt, prntIo, prnt, prntIdx);
+            store(leftPageAddr, leftCnt, prntIo, prntPageAddr, prntIdx);
 
             leftCnt++;
         }
 
-        copyItems(right, left, 0, leftCnt, rightCnt, !emptyBranch);
-        setForward(left, getForward(right));
+        copyItems(rightPageAddr, leftPageAddr, 0, leftCnt, rightCnt, !emptyBranch);
+        setForward(leftPageAddr, getForward(rightPageAddr));
 
-        long rmvId = getRemoveId(right);
+        long rmvId = getRemoveId(rightPageAddr);
 
         // Need to have maximum remove ID.
-        if (rmvId > getRemoveId(left))
-            setRemoveId(left, rmvId);
+        if (rmvId > getRemoveId(leftPageAddr))
+            setRemoveId(leftPageAddr, rmvId);
 
         return true;
     }
 
     /**
-     * @param buf Buffer.
-     * @param pos Position in buffer.
+     * @param pageAddr Page address.
+     * @param pos Position in page.
      * @param bytes Bytes.
      */
-    private static void putBytes(ByteBuffer buf, int pos, byte[] bytes) {
-        int oldPos = buf.position();
-
-        try {
-            buf.position(pos);
-            buf.put(bytes);
-        }
-        finally {
-            buf.position(oldPos);
-        }
+    private static void putBytes(long pageAddr, int pos, byte[] bytes) {
+        PageUtils.putBytes(pageAddr, pos, bytes);
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusInnerIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusInnerIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusInnerIO.java
index 90b0f37..c7018bd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusInnerIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusInnerIO.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.processors.cache.database.tree.io;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageUtils;
 import org.apache.ignite.internal.processors.cache.database.tree.util.PageHandler;
 
 /**
@@ -45,71 +45,71 @@ public abstract class BPlusInnerIO<L> extends BPlusIO<L> {
     }
 
     /** {@inheritDoc} */
-    @Override public int getMaxCount(ByteBuffer buf) {
+    @Override public int getMaxCount(long pageAddr, int pageSize) {
         // The structure of the page is the following:
         // |ITEMS_OFF|w|A|x|B|y|C|z|
         // where capital letters are data items, lowercase letters are 8 byte page references.
-        return (buf.capacity() - ITEMS_OFF - 8) / (itemSize + 8);
+        return (pageSize - ITEMS_OFF - 8) / (itemSize + 8);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param idx Index.
      * @return Page ID.
      */
-    public final long getLeft(ByteBuffer buf, int idx) {
-        return buf.getLong(offset(idx, SHIFT_LEFT));
+    public final long getLeft(long pageAddr, int idx) {
+        return PageUtils.getLong(pageAddr, (offset(idx, SHIFT_LEFT)));
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param idx Index.
      * @param pageId Page ID.
      */
-    public final void setLeft(ByteBuffer buf, int idx, long pageId) {
-        buf.putLong(offset(idx, SHIFT_LEFT), pageId);
+    public final void setLeft(long pageAddr, int idx, long pageId) {
+        PageUtils.putLong(pageAddr, offset(idx, SHIFT_LEFT), pageId);
 
-        assert pageId == getLeft(buf, idx);
+        assert pageId == getLeft(pageAddr, idx);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param idx Index.
      * @return Page ID.
      */
-    public final long getRight(ByteBuffer buf, int idx) {
-        return buf.getLong(offset(idx, SHIFT_RIGHT));
+    public final long getRight(long pageAddr, int idx) {
+        return PageUtils.getLong(pageAddr, offset(idx, SHIFT_RIGHT));
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param idx Index.
      * @param pageId Page ID.
      */
-    public final void setRight(ByteBuffer buf, int idx, long pageId) {
-        buf.putLong(offset(idx, SHIFT_RIGHT), pageId);
+    private void setRight(long pageAddr, int idx, long pageId) {
+        PageUtils.putLong(pageAddr, offset(idx, SHIFT_RIGHT), pageId);
 
-        assert pageId == getRight(buf, idx);
+        assert pageId == getRight(pageAddr, idx);
     }
 
     /** {@inheritDoc} */
-    @Override public final void copyItems(ByteBuffer src, ByteBuffer dst, int srcIdx, int dstIdx, int cnt,
+    @Override public final void copyItems(long srcPageAddr, long dstPageAddr, int srcIdx, int dstIdx, int cnt,
         boolean cpLeft) throws IgniteCheckedException {
-        assert srcIdx != dstIdx || src != dst;
+        assert srcIdx != dstIdx || srcPageAddr != dstPageAddr;
 
         cnt *= itemSize + 8; // From items to bytes.
 
         if (dstIdx > srcIdx) {
-            PageHandler.copyMemory(src, dst, offset(srcIdx), offset(dstIdx), cnt);
+            PageHandler.copyMemory(srcPageAddr, dstPageAddr, offset(srcIdx), offset(dstIdx), cnt);
 
             if (cpLeft)
-                dst.putLong(offset(dstIdx, SHIFT_LEFT), src.getLong(offset(srcIdx, SHIFT_LEFT)));
+                PageUtils.putLong(dstPageAddr, offset(dstIdx, SHIFT_LEFT), PageUtils.getLong(srcPageAddr, (offset(srcIdx, SHIFT_LEFT))));
         }
         else {
             if (cpLeft)
-                dst.putLong(offset(dstIdx, SHIFT_LEFT), src.getLong(offset(srcIdx, SHIFT_LEFT)));
+                PageUtils.putLong(dstPageAddr, offset(dstIdx, SHIFT_LEFT), PageUtils.getLong(srcPageAddr, (offset(srcIdx, SHIFT_LEFT))));
 
-            PageHandler.copyMemory(src, dst, offset(srcIdx), offset(dstIdx), cnt);
+            PageHandler.copyMemory(srcPageAddr, dstPageAddr, offset(srcIdx), offset(dstIdx), cnt);
         }
     }
 
@@ -131,39 +131,42 @@ public abstract class BPlusInnerIO<L> extends BPlusIO<L> {
 
     /** {@inheritDoc} */
     @Override public void insert(
-        ByteBuffer buf,
+        long pageAddr,
         int idx,
         L row,
         byte[] rowBytes,
         long rightId
     ) throws IgniteCheckedException {
-        super.insert(buf, idx, row, rowBytes, rightId);
+        super.insert(pageAddr, idx, row, rowBytes, rightId);
 
         // Setup reference to the right page on split.
-        setRight(buf, idx, rightId);
+        setRight(pageAddr, idx, rightId);
     }
 
     /**
-     * @param newRootBuf New root buffer.
+     * @param newRootPageAddr New root page address.
      * @param newRootId New root ID.
      * @param leftChildId Left child ID.
      * @param row Moved up row.
+     * @param rowBytes Bytes.
      * @param rightChildId Right child ID.
+     * @param pageSize Page size.
      * @throws IgniteCheckedException If failed.
      */
     public void initNewRoot(
-        ByteBuffer newRootBuf,
+        long newRootPageAddr,
         long newRootId,
         long leftChildId,
         L row,
         byte[] rowBytes,
-        long rightChildId
+        long rightChildId,
+        int pageSize
     ) throws IgniteCheckedException {
-        initNewPage(newRootBuf, newRootId);
+        initNewPage(newRootPageAddr, newRootId, pageSize);
 
-        setCount(newRootBuf, 1);
-        setLeft(newRootBuf, 0, leftChildId);
-        store(newRootBuf, 0, row, rowBytes);
-        setRight(newRootBuf, 0, rightChildId);
+        setCount(newRootPageAddr, 1);
+        setLeft(newRootPageAddr, 0, leftChildId);
+        store(newRootPageAddr, 0, row, rowBytes);
+        setRight(newRootPageAddr, 0, rightChildId);
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusLeafIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusLeafIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusLeafIO.java
index 824c943..f3dccee 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusLeafIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusLeafIO.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.internal.processors.cache.database.tree.io;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.processors.cache.database.tree.util.PageHandler;
 
@@ -35,16 +34,16 @@ public abstract class BPlusLeafIO<L> extends BPlusIO<L> {
     }
 
     /** {@inheritDoc} */
-    @Override public int getMaxCount(ByteBuffer buf) {
-        return (buf.capacity() - ITEMS_OFF) / itemSize;
+    @Override public int getMaxCount(long pageAddr, int pageSize) {
+        return (pageSize - ITEMS_OFF) / itemSize;
     }
 
     /** {@inheritDoc} */
-    @Override public final void copyItems(ByteBuffer src, ByteBuffer dst, int srcIdx, int dstIdx, int cnt,
+    @Override public final void copyItems(long srcPageAddr, long dstPageAddr, int srcIdx, int dstIdx, int cnt,
         boolean cpLeft) throws IgniteCheckedException {
-        assert srcIdx != dstIdx || src != dst;
+        assert srcIdx != dstIdx || srcPageAddr != dstPageAddr;
 
-        PageHandler.copyMemory(src, dst, offset(srcIdx), offset(dstIdx), cnt * itemSize);
+        PageHandler.copyMemory(srcPageAddr, dstPageAddr, offset(srcIdx), offset(dstIdx), cnt * itemSize);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusMetaIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusMetaIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusMetaIO.java
index 15a49ef..5925547 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusMetaIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusMetaIO.java
@@ -17,7 +17,7 @@
 
 package org.apache.ignite.internal.processors.cache.database.tree.io;
 
-import java.nio.ByteBuffer;
+import org.apache.ignite.internal.pagemem.PageUtils;
 
 /**
  * IO routines for B+Tree meta pages.
@@ -37,45 +37,48 @@ public class BPlusMetaIO extends PageIO {
     /**
      * @param ver Page format version.
      */
-    protected BPlusMetaIO(int ver) {
+    private BPlusMetaIO(int ver) {
         super(T_BPLUS_META, ver);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAdrr Page address.
      * @param rootId Root page ID.
+     * @param pageSize Page size.
      */
-    public void initRoot(ByteBuffer buf, long rootId) {
-        setLevelsCount(buf, 1);
-        setFirstPageId(buf, 0, rootId);
+    public void initRoot(long pageAdrr, long rootId, int pageSize) {
+        setLevelsCount(pageAdrr, 1, pageSize);
+        setFirstPageId(pageAdrr, 0, rootId);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return Number of levels in this tree.
      */
-    public int getLevelsCount(ByteBuffer buf) {
-        return buf.get(LVLS_OFF);
+    public int getLevelsCount(long pageAddr) {
+        return PageUtils.getByte(pageAddr, LVLS_OFF);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
+     * @param pageSize Page size.
      * @return Max levels possible for this page size.
      */
-    public int getMaxLevels(ByteBuffer buf) {
-        return (buf.capacity() - REFS_OFF) / 8;
+    private int getMaxLevels(long pageAddr, int pageSize) {
+        return (pageSize - REFS_OFF) / 8;
     }
 
     /**
-     * @param buf  Buffer.
+     * @param pageAddr Page address.
      * @param lvls Number of levels in this tree.
+     * @param pageSize Page size.
      */
-    public void setLevelsCount(ByteBuffer buf, int lvls) {
-        assert lvls >= 0 && lvls <= getMaxLevels(buf): lvls;
+    private void setLevelsCount(long pageAddr, int lvls, int pageSize) {
+        assert lvls >= 0 && lvls <= getMaxLevels(pageAddr, pageSize) : lvls;
 
-        buf.put(LVLS_OFF, (byte)lvls);
+        PageUtils.putByte(pageAddr, LVLS_OFF, (byte)lvls);
 
-        assert getLevelsCount(buf) == lvls;
+        assert getLevelsCount(pageAddr) == lvls;
     }
 
     /**
@@ -87,33 +90,33 @@ public class BPlusMetaIO extends PageIO {
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param lvl Level.
      * @return First page ID at that level.
      */
-    public long getFirstPageId(ByteBuffer buf, int lvl) {
-        return buf.getLong(offset(lvl));
+    public long getFirstPageId(long pageAddr, int lvl) {
+        return PageUtils.getLong(pageAddr, offset(lvl));
     }
 
     /**
-     * @param buf    Buffer.
+     * @param pageAddr Page address.
      * @param lvl    Level.
      * @param pageId Page ID.
      */
-    public void setFirstPageId(ByteBuffer buf, int lvl, long pageId) {
-        assert lvl >= 0 && lvl < getLevelsCount(buf);
+    private void setFirstPageId(long pageAddr, int lvl, long pageId) {
+        assert lvl >= 0 && lvl < getLevelsCount(pageAddr);
 
-        buf.putLong(offset(lvl), pageId);
+        PageUtils.putLong(pageAddr, offset(lvl), pageId);
 
-        assert getFirstPageId(buf, lvl) == pageId;
+        assert getFirstPageId(pageAddr, lvl) == pageId;
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return Root level.
      */
-    public int getRootLevel(ByteBuffer buf) {
-        int lvls = getLevelsCount(buf); // The highest level page is root.
+    public int getRootLevel(long pageAddr) {
+        int lvls = getLevelsCount(pageAddr); // The highest level page is root.
 
         assert lvls > 0 : lvls;
 
@@ -121,22 +124,24 @@ public class BPlusMetaIO extends PageIO {
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param rootPageId New root page ID.
+     * @param pageSize Page size.
      */
-    public void addRoot(ByteBuffer buf, long rootPageId) {
-        int lvl = getLevelsCount(buf);
+    public void addRoot(long pageAddr, long rootPageId, int pageSize) {
+        int lvl = getLevelsCount(pageAddr);
 
-        setLevelsCount(buf, lvl + 1);
-        setFirstPageId(buf, lvl, rootPageId);
+        setLevelsCount(pageAddr, lvl + 1, pageSize);
+        setFirstPageId(pageAddr, lvl, rootPageId);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
+     * @param pageSize Page size.
      */
-    public void cutRoot(ByteBuffer buf) {
-        int lvl = getRootLevel(buf);
+    public void cutRoot(long pageAddr, int pageSize) {
+        int lvl = getRootLevel(pageAddr);
 
-        setLevelsCount(buf, lvl); // Decrease tree height.
+        setLevelsCount(pageAddr, lvl, pageSize); // Decrease tree height.
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/CacheVersionIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/CacheVersionIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/CacheVersionIO.java
index 5e9fd6d..8a630cc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/CacheVersionIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/CacheVersionIO.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.cache.database.tree.io;
 
 import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageUtils;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 
 /**
@@ -77,6 +78,29 @@ public class CacheVersionIO {
     }
 
     /**
+     * @param addr Write address.
+     * @param ver Version to write.
+     * @param allowNull Is {@code null} version allowed.
+     */
+    public static void write(long addr, GridCacheVersion ver, boolean allowNull) {
+        if (ver == null) {
+            if (allowNull)
+                PageUtils.putByte(addr, 0, NULL_PROTO_VER);
+            else
+                throw new IllegalStateException("Cache version is null");
+        }
+        else {
+            byte protoVer = 1; // Version of serialization protocol.
+
+            PageUtils.putByte(addr, 0, protoVer);
+            PageUtils.putInt(addr, 1, ver.topologyVersion());
+            PageUtils.putInt(addr, 5, ver.nodeOrderAndDrIdRaw());
+            PageUtils.putLong(addr, 9, ver.globalTime());
+            PageUtils.putLong(addr, 17, ver.order());
+        }
+    }
+
+    /**
      * @param protoVer Serialization protocol version.
      * @param allowNull Is {@code null} version allowed.
      * @return Protocol version.
@@ -139,4 +163,26 @@ public class CacheVersionIO {
 
         return new GridCacheVersion(topVer, nodeOrderDrId, globalTime, order);
     }
+
+    /**
+     * Reads GridCacheVersion instance from the given address.
+     *
+     * @param pageAddr Page address.
+     * @param allowNull Is {@code null} version allowed.
+     * @return Version.
+     * @throws IgniteCheckedException If failed.
+     */
+    public static GridCacheVersion read(long pageAddr, boolean allowNull) throws IgniteCheckedException {
+        byte protoVer = checkProtocolVersion(PageUtils.getByte(pageAddr, 0), allowNull);
+
+        if (protoVer == NULL_PROTO_VER)
+            return null;
+
+        int topVer = PageUtils.getInt(pageAddr, 1);
+        int nodeOrderDrId = PageUtils.getInt(pageAddr, 5);
+        long globalTime = PageUtils.getLong(pageAddr, 9);
+        long order = PageUtils.getLong(pageAddr, 17);
+
+        return new GridCacheVersion(topVer, nodeOrderDrId, globalTime, order);
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPageIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPageIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPageIO.java
index a69caab..548e300 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPageIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPageIO.java
@@ -23,6 +23,8 @@ import java.util.Collection;
 import java.util.HashSet;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.PageMemory;
+import org.apache.ignite.internal.pagemem.PageUtils;
 import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.database.CacheDataRow;
 import org.apache.ignite.internal.processors.cache.database.tree.util.PageHandler;
@@ -88,47 +90,48 @@ public class DataPageIO extends PageIO {
     }
 
     /** {@inheritDoc} */
-    @Override public void initNewPage(ByteBuffer buf, long pageId) {
-        super.initNewPage(buf, pageId);
+    @Override public void initNewPage(long pageAddr, long pageId, int pageSize) {
+        super.initNewPage(pageAddr, pageId, pageSize);
 
-        setEmptyPage(buf);
-        setFreeListPageId(buf, 0L);
+        setEmptyPage(pageAddr, pageSize);
+        setFreeListPageId(pageAddr, 0L);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
+     * @param pageSize Page size.
      */
-    private void setEmptyPage(ByteBuffer buf) {
-        setDirectCount(buf, 0);
-        setIndirectCount(buf, 0);
-        setFirstEntryOffset(buf, buf.capacity());
-        setRealFreeSpace(buf, buf.capacity() - ITEMS_OFF);
+    private void setEmptyPage(long pageAddr, int pageSize) {
+        setDirectCount(pageAddr, 0);
+        setIndirectCount(pageAddr, 0);
+        setFirstEntryOffset(pageAddr, pageSize, pageSize);
+        setRealFreeSpace(pageAddr, pageSize - ITEMS_OFF, pageSize);
     }
 
     /**
-     * @param buf Byte buffer.
+     * @param pageAddr Page address.
      * @param freeListPageId Free list page ID.
      */
-    public void setFreeListPageId(ByteBuffer buf, long freeListPageId) {
-        buf.putLong(FREE_LIST_PAGE_ID_OFF, freeListPageId);
+    public void setFreeListPageId(long pageAddr, long freeListPageId) {
+        PageUtils.putLong(pageAddr, FREE_LIST_PAGE_ID_OFF, freeListPageId);
     }
 
     /**
-     * @param buf Byte buffer.
+     * @param pageAddr Page address.
      * @return Free list page ID.
      */
-    public long getFreeListPageId(ByteBuffer buf) {
-        return buf.getLong(FREE_LIST_PAGE_ID_OFF);
+    public long getFreeListPageId(long pageAddr) {
+        return PageUtils.getLong(pageAddr, FREE_LIST_PAGE_ID_OFF);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param dataOff Data offset.
      * @param show What elements of data page entry to show in the result.
      * @return Data page entry size.
      */
-    private int getPageEntrySize(ByteBuffer buf, int dataOff, int show) {
-        int payloadLen = buf.getShort(dataOff) & 0xFFFF;
+    private int getPageEntrySize(long pageAddr, int dataOff, int show) {
+        int payloadLen = PageUtils.getShort(pageAddr, dataOff) & 0xFFFF;
 
         if ((payloadLen & FRAGMENTED_FLAG) != 0)
             payloadLen &= ~FRAGMENTED_FLAG; // We are fragmented and have a link.
@@ -144,7 +147,7 @@ public class DataPageIO extends PageIO {
      * @return Data page entry size.
      */
     private int getPageEntrySize(int payloadLen, int show) {
-        assert payloadLen > 0: payloadLen;
+        assert payloadLen > 0 : payloadLen;
 
         int res = payloadLen;
 
@@ -161,45 +164,47 @@ public class DataPageIO extends PageIO {
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param dataOff Entry data offset.
+     * @param pageSize Page size.
      */
-    private void setFirstEntryOffset(ByteBuffer buf, int dataOff) {
-        assert dataOff >= ITEMS_OFF + ITEM_SIZE && dataOff <= buf.capacity(): dataOff;
+    private void setFirstEntryOffset(long pageAddr, int dataOff, int pageSize) {
+        assert dataOff >= ITEMS_OFF + ITEM_SIZE && dataOff <= pageSize : dataOff;
 
-        buf.putShort(FIRST_ENTRY_OFF, (short)dataOff);
+        PageUtils.putShort(pageAddr, FIRST_ENTRY_OFF, (short)dataOff);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return Entry data offset.
      */
-    private int getFirstEntryOffset(ByteBuffer buf) {
-        return buf.getShort(FIRST_ENTRY_OFF) & 0xFFFF;
+    private int getFirstEntryOffset(long pageAddr) {
+        return PageUtils.getShort(pageAddr, FIRST_ENTRY_OFF) & 0xFFFF;
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param freeSpace Free space.
+     * @param pageSize Page size.
      */
-    private void setRealFreeSpace(ByteBuffer buf, int freeSpace) {
-        assert freeSpace == actualFreeSpace(buf): freeSpace + " != " + actualFreeSpace(buf);
+    private void setRealFreeSpace(long pageAddr, int freeSpace, int pageSize) {
+        assert freeSpace == actualFreeSpace(pageAddr, pageSize) : freeSpace + " != " + actualFreeSpace(pageAddr, pageSize);
 
-        buf.putShort(FREE_SPACE_OFF, (short)freeSpace);
+        PageUtils.putShort(pageAddr, FREE_SPACE_OFF, (short)freeSpace);
     }
 
     /**
      * Free space refers to a "max row size (without any data page specific overhead) which is
      * guaranteed to fit into this data page".
      *
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return Free space.
      */
-    public int getFreeSpace(ByteBuffer buf) {
-        if (getFreeItemSlots(buf) == 0)
+    public int getFreeSpace(long pageAddr) {
+        if (getFreeItemSlots(pageAddr) == 0)
             return 0;
 
-        int freeSpace = getRealFreeSpace(buf);
+        int freeSpace = getRealFreeSpace(pageAddr);
 
         // We reserve size here because of getFreeSpace() method semantics (see method javadoc).
         // It means that we must be able to accommodate a row of size which is equal to getFreeSpace(),
@@ -211,48 +216,49 @@ public class DataPageIO extends PageIO {
     }
 
     /**
+     * @param pageAddr Page address.
      * @return {@code true} If there is no useful data in this page.
      */
-    public boolean isEmpty(ByteBuffer buf) {
-        return getDirectCount(buf) == 0;
+    public boolean isEmpty(long pageAddr) {
+        return getDirectCount(pageAddr) == 0;
     }
 
     /**
-     * Equivalent for {@link #actualFreeSpace(ByteBuffer)} but reads saved value.
+     * Equivalent for {@link #actualFreeSpace(long, int)} but reads saved value.
      *
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return Free space.
      */
-    private int getRealFreeSpace(ByteBuffer buf) {
-        return buf.getShort(FREE_SPACE_OFF);
+    private int getRealFreeSpace(long pageAddr) {
+        return PageUtils.getShort(pageAddr, FREE_SPACE_OFF);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param cnt Direct count.
      */
-    private void setDirectCount(ByteBuffer buf, int cnt) {
+    private void setDirectCount(long pageAddr, int cnt) {
         assert checkCount(cnt): cnt;
 
-        buf.put(DIRECT_CNT_OFF, (byte)cnt);
+        PageUtils.putByte(pageAddr, DIRECT_CNT_OFF, (byte)cnt);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return Direct count.
      */
-    private int getDirectCount(ByteBuffer buf) {
-        return buf.get(DIRECT_CNT_OFF) & 0xFF;
+    private int getDirectCount(long pageAddr) {
+        return PageUtils.getByte(pageAddr, DIRECT_CNT_OFF) & 0xFF;
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param cnt Indirect count.
      */
-    private void setIndirectCount(ByteBuffer buf, int cnt) {
+    private void setIndirectCount(long pageAddr, int cnt) {
         assert checkCount(cnt): cnt;
 
-        buf.put(INDIRECT_CNT_OFF, (byte)cnt);
+        PageUtils.putByte(pageAddr, INDIRECT_CNT_OFF, (byte)cnt);
     }
 
     /**
@@ -272,36 +278,36 @@ public class DataPageIO extends PageIO {
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return Indirect count.
      */
-    private int getIndirectCount(ByteBuffer buf) {
-        return buf.get(INDIRECT_CNT_OFF) & 0xFF;
+    private int getIndirectCount(long pageAddr) {
+        return PageUtils.getByte(pageAddr, INDIRECT_CNT_OFF) & 0xFF;
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return Number of free entry slots.
      */
-    private int getFreeItemSlots(ByteBuffer buf) {
-        return 0xFF - getDirectCount(buf);
+    private int getFreeItemSlots(long pageAddr) {
+        return 0xFF - getDirectCount(pageAddr);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param itemId Fixed item ID (the index used for referencing an entry from the outside).
      * @param directCnt Direct items count.
      * @param indirectCnt Indirect items count.
      * @return Found index of indirect item.
      */
-    private int findIndirectItemIndex(ByteBuffer buf, int itemId, int directCnt, int indirectCnt) {
+    private int findIndirectItemIndex(long pageAddr, int itemId, int directCnt, int indirectCnt) {
         int low = directCnt;
         int high = directCnt + indirectCnt - 1;
 
         while (low <= high) {
             int mid = (low + high) >>> 1;
 
-            int cmp = Integer.compare(itemId(getItem(buf, mid)), itemId);
+            int cmp = Integer.compare(itemId(getItem(pageAddr, mid)), itemId);
 
             if (cmp < 0)
                 low = mid + 1;
@@ -315,19 +321,20 @@ public class DataPageIO extends PageIO {
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
+     * @param pageSize Page size.
      * @return String representation.
      */
-    public String printPageLayout(ByteBuffer buf) {
-        int directCnt = getDirectCount(buf);
-        int indirectCnt = getIndirectCount(buf);
-        int free = getRealFreeSpace(buf);
+    private String printPageLayout(long pageAddr, int pageSize) {
+        int directCnt = getDirectCount(pageAddr);
+        int indirectCnt = getIndirectCount(pageAddr);
+        int free = getRealFreeSpace(pageAddr);
 
         boolean valid = directCnt >= indirectCnt;
 
         SB b = new SB();
 
-        b.appendHex(PageIO.getPageId(buf)).a(" [");
+        b.appendHex(PageIO.getPageId(pageAddr)).a(" [");
 
         int entriesSize = 0;
 
@@ -335,12 +342,12 @@ public class DataPageIO extends PageIO {
             if (i != 0)
                 b.a(", ");
 
-            short item = getItem(buf, i);
+            short item = getItem(pageAddr, i);
 
-            if (item < ITEMS_OFF || item >= buf.capacity())
+            if (item < ITEMS_OFF || item >= pageSize)
                 valid = false;
 
-            entriesSize += getPageEntrySize(buf, item, SHOW_PAYLOAD_LEN | SHOW_LINK);
+            entriesSize += getPageEntrySize(pageAddr, item, SHOW_PAYLOAD_LEN | SHOW_LINK);
 
             b.a(item);
         }
@@ -353,7 +360,7 @@ public class DataPageIO extends PageIO {
             if (i != directCnt)
                 b.a(", ");
 
-            short item = getItem(buf, i);
+            short item = getItem(pageAddr, i);
 
             int itemId = itemId(item);
             int directIdx = directItemIndex(item);
@@ -366,7 +373,7 @@ public class DataPageIO extends PageIO {
             if (itemId < directCnt || directIdx < 0 || directIdx >= directCnt)
                 valid = false;
 
-            if (i > directCnt && itemId(getItem(buf, i - 1)) >= itemId)
+            if (i > directCnt && itemId(getItem(pageAddr, i - 1)) >= itemId)
                 valid = false;
 
 
@@ -375,7 +382,7 @@ public class DataPageIO extends PageIO {
 
         b.a("][free=").a(free);
 
-        int actualFree = buf.capacity() - ITEMS_OFF - (entriesSize + (directCnt + indirectCnt) * ITEM_SIZE);
+        int actualFree = pageSize - ITEMS_OFF - (entriesSize + (directCnt + indirectCnt) * ITEM_SIZE);
 
         if (free != actualFree) {
             b.a(", actualFree=").a(actualFree);
@@ -391,94 +398,94 @@ public class DataPageIO extends PageIO {
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param itemId Fixed item ID (the index used for referencing an entry from the outside).
+     * @param pageSize Page size.
      * @return Data entry offset in bytes.
      */
-    private int getDataOffset(ByteBuffer buf, int itemId) {
+    private int getDataOffset(long pageAddr, int itemId, int pageSize) {
         assert checkIndex(itemId): itemId;
 
-        int directCnt = getDirectCount(buf);
+        int directCnt = getDirectCount(pageAddr);
 
-        assert directCnt > 0: "itemId=" + itemId + ", directCnt=" + directCnt + ", page=" + printPageLayout(buf);
+        assert directCnt > 0: "itemId=" + itemId + ", directCnt=" + directCnt + ", page=" + printPageLayout(pageAddr, pageSize);
 
         if (itemId >= directCnt) { // Need to do indirect lookup.
-            int indirectCnt = getIndirectCount(buf);
+            int indirectCnt = getIndirectCount(pageAddr);
 
             // Must have indirect items here.
             assert indirectCnt > 0: "itemId=" + itemId + ", directCnt=" + directCnt + ", indirectCnt=" + indirectCnt +
-                ", page=" + printPageLayout(buf);
+                ", page=" + printPageLayout(pageAddr, pageSize);
 
-            int indirectItemIdx = findIndirectItemIndex(buf, itemId, directCnt, indirectCnt);
+            int indirectItemIdx = findIndirectItemIndex(pageAddr, itemId, directCnt, indirectCnt);
 
             assert indirectItemIdx >= directCnt : indirectItemIdx + " " + directCnt;
             assert indirectItemIdx < directCnt + indirectCnt: indirectItemIdx + " " + directCnt + " " + indirectCnt;
 
-            itemId = directItemIndex(getItem(buf, indirectItemIdx));
+            itemId = directItemIndex(getItem(pageAddr, indirectItemIdx));
 
             assert itemId >= 0 && itemId < directCnt: itemId + " " + directCnt + " " + indirectCnt; // Direct item.
         }
 
-        return directItemToOffset(getItem(buf, itemId));
+        return directItemToOffset(getItem(pageAddr, itemId));
     }
 
     /**
-     * @param buf Byte buffer.
+     * @param pageAddr Page address.
      * @param dataOff Points to the entry start.
      * @return Link to the next entry fragment or 0 if no fragments left or if entry is not fragmented.
      */
-    private long getNextFragmentLink(ByteBuffer buf, int dataOff) {
-        assert isFragmented(buf, dataOff);
+    private long getNextFragmentLink(long pageAddr, int dataOff) {
+        assert isFragmented(pageAddr, dataOff);
 
-        return buf.getLong(dataOff + PAYLOAD_LEN_SIZE);
+        return PageUtils.getLong(pageAddr, dataOff + PAYLOAD_LEN_SIZE);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param dataOff Data offset.
      * @return {@code true} If the data row is fragmented across multiple pages.
      */
-    private boolean isFragmented(ByteBuffer buf, int dataOff) {
-        return (buf.getShort(dataOff) & FRAGMENTED_FLAG) != 0;
+    private boolean isFragmented(long pageAddr, int dataOff) {
+        return (PageUtils.getShort(pageAddr, dataOff) & FRAGMENTED_FLAG) != 0;
     }
 
     /**
      * Sets position to start of actual fragment data and limit to it's end.
      *
-     * @param buf Byte buffer.
+     * @param pageAddr Page address.
      * @param itemId Item to position on.
+     * @param pageSize Page size.
      * @return Link to the next fragment or {@code 0} if it is the last fragment or the data row is not fragmented.
      */
-    public long setPositionAndLimitOnPayload(final ByteBuffer buf, final int itemId) {
-        int dataOff = getDataOffset(buf, itemId);
-
-        boolean fragmented = isFragmented(buf, dataOff);
-        long nextLink = fragmented ? getNextFragmentLink(buf, dataOff) : 0;
-        int payloadSize = getPageEntrySize(buf, dataOff, 0);
+    public DataPagePayload readPayload(final long pageAddr, final int itemId, final int pageSize) {
+        int dataOff = getDataOffset(pageAddr, itemId, pageSize);
 
-        buf.position(dataOff + PAYLOAD_LEN_SIZE + (fragmented ? LINK_SIZE : 0));
+        boolean fragmented = isFragmented(pageAddr, dataOff);
+        long nextLink = fragmented ? getNextFragmentLink(pageAddr, dataOff) : 0;
+        int payloadSize = getPageEntrySize(pageAddr, dataOff, 0);
 
-        buf.limit(buf.position() + payloadSize);
-
-        return nextLink;
+        return new DataPagePayload(dataOff + PAYLOAD_LEN_SIZE + (fragmented ? LINK_SIZE : 0),
+            payloadSize,
+            nextLink);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param idx Item index.
      * @return Item.
      */
-    private short getItem(ByteBuffer buf, int idx) {
-        return buf.getShort(itemOffset(idx));
+    private short getItem(long pageAddr, int idx) {
+        return PageUtils.getShort(pageAddr, itemOffset(idx));
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param idx Item index.
      * @param item Item.
      */
-    private void setItem(ByteBuffer buf, int idx, short item) {
-        buf.putShort(itemOffset(idx), item);
+    private void setItem(long pageAddr, int idx, short item) {
+        PageUtils.putShort(pageAddr, itemOffset(idx), item);
     }
 
     /**
@@ -540,14 +547,14 @@ public class DataPageIO extends PageIO {
     /**
      * Move the last direct item to the free slot and reference it with indirect item on the same place.
      *
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param freeDirectIdx Free slot.
      * @param directCnt Direct items count.
      * @param indirectCnt Indirect items count.
      * @return {@code true} If the last direct item already had corresponding indirect item.
      */
-    private boolean moveLastItem(ByteBuffer buf, int freeDirectIdx, int directCnt, int indirectCnt) {
-        int lastIndirectId = findIndirectIndexForLastDirect(buf, directCnt, indirectCnt);
+    private boolean moveLastItem(long pageAddr, int freeDirectIdx, int directCnt, int indirectCnt) {
+        int lastIndirectId = findIndirectIndexForLastDirect(pageAddr, directCnt, indirectCnt);
 
         int lastItemId = directCnt - 1;
 
@@ -557,13 +564,13 @@ public class DataPageIO extends PageIO {
 
         assert itemId(indirectItem) == lastItemId && directItemIndex(indirectItem) == freeDirectIdx;
 
-        setItem(buf, freeDirectIdx, getItem(buf, lastItemId));
-        setItem(buf, lastItemId, indirectItem);
+        setItem(pageAddr, freeDirectIdx, getItem(pageAddr, lastItemId));
+        setItem(pageAddr, lastItemId, indirectItem);
 
-        assert getItem(buf, lastItemId) == indirectItem;
+        assert getItem(pageAddr, lastItemId) == indirectItem;
 
         if (lastIndirectId != -1) { // Fix pointer to direct item.
-            setItem(buf, lastIndirectId, indirectItem(itemId(getItem(buf, lastIndirectId)), freeDirectIdx));
+            setItem(pageAddr, lastIndirectId, indirectItem(itemId(getItem(pageAddr, lastIndirectId)), freeDirectIdx));
 
             return true;
         }
@@ -572,16 +579,16 @@ public class DataPageIO extends PageIO {
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param directCnt Direct items count.
      * @param indirectCnt Indirect items count.
      * @return Index of indirect item for the last direct item.
      */
-    private int findIndirectIndexForLastDirect(ByteBuffer buf, int directCnt, int indirectCnt) {
+    private int findIndirectIndexForLastDirect(long pageAddr, int directCnt, int indirectCnt) {
         int lastDirectId = directCnt - 1;
 
         for (int i = directCnt, end = directCnt + indirectCnt; i < end; i++) {
-            short item = getItem(buf, i);
+            short item = getItem(pageAddr, i);
 
             if (directItemIndex(item) == lastDirectId)
                 return i;
@@ -591,20 +598,21 @@ public class DataPageIO extends PageIO {
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param itemId Fixed item ID (the index used for referencing an entry from the outside).
+     * @param pageSize Page size.
      * @return Next link for fragmented entries or {@code 0} if none.
      * @throws IgniteCheckedException If failed.
      */
-    public long removeRow(ByteBuffer buf, int itemId) throws IgniteCheckedException {
+    public long removeRow(long pageAddr, int itemId, int pageSize) throws IgniteCheckedException {
         assert checkIndex(itemId) : itemId;
 
-        final int dataOff = getDataOffset(buf, itemId);
-        final long nextLink = isFragmented(buf, dataOff) ? getNextFragmentLink(buf, dataOff) : 0;
+        final int dataOff = getDataOffset(pageAddr, itemId, pageSize);
+        final long nextLink = isFragmented(pageAddr, dataOff) ? getNextFragmentLink(pageAddr, dataOff) : 0;
 
         // Record original counts to calculate delta in free space in the end of remove.
-        final int directCnt = getDirectCount(buf);
-        final int indirectCnt = getIndirectCount(buf);
+        final int directCnt = getDirectCount(pageAddr);
+        final int indirectCnt = getIndirectCount(pageAddr);
 
         int curIndirectCnt = indirectCnt;
 
@@ -613,24 +621,24 @@ public class DataPageIO extends PageIO {
         // Remove the last item on the page.
         if (directCnt == 1) {
             assert (indirectCnt == 0 && itemId == 0) ||
-                (indirectCnt == 1 && itemId == itemId(getItem(buf, 1))) : itemId;
+                (indirectCnt == 1 && itemId == itemId(getItem(pageAddr, 1))) : itemId;
 
-            setEmptyPage(buf);
+            setEmptyPage(pageAddr, pageSize);
         }
         else {
             // Get the entry size before the actual remove.
-            int rmvEntrySize = getPageEntrySize(buf, dataOff, SHOW_PAYLOAD_LEN | SHOW_LINK);
+            int rmvEntrySize = getPageEntrySize(pageAddr, dataOff, SHOW_PAYLOAD_LEN | SHOW_LINK);
 
             int indirectId = 0;
 
             if (itemId >= directCnt) { // Need to remove indirect item.
                 assert indirectCnt > 0;
 
-                indirectId = findIndirectItemIndex(buf, itemId, directCnt, indirectCnt);
+                indirectId = findIndirectItemIndex(pageAddr, itemId, directCnt, indirectCnt);
 
                 assert indirectId >= directCnt;
 
-                itemId = directItemIndex(getItem(buf, indirectId));
+                itemId = directItemIndex(getItem(pageAddr, indirectId));
 
                 assert itemId < directCnt;
             }
@@ -638,48 +646,50 @@ public class DataPageIO extends PageIO {
             boolean dropLast = true;
 
             if (itemId + 1 < directCnt) // It is not the last direct item.
-                dropLast = moveLastItem(buf, itemId, directCnt, indirectCnt);
+                dropLast = moveLastItem(pageAddr, itemId, directCnt, indirectCnt);
 
             if (indirectId == 0) {// For the last direct item with no indirect item.
                 if (dropLast)
-                    moveItems(buf, directCnt, indirectCnt, -1);
+                    moveItems(pageAddr, directCnt, indirectCnt, -1, pageSize);
                 else
                     curIndirectCnt++;
             }
             else {
                 if (dropLast)
-                    moveItems(buf, directCnt, indirectId - directCnt, -1);
+                    moveItems(pageAddr, directCnt, indirectId - directCnt, -1, pageSize);
 
-                moveItems(buf, indirectId + 1, directCnt + indirectCnt - indirectId - 1, dropLast ? -2 : -1);
+                moveItems(pageAddr, indirectId + 1, directCnt + indirectCnt - indirectId - 1, dropLast ? -2 : -1, pageSize);
 
                 if (dropLast)
                     curIndirectCnt--;
             }
 
-            setIndirectCount(buf, curIndirectCnt);
-            setDirectCount(buf, directCnt - 1);
+            setIndirectCount(pageAddr, curIndirectCnt);
+            setDirectCount(pageAddr, directCnt - 1);
 
-            assert getIndirectCount(buf) <= getDirectCount(buf);
+            assert getIndirectCount(pageAddr) <= getDirectCount(pageAddr);
 
             // Increase free space.
-            setRealFreeSpace(buf, getRealFreeSpace(buf) + rmvEntrySize +
-                ITEM_SIZE * (directCnt - getDirectCount(buf) + indirectCnt - getIndirectCount(buf)));
+            setRealFreeSpace(pageAddr,
+                getRealFreeSpace(pageAddr) + rmvEntrySize + ITEM_SIZE * (directCnt - getDirectCount(pageAddr) + indirectCnt - getIndirectCount(pageAddr)),
+                pageSize);
         }
 
         return nextLink;
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param idx Index.
      * @param cnt Count.
      * @param step Step.
+     * @param pageSize Page size.
      */
-    private void moveItems(ByteBuffer buf, int idx, int cnt, int step) {
+    private void moveItems(long pageAddr, int idx, int cnt, int step, int pageSize) {
         assert cnt >= 0: cnt;
 
         if (cnt != 0)
-            moveBytes(buf, itemOffset(idx), cnt * ITEM_SIZE, step * ITEM_SIZE);
+            moveBytes(pageAddr, itemOffset(idx), cnt * ITEM_SIZE, step * ITEM_SIZE, pageSize);
     }
 
     /**
@@ -696,73 +706,80 @@ public class DataPageIO extends PageIO {
     /**
      * Adds row to this data page and sets respective link to the given row object.
      *
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param row Cache data row.
      * @param rowSize Row size.
+     * @param pageSize Page size.
      * @throws IgniteCheckedException If failed.
      */
     public void addRow(
-        ByteBuffer buf,
+        final long pageAddr,
         CacheDataRow row,
-        int rowSize
+        final int rowSize,
+        final int pageSize
     ) throws IgniteCheckedException {
-        assert rowSize <= getFreeSpace(buf): "can't call addRow if not enough space for the whole row";
+        assert rowSize <= getFreeSpace(pageAddr): "can't call addRow if not enough space for the whole row";
 
         int fullEntrySize = getPageEntrySize(rowSize, SHOW_PAYLOAD_LEN | SHOW_ITEM);
 
-        int directCnt = getDirectCount(buf);
-        int indirectCnt = getIndirectCount(buf);
+        int directCnt = getDirectCount(pageAddr);
+        int indirectCnt = getIndirectCount(pageAddr);
 
-        int dataOff = getDataOffsetForWrite(buf, fullEntrySize, directCnt, indirectCnt);
+        int dataOff = getDataOffsetForWrite(pageAddr, fullEntrySize, directCnt, indirectCnt, pageSize);
 
-        writeRowData(buf, dataOff, rowSize, row);
+        writeRowData(pageAddr, dataOff, rowSize, row);
 
-        int itemId = addItem(buf, fullEntrySize, directCnt, indirectCnt, dataOff);
+        int itemId = addItem(pageAddr, fullEntrySize, directCnt, indirectCnt, dataOff, pageSize);
 
-        setLink(row, buf, itemId);
+        setLink(row, pageAddr, itemId);
     }
 
     /**
      * Adds row to this data page and sets respective link to the given row object.
      *
-     * @param buf Buffer.
+     * @param pageAddr Page address.
+     * @param payload Payload.
+     * @param pageSize Page size.
      * @throws IgniteCheckedException If failed.
      */
     public void addRow(
-        ByteBuffer buf,
-        byte[] payload
+        long pageAddr,
+        byte[] payload,
+        int pageSize
     ) throws IgniteCheckedException {
-        assert payload.length <= getFreeSpace(buf): "can't call addRow if not enough space for the whole row";
+        assert payload.length <= getFreeSpace(pageAddr): "can't call addRow if not enough space for the whole row";
 
         int fullEntrySize = getPageEntrySize(payload.length, SHOW_PAYLOAD_LEN | SHOW_ITEM);
 
-        int directCnt = getDirectCount(buf);
-        int indirectCnt = getIndirectCount(buf);
+        int directCnt = getDirectCount(pageAddr);
+        int indirectCnt = getIndirectCount(pageAddr);
 
-        int dataOff = getDataOffsetForWrite(buf, fullEntrySize, directCnt, indirectCnt);
+        int dataOff = getDataOffsetForWrite(pageAddr, fullEntrySize, directCnt, indirectCnt, pageSize);
 
-        writeRowData(buf, dataOff, payload);
+        writeRowData(pageAddr, dataOff, payload);
 
-        addItem(buf, fullEntrySize, directCnt, indirectCnt, dataOff);
+        addItem(pageAddr, fullEntrySize, directCnt, indirectCnt, dataOff, pageSize);
     }
 
     /**
-     * @param buf Byte buffer.
+     * @param pageAddr Page address.
      * @param entryFullSize New entry full size (with item, length and link).
      * @param directCnt Direct items count.
      * @param indirectCnt Indirect items count.
      * @param dataOff First entry offset.
+     * @param pageSize Page size.
      * @return First entry offset after compaction.
      */
     private int compactIfNeed(
-        final ByteBuffer buf,
+        final long pageAddr,
         final int entryFullSize,
         final int directCnt,
         final int indirectCnt,
-        int dataOff
+        int dataOff,
+        int pageSize
     ) {
         if (!isEnoughSpace(entryFullSize, dataOff, directCnt, indirectCnt)) {
-            dataOff = compactDataEntries(buf, directCnt);
+            dataOff = compactDataEntries(pageAddr, directCnt, pageSize);
 
             assert isEnoughSpace(entryFullSize, dataOff, directCnt, indirectCnt);
         }
@@ -773,41 +790,49 @@ public class DataPageIO extends PageIO {
     /**
      * Put item reference on entry.
      *
-     * @param buf Byte buffer.
+     * @param pageAddr Page address.
      * @param fullEntrySize Full entry size (with link, payload size and item).
      * @param directCnt Direct items count.
      * @param indirectCnt Indirect items count.
      * @param dataOff Data offset.
+     * @param pageSize Page size.
      * @return Item ID.
      */
-    private int addItem(final ByteBuffer buf, final int fullEntrySize, final int directCnt,
-        final int indirectCnt, final int dataOff) {
-        setFirstEntryOffset(buf, dataOff);
+    private int addItem(final long pageAddr,
+        final int fullEntrySize,
+        final int directCnt,
+        final int indirectCnt,
+        final int dataOff,
+        final int pageSize)
+    {
+        setFirstEntryOffset(pageAddr, dataOff, pageSize);
 
-        int itemId = insertItem(buf, dataOff, directCnt, indirectCnt);
+        int itemId = insertItem(pageAddr, dataOff, directCnt, indirectCnt, pageSize);
 
         assert checkIndex(itemId): itemId;
-        assert getIndirectCount(buf) <= getDirectCount(buf);
+        assert getIndirectCount(pageAddr) <= getDirectCount(pageAddr);
 
         // Update free space. If number of indirect items changed, then we were able to reuse an item slot.
-        setRealFreeSpace(buf, getRealFreeSpace(buf) - fullEntrySize +
-            (getIndirectCount(buf) != indirectCnt ? ITEM_SIZE : 0));
+        setRealFreeSpace(pageAddr,
+            getRealFreeSpace(pageAddr) - fullEntrySize + (getIndirectCount(pageAddr) != indirectCnt ? ITEM_SIZE : 0),
+            pageSize);
 
         return itemId;
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param fullEntrySize Full entry size.
      * @param directCnt Direct items count.
      * @param indirectCnt Indirect items count.
+     * @param pageSize Page size.
      * @return Offset in the buffer where the entry must be written.
      */
-    private int getDataOffsetForWrite(ByteBuffer buf, int fullEntrySize, int directCnt, int indirectCnt) {
-        int dataOff = getFirstEntryOffset(buf);
+    private int getDataOffsetForWrite(long pageAddr, int fullEntrySize, int directCnt, int indirectCnt, int pageSize) {
+        int dataOff = getFirstEntryOffset(pageAddr);
 
         // Compact if we do not have enough space for entry.
-        dataOff = compactIfNeed(buf, fullEntrySize, directCnt, indirectCnt, dataOff);
+        dataOff = compactIfNeed(pageAddr, fullEntrySize, directCnt, indirectCnt, dataOff, pageSize);
 
         // We will write data right before the first entry.
         dataOff -= fullEntrySize - ITEM_SIZE;
@@ -818,91 +843,105 @@ public class DataPageIO extends PageIO {
     /**
      * Adds maximum possible fragment of the given row to this data page and sets respective link to the row.
      *
-     * @param buf Byte buffer.
+     * @param pageMem Page memory.
+     * @param pageAddr Page address.
      * @param row Cache data row.
      * @param written Number of bytes of row size that was already written.
      * @param rowSize Row size.
+     * @param pageSize Page size.
      * @return Written payload size.
      * @throws IgniteCheckedException If failed.
      */
     public int addRowFragment(
-        ByteBuffer buf,
+        PageMemory pageMem,
+        long pageAddr,
         CacheDataRow row,
         int written,
-        int rowSize
+        int rowSize,
+        int pageSize
     ) throws IgniteCheckedException {
-        return addRowFragment(buf, written, rowSize, row.link(), row, null);
+        return addRowFragment(pageMem, pageAddr, written, rowSize, row.link(), row, null, pageSize);
     }
 
     /**
      * Adds this payload as a fragment to this data page.
      *
-     * @param buf Byte buffer.
+     * @param pageAddr Page address.
      * @param payload Payload bytes.
      * @param lastLink Link to the previous written fragment (link to the tail).
+     * @param pageSize Page size.
      * @throws IgniteCheckedException If failed.
      */
     public void addRowFragment(
-        ByteBuffer buf,
+        long pageAddr,
         byte[] payload,
-        long lastLink
+        long lastLink,
+        int pageSize
     ) throws IgniteCheckedException {
-        addRowFragment(buf, 0, 0, lastLink, null, payload);
+        addRowFragment(null, pageAddr, 0, 0, lastLink, null, payload, pageSize);
     }
 
     /**
      * Adds maximum possible fragment of the given row to this data page and sets respective link to the row.
      *
-     * @param buf Byte buffer.
+     * @param pageMem Page memory.
+     * @param pageAddr Page address.
      * @param written Number of bytes of row size that was already written.
      * @param rowSize Row size.
      * @param lastLink Link to the previous written fragment (link to the tail).
      * @param row Row.
      * @param payload Payload bytes.
+     * @param pageSize Page size.
      * @return Written payload size.
      * @throws IgniteCheckedException If failed.
      */
     private int addRowFragment(
-        ByteBuffer buf,
+        PageMemory pageMem,
+        long pageAddr,
         int written,
         int rowSize,
         long lastLink,
         CacheDataRow row,
-        byte[] payload
+        byte[] payload,
+        int pageSize
     ) throws IgniteCheckedException {
         assert payload == null ^ row == null;
 
-        int directCnt = getDirectCount(buf);
-        int indirectCnt = getIndirectCount(buf);
+        int directCnt = getDirectCount(pageAddr);
+        int indirectCnt = getIndirectCount(pageAddr);
 
         int payloadSize = payload != null ? payload.length :
-            Math.min(rowSize - written, getFreeSpace(buf));
+            Math.min(rowSize - written, getFreeSpace(pageAddr));
 
         int fullEntrySize = getPageEntrySize(payloadSize, SHOW_PAYLOAD_LEN | SHOW_LINK | SHOW_ITEM);
-        int dataOff = getDataOffsetForWrite(buf, fullEntrySize, directCnt, indirectCnt);
+        int dataOff = getDataOffsetForWrite(pageAddr, fullEntrySize, directCnt, indirectCnt, pageSize);
+
+        if (payload == null) {
+            ByteBuffer buf = pageMem.pageBuffer(pageAddr);
 
-        try {
             buf.position(dataOff);
 
-            buf.putShort((short) (payloadSize | FRAGMENTED_FLAG));
+            short p = (short)(payloadSize | FRAGMENTED_FLAG);
+
+            buf.putShort(p);
             buf.putLong(lastLink);
 
-            if (payload == null) {
-                int rowOff = rowSize - written - payloadSize;
+            int rowOff = rowSize - written - payloadSize;
 
-                writeFragmentData(row, buf, rowOff, payloadSize);
-            }
-            else
-                buf.put(payload);
+            writeFragmentData(row, buf, rowOff, payloadSize);
         }
-        finally {
-            buf.position(0);
+        else {
+            PageUtils.putShort(pageAddr, dataOff, (short)(payloadSize | FRAGMENTED_FLAG));
+
+            PageUtils.putLong(pageAddr, dataOff + 2, lastLink);
+
+            PageUtils.putBytes(pageAddr, dataOff + 10, payload);
         }
 
-        int itemId = addItem(buf, fullEntrySize, directCnt, indirectCnt, dataOff);
+        int itemId = addItem(pageAddr, fullEntrySize, directCnt, indirectCnt, dataOff, pageSize);
 
         if (row != null)
-            setLink(row, buf, itemId);
+            setLink(row, pageAddr, itemId);
 
         return payloadSize;
     }
@@ -912,7 +951,7 @@ public class DataPageIO extends PageIO {
      * @param buf Page buffer.
      * @param itemId Item ID.
      */
-    private void setLink(CacheDataRow row, ByteBuffer buf, int itemId) {
+    private void setLink(CacheDataRow row, long buf, int itemId) {
         row.link(PageIdUtils.link(getPageId(buf), itemId));
     }
 
@@ -1083,54 +1122,56 @@ public class DataPageIO extends PageIO {
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param dataOff Data offset.
      * @param directCnt Direct items count.
      * @param indirectCnt Indirect items count.
+     * @param pageSize Page size.
      * @return Item ID (insertion index).
      */
-    private int insertItem(ByteBuffer buf, int dataOff, int directCnt, int indirectCnt) {
+    private int insertItem(long pageAddr, int dataOff, int directCnt, int indirectCnt, int pageSize) {
         if (indirectCnt > 0) {
             // If the first indirect item is on correct place to become the last direct item, do the transition
             // and insert the new item into the free slot which was referenced by this first indirect item.
-            short item = getItem(buf, directCnt);
+            short item = getItem(pageAddr, directCnt);
 
             if (itemId(item) == directCnt) {
                 int directItemIdx = directItemIndex(item);
 
-                setItem(buf, directCnt, getItem(buf, directItemIdx));
-                setItem(buf, directItemIdx, directItemFromOffset(dataOff));
+                setItem(pageAddr, directCnt, getItem(pageAddr, directItemIdx));
+                setItem(pageAddr, directItemIdx, directItemFromOffset(dataOff));
 
-                setDirectCount(buf, directCnt + 1);
-                setIndirectCount(buf, indirectCnt - 1);
+                setDirectCount(pageAddr, directCnt + 1);
+                setIndirectCount(pageAddr, indirectCnt - 1);
 
                 return directItemIdx;
             }
         }
 
         // Move all the indirect items forward to make a free slot and insert new item at the end of direct items.
-        moveItems(buf, directCnt, indirectCnt, +1);
+        moveItems(pageAddr, directCnt, indirectCnt, +1, pageSize);
 
-        setItem(buf, directCnt, directItemFromOffset(dataOff));
+        setItem(pageAddr, directCnt, directItemFromOffset(dataOff));
 
-        setDirectCount(buf, directCnt + 1);
-        assert getDirectCount(buf) == directCnt + 1;
+        setDirectCount(pageAddr, directCnt + 1);
+        assert getDirectCount(pageAddr) == directCnt + 1;
 
         return directCnt; // Previous directCnt will be our itemId.
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param directCnt Direct items count.
+     * @param pageSize Page size.
      * @return New first entry offset.
      */
-    private int compactDataEntries(ByteBuffer buf, int directCnt) {
+    private int compactDataEntries(long pageAddr, int directCnt, int pageSize) {
         assert checkCount(directCnt): directCnt;
 
         int[] offs = new int[directCnt];
 
         for (int i = 0; i < directCnt; i++) {
-            int off = directItemToOffset(getItem(buf, i));
+            int off = directItemToOffset(getItem(pageAddr, i));
 
             offs[i] = (off << 8) | i; // This way we'll be able to sort by offset using Arrays.sort(...).
         }
@@ -1138,27 +1179,27 @@ public class DataPageIO extends PageIO {
         Arrays.sort(offs);
 
         // Move right all of the entries if possible to make the page as compact as possible to its tail.
-        int prevOff = buf.capacity();
+        int prevOff = pageSize;
 
         for (int i = directCnt - 1; i >= 0; i--) {
             int off = offs[i] >>> 8;
 
             assert off < prevOff: off;
 
-            int entrySize = getPageEntrySize(buf, off, SHOW_PAYLOAD_LEN | SHOW_LINK);
+            int entrySize = getPageEntrySize(pageAddr, off, SHOW_PAYLOAD_LEN | SHOW_LINK);
 
             int delta = prevOff - (off + entrySize);
 
             if (delta != 0) { // Move right.
                 assert delta > 0: delta;
 
-                moveBytes(buf, off, entrySize, delta);
+                moveBytes(pageAddr, off, entrySize, delta, pageSize);
 
                 int itemId = offs[i] & 0xFF;
 
                 off += delta;
 
-                setItem(buf, itemId, directItemFromOffset(off));
+                setItem(pageAddr, itemId, directItemFromOffset(off));
             }
 
             prevOff = off;
@@ -1170,94 +1211,82 @@ public class DataPageIO extends PageIO {
     /**
      * Full-scan free space calculation procedure.
      *
-     * @param buf Buffer to scan.
+     * @param pageAddr Page to scan.
+     * @param pageSize Page size.
      * @return Actual free space in the buffer.
      */
-    private int actualFreeSpace(ByteBuffer buf) {
-        int directCnt = getDirectCount(buf);
+    private int actualFreeSpace(long pageAddr, int pageSize) {
+        int directCnt = getDirectCount(pageAddr);
 
         int entriesSize = 0;
 
         for (int i = 0; i < directCnt; i++) {
-            int off = directItemToOffset(getItem(buf, i));
+            int off = directItemToOffset(getItem(pageAddr, i));
 
-            int entrySize = getPageEntrySize(buf, off, SHOW_PAYLOAD_LEN | SHOW_LINK);
+            int entrySize = getPageEntrySize(pageAddr, off, SHOW_PAYLOAD_LEN | SHOW_LINK);
 
             entriesSize += entrySize;
         }
 
-        return buf.capacity() - ITEMS_OFF - entriesSize - (directCnt + getIndirectCount(buf)) * ITEM_SIZE;
+        return pageSize - ITEMS_OFF - entriesSize - (directCnt + getIndirectCount(pageAddr)) * ITEM_SIZE;
     }
 
     /**
-     * @param buf Buffer.
+     * @param addr Address.
      * @param off Offset.
      * @param cnt Count.
      * @param step Step.
+     * @param pageSize Page size.
      */
-    private void moveBytes(ByteBuffer buf, int off, int cnt, int step) {
+    private void moveBytes(long addr, int off, int cnt, int step, int pageSize) {
         assert step != 0: step;
         assert off + step >= 0;
-        assert off + step + cnt <= buf.capacity() : "[off=" + off + ", step=" + step + ", cnt=" + cnt +
-            ", cap=" + buf.capacity() + ']';
+        assert off + step + cnt <= pageSize : "[off=" + off + ", step=" + step + ", cnt=" + cnt +
+            ", cap=" + pageSize + ']';
 
-        PageHandler.copyMemory(buf, buf, off, off + step, cnt);
+        PageHandler.copyMemory(addr, addr, off, off + step, cnt);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param dataOff Data offset.
      * @param payloadSize Payload size.
      * @param row Data row.
      * @throws IgniteCheckedException If failed.
      */
     private void writeRowData(
-        ByteBuffer buf,
+        long pageAddr,
         int dataOff,
         int payloadSize,
         CacheDataRow row
     ) throws IgniteCheckedException {
-        try {
-            buf.position(dataOff);
+        long addr = pageAddr + dataOff;
 
-            buf.putShort((short)payloadSize);
+        PageUtils.putShort(addr, 0, (short)payloadSize);
+        addr += 2;
 
-            boolean ok = row.key().putValue(buf);
+        addr += row.key().putValue(addr);
+        addr += row.value().putValue(addr);
 
-            assert ok;
+        CacheVersionIO.write(addr, row.version(), false);
+        addr += CacheVersionIO.size(row.version(), false);
 
-            ok = row.value().putValue(buf);
-
-            assert ok;
-
-            CacheVersionIO.write(buf, row.version(), false);
-
-            buf.putLong(row.expireTime());
-        }
-        finally {
-            buf.position(0);
-        }
+        PageUtils.putLong(addr, 0, row.expireTime());
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param dataOff Data offset.
      * @param payload Payload
      */
     private void writeRowData(
-        ByteBuffer buf,
+        long pageAddr,
         int dataOff,
         byte[] payload
     ) {
-        try {
-            buf.position(dataOff);
-
-            buf.putShort((short)payload.length);
+        PageUtils.putShort(pageAddr, dataOff, (short)payload.length);
+        dataOff += 2;
 
-            buf.put(payload);
-        }
-        finally {
-            buf.position(0);
-        }
+        PageUtils.putBytes(pageAddr, dataOff, payload);
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPagePayload.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPagePayload.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPagePayload.java
new file mode 100644
index 0000000..7dedc00
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPagePayload.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.database.tree.io;
+
+/**
+ *
+ */
+public class DataPagePayload {
+    /** */
+    private final int off;
+
+    /** */
+    private final int payloadSize;
+
+    /** */
+    private final long nextLink;
+
+    /**
+     * @param off Offset.
+     * @param payloadSize Payload size.
+     * @param nextLink Next link.
+     */
+    DataPagePayload(int off, int payloadSize, long nextLink) {
+        this.off = off;
+        this.payloadSize = payloadSize;
+        this.nextLink = nextLink;
+    }
+
+    /**
+     * @return Offset.
+     */
+    public int offset() {
+        return off;
+    }
+
+    /**
+     * @return Payload size.
+     */
+    public int payloadSize() {
+        return payloadSize;
+    }
+
+    /**
+     * @return Next link.
+     */
+    public long nextLink() {
+        return nextLink;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/IOVersions.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/IOVersions.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/IOVersions.java
index bd1139d..428cb3f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/IOVersions.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/IOVersions.java
@@ -17,8 +17,6 @@
 
 package org.apache.ignite.internal.processors.cache.database.tree.io;
 
-import java.nio.ByteBuffer;
-
 /**
  * Registry for IO versions.
  */
@@ -80,16 +78,16 @@ public final class IOVersions<V extends PageIO> {
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return IO.
      */
-    public V forPage(ByteBuffer buf) {
-        int ver = PageIO.getVersion(buf);
+    public V forPage(long pageAddr) {
+        int ver = PageIO.getVersion(pageAddr);
 
         V res = forVersion(ver);
 
-        assert res.getType() == PageIO.getType(buf) : "resType=" + res.getType() +
-            ", pageType=" + PageIO.getType(buf);
+        assert res.getType() == PageIO.getType(pageAddr) : "resType=" + res.getType() +
+            ", pageType=" + PageIO.getType(pageAddr);
 
         return res;
     }