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/08/05 13:38:03 UTC

[ignite-3] branch main updated: IGNITE-17085 Support persistent case for page-memory-based MV storage (#976)

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 2a6d30702 IGNITE-17085 Support persistent case for page-memory-based MV storage (#976)
2a6d30702 is described below

commit 2a6d30702c873bd61b6230784d86a9adfcd4d4cd
Author: Kirill Tkalenko <tk...@yandex.ru>
AuthorDate: Fri Aug 5 16:37:57 2022 +0300

    IGNITE-17085 Support persistent case for page-memory-based MV storage (#976)
---
 .../pagememory/persistence/PartitionMeta.java      | 124 +++++-
 .../pagememory/persistence/io/PartitionMetaIo.java |  81 +++-
 .../persistence/PartitionMetaManagerTest.java      |  17 +-
 .../pagememory/persistence/PartitionMetaTest.java  |  98 ++++-
 .../persistence/checkpoint/CheckpointerTest.java   |   2 +-
 .../storage/AbstractMvPartitionStorageTest.java    |   8 +-
 .../storage/basic/TestMvPartitionStorageTest.java  |   3 +-
 ...ava => AbstractPageMemoryPartitionStorage.java} |   9 +-
 .../pagememory/AbstractPageMemoryTableStorage.java |  23 +-
 .../PersistentPageMemoryPartitionStorage.java      |   5 +-
 .../PersistentPageMemoryStorageEngine.java         |   7 +
 .../PersistentPageMemoryTableStorage.java          | 270 ++++++++++--
 .../pagememory/VolatilePageMemoryDataRegion.java   |  21 +-
 .../VolatilePageMemoryPartitionStorage.java        | 453 +--------------------
 .../pagememory/VolatilePageMemoryTableStorage.java |  78 ++--
 ...a => AbstractPageMemoryMvPartitionStorage.java} |  82 ++--
 .../pagememory/mv/PageMemoryMvStorageIoModule.java |   3 +-
 .../mv/PersistentPageMemoryMvPartitionStorage.java |  69 ++++
 .../pagememory/mv/VersionChainFreeList.java        |   5 +-
 .../mv/VolatilePageMemoryMvPartitionStorage.java   |  49 +++
 ... AbstractPageMemoryMvPartitionStorageTest.java} | 144 +++----
 ...PersistentPageMemoryMvPartitionStorageTest.java | 127 ++++++
 .../VolatilePageMemoryMvPartitionStorageTest.java  | 103 +++++
 .../rocksdb/RocksDbMvPartitionStorageTest.java     |   2 +-
 24 files changed, 1076 insertions(+), 707 deletions(-)

diff --git a/modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/PartitionMeta.java b/modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/PartitionMeta.java
index a869cb619..7720cf23a 100644
--- a/modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/PartitionMeta.java
+++ b/modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/PartitionMeta.java
@@ -45,25 +45,54 @@ public class PartitionMeta {
         }
     }
 
+    // TODO: IGNITE-17466 Delete it
     private volatile long treeRootPageId;
 
+    // TODO: IGNITE-17466 Delete it
     private volatile long reuseListRootPageId;
 
+    private volatile long versionChainTreeRootPageId;
+
+    private volatile long versionChainFreeListRootPageId;
+
+    private volatile long rowVersionFreeListRootPageId;
+
     private volatile int pageCount;
 
     private volatile PartitionMetaSnapshot metaSnapshot;
 
+    /**
+     * Default constructor.
+     */
+    public PartitionMeta() {
+        metaSnapshot = new PartitionMetaSnapshot(null, this);
+    }
+
     /**
      * Constructor.
      *
      * @param checkpointId Checkpoint ID.
      * @param treeRootPageId Tree root page ID.
      * @param reuseListRootPageId Reuse list root page ID.
+     * @param versionChainTreeRootPageId Version chain tree root page ID.
+     * @param versionChainFreeListRootPageId Version chain free list root page ID.
+     * @param rowVersionFreeListRootPageId Row version free list root page ID.
      * @param pageCount Count of pages in the partition.
      */
-    public PartitionMeta(@Nullable UUID checkpointId, long treeRootPageId, long reuseListRootPageId, int pageCount) {
+    public PartitionMeta(
+            @Nullable UUID checkpointId,
+            long treeRootPageId,
+            long reuseListRootPageId,
+            long versionChainTreeRootPageId,
+            long versionChainFreeListRootPageId,
+            long rowVersionFreeListRootPageId,
+            int pageCount
+    ) {
         this.treeRootPageId = treeRootPageId;
         this.reuseListRootPageId = reuseListRootPageId;
+        this.versionChainTreeRootPageId = versionChainTreeRootPageId;
+        this.versionChainFreeListRootPageId = versionChainFreeListRootPageId;
+        this.rowVersionFreeListRootPageId = rowVersionFreeListRootPageId;
         this.pageCount = pageCount;
 
         metaSnapshot = new PartitionMetaSnapshot(checkpointId, this);
@@ -81,6 +110,9 @@ public class PartitionMeta {
                 checkpointId,
                 metaIo.getTreeRootPageId(pageAddr),
                 metaIo.getReuseListRootPageId(pageAddr),
+                metaIo.getVersionChainTreeRootPageId(pageAddr),
+                metaIo.getVersionChainFreeListRootPageId(pageAddr),
+                metaIo.getRowVersionFreeListRootPageId(pageAddr),
                 metaIo.getPageCount(pageAddr)
         );
     }
@@ -123,6 +155,63 @@ public class PartitionMeta {
         this.reuseListRootPageId = reuseListRootPageId;
     }
 
+    /**
+     * Returns version chain tree root page ID.
+     */
+    public long versionChainTreeRootPageId() {
+        return versionChainTreeRootPageId;
+    }
+
+    /**
+     * Sets version chain root page ID.
+     *
+     * @param checkpointId Checkpoint ID.
+     * @param versionChainTreeRootPageId Version chain root page ID.
+     */
+    public void versionChainTreeRootPageId(@Nullable UUID checkpointId, long versionChainTreeRootPageId) {
+        updateSnapshot(checkpointId);
+
+        this.versionChainTreeRootPageId = versionChainTreeRootPageId;
+    }
+
+    /**
+     * Returns version chain free list root page ID.
+     */
+    public long versionChainFreeListRootPageId() {
+        return versionChainFreeListRootPageId;
+    }
+
+    /**
+     * Sets version chain free list root page ID.
+     *
+     * @param checkpointId Checkpoint ID.
+     * @param versionChainFreeListRootPageId Version chain free list root page ID.
+     */
+    public void versionChainFreeListRootPageId(@Nullable UUID checkpointId, long versionChainFreeListRootPageId) {
+        updateSnapshot(checkpointId);
+
+        this.versionChainFreeListRootPageId = versionChainFreeListRootPageId;
+    }
+
+    /**
+     * Returns row version free list root page ID.
+     */
+    public long rowVersionFreeListRootPageId() {
+        return rowVersionFreeListRootPageId;
+    }
+
+    /**
+     * Sets row version free list root page ID.
+     *
+     * @param checkpointId Checkpoint ID.
+     * @param rowVersionFreeListRootPageId Row version free list root page ID.
+     */
+    public void rowVersionFreeListRootPageId(@Nullable UUID checkpointId, long rowVersionFreeListRootPageId) {
+        updateSnapshot(checkpointId);
+
+        this.rowVersionFreeListRootPageId = rowVersionFreeListRootPageId;
+    }
+
     /**
      * Returns count of pages in the partition.
      */
@@ -180,6 +269,12 @@ public class PartitionMeta {
 
         private final long reuseListRootPageId;
 
+        private final long versionChainTreeRootPageId;
+
+        private final long versionChainFreeListRootPageId;
+
+        private final long rowVersionFreeListRootPageId;
+
         private final int pageCount;
 
         /**
@@ -192,6 +287,9 @@ public class PartitionMeta {
             this.checkpointId = checkpointId;
             this.treeRootPageId = partitionMeta.treeRootPageId;
             this.reuseListRootPageId = partitionMeta.reuseListRootPageId;
+            this.versionChainTreeRootPageId = partitionMeta.versionChainTreeRootPageId;
+            this.versionChainFreeListRootPageId = partitionMeta.versionChainFreeListRootPageId;
+            this.rowVersionFreeListRootPageId = partitionMeta.rowVersionFreeListRootPageId;
             this.pageCount = partitionMeta.pageCount;
         }
 
@@ -209,6 +307,27 @@ public class PartitionMeta {
             return reuseListRootPageId;
         }
 
+        /**
+         * Returns version chain tree root page ID.
+         */
+        public long versionChainTreeRootPageId() {
+            return versionChainTreeRootPageId;
+        }
+
+        /**
+         * Returns version chain free list root page ID.
+         */
+        public long versionChainFreeListRootPageId() {
+            return versionChainFreeListRootPageId;
+        }
+
+        /**
+         * Returns row version free list root page ID.
+         */
+        public long rowVersionFreeListRootPageId() {
+            return rowVersionFreeListRootPageId;
+        }
+
         /**
          * Returns count of pages in the partition.
          */
@@ -225,6 +344,9 @@ public class PartitionMeta {
         void writeTo(PartitionMetaIo metaIo, long pageAddr) {
             metaIo.setTreeRootPageId(pageAddr, treeRootPageId);
             metaIo.setReuseListRootPageId(pageAddr, reuseListRootPageId);
+            metaIo.setVersionChainTreeRootPageId(pageAddr, versionChainTreeRootPageId);
+            metaIo.setVersionChainFreeListRootPageId(pageAddr, versionChainFreeListRootPageId);
+            metaIo.setRowVersionFreeListRootPageId(pageAddr, rowVersionFreeListRootPageId);
             metaIo.setPageCount(pageAddr, pageCount);
         }
 
diff --git a/modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/io/PartitionMetaIo.java b/modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/io/PartitionMetaIo.java
index 3a4733adb..c86a7f45f 100644
--- a/modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/io/PartitionMetaIo.java
+++ b/modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/io/PartitionMetaIo.java
@@ -35,7 +35,13 @@ public class PartitionMetaIo extends PageIo {
 
     private static final int REUSE_LIST_ROOT_PAGE_ID_OFF = TREE_ROOT_PAGE_ID_OFF + Long.BYTES;
 
-    private static final int PAGE_COUNT_OFF = REUSE_LIST_ROOT_PAGE_ID_OFF + Long.BYTES;
+    private static final int VERSION_CHAIN_TREE_ROOT_PAGE_ID_OFF = REUSE_LIST_ROOT_PAGE_ID_OFF + Long.BYTES;
+
+    private static final int VERSION_CHAIN_FREE_LIST_ROOT_PAGE_ID_OFF = VERSION_CHAIN_TREE_ROOT_PAGE_ID_OFF + Long.BYTES;
+
+    private static final int ROW_VERSION_FREE_LIST_ROOT_PAGE_ID_OFF = VERSION_CHAIN_FREE_LIST_ROOT_PAGE_ID_OFF + Long.BYTES;
+
+    private static final int PAGE_COUNT_OFF = ROW_VERSION_FREE_LIST_ROOT_PAGE_ID_OFF + Long.BYTES;
 
     /** Page IO type. */
     public static final short T_TABLE_PARTITION_META_IO = 7;
@@ -59,6 +65,9 @@ public class PartitionMetaIo extends PageIo {
 
         setTreeRootPageId(pageAddr, 0);
         setReuseListRootPageId(pageAddr, 0);
+        setVersionChainTreeRootPageId(pageAddr, 0);
+        setVersionChainFreeListRootPageId(pageAddr, 0);
+        setRowVersionFreeListRootPageId(pageAddr, 0);
         setPageCount(pageAddr, 0);
     }
 
@@ -68,6 +77,7 @@ public class PartitionMetaIo extends PageIo {
      * @param pageAddr Page address.
      * @param pageId Tree root page ID.
      */
+    // TODO: IGNITE-17466 Delete it
     public void setTreeRootPageId(long pageAddr, long pageId) {
         assertPageType(pageAddr);
 
@@ -79,6 +89,7 @@ public class PartitionMetaIo extends PageIo {
      *
      * @param pageAddr Page address.
      */
+    // TODO: IGNITE-17466 Delete it
     public long getTreeRootPageId(long pageAddr) {
         return getLong(pageAddr, TREE_ROOT_PAGE_ID_OFF);
     }
@@ -89,6 +100,7 @@ public class PartitionMetaIo extends PageIo {
      * @param pageAddr Page address.
      * @param pageId Reuse list root page ID.
      */
+    // TODO: IGNITE-17466 Delete it
     public void setReuseListRootPageId(long pageAddr, long pageId) {
         assertPageType(pageAddr);
 
@@ -100,10 +112,74 @@ public class PartitionMetaIo extends PageIo {
      *
      * @param pageAddr Page address.
      */
+    // TODO: IGNITE-17466 Delete it
     public long getReuseListRootPageId(long pageAddr) {
         return getLong(pageAddr, REUSE_LIST_ROOT_PAGE_ID_OFF);
     }
 
+    /**
+     * Sets version chain tree root page ID.
+     *
+     * @param pageAddr Page address.
+     * @param pageId Version chain tree root page ID.
+     */
+    public void setVersionChainTreeRootPageId(long pageAddr, long pageId) {
+        assertPageType(pageAddr);
+
+        putLong(pageAddr, VERSION_CHAIN_TREE_ROOT_PAGE_ID_OFF, pageId);
+    }
+
+    /**
+     * Returns version chain tree root page ID.
+     *
+     * @param pageAddr Page address.
+     */
+    public long getVersionChainTreeRootPageId(long pageAddr) {
+        return getLong(pageAddr, VERSION_CHAIN_TREE_ROOT_PAGE_ID_OFF);
+    }
+
+    /**
+     * Sets version chain free list root page ID.
+     *
+     * @param pageAddr Page address.
+     * @param pageId Version chain free list root page ID.
+     */
+    public void setVersionChainFreeListRootPageId(long pageAddr, long pageId) {
+        assertPageType(pageAddr);
+
+        putLong(pageAddr, VERSION_CHAIN_FREE_LIST_ROOT_PAGE_ID_OFF, pageId);
+    }
+
+    /**
+     * Returns version chain free list root page ID.
+     *
+     * @param pageAddr Page address.
+     */
+    public long getVersionChainFreeListRootPageId(long pageAddr) {
+        return getLong(pageAddr, VERSION_CHAIN_FREE_LIST_ROOT_PAGE_ID_OFF);
+    }
+
+    /**
+     * Sets row version free list root page ID.
+     *
+     * @param pageAddr Page address.
+     * @param pageId Row version free list root page ID.
+     */
+    public void setRowVersionFreeListRootPageId(long pageAddr, long pageId) {
+        assertPageType(pageAddr);
+
+        putLong(pageAddr, ROW_VERSION_FREE_LIST_ROOT_PAGE_ID_OFF, pageId);
+    }
+
+    /**
+     * Returns row version free list root page ID.
+     *
+     * @param pageAddr Page address.
+     */
+    public long getRowVersionFreeListRootPageId(long pageAddr) {
+        return getLong(pageAddr, ROW_VERSION_FREE_LIST_ROOT_PAGE_ID_OFF);
+    }
+
     /**
      * Sets the count of pages.
      *
@@ -131,6 +207,9 @@ public class PartitionMetaIo extends PageIo {
         sb.app("TablePartitionMeta [").nl()
                 .app("treeRootPageId=").appendHex(getTreeRootPageId(addr)).nl()
                 .app(", reuseListRootPageId=").appendHex(getReuseListRootPageId(addr)).nl()
+                .app(", versionChainTreeRootPageId=").appendHex(getVersionChainTreeRootPageId(addr)).nl()
+                .app(", versionChainFreeListRootPageId=").appendHex(getVersionChainFreeListRootPageId(addr)).nl()
+                .app(", rowVersionFreeListRootPageId=").appendHex(getRowVersionFreeListRootPageId(addr)).nl()
                 .app(", pageCount=").app(getPageCount(addr)).nl()
                 .app(']');
     }
diff --git a/modules/page-memory/src/test/java/org/apache/ignite/internal/pagememory/persistence/PartitionMetaManagerTest.java b/modules/page-memory/src/test/java/org/apache/ignite/internal/pagememory/persistence/PartitionMetaManagerTest.java
index 1c510cbdf..974a65378 100644
--- a/modules/page-memory/src/test/java/org/apache/ignite/internal/pagememory/persistence/PartitionMetaManagerTest.java
+++ b/modules/page-memory/src/test/java/org/apache/ignite/internal/pagememory/persistence/PartitionMetaManagerTest.java
@@ -101,11 +101,17 @@ public class PartitionMetaManagerTest {
 
                 assertEquals(0, meta.treeRootPageId());
                 assertEquals(0, meta.reuseListRootPageId());
+                assertEquals(0, meta.versionChainTreeRootPageId());
+                assertEquals(0, meta.versionChainFreeListRootPageId());
+                assertEquals(0, meta.rowVersionFreeListRootPageId());
                 assertEquals(1, meta.pageCount());
 
                 // Change the meta and write it to the file.
                 meta.treeRootPageId(null, 100);
                 meta.reuseListRootPageId(null, 500);
+                meta.versionChainTreeRootPageId(null, 300);
+                meta.versionChainFreeListRootPageId(null, 600);
+                meta.rowVersionFreeListRootPageId(null, 900);
                 meta.incrementPageCount(null);
 
                 manager.writeMetaToBuffer(partId, meta.metaSnapshot(UUID.randomUUID()), buffer);
@@ -123,6 +129,9 @@ public class PartitionMetaManagerTest {
 
                 assertEquals(100, meta.treeRootPageId());
                 assertEquals(500, meta.reuseListRootPageId());
+                assertEquals(300, meta.versionChainTreeRootPageId());
+                assertEquals(600, meta.versionChainFreeListRootPageId());
+                assertEquals(900, meta.rowVersionFreeListRootPageId());
                 assertEquals(2, meta.pageCount());
             }
 
@@ -130,7 +139,7 @@ public class PartitionMetaManagerTest {
             try (FilePageStore filePageStore = createFilePageStore(testFilePath)) {
                 manager.writeMetaToBuffer(
                         partId,
-                        new PartitionMeta(UUID.randomUUID(), 200, 1000, 4).metaSnapshot(null),
+                        new PartitionMeta(UUID.randomUUID(), 200, 1000, 300, 600, 900, 4).metaSnapshot(null),
                         buffer.rewind()
                 );
 
@@ -147,6 +156,9 @@ public class PartitionMetaManagerTest {
 
                 assertEquals(200, meta.treeRootPageId());
                 assertEquals(1000, meta.reuseListRootPageId());
+                assertEquals(300, meta.versionChainTreeRootPageId());
+                assertEquals(600, meta.versionChainFreeListRootPageId());
+                assertEquals(900, meta.rowVersionFreeListRootPageId());
                 assertEquals(4, meta.pageCount());
             }
 
@@ -163,6 +175,9 @@ public class PartitionMetaManagerTest {
 
                 assertEquals(0, meta.treeRootPageId());
                 assertEquals(0, meta.reuseListRootPageId());
+                assertEquals(0, meta.versionChainTreeRootPageId());
+                assertEquals(0, meta.versionChainFreeListRootPageId());
+                assertEquals(0, meta.rowVersionFreeListRootPageId());
                 assertEquals(1, meta.pageCount());
             }
         } finally {
diff --git a/modules/page-memory/src/test/java/org/apache/ignite/internal/pagememory/persistence/PartitionMetaTest.java b/modules/page-memory/src/test/java/org/apache/ignite/internal/pagememory/persistence/PartitionMetaTest.java
index 03e56709d..9b21b7e7f 100644
--- a/modules/page-memory/src/test/java/org/apache/ignite/internal/pagememory/persistence/PartitionMetaTest.java
+++ b/modules/page-memory/src/test/java/org/apache/ignite/internal/pagememory/persistence/PartitionMetaTest.java
@@ -37,7 +37,7 @@ import org.junit.jupiter.api.Test;
 public class PartitionMetaTest {
     @Test
     void testTreeRootPageId() {
-        PartitionMeta meta = new PartitionMeta(null, 0, 0, 0);
+        PartitionMeta meta = new PartitionMeta();
 
         assertEquals(0, meta.treeRootPageId());
 
@@ -52,7 +52,7 @@ public class PartitionMetaTest {
 
     @Test
     void testReuseListRootPageId() {
-        PartitionMeta meta = new PartitionMeta(null, 0, 0, 0);
+        PartitionMeta meta = new PartitionMeta();
 
         assertEquals(0, meta.reuseListRootPageId());
 
@@ -67,7 +67,7 @@ public class PartitionMetaTest {
 
     @Test
     void testPageCount() {
-        PartitionMeta meta = new PartitionMeta(null, 0, 0, 0);
+        PartitionMeta meta = new PartitionMeta();
 
         assertEquals(0, meta.pageCount());
 
@@ -80,32 +80,89 @@ public class PartitionMetaTest {
         assertEquals(2, meta.pageCount());
     }
 
+    @Test
+    void testVersionChainTreeRootPageId() {
+        PartitionMeta meta = new PartitionMeta();
+
+        assertEquals(0, meta.versionChainTreeRootPageId());
+
+        assertDoesNotThrow(() -> meta.versionChainTreeRootPageId(null, 100));
+
+        assertEquals(100, meta.versionChainTreeRootPageId());
+
+        assertDoesNotThrow(() -> meta.versionChainTreeRootPageId(UUID.randomUUID(), 500));
+
+        assertEquals(500, meta.versionChainTreeRootPageId());
+    }
+
+    @Test
+    void testVersionChainFreeListRootPageId() {
+        PartitionMeta meta = new PartitionMeta();
+
+        assertEquals(0, meta.versionChainFreeListRootPageId());
+
+        assertDoesNotThrow(() -> meta.versionChainFreeListRootPageId(null, 100));
+
+        assertEquals(100, meta.versionChainFreeListRootPageId());
+
+        assertDoesNotThrow(() -> meta.versionChainFreeListRootPageId(UUID.randomUUID(), 500));
+
+        assertEquals(500, meta.versionChainFreeListRootPageId());
+    }
+
+    @Test
+    void testRowVersionFreeListRootPageId() {
+        PartitionMeta meta = new PartitionMeta();
+
+        assertEquals(0, meta.rowVersionFreeListRootPageId());
+
+        assertDoesNotThrow(() -> meta.rowVersionFreeListRootPageId(null, 100));
+
+        assertEquals(100, meta.rowVersionFreeListRootPageId());
+
+        assertDoesNotThrow(() -> meta.rowVersionFreeListRootPageId(UUID.randomUUID(), 500));
+
+        assertEquals(500, meta.rowVersionFreeListRootPageId());
+    }
+
     @Test
     void testSnapshot() {
         UUID checkpointId = null;
 
-        PartitionMeta meta = new PartitionMeta(checkpointId, 0, 0, 0);
+        PartitionMeta meta = new PartitionMeta(checkpointId, 0, 0, 0, 0, 0, 0);
 
-        checkSnapshot(meta.metaSnapshot(checkpointId), 0, 0, 0);
-        checkSnapshot(meta.metaSnapshot(checkpointId = UUID.randomUUID()), 0, 0, 0);
+        checkSnapshot(meta.metaSnapshot(checkpointId), 0, 0, 0, 0, 0, 0);
+        checkSnapshot(meta.metaSnapshot(checkpointId = UUID.randomUUID()), 0, 0, 0, 0, 0, 0);
 
         meta.treeRootPageId(checkpointId, 100);
         meta.reuseListRootPageId(checkpointId, 500);
+        meta.versionChainTreeRootPageId(checkpointId, 300);
+        meta.versionChainFreeListRootPageId(checkpointId, 600);
+        meta.rowVersionFreeListRootPageId(checkpointId, 900);
         meta.incrementPageCount(checkpointId);
 
-        checkSnapshot(meta.metaSnapshot(checkpointId), 0, 0, 0);
-        checkSnapshot(meta.metaSnapshot(UUID.randomUUID()), 100, 500, 1);
+        checkSnapshot(meta.metaSnapshot(checkpointId), 0, 0, 0, 0, 0, 0);
+        checkSnapshot(meta.metaSnapshot(UUID.randomUUID()), 100, 500, 300, 600, 900, 1);
 
         meta.treeRootPageId(checkpointId = UUID.randomUUID(), 101);
-        checkSnapshot(meta.metaSnapshot(checkpointId), 100, 500, 1);
+        checkSnapshot(meta.metaSnapshot(checkpointId), 100, 500, 300, 600, 900, 1);
 
         meta.reuseListRootPageId(checkpointId = UUID.randomUUID(), 505);
-        checkSnapshot(meta.metaSnapshot(checkpointId), 101, 500, 1);
+        checkSnapshot(meta.metaSnapshot(checkpointId), 101, 500, 300, 600, 900, 1);
+
+        meta.versionChainTreeRootPageId(checkpointId = UUID.randomUUID(), 303);
+        checkSnapshot(meta.metaSnapshot(checkpointId), 101, 505, 300, 600, 900, 1);
+
+        meta.versionChainFreeListRootPageId(checkpointId = UUID.randomUUID(), 606);
+        checkSnapshot(meta.metaSnapshot(checkpointId), 101, 505, 303, 600, 900, 1);
+
+        meta.rowVersionFreeListRootPageId(checkpointId = UUID.randomUUID(), 909);
+        checkSnapshot(meta.metaSnapshot(checkpointId), 101, 505, 303, 606, 900, 1);
 
         meta.incrementPageCount(checkpointId = UUID.randomUUID());
-        checkSnapshot(meta.metaSnapshot(checkpointId), 101, 505, 1);
+        checkSnapshot(meta.metaSnapshot(checkpointId), 101, 505, 303, 606, 909, 1);
 
-        checkSnapshot(meta.metaSnapshot(UUID.randomUUID()), 101, 505, 2);
+        checkSnapshot(meta.metaSnapshot(UUID.randomUUID()), 101, 505, 303, 606, 909, 2);
     }
 
     @Test
@@ -117,9 +174,20 @@ public class PartitionMetaTest {
         assertEquals(0, pageIndex(pageId));
     }
 
-    private static void checkSnapshot(PartitionMetaSnapshot snapshot, long expTreeRootPageId, long reuseListPageId, int pageCount) {
+    private static void checkSnapshot(
+            PartitionMetaSnapshot snapshot,
+            long expTreeRootPageId,
+            long expReuseListPageId,
+            long expVersionChainTreeRootPageId,
+            long expVersionChainFreeListRootPageId,
+            long expRowVersionFreeListRootPageId,
+            int expPageCount
+    ) {
         assertThat(snapshot.treeRootPageId(), equalTo(expTreeRootPageId));
-        assertThat(snapshot.reuseListRootPageId(), equalTo(reuseListPageId));
-        assertThat(snapshot.pageCount(), equalTo(pageCount));
+        assertThat(snapshot.reuseListRootPageId(), equalTo(expReuseListPageId));
+        assertThat(snapshot.versionChainTreeRootPageId(), equalTo(expVersionChainTreeRootPageId));
+        assertThat(snapshot.versionChainFreeListRootPageId(), equalTo(expVersionChainFreeListRootPageId));
+        assertThat(snapshot.rowVersionFreeListRootPageId(), equalTo(expRowVersionFreeListRootPageId));
+        assertThat(snapshot.pageCount(), equalTo(expPageCount));
     }
 }
diff --git a/modules/page-memory/src/test/java/org/apache/ignite/internal/pagememory/persistence/checkpoint/CheckpointerTest.java b/modules/page-memory/src/test/java/org/apache/ignite/internal/pagememory/persistence/checkpoint/CheckpointerTest.java
index 3e5f2b05f..6a47e317f 100644
--- a/modules/page-memory/src/test/java/org/apache/ignite/internal/pagememory/persistence/checkpoint/CheckpointerTest.java
+++ b/modules/page-memory/src/test/java/org/apache/ignite/internal/pagememory/persistence/checkpoint/CheckpointerTest.java
@@ -351,7 +351,7 @@ public class CheckpointerTest {
 
         partitionMetaManager.addMeta(
                 new GroupPartitionId(0, 0),
-                new PartitionMeta(null, 0, 0, 3)
+                new PartitionMeta(null, 0, 0, 0, 0, 0, 3)
         );
 
         FilePageStore filePageStore = mock(FilePageStore.class);
diff --git a/modules/storage-api/src/test/java/org/apache/ignite/internal/storage/AbstractMvPartitionStorageTest.java b/modules/storage-api/src/test/java/org/apache/ignite/internal/storage/AbstractMvPartitionStorageTest.java
index 38445e461..6e8cfc48a 100644
--- a/modules/storage-api/src/test/java/org/apache/ignite/internal/storage/AbstractMvPartitionStorageTest.java
+++ b/modules/storage-api/src/test/java/org/apache/ignite/internal/storage/AbstractMvPartitionStorageTest.java
@@ -45,8 +45,8 @@ import org.junit.jupiter.api.Test;
 /**
  * Base test for MV partition storages.
  */
-public abstract class AbstractMvPartitionStorageTest<S extends MvPartitionStorage> extends BaseMvStoragesTest {
-    protected S storage;
+public abstract class AbstractMvPartitionStorageTest extends BaseMvStoragesTest {
+    protected MvPartitionStorage storage;
 
     protected final UUID txId = newTransactionId();
 
@@ -87,14 +87,14 @@ public abstract class AbstractMvPartitionStorageTest<S extends MvPartitionStorag
     /**
      * Inserts a row inside of consistency closure.
      */
-    private RowId insert(BinaryRow binaryRow, UUID txId) {
+    protected RowId insert(BinaryRow binaryRow, UUID txId) {
         return storage.runConsistently(() -> storage.insert(binaryRow, txId));
     }
 
     /**
      * Adds/updates a write-intent inside of consistency closure.
      */
-    private BinaryRow addWrite(RowId rowId, BinaryRow binaryRow, UUID txId) {
+    protected BinaryRow addWrite(RowId rowId, BinaryRow binaryRow, UUID txId) {
         return storage.runConsistently(() -> storage.addWrite(rowId, binaryRow, txId));
     }
 
diff --git a/modules/storage-api/src/test/java/org/apache/ignite/internal/storage/basic/TestMvPartitionStorageTest.java b/modules/storage-api/src/test/java/org/apache/ignite/internal/storage/basic/TestMvPartitionStorageTest.java
index ba32f5d24..a7d321048 100644
--- a/modules/storage-api/src/test/java/org/apache/ignite/internal/storage/basic/TestMvPartitionStorageTest.java
+++ b/modules/storage-api/src/test/java/org/apache/ignite/internal/storage/basic/TestMvPartitionStorageTest.java
@@ -23,12 +23,11 @@ import org.apache.ignite.internal.storage.AbstractMvPartitionStorageTest;
 /**
  * MV partition storage test implementation for {@link TestMvPartitionStorage} class.
  */
-public class TestMvPartitionStorageTest extends AbstractMvPartitionStorageTest<TestMvPartitionStorage> {
+public class TestMvPartitionStorageTest extends AbstractMvPartitionStorageTest {
     /**
      * Creates new instance.
      */
     public TestMvPartitionStorageTest() {
         storage = new TestMvPartitionStorage(List.of(), 0);
     }
-
 }
diff --git a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/VolatilePageMemoryPartitionStorage.java b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/AbstractPageMemoryPartitionStorage.java
similarity index 97%
copy from modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/VolatilePageMemoryPartitionStorage.java
copy to modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/AbstractPageMemoryPartitionStorage.java
index 5451bc810..5cf3be56a 100644
--- a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/VolatilePageMemoryPartitionStorage.java
+++ b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/AbstractPageMemoryPartitionStorage.java
@@ -44,10 +44,10 @@ import org.apache.ignite.lang.IgniteInternalException;
 import org.jetbrains.annotations.Nullable;
 
 /**
- * Implementation of {@link PartitionStorage} based on a {@link BplusTree} for in-memory case.
+ * Abstract implementation of {@link PartitionStorage} based on a {@link BplusTree}.
  */
 // TODO: IGNITE-16644 Support snapshots.
-class VolatilePageMemoryPartitionStorage implements PartitionStorage {
+abstract class AbstractPageMemoryPartitionStorage implements PartitionStorage {
     protected final int partId;
 
     protected final TableTree tree;
@@ -60,13 +60,12 @@ class VolatilePageMemoryPartitionStorage implements PartitionStorage {
      * @param partId Partition id.
      * @param freeList Table free list.
      * @param tree Table tree.
-     * @throws StorageException If there is an error while creating the partition storage.
      */
-    public VolatilePageMemoryPartitionStorage(
+    public AbstractPageMemoryPartitionStorage(
             int partId,
             TableFreeList freeList,
             TableTree tree
-    ) throws StorageException {
+    ) {
         assert partId >= 0 && partId < MAX_PARTITION_ID : partId;
 
         this.partId = partId;
diff --git a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/AbstractPageMemoryTableStorage.java b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/AbstractPageMemoryTableStorage.java
index 058421613..5c508ad90 100644
--- a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/AbstractPageMemoryTableStorage.java
+++ b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/AbstractPageMemoryTableStorage.java
@@ -32,11 +32,10 @@ import org.apache.ignite.internal.storage.StorageException;
 import org.apache.ignite.internal.storage.engine.MvTableStorage;
 import org.apache.ignite.internal.storage.engine.TableStorage;
 import org.apache.ignite.internal.storage.index.SortedIndexStorage;
-import org.apache.ignite.internal.storage.pagememory.mv.PageMemoryMvPartitionStorage;
+import org.apache.ignite.internal.storage.pagememory.mv.AbstractPageMemoryMvPartitionStorage;
 import org.apache.ignite.internal.tostring.S;
 import org.apache.ignite.internal.util.IgniteUtils;
 import org.jetbrains.annotations.Nullable;
-import org.jetbrains.annotations.TestOnly;
 
 /**
  * Abstract table storage implementation based on {@link PageMemory}.
@@ -147,12 +146,20 @@ public abstract class AbstractPageMemoryTableStorage implements TableStorage, Mv
     }
 
     /**
-     * Returns a new instance of {@link VolatilePageMemoryPartitionStorage}.
+     * Returns a new instance of {@link AbstractPageMemoryPartitionStorage}.
      *
-     * @param partId Partition id.
+     * @param partitionId Partition id.
      * @throws StorageException If there is an error while creating the partition storage.
      */
-    protected abstract VolatilePageMemoryPartitionStorage createPartitionStorage(int partId) throws StorageException;
+    protected abstract AbstractPageMemoryPartitionStorage createPartitionStorage(int partitionId) throws StorageException;
+
+    /**
+     * Returns a new instance of {@link AbstractPageMemoryMvPartitionStorage}.
+     *
+     * @param partitionId Partition id.
+     * @throws StorageException If there is an error while creating the mv partition storage.
+     */
+    public abstract AbstractPageMemoryMvPartitionStorage createMvPartitionStorage(int partitionId) throws StorageException;
 
     /** {@inheritDoc} */
     @Override
@@ -205,12 +212,6 @@ public abstract class AbstractPageMemoryTableStorage implements TableStorage, Mv
         return CompletableFuture.completedFuture(null);
     }
 
-    /**
-     * This API is not yet ready. But we need to test mv storages anyways.
-     */
-    @TestOnly
-    public abstract PageMemoryMvPartitionStorage createMvPartitionStorage(int partitionId);
-
     /**
      * Closes all {@link #partitions} and {@link #autoCloseables}.
      *
diff --git a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/PersistentPageMemoryPartitionStorage.java b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/PersistentPageMemoryPartitionStorage.java
index 0a83869c2..4f7d7d547 100644
--- a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/PersistentPageMemoryPartitionStorage.java
+++ b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/PersistentPageMemoryPartitionStorage.java
@@ -33,7 +33,7 @@ import org.jetbrains.annotations.Nullable;
 /**
  * Implementation of {@link PartitionStorage} based on a {@link BplusTree} for persistent case.
  */
-public class PersistentPageMemoryPartitionStorage extends VolatilePageMemoryPartitionStorage {
+public class PersistentPageMemoryPartitionStorage extends AbstractPageMemoryPartitionStorage {
     private final CheckpointTimeoutLock checkpointTimeoutLock;
 
     /**
@@ -43,14 +43,13 @@ public class PersistentPageMemoryPartitionStorage extends VolatilePageMemoryPart
      * @param freeList Table free list.
      * @param tree Table tree.
      * @param checkpointTimeoutLock Checkpoint timeout lock.
-     * @throws StorageException If there is an error while creating the partition storage.
      */
     public PersistentPageMemoryPartitionStorage(
             int partId,
             TableFreeList freeList,
             TableTree tree,
             CheckpointTimeoutLock checkpointTimeoutLock
-    ) throws StorageException {
+    ) {
         super(partId, freeList, tree);
 
         this.checkpointTimeoutLock = checkpointTimeoutLock;
diff --git a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/PersistentPageMemoryStorageEngine.java b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/PersistentPageMemoryStorageEngine.java
index e280ce2b8..ee80383e7 100644
--- a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/PersistentPageMemoryStorageEngine.java
+++ b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/PersistentPageMemoryStorageEngine.java
@@ -42,6 +42,7 @@ import org.apache.ignite.internal.pagememory.persistence.PartitionMetaManager;
 import org.apache.ignite.internal.pagememory.persistence.checkpoint.CheckpointManager;
 import org.apache.ignite.internal.pagememory.persistence.store.FilePageStoreManager;
 import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.MvTableStorage;
 import org.apache.ignite.internal.storage.engine.StorageEngine;
 import org.apache.ignite.internal.storage.pagememory.configuration.schema.PersistentPageMemoryDataStorageView;
 import org.apache.ignite.internal.storage.pagememory.configuration.schema.PersistentPageMemoryStorageEngineConfiguration;
@@ -190,6 +191,12 @@ public class PersistentPageMemoryStorageEngine implements StorageEngine {
         return new PersistentPageMemoryTableStorage(tableCfg, regions.get(dataStorageView.dataRegion()));
     }
 
+    /** {@inheritDoc} */
+    @Override
+    public MvTableStorage createMvTable(TableConfiguration tableCfg) throws StorageException {
+        return createTable(tableCfg);
+    }
+
     /**
      * Returns checkpoint manager, {@code null} if engine not started.
      */
diff --git a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/PersistentPageMemoryTableStorage.java b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/PersistentPageMemoryTableStorage.java
index 593ce5c59..5add88f72 100644
--- a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/PersistentPageMemoryTableStorage.java
+++ b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/PersistentPageMemoryTableStorage.java
@@ -34,13 +34,17 @@ import org.apache.ignite.internal.pagememory.persistence.checkpoint.CheckpointTi
 import org.apache.ignite.internal.pagememory.persistence.store.FilePageStore;
 import org.apache.ignite.internal.pagememory.util.PageLockListenerNoOp;
 import org.apache.ignite.internal.storage.StorageException;
-import org.apache.ignite.internal.storage.pagememory.mv.PageMemoryMvPartitionStorage;
+import org.apache.ignite.internal.storage.pagememory.mv.PersistentPageMemoryMvPartitionStorage;
+import org.apache.ignite.internal.storage.pagememory.mv.RowVersionFreeList;
+import org.apache.ignite.internal.storage.pagememory.mv.VersionChain;
+import org.apache.ignite.internal.storage.pagememory.mv.VersionChainFreeList;
+import org.apache.ignite.internal.storage.pagememory.mv.VersionChainTree;
 import org.apache.ignite.lang.IgniteInternalCheckedException;
 
 /**
  * Implementation of {@link AbstractPageMemoryTableStorage} for persistent case.
  */
-class PersistentPageMemoryTableStorage extends AbstractPageMemoryTableStorage {
+public class PersistentPageMemoryTableStorage extends AbstractPageMemoryTableStorage {
     private final PersistentPageMemoryDataRegion dataRegion;
 
     /**
@@ -58,6 +62,7 @@ class PersistentPageMemoryTableStorage extends AbstractPageMemoryTableStorage {
         this.dataRegion = dataRegion;
     }
 
+    /** {@inheritDoc} */
     @Override
     public boolean isVolatile() {
         return false;
@@ -85,10 +90,10 @@ class PersistentPageMemoryTableStorage extends AbstractPageMemoryTableStorage {
 
     /** {@inheritDoc} */
     @Override
-    protected PersistentPageMemoryPartitionStorage createPartitionStorage(int partId) throws StorageException {
+    protected PersistentPageMemoryPartitionStorage createPartitionStorage(int partitionId) throws StorageException {
         TableView tableView = tableCfg.value();
 
-        FilePageStore filePageStore = ensurePartitionFilePageStore(tableView, partId);
+        FilePageStore filePageStore = ensurePartitionFilePageStore(tableView, partitionId);
 
         CheckpointManager checkpointManager = dataRegion.checkpointManager();
 
@@ -107,11 +112,11 @@ class PersistentPageMemoryTableStorage extends AbstractPageMemoryTableStorage {
 
             PartitionMeta meta = dataRegion.partitionMetaManager().readOrCreateMeta(
                     checkpointId,
-                    new GroupPartitionId(grpId, partId),
+                    new GroupPartitionId(grpId, partitionId),
                     filePageStore
             );
 
-            dataRegion.partitionMetaManager().addMeta(new GroupPartitionId(grpId, partId), meta);
+            dataRegion.partitionMetaManager().addMeta(new GroupPartitionId(grpId, partitionId), meta);
 
             filePageStore.pages(meta.pageCount());
 
@@ -126,7 +131,7 @@ class PersistentPageMemoryTableStorage extends AbstractPageMemoryTableStorage {
             boolean initNewTree = false;
 
             if (meta.treeRootPageId() == 0) {
-                meta.treeRootPageId(checkpointId, persistentPageMemory.allocatePage(grpId, partId, FLAG_AUX));
+                meta.treeRootPageId(checkpointId, persistentPageMemory.allocatePage(grpId, partitionId, FLAG_AUX));
 
                 initNewTree = true;
             }
@@ -134,21 +139,21 @@ class PersistentPageMemoryTableStorage extends AbstractPageMemoryTableStorage {
             boolean initNewReuseList = false;
 
             if (meta.reuseListRootPageId() == 0) {
-                meta.reuseListRootPageId(checkpointId, persistentPageMemory.allocatePage(grpId, partId, FLAG_AUX));
+                meta.reuseListRootPageId(checkpointId, persistentPageMemory.allocatePage(grpId, partitionId, FLAG_AUX));
 
                 initNewReuseList = true;
             }
 
-            TableFreeList tableFreeList = createTableFreeList(tableView, partId, meta, initNewReuseList);
+            TableFreeList tableFreeList = createTableFreeList(tableView, partitionId, meta.reuseListRootPageId(), initNewReuseList);
 
             autoCloseables.add(tableFreeList::close);
 
-            TableTree tableTree = createTableTree(tableView, partId, tableFreeList, meta, initNewTree);
+            TableTree tableTree = createTableTree(tableView, partitionId, tableFreeList, meta.treeRootPageId(), initNewTree);
 
-            return new PersistentPageMemoryPartitionStorage(partId, tableFreeList, tableTree, checkpointTimeoutLock);
+            return new PersistentPageMemoryPartitionStorage(partitionId, tableFreeList, tableTree, checkpointTimeoutLock);
         } catch (IgniteInternalCheckedException e) {
             throw new StorageException(
-                    String.format("Error getting or creating partition metadata [tableName=%s, partitionId=%s]", tableView.name(), partId),
+                    String.format("Error getting or creating partition [tableName=%s, partitionId=%s]", tableView.name(), partitionId),
                     e
             );
         } finally {
@@ -164,8 +169,112 @@ class PersistentPageMemoryTableStorage extends AbstractPageMemoryTableStorage {
 
     /** {@inheritDoc} */
     @Override
-    public PageMemoryMvPartitionStorage createMvPartitionStorage(int partitionId) {
-        throw new UnsupportedOperationException("Not supported yet");
+    public PersistentPageMemoryMvPartitionStorage createMvPartitionStorage(int partitionId) {
+        TableView tableView = tableCfg.value();
+
+        FilePageStore filePageStore = ensurePartitionFilePageStore(tableView, partitionId);
+
+        CheckpointManager checkpointManager = dataRegion.checkpointManager();
+
+        CheckpointTimeoutLock checkpointTimeoutLock = checkpointManager.checkpointTimeoutLock();
+
+        checkpointTimeoutLock.checkpointReadLock();
+
+        try {
+            PersistentPageMemory persistentPageMemory = dataRegion.pageMemory();
+
+            int grpId = tableView.tableId();
+
+            CheckpointProgress lastCheckpointProgress = checkpointManager.lastCheckpointProgress();
+
+            UUID checkpointId = lastCheckpointProgress == null ? null : lastCheckpointProgress.id();
+
+            PartitionMeta meta = dataRegion.partitionMetaManager().readOrCreateMeta(
+                    checkpointId,
+                    new GroupPartitionId(grpId, partitionId),
+                    filePageStore
+            );
+
+            dataRegion.partitionMetaManager().addMeta(new GroupPartitionId(grpId, partitionId), meta);
+
+            filePageStore.pages(meta.pageCount());
+
+            filePageStore.setPageAllocationListener(pageIdx -> {
+                assert checkpointTimeoutLock.checkpointLockIsHeldByThread();
+
+                CheckpointProgress last = checkpointManager.lastCheckpointProgress();
+
+                meta.incrementPageCount(last == null ? null : last.id());
+            });
+
+            boolean initNewVersionChainTree = false;
+
+            if (meta.versionChainTreeRootPageId() == 0) {
+                meta.versionChainTreeRootPageId(checkpointId, persistentPageMemory.allocatePage(grpId, partitionId, FLAG_AUX));
+
+                initNewVersionChainTree = true;
+            }
+
+            boolean initVersionChainFreeList = false;
+
+            if (meta.versionChainFreeListRootPageId() == 0) {
+                meta.versionChainFreeListRootPageId(checkpointId, persistentPageMemory.allocatePage(grpId, partitionId, FLAG_AUX));
+
+                initVersionChainFreeList = true;
+            }
+
+            boolean initRowVersionFreeList = false;
+
+            if (meta.rowVersionFreeListRootPageId() == 0) {
+                meta.rowVersionFreeListRootPageId(checkpointId, persistentPageMemory.allocatePage(grpId, partitionId, FLAG_AUX));
+
+                initRowVersionFreeList = true;
+            }
+
+            VersionChainFreeList versionChainFreeList = createVersionChainFreeList(
+                    tableView,
+                    partitionId,
+                    meta.versionChainFreeListRootPageId(),
+                    initVersionChainFreeList
+            );
+
+            autoCloseables.add(versionChainFreeList::close);
+
+            RowVersionFreeList rowVersionFreeList = createRowVersionFreeList(
+                    tableView,
+                    partitionId,
+                    versionChainFreeList,
+                    meta.rowVersionFreeListRootPageId(),
+                    initRowVersionFreeList
+            );
+
+            autoCloseables.add(rowVersionFreeList::close);
+
+            VersionChainTree versionChainTree = createVersionChainTree(
+                    tableView,
+                    partitionId,
+                    versionChainFreeList,
+                    meta.versionChainTreeRootPageId(),
+                    initNewVersionChainTree
+            );
+
+            return new PersistentPageMemoryMvPartitionStorage(
+                    partitionId,
+                    tableView,
+                    dataRegion.pageMemory(),
+                    versionChainFreeList,
+                    rowVersionFreeList,
+                    versionChainTree,
+                    checkpointTimeoutLock
+            );
+        } catch (IgniteInternalCheckedException e) {
+            throw new StorageException(
+                    String.format("Error getting or creating partition [tableName=%s, partitionId=%s]", tableView.name(), partitionId),
+                    e
+            );
+        } finally {
+            checkpointTimeoutLock.checkpointReadUnlock();
+        }
     }
 
     /**
@@ -176,7 +285,7 @@ class PersistentPageMemoryTableStorage extends AbstractPageMemoryTableStorage {
      * @return Partition file page store.
      * @throws StorageException If failed.
      */
-    FilePageStore ensurePartitionFilePageStore(TableView tableView, int partId) throws StorageException {
+    private FilePageStore ensurePartitionFilePageStore(TableView tableView, int partId) throws StorageException {
         try {
             FilePageStore filePageStore = dataRegion.filePageStoreManager().getStore(tableView.tableId(), partId);
 
@@ -196,14 +305,14 @@ class PersistentPageMemoryTableStorage extends AbstractPageMemoryTableStorage {
      *
      * @param tableView Table configuration.
      * @param partId Partition ID.
-     * @param partitionMeta Partition metadata.
+     * @param rootPageId Root page ID.
      * @param initNew {@code True} if new metadata should be initialized.
      * @throws StorageException If failed.
      */
-    TableFreeList createTableFreeList(
+    private TableFreeList createTableFreeList(
             TableView tableView,
             int partId,
-            PartitionMeta partitionMeta,
+            long rootPageId,
             boolean initNew
     ) throws StorageException {
         try {
@@ -212,7 +321,7 @@ class PersistentPageMemoryTableStorage extends AbstractPageMemoryTableStorage {
                     partId,
                     dataRegion.pageMemory(),
                     PageLockListenerNoOp.INSTANCE,
-                    partitionMeta.reuseListRootPageId(),
+                    rootPageId,
                     initNew,
                     null,
                     PageEvictionTrackerNoOp.INSTANCE,
@@ -226,21 +335,94 @@ class PersistentPageMemoryTableStorage extends AbstractPageMemoryTableStorage {
         }
     }
 
+    /**
+     * Returns new {@link VersionChainFreeList} instance for partition.
+     *
+     * @param tableView Table configuration.
+     * @param partId Partition ID.
+     * @param rootPageId Root page ID.
+     * @param initNew {@code True} if new metadata should be initialized.
+     * @throws StorageException If failed.
+     */
+    private VersionChainFreeList createVersionChainFreeList(
+            TableView tableView,
+            int partId,
+            long rootPageId,
+            boolean initNew
+    ) throws StorageException {
+        try {
+            return new VersionChainFreeList(
+                    tableView.tableId(),
+                    partId,
+                    dataRegion.pageMemory(),
+                    PageLockListenerNoOp.INSTANCE,
+                    rootPageId,
+                    initNew,
+                    null,
+                    PageEvictionTrackerNoOp.INSTANCE,
+                    IoStatisticsHolderNoOp.INSTANCE
+            );
+        } catch (IgniteInternalCheckedException e) {
+            throw new StorageException(
+                    String.format("Error creating VersionChainFreeList [tableName=%s, partitionId=%s]", tableView.name(), partId),
+                    e
+            );
+        }
+    }
+
+    /**
+     * Returns new {@link RowVersionFreeList} instance for partition.
+     *
+     * @param tableView Table configuration.
+     * @param partId Partition ID.
+     * @param reuseList Reuse list.
+     * @param rootPageId Root page ID.
+     * @param initNew {@code True} if new metadata should be initialized.
+     * @throws StorageException If failed.
+     */
+    private RowVersionFreeList createRowVersionFreeList(
+            TableView tableView,
+            int partId,
+            VersionChainFreeList reuseList,
+            long rootPageId,
+            boolean initNew
+    ) throws StorageException {
+        try {
+            return new RowVersionFreeList(
+                    tableView.tableId(),
+                    partId,
+                    dataRegion.pageMemory(),
+                    reuseList,
+                    PageLockListenerNoOp.INSTANCE,
+                    rootPageId,
+                    initNew,
+                    null,
+                    PageEvictionTrackerNoOp.INSTANCE,
+                    IoStatisticsHolderNoOp.INSTANCE
+            );
+        } catch (IgniteInternalCheckedException e) {
+            throw new StorageException(
+                    String.format("Error creating RowVersionFreeList [tableName=%s, partitionId=%s]", tableView.name(), partId),
+                    e
+            );
+        }
+    }
+
     /**
      * Returns new {@link TableTree} instance for partition.
      *
      * @param tableView Table configuration.
      * @param partId Partition ID.
      * @param freeList Table free list.
-     * @param partitionMeta Partition metadata.
+     * @param rootPageId Root page ID.
      * @param initNewTree {@code True} if new tree should be created.
      * @throws StorageException If failed.
      */
-    TableTree createTableTree(
+    private TableTree createTableTree(
             TableView tableView,
             int partId,
             TableFreeList freeList,
-            PartitionMeta partitionMeta,
+            long rootPageId,
             boolean initNewTree
     ) throws StorageException {
         int grpId = tableView.tableId();
@@ -249,10 +431,11 @@ class PersistentPageMemoryTableStorage extends AbstractPageMemoryTableStorage {
             return new TableTree(
                     grpId,
                     tableView.name(),
-                    partId, dataRegion.pageMemory(),
+                    partId,
+                    dataRegion.pageMemory(),
                     PageLockListenerNoOp.INSTANCE,
                     new AtomicLong(),
-                    partitionMeta.treeRootPageId(),
+                    rootPageId,
                     freeList,
                     initNewTree
             );
@@ -263,4 +446,43 @@ class PersistentPageMemoryTableStorage extends AbstractPageMemoryTableStorage {
             );
         }
     }
+
+    /**
+     * Returns new {@link TableTree} instance for partition.
+     *
+     * @param tableView Table configuration.
+     * @param partId Partition ID.
+     * @param freeList {@link VersionChain} free list.
+     * @param rootPageId Root page ID.
+     * @param initNewTree {@code True} if new tree should be created.
+     * @throws StorageException If failed.
+     */
+    private VersionChainTree createVersionChainTree(
+            TableView tableView,
+            int partId,
+            VersionChainFreeList freeList,
+            long rootPageId,
+            boolean initNewTree
+    ) throws StorageException {
+        int grpId = tableView.tableId();
+
+        try {
+            return new VersionChainTree(
+                    grpId,
+                    tableView.name(),
+                    partId,
+                    dataRegion.pageMemory(),
+                    PageLockListenerNoOp.INSTANCE,
+                    new AtomicLong(),
+                    rootPageId,
+                    freeList,
+                    initNewTree
+            );
+        } catch (IgniteInternalCheckedException e) {
+            throw new StorageException(
+                    String.format("Error creating VersionChainTree [tableName=%s, partitionId=%s]", tableView.name(), partId),
+                    e
+            );
+        }
+    }
 }
diff --git a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/VolatilePageMemoryDataRegion.java b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/VolatilePageMemoryDataRegion.java
index 4792aac0c..e9d06e3ec 100644
--- a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/VolatilePageMemoryDataRegion.java
+++ b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/VolatilePageMemoryDataRegion.java
@@ -27,7 +27,6 @@ import org.apache.ignite.internal.pagememory.evict.PageEvictionTrackerNoOp;
 import org.apache.ignite.internal.pagememory.inmemory.VolatilePageMemory;
 import org.apache.ignite.internal.pagememory.io.PageIoRegistry;
 import org.apache.ignite.internal.pagememory.metric.IoStatisticsHolderNoOp;
-import org.apache.ignite.internal.pagememory.reuse.ReuseList;
 import org.apache.ignite.internal.pagememory.util.PageLockListenerNoOp;
 import org.apache.ignite.internal.storage.StorageException;
 import org.apache.ignite.internal.storage.pagememory.mv.RowVersionFreeList;
@@ -37,7 +36,7 @@ import org.apache.ignite.lang.IgniteInternalCheckedException;
 /**
  * Implementation of {@link DataRegion} for in-memory case.
  */
-class VolatilePageMemoryDataRegion implements DataRegion<VolatilePageMemory> {
+public class VolatilePageMemoryDataRegion implements DataRegion<VolatilePageMemory> {
     private static final int FREE_LIST_GROUP_ID = 0;
 
     private static final int FREE_LIST_PARTITION_ID = 0;
@@ -89,13 +88,13 @@ class VolatilePageMemoryDataRegion implements DataRegion<VolatilePageMemory> {
         }
 
         try {
-            versionChainFreeList = createVersionChainFreeList(pageMemory, null);
+            versionChainFreeList = createVersionChainFreeList(pageMemory);
         } catch (IgniteInternalCheckedException e) {
             throw new StorageException("Error creating a VersionChainFreeList", e);
         }
 
         try {
-            rowVersionFreeList = createRowVersionFreeList(pageMemory, tableFreeList);
+            rowVersionFreeList = createRowVersionFreeList(pageMemory, versionChainFreeList);
         } catch (IgniteInternalCheckedException e) {
             throw new StorageException("Error creating a RowVersionFreeList", e);
         }
@@ -119,17 +118,13 @@ class VolatilePageMemoryDataRegion implements DataRegion<VolatilePageMemory> {
         );
     }
 
-    private static VersionChainFreeList createVersionChainFreeList(
-            PageMemory pageMemory,
-            ReuseList reuseList
-    ) throws IgniteInternalCheckedException {
+    private static VersionChainFreeList createVersionChainFreeList(PageMemory pageMemory) throws IgniteInternalCheckedException {
         long metaPageId = pageMemory.allocatePage(FREE_LIST_GROUP_ID, FREE_LIST_PARTITION_ID, FLAG_AUX);
 
         return new VersionChainFreeList(
                 FREE_LIST_GROUP_ID,
                 FREE_LIST_PARTITION_ID,
                 pageMemory,
-                reuseList,
                 PageLockListenerNoOp.INSTANCE,
                 metaPageId,
                 true,
@@ -141,13 +136,9 @@ class VolatilePageMemoryDataRegion implements DataRegion<VolatilePageMemory> {
 
     private static RowVersionFreeList createRowVersionFreeList(
             PageMemory pageMemory,
-            ReuseList reuseList
+            VersionChainFreeList reuseList
     ) throws IgniteInternalCheckedException {
-        long metaPageId = pageMemory.allocatePage(
-                FREE_LIST_GROUP_ID,
-                FREE_LIST_PARTITION_ID,
-                FLAG_AUX
-        );
+        long metaPageId = pageMemory.allocatePage(FREE_LIST_GROUP_ID, FREE_LIST_PARTITION_ID, FLAG_AUX);
 
         return new RowVersionFreeList(
                 FREE_LIST_GROUP_ID,
diff --git a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/VolatilePageMemoryPartitionStorage.java b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/VolatilePageMemoryPartitionStorage.java
index 5451bc810..ae3ad529a 100644
--- a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/VolatilePageMemoryPartitionStorage.java
+++ b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/VolatilePageMemoryPartitionStorage.java
@@ -17,472 +17,25 @@
 
 package org.apache.ignite.internal.storage.pagememory;
 
-import static org.apache.ignite.internal.pagememory.PageIdAllocator.MAX_PARTITION_ID;
-
-import java.nio.ByteBuffer;
-import java.nio.file.Path;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Iterator;
-import java.util.List;
-import java.util.NoSuchElementException;
-import java.util.concurrent.CompletableFuture;
-import java.util.function.Predicate;
 import org.apache.ignite.internal.pagememory.tree.BplusTree;
-import org.apache.ignite.internal.pagememory.tree.IgniteTree;
-import org.apache.ignite.internal.storage.DataRow;
-import org.apache.ignite.internal.storage.InvokeClosure;
-import org.apache.ignite.internal.storage.OperationType;
 import org.apache.ignite.internal.storage.PartitionStorage;
-import org.apache.ignite.internal.storage.SearchRow;
-import org.apache.ignite.internal.storage.StorageException;
-import org.apache.ignite.internal.storage.StorageUtils;
-import org.apache.ignite.internal.util.Cursor;
-import org.apache.ignite.internal.util.IgniteCursor;
-import org.apache.ignite.lang.IgniteInternalCheckedException;
-import org.apache.ignite.lang.IgniteInternalException;
-import org.jetbrains.annotations.Nullable;
 
 /**
  * Implementation of {@link PartitionStorage} based on a {@link BplusTree} for in-memory case.
  */
-// TODO: IGNITE-16644 Support snapshots.
-class VolatilePageMemoryPartitionStorage implements PartitionStorage {
-    protected final int partId;
-
-    protected final TableTree tree;
-
-    protected final TableFreeList freeList;
-
+class VolatilePageMemoryPartitionStorage extends AbstractPageMemoryPartitionStorage {
     /**
      * Constructor.
      *
      * @param partId Partition id.
      * @param freeList Table free list.
      * @param tree Table tree.
-     * @throws StorageException If there is an error while creating the partition storage.
      */
     public VolatilePageMemoryPartitionStorage(
             int partId,
             TableFreeList freeList,
             TableTree tree
-    ) throws StorageException {
-        assert partId >= 0 && partId < MAX_PARTITION_ID : partId;
-
-        this.partId = partId;
-        this.freeList = freeList;
-        this.tree = tree;
-    }
-
-    /** {@inheritDoc} */
-    @Override
-    public int partitionId() {
-        return partId;
-    }
-
-    /** {@inheritDoc} */
-    @Override
-    public @Nullable DataRow read(SearchRow key) throws StorageException {
-        try {
-            return wrap(tree.findOne(wrap(key)));
-        } catch (IgniteInternalCheckedException e) {
-            throw new StorageException("Error reading row", e);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override
-    public Collection<DataRow> readAll(List<? extends SearchRow> keys) throws StorageException {
-        Collection<DataRow> res = new ArrayList<>(keys.size());
-
-        try {
-            for (SearchRow key : keys) {
-                res.add(wrap(tree.findOne(wrap(key))));
-            }
-        } catch (IgniteInternalCheckedException e) {
-            throw new StorageException("Error reading rows", e);
-        }
-
-        return res;
-    }
-
-    /** {@inheritDoc} */
-    @Override
-    public void write(DataRow row) throws StorageException {
-        try {
-            TableDataRow dataRow = wrap(row);
-
-            freeList.insertDataRow(dataRow);
-
-            tree.put(dataRow);
-        } catch (IgniteInternalCheckedException e) {
-            throw new StorageException("Error writing row", e);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override
-    public void writeAll(List<? extends DataRow> rows) throws StorageException {
-        try {
-            for (DataRow row : rows) {
-                TableDataRow dataRow = wrap(row);
-
-                freeList.insertDataRow(dataRow);
-
-                tree.put(dataRow);
-            }
-        } catch (IgniteInternalCheckedException e) {
-            throw new StorageException("Error writing rows", e);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override
-    public Collection<DataRow> insertAll(List<? extends DataRow> rows) throws StorageException {
-        Collection<DataRow> cantInsert = new ArrayList<>();
-
-        try {
-            InsertClosure insertClosure = new InsertClosure(freeList);
-
-            for (DataRow row : rows) {
-                TableDataRow dataRow = wrap(row);
-
-                insertClosure.reset();
-
-                insertClosure.newRow = dataRow;
-
-                tree.invoke(dataRow, null, insertClosure);
-
-                if (insertClosure.oldRow != null) {
-                    cantInsert.add(row);
-                }
-            }
-        } catch (IgniteInternalCheckedException e) {
-            throw new StorageException("Error inserting rows", e);
-        }
-
-        return cantInsert;
-    }
-
-    /** {@inheritDoc} */
-    @Override
-    public void remove(SearchRow key) throws StorageException {
-        try {
-            TableSearchRow searchRow = wrap(key);
-
-            TableDataRow removed = tree.remove(searchRow);
-
-            if (removed != null) {
-                freeList.removeDataRowByLink(removed.link());
-            }
-        } catch (IgniteInternalCheckedException e) {
-            throw new StorageException("Error removing row", e);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override
-    public Collection<SearchRow> removeAll(List<? extends SearchRow> keys) throws StorageException {
-        Collection<SearchRow> skippedRows = new ArrayList<>();
-
-        try {
-            for (SearchRow key : keys) {
-                TableDataRow removed = tree.remove(wrap(key));
-
-                if (removed != null) {
-                    freeList.removeDataRowByLink(removed.link());
-                } else {
-                    skippedRows.add(key);
-                }
-            }
-        } catch (IgniteInternalCheckedException e) {
-            throw new StorageException("Error removing rows", e);
-        }
-
-        return skippedRows;
-    }
-
-    /** {@inheritDoc} */
-    @Override
-    public Collection<DataRow> removeAllExact(List<? extends DataRow> keyValues) throws StorageException {
-        Collection<DataRow> skipped = new ArrayList<>();
-
-        try {
-            RemoveExactClosure removeExactClosure = new RemoveExactClosure();
-
-            for (DataRow keyValue : keyValues) {
-                TableDataRow dataRow = wrap(keyValue);
-
-                removeExactClosure.reset();
-
-                removeExactClosure.forRemoveRow = dataRow;
-
-                tree.invoke(dataRow, null, removeExactClosure);
-
-                if (removeExactClosure.foundRow == null) {
-                    skipped.add(keyValue);
-                } else {
-                    freeList.removeDataRowByLink(removeExactClosure.foundRow.link());
-                }
-            }
-        } catch (IgniteInternalCheckedException e) {
-            throw new StorageException("Error while removing exact rows", e);
-        }
-
-        return skipped;
-    }
-
-    /** {@inheritDoc} */
-    @Override
-    public <T> @Nullable T invoke(SearchRow key, InvokeClosure<T> clo) throws StorageException {
-        IgniteTree.InvokeClosure<TableDataRow> treeClosure = new IgniteTree.InvokeClosure<>() {
-            /** {@inheritDoc} */
-            @Override
-            public void call(@Nullable TableDataRow oldRow) {
-                clo.call(wrap(oldRow));
-            }
-
-            /** {@inheritDoc} */
-            @Override
-            public @Nullable TableDataRow newRow() {
-                DataRow newRow = clo.newRow();
-
-                if (newRow == null) {
-                    return null;
-                }
-
-                TableDataRow dataRow = wrap(newRow);
-
-                try {
-                    freeList.insertDataRow(dataRow);
-                } catch (IgniteInternalCheckedException e) {
-                    throw new IgniteInternalException(e);
-                }
-
-                return dataRow;
-            }
-
-            /** {@inheritDoc} */
-            @Override
-            public IgniteTree.OperationType operationType() {
-                OperationType operationType = clo.operationType();
-
-                switch (operationType) {
-                    case WRITE:
-                        return IgniteTree.OperationType.PUT;
-
-                    case REMOVE:
-                        return IgniteTree.OperationType.REMOVE;
-
-                    case NOOP:
-                        return IgniteTree.OperationType.NOOP;
-
-                    default:
-                        throw new UnsupportedOperationException(String.valueOf(clo.operationType()));
-                }
-            }
-        };
-
-        try {
-            tree.invoke(wrap(key), null, treeClosure);
-        } catch (IgniteInternalCheckedException e) {
-            throw new StorageException("Error invoking a closure for a row", e);
-        }
-
-        return clo.result();
-    }
-
-    /** {@inheritDoc} */
-    @Override
-    public Cursor<DataRow> scan(Predicate<SearchRow> filter) throws StorageException {
-        try {
-            IgniteCursor<TableDataRow> treeCursor = tree.find(null, null);
-
-            return new Cursor<DataRow>() {
-                @Nullable TableDataRow cur = advance();
-
-                /** {@inheritDoc} */
-                @Override
-                public void close() {
-                }
-
-                /** {@inheritDoc} */
-                @Override
-                public Iterator<DataRow> iterator() {
-                    return this;
-                }
-
-                /** {@inheritDoc} */
-                @Override
-                public boolean hasNext() {
-                    return cur != null;
-                }
-
-                /** {@inheritDoc} */
-                @Override
-                public DataRow next() {
-                    DataRow next = wrap(cur);
-
-                    if (next == null) {
-                        throw new NoSuchElementException();
-                    }
-
-                    try {
-                        cur = advance();
-                    } catch (IgniteInternalCheckedException e) {
-                        throw new StorageException("Error getting next row", e);
-                    }
-
-                    return next;
-                }
-
-                @Nullable TableDataRow advance() throws IgniteInternalCheckedException {
-                    while (treeCursor.next()) {
-                        TableDataRow dataRow = treeCursor.get();
-
-                        if (filter.test(wrap(dataRow))) {
-                            return dataRow;
-                        }
-                    }
-
-                    return null;
-                }
-            };
-        } catch (IgniteInternalCheckedException e) {
-            throw new StorageException("Error while scanning rows", e);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override
-    public CompletableFuture<Void> snapshot(Path snapshotPath) {
-        throw new UnsupportedOperationException("Snapshots are not supported yet.");
-    }
-
-    /** {@inheritDoc} */
-    @Override
-    public void restoreSnapshot(Path snapshotPath) {
-        throw new UnsupportedOperationException("Snapshots are not supported yet.");
-    }
-
-    /** {@inheritDoc} */
-    @Override
-    public void destroy() throws StorageException {
-        try {
-            tree.destroy();
-        } catch (IgniteInternalCheckedException e) {
-            throw new StorageException("Error while destroying data", e);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override
-    public long rowsCount() {
-        try {
-            return tree.size();
-        } catch (IgniteInternalCheckedException e) {
-            throw new StorageException("Error occurred while fetching the size.", e);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override
-    public void close() {
-        tree.close();
-    }
-
-    private static TableSearchRow wrap(SearchRow searchRow) {
-        ByteBuffer key = searchRow.key();
-
-        return new TableSearchRow(StorageUtils.hashCode(key), key);
-    }
-
-    private static TableDataRow wrap(DataRow dataRow) {
-        ByteBuffer key = dataRow.key();
-        ByteBuffer value = dataRow.value();
-
-        return new TableDataRow(StorageUtils.hashCode(key), key, value);
-    }
-
-    private static @Nullable DataRow wrap(TableDataRow tableDataRow) {
-        return tableDataRow == null ? null : new TableDataRowAdapter(tableDataRow);
-    }
-
-    private static class InsertClosure implements IgniteTree.InvokeClosure<TableDataRow> {
-        final TableFreeList freeList;
-
-        TableDataRow newRow;
-
-        @Nullable TableDataRow oldRow;
-
-        InsertClosure(TableFreeList freeList) {
-            this.freeList = freeList;
-        }
-
-        /** {@inheritDoc} */
-        @Override
-        public void call(@Nullable TableDataRow oldRow) {
-            this.oldRow = oldRow;
-        }
-
-        /** {@inheritDoc} */
-        @Override
-        public @Nullable TableDataRow newRow() {
-            assert newRow != null;
-
-            try {
-                freeList.insertDataRow(newRow);
-            } catch (IgniteInternalCheckedException e) {
-                throw new IgniteInternalException(e);
-            }
-
-            return newRow;
-        }
-
-        /** {@inheritDoc} */
-        @Override
-        public IgniteTree.OperationType operationType() {
-            return oldRow == null ? IgniteTree.OperationType.PUT : IgniteTree.OperationType.NOOP;
-        }
-
-        void reset() {
-            newRow = null;
-
-            oldRow = null;
-        }
-    }
-
-    private static class RemoveExactClosure implements IgniteTree.InvokeClosure<TableDataRow> {
-        TableDataRow forRemoveRow;
-
-        @Nullable TableDataRow foundRow;
-
-        /** {@inheritDoc} */
-        @Override
-        public void call(@Nullable TableDataRow oldRow) {
-            assert forRemoveRow != null;
-
-            if (oldRow != null && oldRow.value().equals(forRemoveRow.value())) {
-                foundRow = oldRow;
-            }
-        }
-
-        /** {@inheritDoc} */
-        @Override
-        public @Nullable TableDataRow newRow() {
-            return null;
-        }
-
-        /** {@inheritDoc} */
-        @Override
-        public IgniteTree.OperationType operationType() {
-            return foundRow == null ? IgniteTree.OperationType.NOOP : IgniteTree.OperationType.REMOVE;
-        }
-
-        void reset() {
-            forRemoveRow = null;
-
-            foundRow = null;
-        }
+    ) {
+        super(partId, freeList, tree);
     }
 }
diff --git a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/VolatilePageMemoryTableStorage.java b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/VolatilePageMemoryTableStorage.java
index 46fd5f8f2..5b95b67e0 100644
--- a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/VolatilePageMemoryTableStorage.java
+++ b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/VolatilePageMemoryTableStorage.java
@@ -24,13 +24,14 @@ import org.apache.ignite.configuration.schemas.table.TableConfiguration;
 import org.apache.ignite.configuration.schemas.table.TableView;
 import org.apache.ignite.internal.pagememory.util.PageLockListenerNoOp;
 import org.apache.ignite.internal.storage.StorageException;
-import org.apache.ignite.internal.storage.pagememory.mv.PageMemoryMvPartitionStorage;
+import org.apache.ignite.internal.storage.pagememory.mv.VersionChainTree;
+import org.apache.ignite.internal.storage.pagememory.mv.VolatilePageMemoryMvPartitionStorage;
 import org.apache.ignite.lang.IgniteInternalCheckedException;
 
 /**
  * Implementation of {@link AbstractPageMemoryTableStorage} for in-memory case.
  */
-class VolatilePageMemoryTableStorage extends AbstractPageMemoryTableStorage {
+public class VolatilePageMemoryTableStorage extends AbstractPageMemoryTableStorage {
     private VolatilePageMemoryDataRegion dataRegion;
 
     /**
@@ -48,17 +49,31 @@ class VolatilePageMemoryTableStorage extends AbstractPageMemoryTableStorage {
     /** {@inheritDoc} */
     @Override
     protected VolatilePageMemoryPartitionStorage createPartitionStorage(int partId) throws StorageException {
-        TableFreeList tableFreeList = dataRegion.tableFreeList();
-
-        TableTree tableTree = createTableTree(tableCfg.value(), partId, tableFreeList);
+        TableTree tableTree = createTableTree(partId, tableCfg.value());
 
         return new VolatilePageMemoryPartitionStorage(
                 partId,
-                tableFreeList,
+                dataRegion.tableFreeList(),
                 tableTree
         );
     }
 
+    /** {@inheritDoc} */
+    @Override
+    public VolatilePageMemoryMvPartitionStorage createMvPartitionStorage(int partitionId) throws StorageException {
+        VersionChainTree versionChainTree = createVersionChainTree(partitionId, tableCfg.value());
+
+        return new VolatilePageMemoryMvPartitionStorage(
+                partitionId,
+                tableCfg.value(),
+                dataRegion.pageMemory(),
+                dataRegion.versionChainFreeList(),
+                dataRegion.rowVersionFreeList(),
+                versionChainTree
+        );
+    }
+
+    /** {@inheritDoc} */
     @Override
     public boolean isVolatile() {
         return true;
@@ -70,35 +85,48 @@ class VolatilePageMemoryTableStorage extends AbstractPageMemoryTableStorage {
         stop();
     }
 
-    /** {@inheritDoc} */
-    @Override
-    public PageMemoryMvPartitionStorage createMvPartitionStorage(int partitionId) {
-        return new PageMemoryMvPartitionStorage(
-                partitionId,
-                tableCfg.value(),
-                dataRegion,
-                dataRegion.versionChainFreeList(),
-                dataRegion.rowVersionFreeList()
-        );
+    /**
+     * Returns new {@link TableTree} instance for partition.
+     *
+     * @param partId Partition ID.
+     * @param tableView Table configuration.
+     * @throws StorageException If failed.
+     */
+    TableTree createTableTree(int partId, TableView tableView) throws StorageException {
+        int grpId = tableView.tableId();
+
+        try {
+            return new TableTree(
+                    grpId,
+                    tableView.name(),
+                    partId,
+                    dataRegion.pageMemory(),
+                    PageLockListenerNoOp.INSTANCE,
+                    new AtomicLong(),
+                    dataRegion.pageMemory().allocatePage(grpId, partId, FLAG_AUX),
+                    dataRegion.tableFreeList(),
+                    true
+            );
+        } catch (IgniteInternalCheckedException e) {
+            throw new StorageException(
+                    String.format("Error creating TableTree [tableName=%s, partitionId=%s]", tableView.name(), partId),
+                    e
+            );
+        }
     }
 
     /**
      * Returns new {@link TableTree} instance for partition.
      *
-     * @param tableView Table configuration.
      * @param partId Partition ID.
-     * @param freeList Table free list.
+     * @param tableView Table configuration.
      * @throws StorageException If failed.
      */
-    TableTree createTableTree(
-            TableView tableView,
-            int partId,
-            TableFreeList freeList
-    ) throws StorageException {
+    VersionChainTree createVersionChainTree(int partId, TableView tableView) throws StorageException {
         int grpId = tableView.tableId();
 
         try {
-            return new TableTree(
+            return new VersionChainTree(
                     grpId,
                     tableView.name(),
                     partId,
@@ -106,7 +134,7 @@ class VolatilePageMemoryTableStorage extends AbstractPageMemoryTableStorage {
                     PageLockListenerNoOp.INSTANCE,
                     new AtomicLong(),
                     dataRegion.pageMemory().allocatePage(grpId, partId, FLAG_AUX),
-                    freeList,
+                    dataRegion.versionChainFreeList(),
                     true
             );
         } catch (IgniteInternalCheckedException e) {
diff --git a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/PageMemoryMvPartitionStorage.java b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/AbstractPageMemoryMvPartitionStorage.java
similarity index 91%
rename from modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/PageMemoryMvPartitionStorage.java
rename to modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/AbstractPageMemoryMvPartitionStorage.java
index 26ddc159f..904251a3a 100644
--- a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/PageMemoryMvPartitionStorage.java
+++ b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/AbstractPageMemoryMvPartitionStorage.java
@@ -17,20 +17,16 @@
 
 package org.apache.ignite.internal.storage.pagememory.mv;
 
-import static org.apache.ignite.internal.pagememory.PageIdAllocator.FLAG_AUX;
-
 import java.nio.ByteBuffer;
 import java.util.NoSuchElementException;
 import java.util.Objects;
 import java.util.UUID;
-import java.util.concurrent.atomic.AtomicLong;
 import java.util.function.BiConsumer;
 import java.util.function.Predicate;
 import org.apache.ignite.configuration.schemas.table.TableView;
-import org.apache.ignite.internal.pagememory.DataRegion;
+import org.apache.ignite.internal.pagememory.PageMemory;
 import org.apache.ignite.internal.pagememory.datapage.DataPageReader;
 import org.apache.ignite.internal.pagememory.metric.IoStatisticsHolderNoOp;
-import org.apache.ignite.internal.pagememory.util.PageLockListenerNoOp;
 import org.apache.ignite.internal.schema.BinaryRow;
 import org.apache.ignite.internal.schema.ByteBufferRow;
 import org.apache.ignite.internal.storage.MvPartitionStorage;
@@ -44,11 +40,11 @@ import org.apache.ignite.lang.IgniteInternalCheckedException;
 import org.jetbrains.annotations.Nullable;
 
 /**
- * Implementation of {@link MvPartitionStorage} using Page Memory.
+ * Abstract implementation of {@link MvPartitionStorage} using Page Memory.
  *
  * @see MvPartitionStorage
  */
-public class PageMemoryMvPartitionStorage implements MvPartitionStorage {
+public abstract class AbstractPageMemoryMvPartitionStorage implements MvPartitionStorage {
     private static final byte[] TOMBSTONE_PAYLOAD = new byte[0];
 
     private static final Predicate<BinaryRow> MATCH_ALL = row -> true;
@@ -78,54 +74,32 @@ public class PageMemoryMvPartitionStorage implements MvPartitionStorage {
 
     /**
      * Constructor.
+     *
+     * @param partId Partition id.
+     * @param tableView Table configuration.
+     * @param pageMemory Page memory.
+     * @param versionChainFreeList Free list for {@link VersionChain}.
+     * @param rowVersionFreeList Free list for {@link RowVersion}.
+     * @param versionChainTree Table tree for {@link VersionChain}.
      */
-    public PageMemoryMvPartitionStorage(
+    public AbstractPageMemoryMvPartitionStorage(
             int partId,
-            TableView tableConfig,
-            DataRegion<?> dataRegion,
+            TableView tableView,
+            PageMemory pageMemory,
             VersionChainFreeList versionChainFreeList,
-            RowVersionFreeList rowVersionFreeList
+            RowVersionFreeList rowVersionFreeList,
+            VersionChainTree versionChainTree
     ) {
         this.partId = partId;
 
         this.versionChainFreeList = versionChainFreeList;
         this.rowVersionFreeList = rowVersionFreeList;
+        this.versionChainTree = versionChainTree;
 
-        groupId = tableConfig.tableId();
-
-        try {
-            versionChainTree = createVersionChainTree(partId, tableConfig, dataRegion, versionChainFreeList);
-        } catch (IgniteInternalCheckedException e) {
-            throw new StorageException("Error occurred while creating the partition storage", e);
-        }
-
-        versionChainDataPageReader = new VersionChainDataPageReader(dataRegion.pageMemory(), groupId, IoStatisticsHolderNoOp.INSTANCE);
-        rowVersionDataPageReader = new DataPageReader(dataRegion.pageMemory(), groupId, IoStatisticsHolderNoOp.INSTANCE);
-    }
+        groupId = tableView.tableId();
 
-    private VersionChainTree createVersionChainTree(
-            int partId,
-            TableView tableConfig,
-            DataRegion<?> dataRegion,
-            VersionChainFreeList versionChainFreeList1
-    ) throws IgniteInternalCheckedException {
-        // TODO: IGNITE-17085 It is necessary to do getting the tree root for the persistent case.
-        long metaPageId = dataRegion.pageMemory().allocatePage(groupId, partId, FLAG_AUX);
-
-        // TODO: IGNITE-17085 It is necessary to take into account the persistent case.
-        boolean initNew = true;
-
-        return new VersionChainTree(
-                groupId,
-                tableConfig.name(),
-                partId,
-                dataRegion.pageMemory(),
-                PageLockListenerNoOp.INSTANCE,
-                new AtomicLong(),
-                metaPageId,
-                versionChainFreeList1,
-                initNew
-        );
+        versionChainDataPageReader = new VersionChainDataPageReader(pageMemory, groupId, IoStatisticsHolderNoOp.INSTANCE);
+        rowVersionDataPageReader = new DataPageReader(pageMemory, groupId, IoStatisticsHolderNoOp.INSTANCE);
     }
 
     /** {@inheritDoc} */
@@ -150,6 +124,7 @@ public class PageMemoryMvPartitionStorage implements MvPartitionStorage {
     @Override
     public @Nullable BinaryRow read(RowId rowId, UUID txId) throws TxIdMismatchException, StorageException {
         VersionChain versionChain = findVersionChain(rowId);
+
         if (versionChain == null) {
             return null;
         }
@@ -161,6 +136,7 @@ public class PageMemoryMvPartitionStorage implements MvPartitionStorage {
     @Override
     public @Nullable BinaryRow read(RowId rowId, Timestamp timestamp) throws StorageException {
         VersionChain versionChain = findVersionChain(rowId);
+
         if (versionChain == null) {
             return null;
         }
@@ -189,6 +165,7 @@ public class PageMemoryMvPartitionStorage implements MvPartitionStorage {
 
     private @Nullable ByteBufferRow findLatestRowVersion(VersionChain versionChain, UUID txId, Predicate<BinaryRow> keyFilter) {
         RowVersion rowVersion = findLatestRowVersion(versionChain, ALWAYS_LOAD_VALUE);
+
         ByteBufferRow row = rowVersionToBinaryRow(rowVersion);
 
         if (!keyFilter.test(row)) {
@@ -220,7 +197,9 @@ public class PageMemoryMvPartitionStorage implements MvPartitionStorage {
 
     private ReadRowVersion freshReadRowVersion() {
         ReadRowVersion traversal = readRowVersionCache.get();
+
         traversal.reset();
+
         return traversal;
     }
 
@@ -250,6 +229,7 @@ public class PageMemoryMvPartitionStorage implements MvPartitionStorage {
             return findLatestRowVersion(versionChain, transactionId, keyFilter);
         } else {
             ByteBufferRow row = findRowVersionByTimestamp(versionChain, timestamp);
+
             return keyFilter.test(row) ? row : null;
         }
     }
@@ -275,7 +255,9 @@ public class PageMemoryMvPartitionStorage implements MvPartitionStorage {
 
     private ScanVersionChainByTimestamp freshScanByTimestamp() {
         ScanVersionChainByTimestamp traversal = scanVersionChainByTimestampCache.get();
+
         traversal.reset();
+
         return traversal;
     }
 
@@ -358,9 +340,11 @@ public class PageMemoryMvPartitionStorage implements MvPartitionStorage {
 
     private VersionChain findVersionChainForModification(RowId rowId) {
         VersionChain currentChain = findVersionChain(rowId);
+
         if (currentChain == null) {
             throw new RowIdIsInvalidForModificationsException();
         }
+
         return currentChain;
     }
 
@@ -375,6 +359,7 @@ public class PageMemoryMvPartitionStorage implements MvPartitionStorage {
         }
 
         RowVersion latestVersion = findLatestRowVersion(currentVersionChain, ALWAYS_LOAD_VALUE);
+
         assert latestVersion.isUncommitted();
 
         removeRowVersion(latestVersion);
@@ -393,6 +378,7 @@ public class PageMemoryMvPartitionStorage implements MvPartitionStorage {
                     // as aborts are pretty rare; let's strive for internal consistency for now and write the correct value.
                     latestCommittedVersion.nextLink()
             );
+
             updateVersionChain(currentVersionChain, versionChainReplacement);
         } else {
             // it was the only version, let's remove the chain as well
@@ -426,6 +412,7 @@ public class PageMemoryMvPartitionStorage implements MvPartitionStorage {
     @Override
     public void commitWrite(RowId rowId, Timestamp timestamp) throws StorageException {
         VersionChain currentVersionChain = findVersionChainForModification(rowId);
+
         long chainLink = PartitionlessLinks.addPartitionIdToPartititionlessLink(currentVersionChain.headLink(), partId);
 
         if (currentVersionChain.transactionId() == null) {
@@ -450,13 +437,14 @@ public class PageMemoryMvPartitionStorage implements MvPartitionStorage {
         try {
             rowVersionFreeList.removeDataRowByLink(currentVersion.link());
         } catch (IgniteInternalCheckedException e) {
-            throw new StorageException("Cannot update row version");
+            throw new StorageException("Cannot update row version", e);
         }
     }
 
     private void updateVersionChain(VersionChain currentVersionChain, VersionChain versionChainReplacement) {
         try {
             boolean updatedInPlace = versionChainFreeList.updateDataRow(currentVersionChain.link(), versionChainReplacement);
+
             if (!updatedInPlace) {
                 throw new StorageException("Only in-place updates are supported");
             }
@@ -481,6 +469,7 @@ public class PageMemoryMvPartitionStorage implements MvPartitionStorage {
         assert transactionId != null ^ timestamp != null;
 
         IgniteCursor<VersionChain> treeCursor;
+
         try {
             treeCursor = versionChainTree.find(null, null);
         } catch (IgniteInternalCheckedException e) {
@@ -539,6 +528,7 @@ public class PageMemoryMvPartitionStorage implements MvPartitionStorage {
             if (nextRow != null) {
                 return true;
             }
+
             if (iterationExhausted) {
                 return false;
             }
diff --git a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/PageMemoryMvStorageIoModule.java b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/PageMemoryMvStorageIoModule.java
index 4778fbc9a..c7afdd9ef 100644
--- a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/PageMemoryMvStorageIoModule.java
+++ b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/PageMemoryMvStorageIoModule.java
@@ -28,7 +28,8 @@ import org.apache.ignite.internal.storage.pagememory.mv.io.VersionChainLeafIo;
 import org.apache.ignite.internal.storage.pagememory.mv.io.VersionChainMetaIo;
 
 /**
- * {@link PageIoModule} related to {@link PageMemoryMvPartitionStorage} implementation.
+ * {@link PageIoModule} related to {@link VolatilePageMemoryMvPartitionStorage} and {@link PersistentPageMemoryMvPartitionStorage}
+ * implementations.
  */
 public class PageMemoryMvStorageIoModule implements PageIoModule {
     /** {@inheritDoc} */
diff --git a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/PersistentPageMemoryMvPartitionStorage.java b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/PersistentPageMemoryMvPartitionStorage.java
new file mode 100644
index 000000000..70ff1fd28
--- /dev/null
+++ b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/PersistentPageMemoryMvPartitionStorage.java
@@ -0,0 +1,69 @@
+/*
+ * 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.storage.pagememory.mv;
+
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.pagememory.persistence.PersistentPageMemory;
+import org.apache.ignite.internal.pagememory.persistence.checkpoint.CheckpointTimeoutLock;
+import org.apache.ignite.internal.pagememory.tree.BplusTree;
+import org.apache.ignite.internal.storage.MvPartitionStorage;
+import org.apache.ignite.internal.storage.StorageException;
+
+/**
+ * Implementation of {@link MvPartitionStorage} based on a {@link BplusTree} for persistent case.
+ */
+public class PersistentPageMemoryMvPartitionStorage extends AbstractPageMemoryMvPartitionStorage {
+    private final CheckpointTimeoutLock checkpointTimeoutLock;
+
+    /**
+     * Constructor.
+     *
+     * @param partId Partition id.
+     * @param tableView Table configuration.
+     * @param pageMemory Page memory.
+     * @param versionChainFreeList Free list for {@link VersionChain}.
+     * @param rowVersionFreeList Free list for {@link RowVersion}.
+     * @param versionChainTree Table tree for {@link VersionChain}.
+     * @param checkpointTimeoutLock Checkpoint timeout lock.
+     */
+    public PersistentPageMemoryMvPartitionStorage(
+            int partId,
+            TableView tableView,
+            PersistentPageMemory pageMemory,
+            VersionChainFreeList versionChainFreeList,
+            RowVersionFreeList rowVersionFreeList,
+            VersionChainTree versionChainTree,
+            CheckpointTimeoutLock checkpointTimeoutLock
+    ) {
+        super(partId, tableView, pageMemory, versionChainFreeList, rowVersionFreeList, versionChainTree);
+
+        this.checkpointTimeoutLock = checkpointTimeoutLock;
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public <V> V runConsistently(WriteClosure<V> closure) throws StorageException {
+        checkpointTimeoutLock.checkpointReadLock();
+
+        try {
+            return closure.execute();
+        } finally {
+            checkpointTimeoutLock.checkpointReadUnlock();
+        }
+    }
+}
diff --git a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/VersionChainFreeList.java b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/VersionChainFreeList.java
index 07d60a92a..3a3349459 100644
--- a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/VersionChainFreeList.java
+++ b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/VersionChainFreeList.java
@@ -26,7 +26,6 @@ import org.apache.ignite.internal.pagememory.evict.PageEvictionTracker;
 import org.apache.ignite.internal.pagememory.freelist.AbstractFreeList;
 import org.apache.ignite.internal.pagememory.io.PageIo;
 import org.apache.ignite.internal.pagememory.metric.IoStatisticsHolder;
-import org.apache.ignite.internal.pagememory.reuse.ReuseList;
 import org.apache.ignite.internal.pagememory.util.PageHandler;
 import org.apache.ignite.internal.pagememory.util.PageLockListener;
 import org.apache.ignite.internal.storage.pagememory.mv.io.VersionChainDataIo;
@@ -51,7 +50,6 @@ public class VersionChainFreeList extends AbstractFreeList<VersionChain> {
      * @param grpId Group ID.
      * @param partId Partition ID.
      * @param pageMem Page memory.
-     * @param reuseList Reuse list to use.
      * @param lockLsnr Page lock listener.
      * @param metaPageId Metadata page ID.
      * @param initNew {@code True} if new metadata should be initialized.
@@ -64,7 +62,6 @@ public class VersionChainFreeList extends AbstractFreeList<VersionChain> {
             int grpId,
             int partId,
             PageMemory pageMem,
-            ReuseList reuseList,
             PageLockListener lockLsnr,
             long metaPageId,
             boolean initNew,
@@ -77,7 +74,7 @@ public class VersionChainFreeList extends AbstractFreeList<VersionChain> {
                 partId,
                 "VersionChainFreeList_" + grpId,
                 pageMem,
-                reuseList,
+                null,
                 lockLsnr,
                 LOG,
                 metaPageId,
diff --git a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/VolatilePageMemoryMvPartitionStorage.java b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/VolatilePageMemoryMvPartitionStorage.java
new file mode 100644
index 000000000..c076d1ccd
--- /dev/null
+++ b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/VolatilePageMemoryMvPartitionStorage.java
@@ -0,0 +1,49 @@
+/*
+ * 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.storage.pagememory.mv;
+
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.pagememory.inmemory.VolatilePageMemory;
+import org.apache.ignite.internal.pagememory.tree.BplusTree;
+import org.apache.ignite.internal.storage.MvPartitionStorage;
+
+/**
+ * Implementation of {@link MvPartitionStorage} based on a {@link BplusTree} for in-memory case.
+ */
+public class VolatilePageMemoryMvPartitionStorage extends AbstractPageMemoryMvPartitionStorage {
+    /**
+     * Constructor.
+     *
+     * @param partId Partition id.
+     * @param tableView Table configuration.
+     * @param pageMemory Page memory.
+     * @param versionChainFreeList Free list for {@link VersionChain}.
+     * @param rowVersionFreeList Free list for {@link RowVersion}.
+     * @param versionChainTree Table tree for {@link VersionChain}.
+     */
+    public VolatilePageMemoryMvPartitionStorage(
+            int partId,
+            TableView tableView,
+            VolatilePageMemory pageMemory,
+            VersionChainFreeList versionChainFreeList,
+            RowVersionFreeList rowVersionFreeList,
+            VersionChainTree versionChainTree
+    ) {
+        super(partId, tableView, pageMemory, versionChainFreeList, rowVersionFreeList, versionChainTree);
+    }
+}
diff --git a/modules/storage-page-memory/src/test/java/org/apache/ignite/internal/storage/pagememory/mv/PageMemoryMvPartitionStorageTest.java b/modules/storage-page-memory/src/test/java/org/apache/ignite/internal/storage/pagememory/mv/AbstractPageMemoryMvPartitionStorageTest.java
similarity index 52%
rename from modules/storage-page-memory/src/test/java/org/apache/ignite/internal/storage/pagememory/mv/PageMemoryMvPartitionStorageTest.java
rename to modules/storage-page-memory/src/test/java/org/apache/ignite/internal/storage/pagememory/mv/AbstractPageMemoryMvPartitionStorageTest.java
index b2f91d2a0..e93b41e43 100644
--- a/modules/storage-page-memory/src/test/java/org/apache/ignite/internal/storage/pagememory/mv/PageMemoryMvPartitionStorageTest.java
+++ b/modules/storage-page-memory/src/test/java/org/apache/ignite/internal/storage/pagememory/mv/AbstractPageMemoryMvPartitionStorageTest.java
@@ -21,47 +21,30 @@ import static java.util.stream.Collectors.joining;
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.hamcrest.Matchers.is;
 import static org.hamcrest.Matchers.nullValue;
-import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertThrows;
 
 import java.nio.file.Path;
-import java.util.concurrent.TimeUnit;
 import java.util.stream.IntStream;
-import org.apache.ignite.configuration.schemas.store.UnknownDataStorageConfigurationSchema;
-import org.apache.ignite.configuration.schemas.table.ConstantValueDefaultConfigurationSchema;
-import org.apache.ignite.configuration.schemas.table.FunctionCallDefaultConfigurationSchema;
-import org.apache.ignite.configuration.schemas.table.HashIndexConfigurationSchema;
-import org.apache.ignite.configuration.schemas.table.NullValueDefaultConfigurationSchema;
-import org.apache.ignite.configuration.schemas.table.TableConfiguration;
 import org.apache.ignite.internal.configuration.testframework.ConfigurationExtension;
-import org.apache.ignite.internal.configuration.testframework.InjectConfiguration;
-import org.apache.ignite.internal.pagememory.configuration.schema.UnsafeMemoryAllocatorConfigurationSchema;
 import org.apache.ignite.internal.pagememory.io.PageIoRegistry;
 import org.apache.ignite.internal.schema.BinaryRow;
 import org.apache.ignite.internal.storage.AbstractMvPartitionStorageTest;
 import org.apache.ignite.internal.storage.RowId;
-import org.apache.ignite.internal.storage.pagememory.AbstractPageMemoryTableStorage;
-import org.apache.ignite.internal.storage.pagememory.VolatilePageMemoryStorageEngine;
-import org.apache.ignite.internal.storage.pagememory.configuration.schema.VolatilePageMemoryDataStorageChange;
-import org.apache.ignite.internal.storage.pagememory.configuration.schema.VolatilePageMemoryDataStorageConfigurationSchema;
-import org.apache.ignite.internal.storage.pagememory.configuration.schema.VolatilePageMemoryDataStorageView;
-import org.apache.ignite.internal.storage.pagememory.configuration.schema.VolatilePageMemoryStorageEngineConfiguration;
-import org.apache.ignite.internal.storage.pagememory.configuration.schema.VolatilePageMemoryStorageEngineConfigurationSchema;
 import org.apache.ignite.internal.testframework.WorkDirectory;
 import org.apache.ignite.internal.testframework.WorkDirectoryExtension;
 import org.apache.ignite.internal.tx.Timestamp;
 import org.apache.ignite.internal.util.Cursor;
-import org.apache.ignite.internal.util.IgniteUtils;
-import org.junit.jupiter.api.AfterEach;
-import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Test;
 import org.junit.jupiter.api.extension.ExtendWith;
 
+/**
+ * Base test for MV partition storages based on PageMemory.
+ */
 @ExtendWith(ConfigurationExtension.class)
 @ExtendWith(WorkDirectoryExtension.class)
-class PageMemoryMvPartitionStorageTest extends AbstractMvPartitionStorageTest<PageMemoryMvPartitionStorage> {
-    private final PageIoRegistry ioRegistry = new PageIoRegistry();
+abstract class AbstractPageMemoryMvPartitionStorageTest extends AbstractMvPartitionStorageTest {
+    protected final PageIoRegistry ioRegistry = new PageIoRegistry();
 
     private final BinaryRow binaryRow3 = binaryRow(key, new TestValue(22, "bar3"));
 
@@ -69,59 +52,15 @@ class PageMemoryMvPartitionStorageTest extends AbstractMvPartitionStorageTest<Pa
         ioRegistry.loadFromServiceLoader();
     }
 
-    @InjectConfiguration(polymorphicExtensions = UnsafeMemoryAllocatorConfigurationSchema.class)
-    private VolatilePageMemoryStorageEngineConfiguration engineConfig;
-
-    @InjectConfiguration(
-            name = "table",
-            polymorphicExtensions = {
-                    HashIndexConfigurationSchema.class,
-                    UnknownDataStorageConfigurationSchema.class,
-                    VolatilePageMemoryDataStorageConfigurationSchema.class,
-                    ConstantValueDefaultConfigurationSchema.class,
-                    FunctionCallDefaultConfigurationSchema.class,
-                    NullValueDefaultConfigurationSchema.class,
-            }
-    )
-    private TableConfiguration tableCfg;
-
-    private VolatilePageMemoryStorageEngine engine;
-
-    private AbstractPageMemoryTableStorage table;
-
     @WorkDirectory
-    private Path workDir;
-
-    @BeforeEach
-    void setUp() throws Exception {
-        engine = new VolatilePageMemoryStorageEngine(engineConfig, ioRegistry);
+    protected Path workDir;
 
-        engine.start();
-
-        tableCfg
-                .change(c -> c.changeDataStorage(dsc -> dsc.convert(VolatilePageMemoryDataStorageChange.class)))
-                .get(1, TimeUnit.SECONDS);
-
-        assertEquals(
-                VolatilePageMemoryStorageEngineConfigurationSchema.DEFAULT_DATA_REGION_NAME,
-                ((VolatilePageMemoryDataStorageView) tableCfg.dataStorage().value()).dataRegion()
-        );
-
-        table = engine.createTable(tableCfg);
-        table.start();
-
-        storage = table.createMvPartitionStorage(partitionId());
-    }
-
-    @AfterEach
-    void tearDown() throws Exception {
-        IgniteUtils.closeAll(
-                storage,
-                table == null ? null : table::stop,
-                engine == null ? null : engine::stop
-        );
-    }
+    /**
+     * Returns page size in bytes.
+     */
+    abstract int pageSize();
 
+    /** {@inheritDoc} */
     @Override
     protected int partitionId() {
         // 1 instead of the default 0 to make sure that we note cases when we forget to pass the partition ID (in which
@@ -129,6 +68,7 @@ class PageMemoryMvPartitionStorageTest extends AbstractMvPartitionStorageTest<Pa
         return 1;
     }
 
+    /** {@inheritDoc} */
     @SuppressWarnings("JUnit3StyleTestMethodInJUnit4Class")
     @Override
     public void testReadsFromEmpty() {
@@ -140,24 +80,27 @@ class PageMemoryMvPartitionStorageTest extends AbstractMvPartitionStorageTest<Pa
 
     @Test
     void abortOfInsertMakesRowIdInvalidForAddWrite() {
-        RowId rowId = storage.insert(binaryRow, newTransactionId());
-        storage.abortWrite(rowId);
+        RowId rowId = insert(binaryRow, newTransactionId());
+
+        abortWrite(rowId);
 
-        assertThrows(RowIdIsInvalidForModificationsException.class, () -> storage.addWrite(rowId, binaryRow2, txId));
+        assertThrows(RowIdIsInvalidForModificationsException.class, () -> addWrite(rowId, binaryRow2, txId));
     }
 
     @Test
     void abortOfInsertMakesRowIdInvalidForCommitWrite() {
-        RowId rowId = storage.insert(binaryRow, newTransactionId());
-        storage.abortWrite(rowId);
+        RowId rowId = insert(binaryRow, newTransactionId());
+
+        abortWrite(rowId);
 
-        assertThrows(RowIdIsInvalidForModificationsException.class, () -> storage.commitWrite(rowId, Timestamp.nextVersion()));
+        assertThrows(RowIdIsInvalidForModificationsException.class, () -> commitWrite(rowId, Timestamp.nextVersion()));
     }
 
     @Test
     void abortOfInsertMakesRowIdInvalidForAbortWrite() {
-        RowId rowId = storage.insert(binaryRow, newTransactionId());
-        storage.abortWrite(rowId);
+        RowId rowId = insert(binaryRow, newTransactionId());
+
+        abortWrite(rowId);
 
         assertThrows(RowIdIsInvalidForModificationsException.class, () -> storage.abortWrite(rowId));
     }
@@ -165,15 +108,16 @@ class PageMemoryMvPartitionStorageTest extends AbstractMvPartitionStorageTest<Pa
     @Test
     void uncommittedMultiPageValuesAreReadSuccessfully() {
         BinaryRow longRow = rowStoredInFragments();
-        LinkRowId rowId = storage.insert(longRow, txId);
 
-        BinaryRow foundRow = storage.read(rowId, txId);
+        RowId rowId = insert(longRow, txId);
+
+        BinaryRow foundRow = read(rowId, txId);
 
         assertRowMatches(foundRow, longRow);
     }
 
     private BinaryRow rowStoredInFragments() {
-        int pageSize = engineConfig.pageSize().value();
+        int pageSize = pageSize();
 
         // A repetitive pattern of 19 different characters (19 is chosen as a prime number) to reduce probability of 'lucky' matches
         // hiding bugs.
@@ -189,10 +133,11 @@ class PageMemoryMvPartitionStorageTest extends AbstractMvPartitionStorageTest<Pa
     void committedMultiPageValuesAreReadSuccessfully() {
         BinaryRow longRow = rowStoredInFragments();
 
-        LinkRowId rowId = storage.insert(longRow, txId);
-        storage.commitWrite(rowId, Timestamp.nextVersion());
+        RowId rowId = insert(longRow, txId);
 
-        BinaryRow foundRow = storage.read(rowId, Timestamp.nextVersion());
+        commitWrite(rowId, Timestamp.nextVersion());
+
+        BinaryRow foundRow = read(rowId, Timestamp.nextVersion());
 
         assertRowMatches(foundRow, longRow);
     }
@@ -200,7 +145,8 @@ class PageMemoryMvPartitionStorageTest extends AbstractMvPartitionStorageTest<Pa
     @Test
     void uncommittedMultiPageValuesWorkWithScans() throws Exception {
         BinaryRow longRow = rowStoredInFragments();
-        storage.insert(longRow, txId);
+
+        insert(longRow, txId);
 
         try (Cursor<BinaryRow> cursor = storage.scan(row -> true, txId)) {
             BinaryRow foundRow = cursor.next();
@@ -213,8 +159,9 @@ class PageMemoryMvPartitionStorageTest extends AbstractMvPartitionStorageTest<Pa
     void committedMultiPageValuesWorkWithScans() throws Exception {
         BinaryRow longRow = rowStoredInFragments();
 
-        LinkRowId rowId = storage.insert(longRow, txId);
-        storage.commitWrite(rowId, Timestamp.nextVersion());
+        RowId rowId = insert(longRow, txId);
+
+        commitWrite(rowId, Timestamp.nextVersion());
 
         try (Cursor<BinaryRow> cursor = storage.scan(row -> true, txId)) {
             BinaryRow foundRow = cursor.next();
@@ -233,15 +180,18 @@ class PageMemoryMvPartitionStorageTest extends AbstractMvPartitionStorageTest<Pa
     }
 
     private RowId commitAbortAndAddUncommitted() {
-        RowId rowId = storage.insert(binaryRow, txId);
-        storage.commitWrite(rowId, Timestamp.nextVersion());
+        return storage.runConsistently(() -> {
+            RowId rowId = storage.insert(binaryRow, txId);
 
-        storage.addWrite(rowId, binaryRow2, newTransactionId());
-        storage.abortWrite(rowId);
+            storage.commitWrite(rowId, Timestamp.nextVersion());
 
-        storage.addWrite(rowId, binaryRow3, newTransactionId());
+            storage.addWrite(rowId, binaryRow2, newTransactionId());
+            storage.abortWrite(rowId);
 
-        return rowId;
+            storage.addWrite(rowId, binaryRow3, newTransactionId());
+
+            return rowId;
+        });
     }
 
     @Test
@@ -259,9 +209,9 @@ class PageMemoryMvPartitionStorageTest extends AbstractMvPartitionStorageTest<Pa
 
     @Test
     void readByTimestampWorksCorrectlyIfNoUncommittedValueExists() {
-        LinkRowId rowId = storage.insert(binaryRow, txId);
+        RowId rowId = insert(binaryRow, txId);
 
-        BinaryRow foundRow = storage.read(rowId, Timestamp.nextVersion());
+        BinaryRow foundRow = read(rowId, Timestamp.nextVersion());
 
         assertThat(foundRow, is(nullValue()));
     }
diff --git a/modules/storage-page-memory/src/test/java/org/apache/ignite/internal/storage/pagememory/mv/PersistentPageMemoryMvPartitionStorageTest.java b/modules/storage-page-memory/src/test/java/org/apache/ignite/internal/storage/pagememory/mv/PersistentPageMemoryMvPartitionStorageTest.java
new file mode 100644
index 000000000..23f677003
--- /dev/null
+++ b/modules/storage-page-memory/src/test/java/org/apache/ignite/internal/storage/pagememory/mv/PersistentPageMemoryMvPartitionStorageTest.java
@@ -0,0 +1,127 @@
+/*
+ * 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.storage.pagememory.mv;
+
+import static org.apache.ignite.internal.pagememory.persistence.checkpoint.CheckpointState.FINISHED;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+import java.util.concurrent.TimeUnit;
+import org.apache.ignite.configuration.schemas.store.UnknownDataStorageConfigurationSchema;
+import org.apache.ignite.configuration.schemas.table.ConstantValueDefaultConfigurationSchema;
+import org.apache.ignite.configuration.schemas.table.FunctionCallDefaultConfigurationSchema;
+import org.apache.ignite.configuration.schemas.table.HashIndexConfigurationSchema;
+import org.apache.ignite.configuration.schemas.table.NullValueDefaultConfigurationSchema;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.internal.components.LongJvmPauseDetector;
+import org.apache.ignite.internal.configuration.testframework.InjectConfiguration;
+import org.apache.ignite.internal.logger.Loggers;
+import org.apache.ignite.internal.pagememory.configuration.schema.UnsafeMemoryAllocatorConfigurationSchema;
+import org.apache.ignite.internal.storage.RowId;
+import org.apache.ignite.internal.storage.pagememory.PersistentPageMemoryStorageEngine;
+import org.apache.ignite.internal.storage.pagememory.PersistentPageMemoryTableStorage;
+import org.apache.ignite.internal.storage.pagememory.configuration.schema.PersistentPageMemoryDataStorageChange;
+import org.apache.ignite.internal.storage.pagememory.configuration.schema.PersistentPageMemoryDataStorageConfigurationSchema;
+import org.apache.ignite.internal.storage.pagememory.configuration.schema.PersistentPageMemoryDataStorageView;
+import org.apache.ignite.internal.storage.pagememory.configuration.schema.PersistentPageMemoryStorageEngineConfiguration;
+import org.apache.ignite.internal.storage.pagememory.configuration.schema.PersistentPageMemoryStorageEngineConfigurationSchema;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+class PersistentPageMemoryMvPartitionStorageTest extends AbstractPageMemoryMvPartitionStorageTest {
+    @InjectConfiguration(polymorphicExtensions = UnsafeMemoryAllocatorConfigurationSchema.class)
+    private PersistentPageMemoryStorageEngineConfiguration engineConfig;
+
+    @InjectConfiguration(
+            name = "table",
+            polymorphicExtensions = {
+                    HashIndexConfigurationSchema.class,
+                    UnknownDataStorageConfigurationSchema.class,
+                    PersistentPageMemoryDataStorageConfigurationSchema.class,
+                    ConstantValueDefaultConfigurationSchema.class,
+                    FunctionCallDefaultConfigurationSchema.class,
+                    NullValueDefaultConfigurationSchema.class,
+            }
+    )
+    private TableConfiguration tableCfg;
+
+    private LongJvmPauseDetector longJvmPauseDetector;
+
+    private PersistentPageMemoryStorageEngine engine;
+
+    private PersistentPageMemoryTableStorage table;
+
+    @BeforeEach
+    void setUp() throws Exception {
+        longJvmPauseDetector = new LongJvmPauseDetector("test", Loggers.forClass(LongJvmPauseDetector.class));
+
+        longJvmPauseDetector.start();
+
+        engine = new PersistentPageMemoryStorageEngine("test", engineConfig, ioRegistry, workDir, longJvmPauseDetector);
+
+        engine.start();
+
+        tableCfg
+                .change(c -> c.changeDataStorage(dsc -> dsc.convert(PersistentPageMemoryDataStorageChange.class)))
+                .get(1, TimeUnit.SECONDS);
+
+        assertEquals(
+                PersistentPageMemoryStorageEngineConfigurationSchema.DEFAULT_DATA_REGION_NAME,
+                ((PersistentPageMemoryDataStorageView) tableCfg.dataStorage().value()).dataRegion()
+        );
+
+        table = engine.createTable(tableCfg);
+        table.start();
+
+        storage = table.createMvPartitionStorage(partitionId());
+    }
+
+    @AfterEach
+    void tearDown() throws Exception {
+        IgniteUtils.closeAll(
+                storage,
+                table == null ? null : table::stop,
+                engine == null ? null : engine::stop,
+                longJvmPauseDetector == null ? null : longJvmPauseDetector::stop
+        );
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    int pageSize() {
+        return engineConfig.pageSize().value();
+    }
+
+    @Test
+    void testReadAfterRestart() throws Exception {
+        RowId rowId = insert(binaryRow, txId);
+
+        engine
+                .checkpointManager()
+                .forceCheckpoint("before_stop_engine")
+                .futureFor(FINISHED)
+                .get(1, TimeUnit.SECONDS);
+
+        tearDown();
+
+        setUp();
+
+        assertRowMatches(binaryRow, read(rowId, txId));
+    }
+}
diff --git a/modules/storage-page-memory/src/test/java/org/apache/ignite/internal/storage/pagememory/mv/VolatilePageMemoryMvPartitionStorageTest.java b/modules/storage-page-memory/src/test/java/org/apache/ignite/internal/storage/pagememory/mv/VolatilePageMemoryMvPartitionStorageTest.java
new file mode 100644
index 000000000..e650d5fbe
--- /dev/null
+++ b/modules/storage-page-memory/src/test/java/org/apache/ignite/internal/storage/pagememory/mv/VolatilePageMemoryMvPartitionStorageTest.java
@@ -0,0 +1,103 @@
+/*
+ * 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.storage.pagememory.mv;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+import java.nio.file.Path;
+import java.util.concurrent.TimeUnit;
+import org.apache.ignite.configuration.schemas.store.UnknownDataStorageConfigurationSchema;
+import org.apache.ignite.configuration.schemas.table.ConstantValueDefaultConfigurationSchema;
+import org.apache.ignite.configuration.schemas.table.FunctionCallDefaultConfigurationSchema;
+import org.apache.ignite.configuration.schemas.table.HashIndexConfigurationSchema;
+import org.apache.ignite.configuration.schemas.table.NullValueDefaultConfigurationSchema;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.internal.configuration.testframework.InjectConfiguration;
+import org.apache.ignite.internal.pagememory.configuration.schema.UnsafeMemoryAllocatorConfigurationSchema;
+import org.apache.ignite.internal.storage.pagememory.VolatilePageMemoryStorageEngine;
+import org.apache.ignite.internal.storage.pagememory.VolatilePageMemoryTableStorage;
+import org.apache.ignite.internal.storage.pagememory.configuration.schema.VolatilePageMemoryDataStorageChange;
+import org.apache.ignite.internal.storage.pagememory.configuration.schema.VolatilePageMemoryDataStorageConfigurationSchema;
+import org.apache.ignite.internal.storage.pagememory.configuration.schema.VolatilePageMemoryDataStorageView;
+import org.apache.ignite.internal.storage.pagememory.configuration.schema.VolatilePageMemoryStorageEngineConfiguration;
+import org.apache.ignite.internal.storage.pagememory.configuration.schema.VolatilePageMemoryStorageEngineConfigurationSchema;
+import org.apache.ignite.internal.testframework.WorkDirectory;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+
+class VolatilePageMemoryMvPartitionStorageTest extends AbstractPageMemoryMvPartitionStorageTest {
+    @InjectConfiguration(polymorphicExtensions = UnsafeMemoryAllocatorConfigurationSchema.class)
+    private VolatilePageMemoryStorageEngineConfiguration engineConfig;
+
+    @InjectConfiguration(
+            name = "table",
+            polymorphicExtensions = {
+                    HashIndexConfigurationSchema.class,
+                    UnknownDataStorageConfigurationSchema.class,
+                    VolatilePageMemoryDataStorageConfigurationSchema.class,
+                    ConstantValueDefaultConfigurationSchema.class,
+                    FunctionCallDefaultConfigurationSchema.class,
+                    NullValueDefaultConfigurationSchema.class,
+            }
+    )
+    private TableConfiguration tableCfg;
+
+    private VolatilePageMemoryStorageEngine engine;
+
+    private VolatilePageMemoryTableStorage table;
+
+    @WorkDirectory
+    private Path workDir;
+
+    @BeforeEach
+    void setUp() throws Exception {
+        engine = new VolatilePageMemoryStorageEngine(engineConfig, ioRegistry);
+
+        engine.start();
+
+        tableCfg
+                .change(c -> c.changeDataStorage(dsc -> dsc.convert(VolatilePageMemoryDataStorageChange.class)))
+                .get(1, TimeUnit.SECONDS);
+
+        assertEquals(
+                VolatilePageMemoryStorageEngineConfigurationSchema.DEFAULT_DATA_REGION_NAME,
+                ((VolatilePageMemoryDataStorageView) tableCfg.dataStorage().value()).dataRegion()
+        );
+
+        table = engine.createTable(tableCfg);
+        table.start();
+
+        storage = table.createMvPartitionStorage(partitionId());
+    }
+
+    @AfterEach
+    void tearDown() throws Exception {
+        IgniteUtils.closeAll(
+                storage,
+                table == null ? null : table::stop,
+                engine == null ? null : engine::stop
+        );
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    int pageSize() {
+        return engineConfig.pageSize().value();
+    }
+}
diff --git a/modules/storage-rocksdb/src/test/java/org/apache/ignite/internal/storage/rocksdb/RocksDbMvPartitionStorageTest.java b/modules/storage-rocksdb/src/test/java/org/apache/ignite/internal/storage/rocksdb/RocksDbMvPartitionStorageTest.java
index cd5bc54a7..0291ea553 100644
--- a/modules/storage-rocksdb/src/test/java/org/apache/ignite/internal/storage/rocksdb/RocksDbMvPartitionStorageTest.java
+++ b/modules/storage-rocksdb/src/test/java/org/apache/ignite/internal/storage/rocksdb/RocksDbMvPartitionStorageTest.java
@@ -48,7 +48,7 @@ import org.junit.jupiter.api.extension.ExtendWith;
  */
 @ExtendWith(WorkDirectoryExtension.class)
 @ExtendWith(ConfigurationExtension.class)
-public class RocksDbMvPartitionStorageTest extends AbstractMvPartitionStorageTest<RocksDbMvPartitionStorage> {
+public class RocksDbMvPartitionStorageTest extends AbstractMvPartitionStorageTest {
     private RocksDbStorageEngine engine;
 
     private RocksDbTableStorage table;