You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ib...@apache.org on 2022/01/24 09:35:11 UTC

[ignite-3] branch main updated: IGNITE-15817 Basic PageIO abstractions ported from Ignite 2.x (#566)

This is an automated email from the ASF dual-hosted git repository.

ibessonov pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/ignite-3.git


The following commit(s) were added to refs/heads/main by this push:
     new 13aa463  IGNITE-15817 Basic PageIO abstractions ported from Ignite 2.x (#566)
13aa463 is described below

commit 13aa4635f4b01967f1cee8e609618c3bce76a022
Author: ibessonov <be...@gmail.com>
AuthorDate: Mon Jan 24 12:35:06 2022 +0300

    IGNITE-15817 Basic PageIO abstractions ported from Ignite 2.x (#566)
---
 .../apache/ignite/internal/util/IgniteUtils.java   |  23 +
 modules/page-memory/pom.xml                        |  48 ++
 .../ignite/internal/pagememory/FullPageId.java     | 174 +++++++
 .../internal/pagememory/PageIdAllocator.java       |  63 +++
 .../ignite/internal/pagememory/PageMemory.java     |  65 +++
 .../ignite/internal/pagememory/PageSupport.java    | 128 ++++++
 .../ignite/internal/pagememory/io/IoVersions.java  | 128 ++++++
 .../ignite/internal/pagememory/io/PageIo.java      | 507 +++++++++++++++++++++
 .../internal/pagememory/io/PageIoModule.java       |  31 ++
 .../internal/pagememory/io/PageIoRegistry.java     | 100 ++++
 .../pagememory/metric/IoStatisticsHolder.java      |  48 ++
 .../ignite/internal/pagememory/reuse/ReuseBag.java |  38 ++
 .../internal/pagememory/reuse/ReuseList.java       |  64 +++
 .../internal/pagememory/util/PageHandler.java      | 443 ++++++++++++++++++
 .../internal/pagememory/util/PageIdUtils.java      | 265 +++++++++++
 .../internal/pagememory/util/PageLockListener.java |  86 ++++
 .../ignite/internal/pagememory/util/PageUtils.java | 282 ++++++++++++
 .../internal/pagememory/PageIoRegistryTest.java    |  68 +++
 .../internal/pagememory/TestPageIoModule.java      |  61 +++
 .../internal/pagememory/util/PageIdUtilsTest.java  | 147 ++++++
 ...ache.ignite.internal.pagememory.io.PageIoModule |  17 +
 pom.xml                                            |   1 +
 22 files changed, 2787 insertions(+)

diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
index aea480e..355c75b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
@@ -42,6 +42,7 @@ import java.util.concurrent.atomic.AtomicReference;
 import java.util.function.Predicate;
 import java.util.stream.Stream;
 import org.apache.ignite.lang.IgniteLogger;
+import org.apache.ignite.lang.IgniteStringBuilder;
 import org.apache.ignite.lang.IgniteStringFormatter;
 import org.jetbrains.annotations.Nullable;
 
@@ -262,6 +263,28 @@ public class IgniteUtils {
     }
 
     /**
+     * Returns a hex string representation of the given long value.
+     *
+     * @param val Value to convert to string.
+     * @return Hex string.
+     */
+    //TODO IGNITE-16350 Consider renaming or moving into other class.
+    public static String hexLong(long val) {
+        return new IgniteStringBuilder(16).appendHex(val).toString();
+    }
+
+    /**
+     * Returns a hex string representation of the given integer value.
+     *
+     * @param val Value to convert to string.
+     * @return Hex string.
+     */
+    //TODO IGNITE-16350 Consider renaming or moving into other class.
+    public static String hexInt(int val) {
+        return new IgniteStringBuilder(8).appendHex(val).toString();
+    }
+
+    /**
      * Gets absolute value for integer. If integer is {@link Integer#MIN_VALUE}, then {@code 0} is returned.
      *
      * @param i Integer.
diff --git a/modules/page-memory/pom.xml b/modules/page-memory/pom.xml
new file mode 100644
index 0000000..9785896
--- /dev/null
+++ b/modules/page-memory/pom.xml
@@ -0,0 +1,48 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  ~ 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.
+  -->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.ignite</groupId>
+        <artifactId>ignite-parent</artifactId>
+        <version>1</version>
+        <relativePath>../../parent/pom.xml</relativePath>
+    </parent>
+
+    <artifactId>ignite-page-memory</artifactId>
+    <version>3.0.0-SNAPSHOT</version>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-core</artifactId>
+        </dependency>
+
+        <!-- Test dependencies -->
+        <dependency>
+            <groupId>org.junit.jupiter</groupId>
+            <artifactId>junit-jupiter-engine</artifactId>
+            <scope>test</scope>
+        </dependency>
+    </dependencies>
+</project>
diff --git a/modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/FullPageId.java b/modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/FullPageId.java
new file mode 100644
index 0000000..10d973f
--- /dev/null
+++ b/modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/FullPageId.java
@@ -0,0 +1,174 @@
+/*
+ * 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.pagememory;
+
+import org.apache.ignite.internal.pagememory.util.PageIdUtils;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.apache.ignite.lang.IgniteStringBuilder;
+
+/**
+ * Compound object used to address a page in the global page space.
+ * <h3>Page ID structure</h3>
+ *
+ * <p>Generally, a full page ID consists of a group ID and a page ID. A page ID consists of a page index (32 bits), a partition ID (16 bits)
+ * and flags (8 bits). Group ID is an integer identifier of a logical pages group, like a SQL table or metadata storage, for example.
+ * Page index is a unique page identifier inside of a specific partition of a page group. Set of indexes in the partition represents
+ * a continuous range that starts with 0. Higher 8 bits of a page ID are reserved for addressing entries inside data pages or for
+ * page ID rotation.
+ *
+ * <p>Partition ID {@code 0xFFFF} is reserved for index pages.
+ *
+ * <p>The structure of a page ID is shown in the diagram below:
+ * <pre>
+ * +---------+-----------+------------+--------------------------+
+ * | 8 bits  |   8 bits  |  16 bits   |         32 bits          |
+ * +---------+-----------+------------+--------------------------+
+ * |  OFFSET |   FLAGS   |PARTITION ID|       PAGE INDEX         |
+ * +---------+-----------+------------+--------------------------+
+ * </pre>
+ *
+ * <h3>Page ID rotation</h3>
+ * There are scenarios when we reference one page (B) from within another page (A) by page ID. It is also
+ * possible that this first page (B) is concurrently reused for a different purpose. In this
+ * case we should have a mechanism to determine that the reference from page (A) to page (B) is no longer valid.
+ * This is ensured by page ID rotation - together with the ID of page B we write a value that is incremented
+ * each time a page is reused. This value should be verified after every page read and the page
+ * should be discarded if the full ID is different.
+ *
+ * <p>Effective page ID is a page ID with zeroed bits used for page ID rotation.
+ */
+public final class FullPageId {
+    /** Null page ID. */
+    public static final FullPageId NULL_PAGE = new FullPageId(-1, -1);
+
+    /** Page ID. */
+    private final long pageId;
+
+    /** Group ID. */
+    private final int groupId;
+
+    /**
+     * Constructor.
+     *
+     * @param pageId  Page ID.
+     * @param groupId Group ID.
+     */
+    public FullPageId(long pageId, int groupId) {
+        this.pageId = pageId;
+        this.groupId = groupId;
+    }
+
+    /**
+     * Returns a page ID.
+     */
+    public long pageId() {
+        return pageId;
+    }
+
+    /**
+     * Returns an effective page ID: pageId with only pageIdx and partitionId.
+     */
+    public long effectivePageId() {
+        return PageIdUtils.effectivePageId(pageId);
+    }
+
+    /**
+     * Returns a group ID.
+     */
+    public int groupId() {
+        return groupId;
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+
+        if (!(o instanceof FullPageId)) {
+            return false;
+        }
+
+        FullPageId that = (FullPageId) o;
+
+        return effectivePageId() == that.effectivePageId() && groupId == that.groupId;
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public int hashCode() {
+        return hashCode0(groupId, PageIdUtils.effectivePageId(pageId));
+    }
+
+    /**
+     * Returns hash code of the ID.
+     *
+     * @param groupId Group ID.
+     * @param pageId  Page ID.
+     * @return Hash code.
+     */
+    public static int hashCode(int groupId, long pageId) {
+        long effectiveId = PageIdUtils.effectivePageId(pageId);
+
+        return IgniteUtils.hash(hashCode0(groupId, effectiveId));
+    }
+
+    /**
+     * Will not clear link bits.
+     *
+     * @param groupId         Group ID.
+     * @param effectivePageId Effective page ID.
+     * @return Hash code.
+     */
+    private static int hashCode0(int groupId, long effectivePageId) {
+        return (int) (mix64(effectivePageId) ^ mix32(groupId));
+    }
+
+    /**
+     * MH3's plain finalization step.
+     */
+    private static int mix32(int k) {
+        k = (k ^ (k >>> 16)) * 0x85ebca6b;
+        k = (k ^ (k >>> 13)) * 0xc2b2ae35;
+
+        return k ^ (k >>> 16);
+    }
+
+    /**
+     * Computes David Stafford variant 9 of 64bit mix function (MH3 finalization step, with different shifts and constants).
+     *
+     * <p>Variant 9 is picked because it contains two 32-bit shifts which could be possibly optimized into better machine code.
+     *
+     * @see "http://zimbry.blogspot.com/2011/09/better-bit-mixing-improving-on.html"
+     */
+    private static long mix64(long z) {
+        z = (z ^ (z >>> 32)) * 0x4cd6944c5cc20b6dL;
+        z = (z ^ (z >>> 29)) * 0xfc12c5b19d3259e9L;
+
+        return z ^ (z >>> 32);
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public String toString() {
+        return new IgniteStringBuilder("FullPageId [pageId=").appendHex(pageId)
+                .app(", effectivePageId=").appendHex(effectivePageId())
+                .app(", groupId=").app(groupId).app(']').toString();
+    }
+}
diff --git a/modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/PageIdAllocator.java b/modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/PageIdAllocator.java
new file mode 100644
index 0000000..85449e8
--- /dev/null
+++ b/modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/PageIdAllocator.java
@@ -0,0 +1,63 @@
+/*
+ * 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.pagememory;
+
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+
+/**
+ * Class responsible for allocating/freeing page IDs.
+ */
+public interface PageIdAllocator {
+    /**
+     * Flag for a Data page. Also used by partition meta and tracking pages. This type doesn't use the Page ID rotation mechanism.
+     */
+    public static final byte FLAG_DATA = 1;
+
+    /**
+     * Flag for an internal structure page. This type uses the Page ID rotation mechanism.
+     */
+    public static final byte FLAG_AUX = 2;
+
+    /**
+     * Max partition ID that can be used by affinity.
+     */
+    // TODO IGNITE-16280 Use constant from the table configuration.
+    public static final int MAX_PARTITION_ID = 65500;
+
+    /**
+     * Special partition reserved for the index space.
+     */
+    public static final int INDEX_PARTITION = 0xFFFF;
+
+    /**
+     * Allocates a page from the space for the given partition ID and the given flags.
+     *
+     * @param groupId     Group ID.
+     * @param partitionId Partition ID.
+     * @return Allocated page ID.
+     */
+    public long allocatePage(int groupId, int partitionId, byte flags) throws IgniteInternalCheckedException;
+
+    /**
+     * Frees the given page.
+     *
+     * @param groupId Group ID.
+     * @param pageId  Page ID.
+     */
+    public boolean freePage(int groupId, long pageId);
+}
diff --git a/modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/PageMemory.java b/modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/PageMemory.java
new file mode 100644
index 0000000..88e63f3
--- /dev/null
+++ b/modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/PageMemory.java
@@ -0,0 +1,65 @@
+/*
+ * 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.pagememory;
+
+import java.nio.ByteBuffer;
+import org.apache.ignite.internal.pagememory.io.PageIo;
+import org.apache.ignite.internal.pagememory.io.PageIoRegistry;
+
+/**
+ * Class responsible for pages storage and handling.
+ */
+//TODO IGNITE-16350 Improve javadoc in this class.
+public interface PageMemory extends PageIdAllocator, PageSupport {
+    /**
+     * Returns a page's size in bytes.
+     */
+    int pageSize();
+
+    /**
+     * Returns a page size without the encryption overhead, in bytes.
+     *
+     * @param groupId Group id.
+     */
+    //TODO IGNITE-16350 Consider renaming.
+    int realPageSize(int groupId);
+
+    /**
+     * Returns a page's size with system overhead, in bytes.
+     */
+    //TODO IGNITE-16350 Consider renaming.
+    int systemPageSize();
+
+    /**
+     * Wraps a page address, obtained by a {@code readLock}/{@code writeLock} or their variants, into a direct byte buffer.
+     *
+     * @param pageAddr Page address.
+     * @return Page byte buffer.
+     */
+    ByteBuffer pageBuffer(long pageAddr);
+
+    /**
+     * Returns the total number of pages loaded into memory.
+     */
+    long loadedPages();
+
+    /**
+     * Returns a registry to obtain {@link PageIo} instances for pages.
+     */
+    public PageIoRegistry ioRegistry();
+}
diff --git a/modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/PageSupport.java b/modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/PageSupport.java
new file mode 100644
index 0000000..2eaa3c0
--- /dev/null
+++ b/modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/PageSupport.java
@@ -0,0 +1,128 @@
+/*
+ * 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.pagememory;
+
+import org.apache.ignite.internal.pagememory.metric.IoStatisticsHolder;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+
+/**
+ * Class responsible for acquiring/releasing and locking/unlocking pages.
+ */
+//TODO IGNITE-16350 Document a naming convention for "page" and "pageAddr" parameters.
+public interface PageSupport {
+    /**
+     * Returns an absolute pointer to a page, associated with the given page ID. Each pointer obtained with this method must be released by
+     * calling {@link #releasePage(int, long, long)}. This method will allocate a page with the given ID if it doesn't exist.
+     *
+     * @param groupId Group ID.
+     * @param pageId  Page ID.
+     * @return Page pointer.
+     * @throws IgniteInternalCheckedException If failed.
+     */
+    long acquirePage(int groupId, long pageId) throws IgniteInternalCheckedException;
+
+    /**
+     * Returns an absolute pointer to a page, associated with the given page ID. Each page obtained with this method must be released by
+     * calling {@link #releasePage(int, long, long)}. This method will allocate a page with the given ID if it doesn't exist.
+     *
+     * @param groupId    Group ID.
+     * @param pageId     Page ID.
+     * @param statHolder Statistics holder to track IO operations.
+     * @return Page pointer.
+     * @throws IgniteInternalCheckedException If failed.
+     */
+    long acquirePage(int groupId, long pageId, IoStatisticsHolder statHolder) throws IgniteInternalCheckedException;
+
+    /**
+     * Releases pages acquired by any of the {@code acquirePage} methods.
+     *
+     * @param groupId Group ID.
+     * @param pageId  Page ID to release.
+     * @param page    Page pointer returned by the corresponding {@code acquirePage} call.
+     */
+    void releasePage(int groupId, long pageId, long page);
+
+    /**
+     * Acquires a read lock associated with the given page.
+     *
+     * @param groupId Group ID.
+     * @param pageId  Page ID.
+     * @param page    Page pointer.
+     * @return Pointer for reading the page or {@code 0} if page has been reused.
+     */
+    long readLock(int groupId, long pageId, long page);
+
+    /**
+     * Acquires a read lock, associated with a given page, without checking the page tag.
+     *
+     * @param groupId Group ID.
+     * @param pageId  Page ID.
+     * @param page    Page pointer.
+     * @return Pointer for reading the page.
+     */
+    long readLockForce(int groupId, long pageId, long page);
+
+    /**
+     * Releases a read lock, associated with a given page.
+     *
+     * @param groupId Group ID.
+     * @param pageId  Page ID.
+     * @param page    Page pointer.
+     */
+    void readUnlock(int groupId, long pageId, long page);
+
+    /**
+     * Acquired a write lock on the page.
+     *
+     * @param groupId Group ID.
+     * @param pageId  Page ID.
+     * @param page    Page pointer.
+     * @return Address of a buffer with contents of the given page or {@code 0L} if attempt to take the write lock failed.
+     */
+    long writeLock(int groupId, long pageId, long page);
+
+    /**
+     * Tries to acquire a write lock on the page.
+     *
+     * @param groupId Group ID.
+     * @param pageId  Page ID.
+     * @param page    Page pointer.
+     * @return Address of a buffer with contents of the given page or {@code 0L} if attempt to take the write lock failed.
+     */
+    long tryWriteLock(int groupId, long pageId, long page);
+
+    /**
+     * Releases locked page.
+     *
+     * @param groupId   Group ID.
+     * @param pageId    Page ID.
+     * @param page      Page pointer.
+     * @param dirtyFlag Determines whether the page was modified since the last checkpoint.
+     */
+    void writeUnlock(int groupId, long pageId, long page, boolean dirtyFlag);
+
+    /**
+     * Checks whether the page is dirty.
+     *
+     * @param groupId Group ID.
+     * @param pageId  Page ID.
+     * @param page    Page pointer.
+     * @return {@code True} if the page is dirty.
+     */
+    boolean isDirty(int groupId, long pageId, long page);
+}
diff --git a/modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/io/IoVersions.java b/modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/io/IoVersions.java
new file mode 100644
index 0000000..5bf365e
--- /dev/null
+++ b/modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/io/IoVersions.java
@@ -0,0 +1,128 @@
+/*
+ * 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.pagememory.io;
+
+import org.apache.ignite.internal.tostring.S;
+
+/**
+ * Registry for IO versions of the same type.
+ */
+public final class IoVersions<V extends PageIo> {
+    /**
+     * Sorted array of IO objects.
+     */
+    private final V[] vers;
+
+    /**
+     * Page type.
+     */
+    private final int type;
+
+    /**
+     * Last element of {@link #vers} for faster access.
+     */
+    private final V latest;
+
+    /**
+     * Constructor.
+     *
+     * @param vers Array of IOs. All {@link PageIo#getType()} must match and all {@link PageIo#getVersion()} must continuously increase,
+     *            starting with {@code 1}.
+     */
+    @SafeVarargs
+    public IoVersions(V... vers) {
+        assert vers != null;
+        assert vers.length > 0;
+
+        this.vers = vers;
+        this.type = vers[0].getType();
+
+        latest = vers[vers.length - 1];
+
+        assert checkVersions();
+    }
+
+    /**
+     * Returns type of {@link PageIo}s.
+     */
+    public int getType() {
+        return type;
+    }
+
+    /**
+     * Checks versions array invariants.
+     *
+     * @return {@code true} If versions are correct.
+     */
+    private boolean checkVersions() {
+        for (int i = 0; i < vers.length; i++) {
+            V v = vers[i];
+
+            if (v.getType() != type || v.getVersion() != i + 1) {
+                return false;
+            }
+        }
+
+        return true;
+    }
+
+    /**
+     * Returns latest IO version.
+     */
+    public V latest() {
+        return latest;
+    }
+
+    /**
+     * Returns IO for the given version.
+     *
+     * @param ver Version.
+     * @return IO.
+     * @throws IllegalStateException If requested version is not found.
+     */
+    public V forVersion(int ver) {
+        if (ver == 0) {
+            throw new IllegalStateException("Failed to get page IO instance (page content is corrupted)");
+        }
+
+        return vers[ver - 1];
+    }
+
+    /**
+     * Returns IO for the given page.
+     *
+     * @param pageAddr Page address.
+     * @return IO.
+     * @throws IllegalStateException If requested version is not found.
+     */
+    public V forPage(long pageAddr) {
+        int ver = PageIo.getVersion(pageAddr);
+
+        V res = forVersion(ver);
+
+        assert res.getType() == PageIo.getType(pageAddr) : "resType=" + res.getType() + ", pageType=" + PageIo.getType(pageAddr);
+
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public String toString() {
+        return S.toString(IoVersions.class, this);
+    }
+}
diff --git a/modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/io/PageIo.java b/modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/io/PageIo.java
new file mode 100644
index 0000000..45ce03f
--- /dev/null
+++ b/modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/io/PageIo.java
@@ -0,0 +1,507 @@
+/*
+ * 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.pagememory.io;
+
+import java.nio.ByteBuffer;
+import org.apache.ignite.internal.pagememory.util.PageHandler;
+import org.apache.ignite.internal.pagememory.util.PageIdUtils;
+import org.apache.ignite.internal.pagememory.util.PageUtils;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+
+/**
+ * Base format for all page types.
+ *
+ * <p>Checklist for {@code PageIo} implementations and usage (The Rules):
+ *
+ * <ol>
+ *     <li>
+ *         IO should not have any {@code public static} methods.
+ *
+ *         IO implementations are versioned and static methods make it difficult to implement correctly in a backward-compatible way.
+ *         The base {@link PageIo} class has some static methods (like {@link #getPageId(long)}) intentionally:
+ *         this base format can not be changed between versions;
+ *     </li>
+ *     <li>
+ *         IO must correctly override {@link #initNewPage(long, long, int)} method and call the super method;
+ *     </li>
+ *     <li>
+ *         Always keep in mind that IOs are versioned and their format can change from version to version. In this respect it is a good
+ *         practice to avoid exposing details of the internal format through the API. The API should be minimalistic and abstract,
+ *         so the internal format in future IO versions can be changed without any changes to the API of this page IO;
+ *     </li>
+ *     <li>
+ *         Page IO API should not have any version dependent semantics and should not change API semantics in newer versions;
+ *     </li>
+ *     <li>
+ *         It is almost always preferable to read or write (especially write) page contents using static methods declared in
+ *         {@link PageHandler}. To initialize a new page use {@link PageHandler#initPage} method with a corresponding IO instance.
+ *     </li>
+ * </ol>
+ */
+public abstract class PageIo {
+    /**
+     * Maximum value for page type.
+     */
+    public static final int MAX_IO_TYPE = 65535 - 1;
+
+    /**
+     * Offset for "short" page type.
+     */
+    public static final int TYPE_OFF = 0;
+
+    /**
+     * Offset for "short" page version.
+     */
+    public static final int VER_OFF = TYPE_OFF + Short.BYTES;
+
+    /**
+     * Offset for "int" CRC.
+     */
+    public static final int CRC_OFF = VER_OFF + Short.BYTES;
+
+    /**
+     * Offset for "long" page ID.
+     */
+    public static final int PAGE_ID_OFF = CRC_OFF + Integer.BYTES;
+
+    /**
+     * Offset for "byte" rotated ID.
+     */
+    public static final int ROTATED_ID_PART_OFF = PAGE_ID_OFF + Long.BYTES;
+
+    /**
+     * Offset for "byte" compression type.
+     */
+    private static final int COMPRESSION_TYPE_OFF = ROTATED_ID_PART_OFF + Byte.BYTES;
+
+    /**
+     * Offset for "short" compressed size.
+     */
+    private static final int COMPRESSED_SIZE_OFF = COMPRESSION_TYPE_OFF + Byte.BYTES;
+
+    /**
+     * Offset for "short" compacted size.
+     */
+    private static final int COMPACTED_SIZE_OFF = COMPRESSED_SIZE_OFF + Short.BYTES;
+
+    /**
+     * Offset for reserved "short" value.
+     */
+    private static final int RESERVED_SHORT_OFF = COMPACTED_SIZE_OFF + Short.BYTES;
+
+    /**
+     * Offset for reserved "long" value.
+     */
+    private static final int RESERVED_2_OFF = RESERVED_SHORT_OFF + Short.BYTES;
+
+    /**
+     * Offset for reserved "long" value.
+     */
+    private static final int RESERVED_3_OFF = RESERVED_2_OFF + Long.BYTES;
+
+    /**
+     * Total size of common header, including reserved bytes.
+     */
+    public static final int COMMON_HEADER_END = RESERVED_3_OFF + Long.BYTES;
+
+    /**
+     * IO version.
+     */
+    private final int ver;
+
+    /**
+     * IO type.
+     */
+    private final int type;
+
+    /**
+     * Constructor.
+     *
+     * @param type Page type.
+     * @param ver  Page format version.
+     */
+    protected PageIo(int type, int ver) {
+        assert ver > 0 && ver <= MAX_IO_TYPE : ver;
+        assert type > 0 && type <= MAX_IO_TYPE : type;
+
+        this.type = type;
+        this.ver = ver;
+    }
+
+    /**
+     * Returns a type.
+     */
+    public final int getType() {
+        return type;
+    }
+
+    /**
+     * Returns a page type.
+     *
+     * @param buf Buffer.
+     * @return Page type.
+     */
+    public static int getType(ByteBuffer buf) {
+        return buf.getShort(TYPE_OFF) & 0xFFFF;
+    }
+
+    /**
+     * Returns a page type.
+     *
+     * @param pageAddr Page address.
+     * @return Page type.
+     */
+    public static int getType(long pageAddr) {
+        return PageUtils.getShort(pageAddr, TYPE_OFF) & 0xFFFF;
+    }
+
+    /**
+     * Sets the type to the page.
+     *
+     * @param pageAddr Page address.
+     * @param type     Type.
+     */
+    public static void setType(long pageAddr, int type) {
+        PageUtils.putShort(pageAddr, TYPE_OFF, (short) type);
+
+        assert getType(pageAddr) == type : getType(pageAddr);
+    }
+
+    /**
+     * Returns a version.
+     */
+    public final int getVersion() {
+        return ver;
+    }
+
+    /**
+     * Returns a page version.
+     *
+     * @param buf Buffer.
+     * @return Version.
+     */
+    public static int getVersion(ByteBuffer buf) {
+        return buf.getShort(VER_OFF) & 0xFFFF;
+    }
+
+    /**
+     * Returns a page version.
+     *
+     * @param pageAddr Page address.
+     * @return Version.
+     */
+    public static int getVersion(long pageAddr) {
+        return PageUtils.getShort(pageAddr, VER_OFF) & 0xFFFF;
+    }
+
+    /**
+     * Sets the version to the page.
+     *
+     * @param pageAddr Page address.
+     * @param ver      Version.
+     */
+    protected static void setVersion(long pageAddr, int ver) {
+        PageUtils.putShort(pageAddr, VER_OFF, (short) ver);
+
+        assert getVersion(pageAddr) == ver;
+    }
+
+    /**
+     * Returns a page ID.
+     *
+     * @param buf Buffer.
+     * @return Page ID.
+     */
+    public static long getPageId(ByteBuffer buf) {
+        return buf.getLong(PAGE_ID_OFF);
+    }
+
+    /**
+     * Returns a page ID.
+     *
+     * @param pageAddr Page address.
+     * @return Page ID.
+     */
+    public static long getPageId(long pageAddr) {
+        return PageUtils.getLong(pageAddr, PAGE_ID_OFF);
+    }
+
+    /**
+     * Sets the page ID to the page.
+     *
+     * @param pageAddr Page address.
+     * @param pageId   Page ID.
+     */
+    public static void setPageId(long pageAddr, long pageId) {
+        PageUtils.putLong(pageAddr, PAGE_ID_OFF, pageId);
+
+        assert getPageId(pageAddr) == pageId;
+    }
+
+    /**
+     * Returns a rotated ID.
+     *
+     * @param pageAddr Page address.
+     * @return Rotated page ID part.
+     */
+    public static int getRotatedIdPart(long pageAddr) {
+        return PageUtils.getUnsignedByte(pageAddr, ROTATED_ID_PART_OFF);
+    }
+
+    /**
+     * Sets the rotated ID to the page.
+     *
+     * @param pageAddr      Page address.
+     * @param rotatedIdPart Rotated page ID part.
+     */
+    public static void setRotatedIdPart(long pageAddr, int rotatedIdPart) {
+        PageUtils.putUnsignedByte(pageAddr, ROTATED_ID_PART_OFF, rotatedIdPart);
+
+        assert getRotatedIdPart(pageAddr) == rotatedIdPart;
+    }
+
+    /**
+     * Sets the compression type to the page.
+     *
+     * @param page         Page buffer.
+     * @param compressType Compression type.
+     */
+    public static void setCompressionType(ByteBuffer page, byte compressType) {
+        page.put(COMPRESSION_TYPE_OFF, compressType);
+    }
+
+    /**
+     * Returns a compression type.
+     *
+     * @param page Page buffer.
+     * @return Compression type.
+     */
+    public static byte getCompressionType(ByteBuffer page) {
+        return page.get(COMPRESSION_TYPE_OFF);
+    }
+
+    /**
+     * Returns a compression type.
+     *
+     * @param pageAddr Page address.
+     * @return Compression type.
+     */
+    public static byte getCompressionType(long pageAddr) {
+        return PageUtils.getByte(pageAddr, COMPRESSION_TYPE_OFF);
+    }
+
+    /**
+     * Sets the compressed size to the page.
+     *
+     * @param page           Page buffer.
+     * @param compressedSize Compressed size.
+     */
+    public static void setCompressedSize(ByteBuffer page, short compressedSize) {
+        page.putShort(COMPRESSED_SIZE_OFF, compressedSize);
+    }
+
+    /**
+     * Returns a compressed size.
+     *
+     * @param page Page buffer.
+     * @return Compressed size.
+     */
+    public static short getCompressedSize(ByteBuffer page) {
+        return page.getShort(COMPRESSED_SIZE_OFF);
+    }
+
+    /**
+     * Returns a compressed size.
+     *
+     * @param pageAddr Page address.
+     * @return Compressed size.
+     */
+    public static short getCompressedSize(long pageAddr) {
+        return PageUtils.getShort(pageAddr, COMPRESSED_SIZE_OFF);
+    }
+
+    /**
+     * Sets the compacted size to the page.
+     *
+     * @param page          Page buffer.
+     * @param compactedSize Compacted size.
+     */
+    public static void setCompactedSize(ByteBuffer page, short compactedSize) {
+        page.putShort(COMPACTED_SIZE_OFF, compactedSize);
+    }
+
+    /**
+     * Returns a compacted size.
+     *
+     * @param page Page buffer.
+     * @return Compacted size.
+     */
+    public static short getCompactedSize(ByteBuffer page) {
+        return page.getShort(COMPACTED_SIZE_OFF);
+    }
+
+    /**
+     * Returns a compacted size.
+     *
+     * @param pageAddr Page address.
+     * @return Compacted size.
+     */
+    public static short getCompactedSize(long pageAddr) {
+        return PageUtils.getShort(pageAddr, COMPACTED_SIZE_OFF);
+    }
+
+    /**
+     * Sets the CRC value to the page.
+     *
+     * @param buf Buffer.
+     * @param crc Checksum.
+     */
+    public static void setCrc(ByteBuffer buf, int crc) {
+        buf.putInt(CRC_OFF, crc);
+    }
+
+    /**
+     * Sets the CRC value to the page.
+     *
+     * @param pageAddr Page address.
+     * @param crc      Checksum.
+     */
+    public static void setCrc(long pageAddr, int crc) {
+        PageUtils.putInt(pageAddr, CRC_OFF, crc);
+    }
+
+    /**
+     * Returns a CRC value.
+     *
+     * @param buf Buffer.
+     * @return Checksum.
+     */
+    public static int getCrc(ByteBuffer buf) {
+        return buf.getInt(CRC_OFF);
+    }
+
+    /**
+     * Returns a CRC value.
+     *
+     * @param pageAddr Page address.
+     * @return Checksum.
+     */
+    public static int getCrc(long pageAddr) {
+        return PageUtils.getInt(pageAddr, CRC_OFF);
+    }
+
+    /**
+     * Initializes a new page.
+     *
+     * @param pageAddr Page address.
+     * @param pageId   Page ID.
+     * @param pageSize Page size.
+     */
+    public void initNewPage(long pageAddr, long pageId, int pageSize) {
+        setType(pageAddr, getType());
+        setVersion(pageAddr, getVersion());
+        setPageId(pageAddr, pageId);
+        setCrc(pageAddr, 0);
+
+        // rotated(1) + compress_type(1) + compressed_size(2) + compacted_size(2) + reserved(2)
+        PageUtils.putLong(pageAddr, ROTATED_ID_PART_OFF, 0L);
+        PageUtils.putLong(pageAddr, RESERVED_2_OFF, 0L);
+        PageUtils.putLong(pageAddr, RESERVED_3_OFF, 0L);
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public String toString() {
+        return getClass().getSimpleName() + "[ver=" + getVersion() + "]";
+    }
+
+    /**
+     * Copies a page into the output {@link ByteBuffer}.
+     *
+     * @param page     Page.
+     * @param out      Output buffer.
+     * @param pageSize Page size.
+     */
+    protected final void copyPage(ByteBuffer page, ByteBuffer out, int pageSize) {
+        assert out.position() == 0;
+        assert pageSize <= out.remaining();
+        assert pageSize == page.remaining();
+
+        PageUtils.copyMemory(page, 0, out, 0, pageSize);
+        out.limit(pageSize);
+    }
+
+    /**
+     * Prints a page into the output {@link StringBuilder}.
+     *
+     * @param addr     Address.
+     * @param pageSize Page size.
+     * @param sb       Sb.
+     */
+    protected abstract void printPage(long addr, int pageSize, StringBuilder sb) throws IgniteInternalCheckedException;
+
+    /**
+     * Returns a String representation of pages content.
+     *
+     * @param pageAddr Address.
+     */
+    public static String printPage(PageIoRegistry pageIoRegistry, long pageAddr, int pageSize) {
+        StringBuilder sb = new StringBuilder("Header [\n\ttype=");
+
+        try {
+            PageIo io = pageIoRegistry.resolve(pageAddr);
+
+            sb.append(getType(pageAddr))
+                    .append(" (").append(io.getClass().getSimpleName())
+                    .append("),\n\tver=").append(getVersion(pageAddr)).append(",\n\tcrc=").append(getCrc(pageAddr))
+                    .append(",\n\t").append(PageIdUtils.toDetailString(getPageId(pageAddr)))
+                    .append("\n],\n");
+
+            if (getCompressionType(pageAddr) != 0) {
+                sb.append("CompressedPage[\n\tcompressionType=").append(getCompressionType(pageAddr))
+                        .append(",\n\tcompressedSize=").append(getCompressedSize(pageAddr))
+                        .append(",\n\tcompactedSize=").append(getCompactedSize(pageAddr))
+                        .append("\n]");
+            } else {
+                io.printPage(pageAddr, pageSize, sb);
+            }
+        } catch (IgniteInternalCheckedException e) {
+            sb.append("Failed to print page: ").append(e.getMessage());
+        }
+
+        return sb.toString();
+    }
+
+    /**
+     * Asserts that page type of the page stored at pageAddr matches page type of this PageIO.
+     *
+     * @param pageAddr address of a page to use for assertion
+     */
+    protected final void assertPageType(long pageAddr) {
+        assert getType(pageAddr) == getType() : "Expected type " + getType() + ", but got " + getType(pageAddr);
+    }
+
+    /**
+     * Asserts that page type of the page stored in the given buffer matches page type of this PageIO.
+     *
+     * @param buf buffer where the page for assertion is stored
+     */
+    protected final void assertPageType(ByteBuffer buf) {
+        assert getType(buf) == getType() : "Expected type " + getType() + ", but got " + getType(buf);
+    }
+}
diff --git a/modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/io/PageIoModule.java b/modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/io/PageIoModule.java
new file mode 100644
index 0000000..03f8ca1
--- /dev/null
+++ b/modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/io/PageIoModule.java
@@ -0,0 +1,31 @@
+/*
+ * 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.pagememory.io;
+
+import java.util.Collection;
+
+/**
+ * Extension point for modules to provide their own {@link PageIo} implementations by implementing this interface and exporting
+ * it via the {@code META-INF/services} resource.
+ */
+public interface PageIoModule {
+    /**
+     * Returns a collection of {@link IoVersions} instances for all page IO types declared in a module.
+     */
+    Collection<IoVersions<?>> ioVersions();
+}
diff --git a/modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/io/PageIoRegistry.java b/modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/io/PageIoRegistry.java
new file mode 100644
index 0000000..c158072
--- /dev/null
+++ b/modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/io/PageIoRegistry.java
@@ -0,0 +1,100 @@
+/*
+ * 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.pagememory.io;
+
+import java.nio.ByteBuffer;
+import java.util.ServiceLoader;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+
+/**
+ * Page IO Registry. This component registers and provides all available {@link PageIo} types.
+ */
+public class PageIoRegistry {
+    /**
+     * Arrays of {@link IoVersions} for fast access. Element 0 is reserved.
+     */
+    private final IoVersions<?>[] ioVersions = new IoVersions[PageIo.MAX_IO_TYPE + 1];
+
+    /**
+     * Loads all {@link IoVersions} from a {@link PageIoModule} using the {@link ServiceLoader} mechanism.
+     *
+     * @throws IllegalStateException If there's an invalid page type or several different {@link IoVersions} instances for the same type.
+     */
+    public void loadFromServiceLoader() {
+        ServiceLoader<PageIoModule> serviceLoader = ServiceLoader.load(PageIoModule.class);
+
+        for (PageIoModule pageIoModule : serviceLoader) {
+            for (IoVersions<?> ios : pageIoModule.ioVersions()) {
+                if (ios.getType() == 0) {
+                    throw new IllegalStateException("Type 0 is reserved and can't be used: " + ios);
+                }
+
+                if (ioVersions[ios.getType()] != null && !ioVersions[ios.getType()].equals(ios)) {
+                    throw new IllegalStateException("Duplicated IOVersions found: " + ios);
+                }
+
+                ioVersions[ios.getType()] = ios;
+            }
+        }
+    }
+
+    /**
+     * Returns resolved {@link PageIo} by the {@link ByteBuffer} that contains the page.
+     *
+     * @param pageBuf Byte buffer with page content.
+     * @return Resolved page IO instance.
+     * @throws IgniteInternalCheckedException If page type or version are invalid or not registered.
+     */
+    public PageIo resolve(ByteBuffer pageBuf) throws IgniteInternalCheckedException {
+        return resolve(PageIo.getType(pageBuf), PageIo.getVersion(pageBuf));
+    }
+
+
+    /**
+     * Returns resolved {@link PageIo} by the page address.
+     *
+     * @param pageAddr Memory address pointing to the page content.
+     * @return Resolved page IO instance.
+     * @throws IgniteInternalCheckedException If page type or version are invalid or not registered.
+     */
+    public final PageIo resolve(long pageAddr) throws IgniteInternalCheckedException {
+        return resolve(PageIo.getType(pageAddr), PageIo.getVersion(pageAddr));
+    }
+
+    /**
+     * Returns resolved {@link PageIo} by the type and the version.
+     *
+     * @param type Page IO type.
+     * @param ver  Page IO version.
+     * @return Resolved page IO instance.
+     * @throws IgniteInternalCheckedException If page type or version are invalid or not registered.
+     */
+    public PageIo resolve(int type, int ver) throws IgniteInternalCheckedException {
+        if (type <= 0 || type > PageIo.MAX_IO_TYPE) {
+            throw new IgniteInternalCheckedException("Unknown page IO type: " + type);
+        }
+
+        IoVersions<?> ios = ioVersions[type];
+
+        if (ios == null) {
+            throw new IgniteInternalCheckedException("Unknown page IO type: " + type);
+        }
+
+        return ios.forVersion(ver);
+    }
+}
diff --git a/modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/metric/IoStatisticsHolder.java b/modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/metric/IoStatisticsHolder.java
new file mode 100644
index 0000000..3d0ca3a
--- /dev/null
+++ b/modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/metric/IoStatisticsHolder.java
@@ -0,0 +1,48 @@
+/*
+ * 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.pagememory.metric;
+
+/**
+ * Holder of IO statistics.
+ */
+public interface IoStatisticsHolder {
+    /**
+     * Track logical read of given page.
+     *
+     * @param pageAddr Address of page.
+     */
+    public void trackLogicalRead(long pageAddr);
+
+    /**
+     * Track physical and logical read of given page.
+     *
+     * @param pageAddr start address of page.
+     */
+    public void trackPhysicalAndLogicalRead(long pageAddr);
+
+    /**
+     * Returns a number of logical reads.
+     */
+    public long logicalReads();
+
+    /**
+     * Returns a number of physical reads.
+     */
+    public long physicalReads();
+}
diff --git a/modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/reuse/ReuseBag.java b/modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/reuse/ReuseBag.java
new file mode 100644
index 0000000..0c000b0
--- /dev/null
+++ b/modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/reuse/ReuseBag.java
@@ -0,0 +1,38 @@
+/*
+ * 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.pagememory.reuse;
+
+/**
+ * Reuse bag for free pages.
+ */
+public interface ReuseBag {
+    /**
+     * Adds a free page ID for reuse.
+     */
+    void addFreePage(long pageId);
+
+    /**
+     * Returns a free page ID for reuse or {@code 0} if empty.
+     */
+    long pollFreePage();
+
+    /**
+     * Returns {@code true} if bag has no free page IDs for reuse.
+     */
+    boolean isEmpty();
+}
diff --git a/modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/reuse/ReuseList.java b/modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/reuse/ReuseList.java
new file mode 100644
index 0000000..458c5ae
--- /dev/null
+++ b/modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/reuse/ReuseList.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.pagememory.reuse;
+
+import org.apache.ignite.internal.pagememory.FullPageId;
+import org.apache.ignite.internal.pagememory.PageIdAllocator;
+import org.apache.ignite.internal.pagememory.io.PageIo;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Reuse list.
+ */
+public interface ReuseList {
+    /**
+     * Polls all pages from reuse bag and puts them into reuse list.
+     *
+     * @param bag Reuse bag.
+     * @throws IgniteInternalCheckedException If failed.
+     */
+    void addForRecycle(ReuseBag bag) throws IgniteInternalCheckedException;
+
+    /**
+     * Returns a page ID for reuse or {@code 0} if none available.
+     *
+     * @throws IgniteInternalCheckedException If failed.
+     */
+    long takeRecycledPage() throws IgniteInternalCheckedException;
+
+    /**
+     * Returns a number of recycled pages it contains.
+     *
+     * @throws IgniteInternalCheckedException If failed.
+     */
+    long recycledPagesCount() throws IgniteInternalCheckedException;
+
+    /**
+     * Converts recycled page id back to a usable id. Might modify page content as well if flag is changing.
+     *
+     * @param pageId Id of the recycled page.
+     * @param flag   Flag value for the page. One of {@link PageIdAllocator#FLAG_DATA} or {@link PageIdAllocator#FLAG_AUX}.
+     * @param initIo Page IO to reinit reused page.
+     * @return Updated page id.
+     * @throws IgniteInternalCheckedException If failed.
+     *
+     * @see FullPageId
+     */
+    long initRecycledPage(long pageId, byte flag, @Nullable PageIo initIo) throws IgniteInternalCheckedException;
+}
diff --git a/modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/util/PageHandler.java b/modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/util/PageHandler.java
new file mode 100644
index 0000000..77dd0ab
--- /dev/null
+++ b/modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/util/PageHandler.java
@@ -0,0 +1,443 @@
+/*
+ * 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.pagememory.util;
+
+import static java.lang.Boolean.FALSE;
+
+import org.apache.ignite.internal.pagememory.PageMemory;
+import org.apache.ignite.internal.pagememory.io.PageIo;
+import org.apache.ignite.internal.pagememory.metric.IoStatisticsHolder;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+
+/**
+ * Page handler.
+ *
+ * @param <X> Type of the arbitrary parameter.
+ * @param <R> Type of the result.
+ */
+public interface PageHandler<X, R> {
+    /** No-op page handler. */
+    public static final PageHandler<Void, Boolean> NO_OP = (groupId, pageId, page, pageAddr, io, arg, intArg, statHolder) -> Boolean.TRUE;
+
+    /**
+     * Handles the page.
+     *
+     * @param groupId    Group ID.
+     * @param pageId     Page ID.
+     * @param page       Page pointer.
+     * @param pageAddr   Page address.
+     * @param io         IO.
+     * @param arg        Argument.
+     * @param intArg     Argument of type {@code int}.
+     * @param statHolder Statistics holder to track IO operations.
+     * @return Result.
+     * @throws IgniteInternalCheckedException If failed.
+     */
+    public R run(
+            int groupId,
+            long pageId,
+            long page,
+            long pageAddr,
+            PageIo io,
+            X arg,
+            int intArg,
+            IoStatisticsHolder statHolder
+    ) throws IgniteInternalCheckedException;
+
+    /**
+     * Checks whether write lock (and acquiring if applicable) should be released after handling.
+     *
+     * @param groupId  Group ID.
+     * @param pageId   Page ID.
+     * @param page     Page pointer.
+     * @param pageAddr Page address.
+     * @param arg      Argument.
+     * @param intArg   Argument of type {@code int}.
+     * @return {@code true} If release.
+     */
+    default boolean releaseAfterWrite(
+            int groupId,
+            long pageId,
+            long page,
+            long pageAddr,
+            X arg,
+            int intArg
+    ) {
+        return true;
+    }
+
+    /**
+     * Executes handler under the read lock or returns {@code lockFailed} if lock failed.
+     *
+     * @param pageMem    Page memory.
+     * @param groupId    Group ID.
+     * @param pageId     Page ID.
+     * @param lsnr       Lock listener.
+     * @param h          Handler.
+     * @param arg        Argument.
+     * @param intArg     Argument of type {@code int}.
+     * @param lockFailed Result in case of lock failure due to page recycling.
+     * @param statHolder Statistics holder to track IO operations.
+     * @return Handler result.
+     * @throws IgniteInternalCheckedException If failed.
+     */
+    public static <X, R> R readPage(
+            PageMemory pageMem,
+            int groupId,
+            long pageId,
+            PageLockListener lsnr,
+            PageHandler<X, R> h,
+            X arg,
+            int intArg,
+            R lockFailed,
+            IoStatisticsHolder statHolder
+    ) throws IgniteInternalCheckedException {
+        long page = pageMem.acquirePage(groupId, pageId, statHolder);
+
+        try {
+            return readPage(pageMem, groupId, pageId, page, lsnr, h, arg, intArg, lockFailed, statHolder);
+        } finally {
+            pageMem.releasePage(groupId, pageId, page);
+        }
+    }
+
+    /**
+     * Executes handler under the read lock or returns {@code lockFailed} if lock failed. Page must already be acquired.
+     *
+     * @param pageMem    Page memory.
+     * @param groupId    Group ID.
+     * @param pageId     Page ID.
+     * @param page       Page pointer.
+     * @param lsnr       Lock listener.
+     * @param h          Handler.
+     * @param arg        Argument.
+     * @param intArg     Argument of type {@code int}.
+     * @param lockFailed Result in case of lock failure due to page recycling.
+     * @param statHolder Statistics holder to track IO operations.
+     * @return Handler result.
+     * @throws IgniteInternalCheckedException If failed.
+     */
+    public static <X, R> R readPage(
+            PageMemory pageMem,
+            int groupId,
+            long pageId,
+            long page,
+            PageLockListener lsnr,
+            PageHandler<X, R> h,
+            X arg,
+            int intArg,
+            R lockFailed,
+            IoStatisticsHolder statHolder
+    ) throws IgniteInternalCheckedException {
+        long pageAddr = 0L;
+
+        try {
+            if ((pageAddr = readLock(pageMem, groupId, pageId, page, lsnr)) == 0L) {
+                return lockFailed;
+            }
+
+            PageIo io = pageMem.ioRegistry().resolve(pageAddr);
+
+            return h.run(groupId, pageId, page, pageAddr, io, arg, intArg, statHolder);
+        } finally {
+            if (pageAddr != 0L) {
+                readUnlock(pageMem, groupId, pageId, page, pageAddr, lsnr);
+            }
+        }
+    }
+
+    /**
+     * Acquires the read lock on the page.
+     *
+     * @param pageMem Page memory.
+     * @param groupId Group ID.
+     * @param pageId  Page ID.
+     * @param page    Page pointer.
+     * @param lsnr    Lock listener.
+     * @return Page address or {@code 0} if acquiring failed.
+     */
+    public static long readLock(
+            PageMemory pageMem,
+            int groupId,
+            long pageId,
+            long page,
+            PageLockListener lsnr
+    ) {
+        lsnr.onBeforeReadLock(groupId, pageId, page);
+
+        long pageAddr = pageMem.readLock(groupId, pageId, page);
+
+        lsnr.onReadLock(groupId, pageId, page, pageAddr);
+
+        return pageAddr;
+    }
+
+    /**
+     * Releases acquired read lock.
+     *
+     * @param pageMem  Page memory.
+     * @param groupId  Group ID.
+     * @param pageId   Page ID.
+     * @param page     Page pointer.
+     * @param pageAddr Page address.
+     * @param lsnr     Lock listener.
+     */
+    public static void readUnlock(
+            PageMemory pageMem,
+            int groupId,
+            long pageId,
+            long page,
+            long pageAddr,
+            PageLockListener lsnr
+    ) {
+        lsnr.onReadUnlock(groupId, pageId, page, pageAddr);
+
+        pageMem.readUnlock(groupId, pageId, page);
+    }
+
+    /**
+     * Initializes a new page.
+     *
+     * @param pageMem    Page memory.
+     * @param groupId    Group ID.
+     * @param pageId     Page ID.
+     * @param init       IO for new page initialization.
+     * @param lsnr       Lock listener.
+     * @param statHolder Statistics holder to track IO operations.
+     * @throws IgniteInternalCheckedException If failed.
+     * @see PageIo#initNewPage(long, long, int)
+     */
+    public static void initPage(
+            PageMemory pageMem,
+            int groupId,
+            long pageId,
+            PageIo init,
+            PageLockListener lsnr,
+            IoStatisticsHolder statHolder
+    ) throws IgniteInternalCheckedException {
+        Boolean res = writePage(
+                pageMem,
+                groupId,
+                pageId,
+                lsnr,
+                PageHandler.NO_OP,
+                init,
+                null,
+                0,
+                FALSE,
+                statHolder
+        );
+
+        assert res != FALSE;
+    }
+
+    /**
+     * Executes handler under the write lock or returns {@code lockFailed} if lock failed.
+     *
+     * @param pageMem    Page memory.
+     * @param groupId    Group ID.
+     * @param pageId     Page ID.
+     * @param lsnr       Lock listener.
+     * @param h          Handler.
+     * @param init       IO for new page initialization or {@code null} if it is an existing page.
+     * @param arg        Argument.
+     * @param intArg     Argument of type {@code int}.
+     * @param lockFailed Result in case of lock failure due to page recycling.
+     * @param statHolder Statistics holder to track IO operations.
+     * @return Handler result.
+     * @throws IgniteInternalCheckedException If failed.
+     */
+    //TODO IGNITE-16350 Consider splitting into two separate methods for init and regular locking.
+    public static <X, R> R writePage(
+            PageMemory pageMem,
+            int groupId,
+            final long pageId,
+            PageLockListener lsnr,
+            PageHandler<X, R> h,
+            PageIo init,
+            X arg,
+            int intArg,
+            R lockFailed,
+            IoStatisticsHolder statHolder
+    ) throws IgniteInternalCheckedException {
+        boolean releaseAfterWrite = true;
+        long page = pageMem.acquirePage(groupId, pageId, statHolder);
+        try {
+            long pageAddr = writeLock(pageMem, groupId, pageId, page, lsnr, false);
+
+            if (pageAddr == 0L) {
+                return lockFailed;
+            }
+
+            boolean ok = false;
+
+            try {
+                if (init != null) {
+                    // It is a new page and we have to initialize it.
+                    doInitPage(pageMem, groupId, pageId, pageAddr, init);
+                } else {
+                    init = pageMem.ioRegistry().resolve(pageAddr);
+                }
+
+                R res = h.run(groupId, pageId, page, pageAddr, init, arg, intArg, statHolder);
+
+                ok = true;
+
+                return res;
+            } finally {
+                assert PageIo.getCrc(pageAddr) == 0;
+
+                if (releaseAfterWrite = h.releaseAfterWrite(groupId, pageId, page, pageAddr, arg, intArg)) {
+                    writeUnlock(pageMem, groupId, pageId, page, pageAddr, lsnr, ok);
+                }
+            }
+        } finally {
+            if (releaseAfterWrite) {
+                pageMem.releasePage(groupId, pageId, page);
+            }
+        }
+    }
+
+    /**
+     * Executes handler under the write lock or returns {@code lockFailed} if lock failed. Page must already be acquired.
+     *
+     * @param pageMem    Page memory.
+     * @param groupId    Group ID.
+     * @param pageId     Page ID.
+     * @param page       Page pointer.
+     * @param lsnr       Lock listener.
+     * @param h          Handler.
+     * @param init       IO for new page initialization or {@code null} if it is an existing page.
+     * @param arg        Argument.
+     * @param intArg     Argument of type {@code int}.
+     * @param lockFailed Result in case of lock failure due to page recycling.
+     * @param statHolder Statistics holder to track IO operations.
+     * @return Handler result.
+     * @throws IgniteInternalCheckedException If failed.
+     */
+    public static <X, R> R writePage(
+            PageMemory pageMem,
+            int groupId,
+            long pageId,
+            long page,
+            PageLockListener lsnr,
+            PageHandler<X, R> h,
+            PageIo init,
+            X arg,
+            int intArg,
+            R lockFailed,
+            IoStatisticsHolder statHolder
+    ) throws IgniteInternalCheckedException {
+        long pageAddr = writeLock(pageMem, groupId, pageId, page, lsnr, false);
+
+        if (pageAddr == 0L) {
+            return lockFailed;
+        }
+
+        boolean ok = false;
+
+        try {
+            if (init != null) {
+                // It is a new page and we have to initialize it.
+                doInitPage(pageMem, groupId, pageId, pageAddr, init);
+            } else {
+                init = pageMem.ioRegistry().resolve(pageAddr);
+            }
+
+            R res = h.run(groupId, pageId, page, pageAddr, init, arg, intArg, statHolder);
+
+            ok = true;
+
+            return res;
+        } finally {
+            assert PageIo.getCrc(pageAddr) == 0;
+
+            if (h.releaseAfterWrite(groupId, pageId, page, pageAddr, arg, intArg)) {
+                writeUnlock(pageMem, groupId, pageId, page, pageAddr, lsnr, ok);
+            }
+        }
+    }
+
+    /**
+     * Acquires the write lock on the page.
+     *
+     * @param pageMem Page memory.
+     * @param groupId Group ID.
+     * @param pageId  Page ID.
+     * @param page    Page pointer.
+     * @param lsnr    Lock listener.
+     * @param tryLock Only try to lock without waiting.
+     * @return Page address or {@code 0} if failed to lock due to recycling.
+     */
+    public static long writeLock(
+            PageMemory pageMem,
+            int groupId,
+            long pageId,
+            long page,
+            PageLockListener lsnr,
+            boolean tryLock
+    ) {
+        lsnr.onBeforeWriteLock(groupId, pageId, page);
+
+        long pageAddr = tryLock ? pageMem.tryWriteLock(groupId, pageId, page) : pageMem.writeLock(groupId, pageId, page);
+
+        lsnr.onWriteLock(groupId, pageId, page, pageAddr);
+
+        return pageAddr;
+    }
+
+    /**
+     * Releases acquired write lock.
+     *
+     * @param pageMem  Page memory.
+     * @param groupId  Group ID.
+     * @param pageId   Page ID.
+     * @param page     Page pointer.
+     * @param pageAddr Page address.
+     * @param lsnr     Lock listener.
+     * @param dirty    Page is dirty.
+     */
+    public static void writeUnlock(
+            PageMemory pageMem,
+            int groupId,
+            long pageId,
+            long page,
+            long pageAddr,
+            PageLockListener lsnr,
+            boolean dirty
+    ) {
+        lsnr.onWriteUnlock(groupId, pageId, page, pageAddr);
+
+        pageMem.writeUnlock(groupId, pageId, page, dirty);
+    }
+
+    /**
+     * Invokes {@link PageIo#initNewPage(long, long, int)} and does additional checks.
+     */
+    private static void doInitPage(
+            PageMemory pageMem,
+            int groupId,
+            long pageId,
+            long pageAddr,
+            PageIo init
+    ) throws IgniteInternalCheckedException {
+        assert PageIo.getCrc(pageAddr) == 0;
+
+        init.initNewPage(pageAddr, pageId, pageMem.realPageSize(groupId));
+    }
+}
diff --git a/modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/util/PageIdUtils.java b/modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/util/PageIdUtils.java
new file mode 100644
index 0000000..5c5a7a0
--- /dev/null
+++ b/modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/util/PageIdUtils.java
@@ -0,0 +1,265 @@
+/*
+ * 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.pagememory.util;
+
+import org.apache.ignite.internal.pagememory.PageIdAllocator;
+import org.apache.ignite.internal.util.IgniteUtils;
+
+/**
+ * Utility class for page ID parts manipulation.
+ *
+ * @see FullPageId
+ */
+public final class PageIdUtils {
+    /** Size of the page index portion. */
+    public static final int PAGE_IDX_SIZE = Integer.SIZE;
+
+    /** Size of the partition ID portion. */
+    public static final int PART_ID_SIZE = Short.SIZE;
+
+    /** Size of the flags portion. */
+    public static final int FLAG_SIZE = Byte.SIZE;
+
+    /** Size of the offset portion. */
+    public static final int OFFSET_SIZE = Byte.SIZE;
+
+    /** Size of a tag portion. */
+    public static final int TAG_SIZE = 2 * Byte.SIZE;
+
+    /** Page index mask. */
+    public static final long PAGE_IDX_MASK = ~(-1L << PAGE_IDX_SIZE);
+
+    /** Offset mask. */
+    public static final long OFFSET_MASK = ~(-1L << OFFSET_SIZE);
+
+    /** Tag mask. */
+    public static final long TAG_MASK = ~(-1L << TAG_SIZE);
+
+    /** Page Index is a monotonically growing number within each partition. */
+    public static final long PART_ID_MASK = ~(-1L << PART_ID_SIZE);
+
+    /** Flags mask. Flags consists of a number of reserved bits, and page type (data/index page). */
+    public static final long FLAG_MASK = ~(-1L << FLAG_SIZE);
+
+    /** Effective page ID mask. */
+    private static final long EFFECTIVE_PAGE_ID_MASK = ~(-1L << (PAGE_IDX_SIZE + PART_ID_SIZE));
+
+    /**
+     * Offset of a Rotation ID inside a Page ID.
+     */
+    private static final long ROTATION_ID_OFFSET = PAGE_IDX_SIZE + PART_ID_SIZE + FLAG_SIZE;
+
+    /** Page ID mask that excludes link. */
+    private static final long PAGE_ID_MASK = ~(-1L << ROTATION_ID_OFFSET);
+
+    /** Max itemid number. */
+    public static final int MAX_ITEMID_NUM = 0xFE;
+
+    /** Maximum page number. */
+    public static final long MAX_PAGE_NUM = (1L << PAGE_IDX_SIZE) - 1;
+
+    /** Maximum page number. */
+    public static final int MAX_PART_ID = (1 << PART_ID_SIZE) - 1;
+
+    /** Private constructor. */
+    private PageIdUtils() {
+        // No-op.
+    }
+
+    /**
+     * Constructs a page link by the given page ID and 8-byte words within the page.
+     *
+     * @param pageId Page ID.
+     * @param itemId Item ID.
+     * @return Page link.
+     */
+    public static long link(long pageId, int itemId) {
+        assert itemId >= 0 && itemId <= MAX_ITEMID_NUM : itemId;
+        assert (pageId >> ROTATION_ID_OFFSET) == 0 : IgniteUtils.hexLong(pageId);
+
+        return pageId | (((long) itemId) << ROTATION_ID_OFFSET);
+    }
+
+    /**
+     * Extracts a page index from the given page ID.
+     *
+     * @param pageId Page ID.
+     * @return Page index.
+     */
+    public static int pageIndex(long pageId) {
+        return (int) (pageId & PAGE_IDX_MASK); // 4 bytes
+    }
+
+    /**
+     * Extracts a page ID from the given page link.
+     *
+     * @param link Page link.
+     * @return Page ID.
+     */
+    public static long pageId(long link) {
+        return flag(link) == PageIdAllocator.FLAG_DATA ? link & PAGE_ID_MASK : link;
+    }
+
+    /**
+     * Creates page ID from its components.
+     *
+     * @param partitionId Partition ID.
+     * @param flag        Flags (a number of reserved bits, and page type (data/index page))
+     * @param pageIdx     Page index, monotonically growing number within each partition
+     * @return Page ID constructed from the given pageIdx and partition ID, see {@link FullPageId}
+     */
+    public static long pageId(int partitionId, byte flag, int pageIdx) {
+        long pageId = flag & FLAG_MASK;
+
+        pageId = (pageId << PART_ID_SIZE) | (partitionId & PART_ID_MASK);
+        pageId = (pageId << (PAGE_IDX_SIZE)) | (pageIdx & PAGE_IDX_MASK);
+
+        return pageId;
+    }
+
+    /**
+     * Converts link into an effective page ID: pageId with only pageIdx and partitionId.
+     *
+     * @param link Page link.
+     * @return Effective page id.
+     */
+    public static long effectivePageId(long link) {
+        return link & EFFECTIVE_PAGE_ID_MASK;
+    }
+
+    /**
+     * Checks whether page ID matches effective page ID.
+     *
+     * @param pageId Page id.
+     * @return {@code True} if page id is equal to effective page id.
+     */
+    public static boolean isEffectivePageId(long pageId) {
+        return (pageId & ~EFFECTIVE_PAGE_ID_MASK) == 0;
+    }
+
+    /**
+     * Index of the item inside of data page.
+     *
+     * @param link Page link.
+     * @return Offset in 8-byte words.
+     */
+    public static int itemId(long link) {
+        return (int) ((link >> ROTATION_ID_OFFSET) & OFFSET_MASK);
+    }
+
+    /**
+     * Tag of pageId.
+     *
+     * @param link Page link.
+     * @return tag - item id + flags
+     */
+    public static int tag(long link) {
+        return (int) ((link >> (PAGE_IDX_SIZE + PART_ID_SIZE)) & TAG_MASK);
+    }
+
+    /**
+     * Extracts flags byte from the page ID.
+     *
+     * @param pageId Page ID.
+     * @return Flag.
+     */
+    public static byte flag(long pageId) {
+        return (byte) ((pageId >>> (PART_ID_SIZE + PAGE_IDX_SIZE)) & FLAG_MASK);
+    }
+
+    /**
+     * Extracts partition ID from the page ID.
+     *
+     * @param pageId Page ID.
+     * @return Partition.
+     */
+    public static int partitionId(long pageId) {
+        return (int) ((pageId >>> PAGE_IDX_SIZE) & PART_ID_MASK);
+    }
+
+    /**
+     * Returns the Rotation ID of a page identified by the given ID.
+     */
+    public static long rotationId(long pageId) {
+        return pageId >>> ROTATION_ID_OFFSET;
+    }
+
+    /**
+     * Rotates rotated ID part of the page ID.
+     *
+     * @param pageId Page ID.
+     * @return New page ID.
+     */
+    public static long rotatePageId(long pageId) {
+        long updatedRotationId = rotationId(pageId) + 1;
+
+        if (updatedRotationId > MAX_ITEMID_NUM) {
+            updatedRotationId = 1; // We always want non-zero updatedRotationId
+        }
+
+        return (pageId & PAGE_ID_MASK) | (updatedRotationId << ROTATION_ID_OFFSET);
+    }
+
+    /**
+     * Masks partition ID from full page ID. Effectively the same as {@code changePartitionId(pageId, 0)}.
+     *
+     * @param pageId Page ID to mask partition ID from.
+     * @see #changePartitionId(long, int)
+     */
+    public static long maskPartitionId(long pageId) {
+        return pageId & ~((-1L << PAGE_IDX_SIZE) & (~(-1L << PAGE_IDX_SIZE + PART_ID_SIZE)));
+    }
+
+    /**
+     * Change page type.
+     *
+     * @param pageId Old page ID.
+     * @param type   New page type.
+     * @return Changed page ID.
+     */
+    public static long changeType(long pageId, byte type) {
+        return pageId(partitionId(pageId), type, pageIndex(pageId));
+    }
+
+    /**
+     * Returns convenient human-readable page ID representation.
+     *
+     * @param pageId Page id.
+     */
+    public static String toDetailString(long pageId) {
+        return "pageId=" + pageId
+                + "(offset=" + itemId(pageId)
+                + ", flags=" + Integer.toBinaryString(flag(pageId))
+                + ", partitionId=" + partitionId(pageId)
+                + ", index=" + pageIndex(pageId)
+                + ")";
+    }
+
+    /**
+     * Replaces partition ID in the page ID.
+     *
+     * @param pageId      Page ID.
+     * @param partitionId Partition ID.
+     */
+    public static long changePartitionId(long pageId, int partitionId) {
+        byte flag = flag(pageId);
+        int pageIdx = pageIndex(pageId);
+
+        return pageId(partitionId, flag, pageIdx);
+    }
+}
diff --git a/modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/util/PageLockListener.java b/modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/util/PageLockListener.java
new file mode 100644
index 0000000..3d33316
--- /dev/null
+++ b/modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/util/PageLockListener.java
@@ -0,0 +1,86 @@
+/*
+ * 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.pagememory.util;
+
+/**
+ * Page lock listener.
+ */
+//TODO IGNITE-16350 Consider froper Before/After naming convention for all methods in this class.
+public interface PageLockListener extends AutoCloseable {
+    /**
+     * Callback that's called before write lock acquiring.
+     *
+     * @param groupId Group ID.
+     * @param pageId  Page ID.
+     * @param page    Page pointer.
+     */
+    public void onBeforeWriteLock(int groupId, long pageId, long page);
+
+    /**
+     * Callback that's called after lock acquiring.
+     *
+     * @param groupId  Group ID.
+     * @param pageId   Page ID.
+     * @param page     Page pointer.
+     * @param pageAddr Page address.
+     */
+    public void onWriteLock(int groupId, long pageId, long page, long pageAddr);
+
+    /**
+     * Callback that's called before write lock releasing.
+     *
+     * @param groupId  Group ID.
+     * @param pageId   Page ID.
+     * @param page     Page pointer.
+     * @param pageAddr Page address.
+     */
+    public void onWriteUnlock(int groupId, long pageId, long page, long pageAddr);
+
+    /**
+     * Callback that's called before read lock acquiring.
+     *
+     * @param groupId Group ID.
+     * @param pageId  Page ID.
+     * @param page    Page pointer.
+     */
+    public void onBeforeReadLock(int groupId, long pageId, long page);
+
+    /**
+     * Callback that's called after read lock acquiring.
+     *
+     * @param groupId  Group ID.
+     * @param pageId   Page ID.
+     * @param page     Page pointer.
+     * @param pageAddr Page address.
+     */
+    public void onReadLock(int groupId, long pageId, long page, long pageAddr);
+
+    /**
+     * Callback that's called before read lock releasing.
+     *
+     * @param groupId  Group ID.
+     * @param pageId   Page ID.
+     * @param page     Page pointer.
+     * @param pageAddr Page address.
+     */
+    public void onReadUnlock(int groupId, long pageId, long page, long pageAddr);
+
+    /** {@inheritDoc} */
+    @Override
+    public void close();
+}
diff --git a/modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/util/PageUtils.java b/modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/util/PageUtils.java
new file mode 100644
index 0000000..a5878f4
--- /dev/null
+++ b/modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/util/PageUtils.java
@@ -0,0 +1,282 @@
+/*
+ * 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.pagememory.util;
+
+import java.nio.ByteBuffer;
+import org.apache.ignite.internal.util.GridUnsafe;
+
+/**
+ * Page utils.
+ */
+public class PageUtils {
+    /**
+     * Reads a byte from the memory.
+     *
+     * @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);
+    }
+
+    /**
+     * Reads an unsigned byte from the memory.
+     *
+     * @param addr Start address.
+     * @param off  Offset.
+     * @return Byte value from given address.
+     */
+    public static int getUnsignedByte(long addr, int off) {
+        assert addr > 0 : addr;
+        assert off >= 0;
+
+        return GridUnsafe.getByte(addr + off) & 0xFF;
+    }
+
+    /**
+     * Reads a byte array from the memory.
+     *
+     * @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;
+    }
+
+    /**
+     * Reads a byte array from the memory.
+     *
+     * @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);
+    }
+
+    /**
+     * Reads a short value from the memory.
+     *
+     * @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);
+    }
+
+    /**
+     * Reads an int value from the memory.
+     *
+     * @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);
+    }
+
+    /**
+     * Reads a long value from the memory.
+     *
+     * @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);
+    }
+
+    /**
+     * Writes a byte array into the memory.
+     *
+     * @param addr  Address.
+     * @param off   Offset.
+     * @param bytes Bytes.
+     */
+    public static void putBytes(long addr, int off, byte[] bytes) {
+        putBytes(addr, off, bytes, 0, bytes.length);
+    }
+
+    /**
+     * Writes a byte array into the memory.
+     *
+     * @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) {
+        putBytes(addr, off, bytes, bytesOff, bytes.length - bytesOff);
+    }
+
+    /**
+     * Writes a byte array into the memory.
+     *
+     * @param addr     Address.
+     * @param off      Offset.
+     * @param bytes    Bytes array.
+     * @param bytesOff Bytes array offset.
+     * @param len      Length.
+     */
+    public static void putBytes(long addr, int off, byte[] bytes, int bytesOff, int len) {
+        assert addr > 0 : addr;
+        assert off >= 0;
+        assert bytes != null;
+        assert bytesOff >= 0 && (bytesOff < bytes.length || bytes.length == 0) : bytesOff;
+        assert len >= 0 && (bytesOff + len <= bytes.length);
+
+        GridUnsafe.copyMemory(bytes, GridUnsafe.BYTE_ARR_OFF + bytesOff, null, addr + off, len);
+    }
+
+    /**
+     * Writes a byte into the memory.
+     *
+     * @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);
+    }
+
+    /**
+     * Writes an unsigned byte array into the memory.
+     *
+     * @param addr Address.
+     * @param off  Offset.
+     * @param v    Value.
+     */
+    public static void putUnsignedByte(long addr, int off, int v) {
+        assert addr > 0 : addr;
+        assert off >= 0;
+        assert v >= 0 && v <= 255;
+
+        GridUnsafe.putByte(addr + off, (byte) v);
+    }
+
+    /**
+     * Writes a short value into the memory.
+     *
+     * @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);
+    }
+
+    /**
+     * Writes an int value into the memory.
+     *
+     * @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);
+    }
+
+    /**
+     * Writes a long value into the memory.
+     *
+     * @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);
+    }
+
+    /**
+     * Copies memory from one buffer to another.
+     *
+     * @param src    Source.
+     * @param srcOff Source offset in bytes.
+     * @param dst    Destination.
+     * @param dstOff Destination offset in bytes.
+     * @param cnt    Bytes count to copy.
+     */
+    public static void copyMemory(ByteBuffer src, long srcOff, ByteBuffer dst, long dstOff, long cnt) {
+        byte[] srcArr = src.hasArray() ? src.array() : null;
+        byte[] dstArr = dst.hasArray() ? dst.array() : null;
+        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() ? GridUnsafe.bufferAddress(src) : 0;
+        long dstPtr = dst.isDirect() ? GridUnsafe.bufferAddress(dst) : 0;
+
+        GridUnsafe.copyMemory(srcArr, srcPtr + srcArrOff + srcOff, dstArr, dstPtr + dstArrOff + dstOff, cnt);
+    }
+
+    /**
+     * Copies memory from one address to another.
+     *
+     * @param srcAddr Source.
+     * @param srcOff  Source offset in bytes.
+     * @param dstAddr Destination.
+     * @param dstOff  Destination offset in bytes.
+     * @param cnt     Bytes count to copy.
+     */
+    public static void copyMemory(long srcAddr, long srcOff, long dstAddr, long dstOff, long cnt) {
+        GridUnsafe.copyMemory(null, srcAddr + srcOff, null, dstAddr + dstOff, cnt);
+    }
+}
diff --git a/modules/page-memory/src/test/java/org/apache/ignite/internal/pagememory/PageIoRegistryTest.java b/modules/page-memory/src/test/java/org/apache/ignite/internal/pagememory/PageIoRegistryTest.java
new file mode 100644
index 0000000..26d3bd2
--- /dev/null
+++ b/modules/page-memory/src/test/java/org/apache/ignite/internal/pagememory/PageIoRegistryTest.java
@@ -0,0 +1,68 @@
+/*
+ * 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.pagememory;
+
+import static org.apache.ignite.internal.pagememory.TestPageIoModule.TEST_PAGE_TYPE;
+import static org.apache.ignite.internal.pagememory.TestPageIoModule.TEST_PAGE_VER;
+import static org.apache.ignite.internal.util.GridUnsafe.bufferAddress;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import java.nio.ByteBuffer;
+import org.apache.ignite.internal.pagememory.TestPageIoModule.TestPageIo;
+import org.apache.ignite.internal.pagememory.io.PageIo;
+import org.apache.ignite.internal.pagememory.io.PageIoRegistry;
+import org.apache.ignite.internal.util.GridUnsafe;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+import org.junit.jupiter.api.Test;
+
+/**
+ * Tests for {@link PageIoRegistry} methods.
+ */
+public class PageIoRegistryTest {
+    private PageIoRegistry ioRegistry = new PageIoRegistry();
+
+    @Test
+    void testResolve() throws Exception {
+        // Module is not yet loaded.
+        assertThrows(IgniteInternalCheckedException.class, () -> ioRegistry.resolve(TEST_PAGE_TYPE, TEST_PAGE_VER));
+
+        // Load all PageIOModule-s from the classpath.
+        ioRegistry.loadFromServiceLoader();
+
+        // Test base resolve method.
+        PageIo pageIo = ioRegistry.resolve(TEST_PAGE_TYPE, TEST_PAGE_VER);
+
+        assertTrue(pageIo instanceof TestPageIo);
+        assertEquals(TEST_PAGE_TYPE, pageIo.getType());
+        assertEquals(TEST_PAGE_VER, pageIo.getVersion());
+
+        ByteBuffer pageBuffer = ByteBuffer.allocateDirect(4);
+        pageBuffer.order(GridUnsafe.NATIVE_BYTE_ORDER);
+
+        pageBuffer.putShort(PageIo.TYPE_OFF, (short) TEST_PAGE_TYPE);
+        pageBuffer.putShort(PageIo.VER_OFF, (short) TEST_PAGE_VER);
+
+        // Test resolve from a pointer.
+        assertEquals(pageIo, ioRegistry.resolve(bufferAddress(pageBuffer)));
+
+        // Test resolve from ByteBuffer.
+        assertEquals(pageIo, ioRegistry.resolve(pageBuffer));
+    }
+}
diff --git a/modules/page-memory/src/test/java/org/apache/ignite/internal/pagememory/TestPageIoModule.java b/modules/page-memory/src/test/java/org/apache/ignite/internal/pagememory/TestPageIoModule.java
new file mode 100644
index 0000000..21f33c8
--- /dev/null
+++ b/modules/page-memory/src/test/java/org/apache/ignite/internal/pagememory/TestPageIoModule.java
@@ -0,0 +1,61 @@
+/*
+ * 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.pagememory;
+
+import java.util.Collection;
+import java.util.List;
+import org.apache.ignite.internal.pagememory.io.IoVersions;
+import org.apache.ignite.internal.pagememory.io.PageIo;
+import org.apache.ignite.internal.pagememory.io.PageIoModule;
+
+/**
+ * Test implementation of {@link PageIoModule}.
+ */
+public class TestPageIoModule implements PageIoModule {
+    /** Last possible value for IO type. */
+    public static final int TEST_PAGE_TYPE = PageIo.MAX_IO_TYPE;
+
+    /** Version 1, minimal possible value. */
+    public static final int TEST_PAGE_VER = 1;
+
+    /** {@inheritDoc} */
+    @Override
+    public Collection<IoVersions<?>> ioVersions() {
+        return List.of(new IoVersions<>(new TestPageIo()));
+    }
+
+    /**
+     * Test implementation of {@link PageIo}.
+     */
+    public static class TestPageIo extends PageIo {
+        /**
+         * Constructor.
+         *
+         * @param type Page type.
+         * @param ver  Page format version.
+         */
+        protected TestPageIo() {
+            super(TEST_PAGE_TYPE, TEST_PAGE_VER);
+        }
+
+        /** {@inheritDoc} */
+        @Override
+        protected void printPage(long addr, int pageSize, StringBuilder sb) {
+        }
+    }
+}
diff --git a/modules/page-memory/src/test/java/org/apache/ignite/internal/pagememory/util/PageIdUtilsTest.java b/modules/page-memory/src/test/java/org/apache/ignite/internal/pagememory/util/PageIdUtilsTest.java
new file mode 100644
index 0000000..7760e22
--- /dev/null
+++ b/modules/page-memory/src/test/java/org/apache/ignite/internal/pagememory/util/PageIdUtilsTest.java
@@ -0,0 +1,147 @@
+/*
+ * 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.pagememory.util;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+import java.util.Random;
+import org.apache.ignite.internal.pagememory.PageIdAllocator;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.junit.jupiter.api.Test;
+
+/**
+ * Tests for methods of {@link PageIdUtils}.
+ */
+public class PageIdUtilsTest {
+    @Test
+    public void testRotatePageId() throws Exception {
+        assertEquals(0x0102FFFFFFFFFFFFL, PageIdUtils.rotatePageId(0x0002FFFFFFFFFFFFL));
+        assertEquals(0x0B02FFFFFFFFFFFFL, PageIdUtils.rotatePageId(0x0A02FFFFFFFFFFFFL));
+        assertEquals(0x1002FFFFFFFFFFFFL, PageIdUtils.rotatePageId(0x0F02FFFFFFFFFFFFL));
+        assertEquals(0x0102FFFFFFFFFFFFL, PageIdUtils.rotatePageId(0xFE02FFFFFFFFFFFFL));
+        assertEquals(0x0102FFFFFFFFFFFFL, PageIdUtils.rotatePageId(0xFF02FFFFFFFFFFFFL));
+    }
+
+    @Test
+    public void testEffectivePageId() throws Exception {
+        assertEquals(0x0000FFFFFFFFFFFFL, PageIdUtils.effectivePageId(0x0002FFFFFFFFFFFFL));
+        assertEquals(0x0000FFFFFFFFFFFFL, PageIdUtils.effectivePageId(0x0A02FFFFFFFFFFFFL));
+        assertEquals(0x0000FFFFFFFFFFFFL, PageIdUtils.effectivePageId(0x0F02FFFFFFFFFFFFL));
+        assertEquals(0x0000FFFFFFFFFFFFL, PageIdUtils.effectivePageId(0xFF02FFFFFFFFFFFFL));
+    }
+
+    @Test
+    public void testLinkConstruction() throws Exception {
+        assertEquals(0x00FFFFFFFFFFFFFFL, PageIdUtils.link(0xFFFFFFFFFFFFFFL, 0));
+        assertEquals(0x01FFFFFFFFFFFFFFL, PageIdUtils.link(0xFFFFFFFFFFFFFFL, 1));
+
+        assertEquals(0x0000000000000000L, PageIdUtils.link(0, 0));
+        assertEquals(0x0100000000000000L, PageIdUtils.link(0, 1));
+
+        assertEquals(0xF000000000000000L, PageIdUtils.link(0, 0xF0));
+        assertEquals(0xF0FFFFFFFFFFFFFFL, PageIdUtils.link(0xFFFFFFFFFFFFFFL, 0xF0));
+
+        assertEquals(0xFE00000000000000L, PageIdUtils.link(0, 0xFE));
+        assertEquals(0xFEFFFFFFFFFFFFFFL, PageIdUtils.link(0xFFFFFFFFFFFFFFL, 0xFE));
+
+        assertEquals(0x0F00000000000000L, PageIdUtils.link(0, 0xF));
+        assertEquals(0x0FFFFFFFFFFFFFFFL, PageIdUtils.link(0xFFFFFFFFFFFFFFL, 0xF));
+    }
+
+    @Test
+    public void testOffsetExtraction() throws Exception {
+        assertEquals(0, PageIdUtils.itemId(0x00FFFFFFFFFFFFFFL));
+        assertEquals(1, PageIdUtils.itemId(0x01FFFFFFFFFFFFFFL));
+
+        assertEquals(0, PageIdUtils.itemId(0x0000000000000000L));
+        assertEquals(1, PageIdUtils.itemId(0x0100000000000000L));
+
+        assertEquals(0xFA, PageIdUtils.itemId(0xFA00000000000000L));
+        assertEquals(0xFA, PageIdUtils.itemId(0xFAFFFFFFFFFFFFFFL));
+
+        assertEquals(0xF, PageIdUtils.itemId(0x0F00000000000000L));
+        assertEquals(0xF, PageIdUtils.itemId(0x0FFFFFFFFFFFFFFFL));
+
+        assertEquals(0xF0, PageIdUtils.itemId(0xF000000000000000L));
+        assertEquals(0xF0, PageIdUtils.itemId(0xF0FFFFFFFFFFFFFFL));
+    }
+
+    @Test
+    public void testPageIdFromLink() throws Exception {
+        assertEquals(0x00FFFFFFFFFFFFFFL, PageIdUtils.pageId(0x00FFFFFFFFFFFFFFL));
+
+        assertEquals(0x0001FFFFFFFFFFFFL, PageIdUtils.pageId(0x0001FFFFFFFFFFFFL));
+        assertEquals(0x0001FFFFFFFFFFFFL, PageIdUtils.pageId(0x1001FFFFFFFFFFFFL));
+        assertEquals(0x0001FFFFFFFFFFFFL, PageIdUtils.pageId(0x0101FFFFFFFFFFFFL));
+        assertEquals(0x0001FFFFFFFFFFFFL, PageIdUtils.pageId(0x1101FFFFFFFFFFFFL));
+        assertEquals(0x0001FFFFFFFFFFFFL, PageIdUtils.pageId(0x8001FFFFFFFFFFFFL));
+        assertEquals(0x0001FFFFFFFFFFFFL, PageIdUtils.pageId(0x8801FFFFFFFFFFFFL));
+        assertEquals(0x0001FFFFFFFFFFFFL, PageIdUtils.pageId(0x0801FFFFFFFFFFFFL));
+        assertEquals(0x0001FFFFFFFFFFFFL, PageIdUtils.pageId(0xFF01FFFFFFFFFFFFL));
+
+        assertEquals(0x0002FFFFFFFFFFFFL, PageIdUtils.pageId(0x0002FFFFFFFFFFFFL));
+        assertEquals(0x1002FFFFFFFFFFFFL, PageIdUtils.pageId(0x1002FFFFFFFFFFFFL));
+        assertEquals(0x0102FFFFFFFFFFFFL, PageIdUtils.pageId(0x0102FFFFFFFFFFFFL));
+        assertEquals(0x1102FFFFFFFFFFFFL, PageIdUtils.pageId(0x1102FFFFFFFFFFFFL));
+        assertEquals(0x8002FFFFFFFFFFFFL, PageIdUtils.pageId(0x8002FFFFFFFFFFFFL));
+        assertEquals(0x8802FFFFFFFFFFFFL, PageIdUtils.pageId(0x8802FFFFFFFFFFFFL));
+        assertEquals(0x0802FFFFFFFFFFFFL, PageIdUtils.pageId(0x0802FFFFFFFFFFFFL));
+        assertEquals(0xFF02FFFFFFFFFFFFL, PageIdUtils.pageId(0xFF02FFFFFFFFFFFFL));
+
+        assertEquals(0x0004FFFFFFFFFFFFL, PageIdUtils.pageId(0x0004FFFFFFFFFFFFL));
+        assertEquals(0x1004FFFFFFFFFFFFL, PageIdUtils.pageId(0x1004FFFFFFFFFFFFL));
+        assertEquals(0x0104FFFFFFFFFFFFL, PageIdUtils.pageId(0x0104FFFFFFFFFFFFL));
+        assertEquals(0x1104FFFFFFFFFFFFL, PageIdUtils.pageId(0x1104FFFFFFFFFFFFL));
+        assertEquals(0x8004FFFFFFFFFFFFL, PageIdUtils.pageId(0x8004FFFFFFFFFFFFL));
+        assertEquals(0x8804FFFFFFFFFFFFL, PageIdUtils.pageId(0x8804FFFFFFFFFFFFL));
+        assertEquals(0x0804FFFFFFFFFFFFL, PageIdUtils.pageId(0x0804FFFFFFFFFFFFL));
+        assertEquals(0xFF04FFFFFFFFFFFFL, PageIdUtils.pageId(0xFF04FFFFFFFFFFFFL));
+
+        assertEquals(0x0000000000000000L, PageIdUtils.pageId(0x0000000000000000L));
+        assertEquals(0x1000000000000000L, PageIdUtils.pageId(0x1000000000000000L));
+        assertEquals(0x0100000000000000L, PageIdUtils.pageId(0x0100000000000000L));
+        assertEquals(0x8000000000000000L, PageIdUtils.pageId(0x8000000000000000L));
+        assertEquals(0x0800000000000000L, PageIdUtils.pageId(0x0800000000000000L));
+        assertEquals(0xFF00000000000000L, PageIdUtils.pageId(0xFF00000000000000L));
+    }
+
+    @Test
+    public void testRandomIds() throws Exception {
+        Random rnd = new Random();
+
+        for (int i = 0; i < 50_000; i++) {
+            int off = rnd.nextInt(PageIdUtils.MAX_ITEMID_NUM + 1);
+            int partId = rnd.nextInt(PageIdUtils.MAX_PART_ID + 1);
+            int pageNum = rnd.nextInt();
+
+            long pageId = PageIdUtils.pageId(partId, PageIdAllocator.FLAG_DATA, pageNum);
+
+            String msg = "For values [offset=" + IgniteUtils.hexLong(off) + ", fileId=" + IgniteUtils.hexLong(partId)
+                    + ", pageNum=" + IgniteUtils.hexLong(pageNum) + ']';
+
+            assertEquals(pageId, PageIdUtils.pageId(pageId), msg);
+            assertEquals(0, PageIdUtils.itemId(pageId), msg);
+
+            long link = PageIdUtils.link(pageId, off);
+
+            assertEquals(pageId, PageIdUtils.pageId(link), msg);
+            assertEquals(off, PageIdUtils.itemId(link), msg);
+            assertEquals(pageId, PageIdUtils.pageId(link), msg);
+        }
+    }
+}
diff --git a/modules/page-memory/src/test/resources/META-INF/services/org.apache.ignite.internal.pagememory.io.PageIoModule b/modules/page-memory/src/test/resources/META-INF/services/org.apache.ignite.internal.pagememory.io.PageIoModule
new file mode 100644
index 0000000..bb6ce0b
--- /dev/null
+++ b/modules/page-memory/src/test/resources/META-INF/services/org.apache.ignite.internal.pagememory.io.PageIoModule
@@ -0,0 +1,17 @@
+#
+# 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.
+#
+org.apache.ignite.internal.pagememory.TestPageIoModule
diff --git a/pom.xml b/pom.xml
index 4d6f4ed..ac1f885 100644
--- a/pom.xml
+++ b/pom.xml
@@ -56,6 +56,7 @@
         <module>modules/network</module>
         <module>modules/network-annotation-processor</module>
         <module>modules/network-api</module>
+        <module>modules/page-memory</module>
         <module>modules/raft</module>
         <module>modules/raft-client</module>
         <module>modules/rest</module>