You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by tk...@apache.org on 2022/10/20 08:24:54 UTC

[ignite-3] branch ignite-3.0.0-beta1 updated: IGNITE-17671 Implement BinaryTuple inlining in a sorted index B+Tree (#1231)

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

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


The following commit(s) were added to refs/heads/ignite-3.0.0-beta1 by this push:
     new 692b05396e IGNITE-17671 Implement BinaryTuple inlining in a sorted index B+Tree  (#1231)
692b05396e is described below

commit 692b05396e3d706d70937459eed6e5f35ff97eca
Author: Kirill Tkalenko <tk...@yandex.ru>
AuthorDate: Thu Oct 20 11:24:48 2022 +0300

    IGNITE-17671 Implement BinaryTuple inlining in a sorted index B+Tree  (#1231)
---
 .../index/AbstractSortedIndexStorageTest.java      |  93 +++++--------
 .../pagememory/index/IndexPageIoModule.java        |   6 +-
 .../storage/pagememory/index/IndexPageTypes.java   |  12 +-
 .../storage/pagememory/index/InlineUtils.java      |   2 +-
 .../hash/InsertHashIndexRowInvokeClosure.java      |   4 +-
 .../pagememory/index/hash/io/HashIndexTreeIo.java  |  40 +++---
 .../sorted/InsertSortedIndexRowInvokeClosure.java  |  18 ++-
 .../index/sorted/PageMemorySortedIndexStorage.java |   2 +-
 .../pagememory/index/sorted/SortedIndexTree.java   |  65 ++++++++-
 .../index/sorted/io/SortedIndexTreeInnerIo.java    |  20 ++-
 .../index/sorted/io/SortedIndexTreeIo.java         | 151 ++++++++++++++++-----
 .../index/sorted/io/SortedIndexTreeLeafIo.java     |  20 ++-
 .../index/sorted/io/SortedIndexTreeMetaIo.java     |  34 ++++-
 .../mv/AbstractPageMemoryMvPartitionStorage.java   |   5 +-
 .../AbstractPageMemoryHashIndexStorageTest.java    |  36 ++---
 .../AbstractPageMemorySortedIndexStorageTest.java  | 123 +++++++++++++++++
 ...PersistentPageMemorySortedIndexStorageTest.java |   5 +-
 .../VolatilePageMemorySortedIndexStorageTest.java  |   5 +-
 18 files changed, 456 insertions(+), 185 deletions(-)

diff --git a/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/AbstractSortedIndexStorageTest.java b/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/AbstractSortedIndexStorageTest.java
index 982bc87898..3b1d5ebc01 100644
--- a/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/AbstractSortedIndexStorageTest.java
+++ b/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/AbstractSortedIndexStorageTest.java
@@ -62,6 +62,7 @@ import org.apache.ignite.internal.schema.testutils.builder.SortedIndexDefinition
 import org.apache.ignite.internal.schema.testutils.builder.SortedIndexDefinitionBuilder.SortedIndexColumnBuilder;
 import org.apache.ignite.internal.schema.testutils.definition.ColumnDefinition;
 import org.apache.ignite.internal.schema.testutils.definition.ColumnType;
+import org.apache.ignite.internal.schema.testutils.definition.ColumnType.ColumnTypeSpec;
 import org.apache.ignite.internal.schema.testutils.definition.TableDefinition;
 import org.apache.ignite.internal.schema.testutils.definition.index.ColumnarIndexDefinition;
 import org.apache.ignite.internal.schema.testutils.definition.index.SortedIndexDefinition;
@@ -88,7 +89,7 @@ public abstract class AbstractSortedIndexStorageTest {
     /** Definitions of all supported column types. */
     public static final List<ColumnDefinition> ALL_TYPES_COLUMN_DEFINITIONS = allTypesColumnDefinitions();
 
-    private static final int TEST_PARTITION = 0;
+    protected static final int TEST_PARTITION = 0;
 
     private static List<ColumnDefinition> allTypesColumnDefinitions() {
         Stream<ColumnType> allColumnTypes = Stream.of(
@@ -189,7 +190,7 @@ public abstract class AbstractSortedIndexStorageTest {
     /**
      * Creates a Sorted Index using the given index definition.
      */
-    private SortedIndexStorage createIndexStorage(ColumnarIndexDefinition indexDefinition) {
+    protected SortedIndexStorage createIndexStorage(ColumnarIndexDefinition indexDefinition) {
         CompletableFuture<Void> createIndexFuture =
                 tablesCfg.indexes().change(chg -> chg.create(indexDefinition.name(), idx -> {
                     UUID tableId = ConfigurationUtil.internalId(tablesCfg.tables().value(), "foo");
@@ -218,7 +219,7 @@ public abstract class AbstractSortedIndexStorageTest {
 
         var serializer = new BinaryTupleRowSerializer(indexStorage.indexDescriptor());
 
-        IndexRow row = serializer.serializeRow(columns, new RowId(0));
+        IndexRow row = serializer.serializeRow(columns, new RowId(TEST_PARTITION));
 
         Object[] actual = serializer.deserializeColumns(row);
 
@@ -234,24 +235,19 @@ public abstract class AbstractSortedIndexStorageTest {
 
     @Test
     void testGet() throws Exception {
-        List<ColumnDefinition> columns = List.of(
-                column(ColumnType.string().typeSpec().name(), ColumnType.string()).asNullable(false).build(),
-                column(ColumnType.INT32.typeSpec().name(), ColumnType.INT32).asNullable(false).build()
-        );
-
         SortedIndexDefinition indexDefinition = SchemaBuilders.sortedIndex("TEST_INDEX")
-                .addIndexColumn(columns.get(0).name()).asc().done()
-                .addIndexColumn(columns.get(1).name()).asc().done()
+                .addIndexColumn(ColumnTypeSpec.STRING.name()).asc().done()
+                .addIndexColumn(ColumnTypeSpec.INT32.name()).asc().done()
                 .build();
 
         SortedIndexStorage index = createIndexStorage(indexDefinition);
 
         var serializer = new BinaryTupleRowSerializer(index.indexDescriptor());
 
-        IndexRow val1090 = serializer.serializeRow(new Object[]{ "10", 90 }, new RowId(0));
-        IndexRow otherVal1090 = serializer.serializeRow(new Object[]{ "10", 90 }, new RowId(0));
-        IndexRow val1080 = serializer.serializeRow(new Object[]{ "10", 80 }, new RowId(0));
-        IndexRow val2090 = serializer.serializeRow(new Object[]{ "20", 90 }, new RowId(0));
+        IndexRow val1090 = serializer.serializeRow(new Object[]{ "10", 90 }, new RowId(TEST_PARTITION));
+        IndexRow otherVal1090 = serializer.serializeRow(new Object[]{ "10", 90 }, new RowId(TEST_PARTITION));
+        IndexRow val1080 = serializer.serializeRow(new Object[]{ "10", 80 }, new RowId(TEST_PARTITION));
+        IndexRow val2090 = serializer.serializeRow(new Object[]{ "20", 90 }, new RowId(TEST_PARTITION));
 
         put(index, val1090);
         put(index, otherVal1090);
@@ -280,20 +276,15 @@ public abstract class AbstractSortedIndexStorageTest {
      */
     @Test
     void testPutIdempotence() throws Exception {
-        List<ColumnDefinition> columns = List.of(
-                column(ColumnType.string().typeSpec().name(), ColumnType.string()).asNullable(false).build(),
-                column(ColumnType.INT32.typeSpec().name(), ColumnType.INT32).asNullable(false).build()
-        );
-
         SortedIndexDefinition indexDefinition = SchemaBuilders.sortedIndex("TEST_INDEX")
-                .addIndexColumn(columns.get(0).name()).asc().done()
-                .addIndexColumn(columns.get(1).name()).asc().done()
+                .addIndexColumn(ColumnTypeSpec.STRING.name()).asc().done()
+                .addIndexColumn(ColumnTypeSpec.INT32.name()).asc().done()
                 .build();
 
         SortedIndexStorage index = createIndexStorage(indexDefinition);
 
         var columnValues = new Object[] { "foo", 1 };
-        var rowId = new RowId(0);
+        var rowId = new RowId(TEST_PARTITION);
 
         var serializer = new BinaryTupleRowSerializer(index.indexDescriptor());
 
@@ -312,23 +303,18 @@ public abstract class AbstractSortedIndexStorageTest {
      */
     @Test
     void testMultiplePuts() throws Exception {
-        List<ColumnDefinition> columns = List.of(
-                column(ColumnType.string().typeSpec().name(), ColumnType.string()).asNullable(false).build(),
-                column(ColumnType.INT32.typeSpec().name(), ColumnType.INT32).asNullable(false).build()
-        );
-
         SortedIndexDefinition indexDefinition = SchemaBuilders.sortedIndex("TEST_INDEX")
-                .addIndexColumn(columns.get(0).name()).asc().done()
-                .addIndexColumn(columns.get(1).name()).asc().done()
+                .addIndexColumn(ColumnTypeSpec.STRING.name()).asc().done()
+                .addIndexColumn(ColumnTypeSpec.INT32.name()).asc().done()
                 .build();
 
         SortedIndexStorage index = createIndexStorage(indexDefinition);
 
         var columnValues1 = new Object[] { "foo", 1 };
         var columnValues2 = new Object[] { "bar", 3 };
-        var rowId1 = new RowId(0);
-        var rowId2 = new RowId(0);
-        var rowId3 = new RowId(0);
+        var rowId1 = new RowId(TEST_PARTITION);
+        var rowId2 = new RowId(TEST_PARTITION);
+        var rowId3 = new RowId(TEST_PARTITION);
 
         var serializer = new BinaryTupleRowSerializer(index.indexDescriptor());
 
@@ -350,23 +336,18 @@ public abstract class AbstractSortedIndexStorageTest {
      */
     @Test
     void testRemove() throws Exception {
-        List<ColumnDefinition> columns = List.of(
-                column(ColumnType.string().typeSpec().name(), ColumnType.string()).asNullable(false).build(),
-                column(ColumnType.INT32.typeSpec().name(), ColumnType.INT32).asNullable(false).build()
-        );
-
         SortedIndexDefinition indexDefinition = SchemaBuilders.sortedIndex("TEST_INDEX")
-                .addIndexColumn(columns.get(0).name()).asc().done()
-                .addIndexColumn(columns.get(1).name()).asc().done()
+                .addIndexColumn(ColumnTypeSpec.STRING.name()).asc().done()
+                .addIndexColumn(ColumnTypeSpec.INT32.name()).asc().done()
                 .build();
 
         SortedIndexStorage index = createIndexStorage(indexDefinition);
 
         var columnValues1 = new Object[] { "foo", 1 };
         var columnValues2 = new Object[] { "bar", 3 };
-        var rowId1 = new RowId(0);
-        var rowId2 = new RowId(0);
-        var rowId3 = new RowId(0);
+        var rowId1 = new RowId(TEST_PARTITION);
+        var rowId2 = new RowId(TEST_PARTITION);
+        var rowId3 = new RowId(TEST_PARTITION);
 
         var serializer = new BinaryTupleRowSerializer(index.indexDescriptor());
 
@@ -454,19 +435,17 @@ public abstract class AbstractSortedIndexStorageTest {
 
     @Test
     public void testBoundsAndOrder() throws Exception {
-        List<ColumnDefinition> columns = List.of(
-                column(ColumnType.string().typeSpec().name(), ColumnType.string()).asNullable(false).build(),
-                column(ColumnType.INT32.typeSpec().name(), ColumnType.INT32).asNullable(false).build()
-        );
+        ColumnTypeSpec string = ColumnTypeSpec.STRING;
+        ColumnTypeSpec int32 = ColumnTypeSpec.INT32;
 
         SortedIndexDefinition index1Definition = SchemaBuilders.sortedIndex("TEST_INDEX_1")
-                .addIndexColumn(columns.get(0).name()).asc().done()
-                .addIndexColumn(columns.get(1).name()).asc().done()
+                .addIndexColumn(string.name()).asc().done()
+                .addIndexColumn(int32.name()).asc().done()
                 .build();
 
         SortedIndexDefinition index2Definition = SchemaBuilders.sortedIndex("TEST_INDEX_2")
-                .addIndexColumn(columns.get(0).name()).asc().done()
-                .addIndexColumn(columns.get(1).name()).desc().done()
+                .addIndexColumn(string.name()).asc().done()
+                .addIndexColumn(int32.name()).desc().done()
                 .build();
 
         SortedIndexStorage index1 = createIndexStorage(index1Definition);
@@ -480,10 +459,10 @@ public abstract class AbstractSortedIndexStorageTest {
         for (SortedIndexStorage index : Arrays.asList(index1, index2)) {
             var serializer = new BinaryTupleRowSerializer(index.indexDescriptor());
 
-            put(index, serializer.serializeRow(val1090, new RowId(0)));
-            put(index, serializer.serializeRow(val1080, new RowId(0)));
-            put(index, serializer.serializeRow(val2090, new RowId(0)));
-            put(index, serializer.serializeRow(val2080, new RowId(0)));
+            put(index, serializer.serializeRow(val1090, new RowId(TEST_PARTITION)));
+            put(index, serializer.serializeRow(val1080, new RowId(TEST_PARTITION)));
+            put(index, serializer.serializeRow(val2090, new RowId(TEST_PARTITION)));
+            put(index, serializer.serializeRow(val2080, new RowId(TEST_PARTITION)));
         }
 
         // Test without bounds.
@@ -579,7 +558,7 @@ public abstract class AbstractSortedIndexStorageTest {
 
         var serializer = new BinaryTupleRowSerializer(storage.indexDescriptor());
 
-        IndexRow nullRow = serializer.serializeRow(nullArray, new RowId(0));
+        IndexRow nullRow = serializer.serializeRow(nullArray, new RowId(TEST_PARTITION));
 
         var entry2 = new TestIndexRow(storage, serializer, nullRow, nullArray);
 
@@ -694,13 +673,13 @@ public abstract class AbstractSortedIndexStorageTest {
         }
     }
 
-    private static List<RowId> get(SortedIndexStorage index, BinaryTuple key) throws Exception {
+    protected static List<RowId> get(SortedIndexStorage index, BinaryTuple key) throws Exception {
         try (Cursor<RowId> cursor = index.get(key)) {
             return cursor.stream().collect(toUnmodifiableList());
         }
     }
 
-    private void put(SortedIndexStorage indexStorage, IndexRow row) {
+    protected void put(SortedIndexStorage indexStorage, IndexRow row) {
         partitionStorage.runConsistently(() -> {
             indexStorage.put(row);
 
diff --git a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/IndexPageIoModule.java b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/IndexPageIoModule.java
index 36a5f18207..3f6c43721e 100644
--- a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/IndexPageIoModule.java
+++ b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/IndexPageIoModule.java
@@ -55,12 +55,10 @@ public class IndexPageIoModule implements PageIoModule {
         ioVersions.addAll(HashIndexTreeInnerIo.VERSIONS);
         ioVersions.addAll(HashIndexTreeLeafIo.VERSIONS);
 
-        // TODO: IGNITE-17671 Need to implement for sorted indexes
-
         // Sorted index IO.
         ioVersions.add(SortedIndexTreeMetaIo.VERSIONS);
-        ioVersions.add(SortedIndexTreeInnerIo.VERSIONS);
-        ioVersions.add(SortedIndexTreeLeafIo.VERSIONS);
+        ioVersions.addAll(SortedIndexTreeInnerIo.VERSIONS);
+        ioVersions.addAll(SortedIndexTreeLeafIo.VERSIONS);
 
         return ioVersions;
     }
diff --git a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/IndexPageTypes.java b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/IndexPageTypes.java
index f91aee4d2c..8eab57d453 100644
--- a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/IndexPageTypes.java
+++ b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/IndexPageTypes.java
@@ -39,17 +39,15 @@ public interface IndexPageTypes {
     /** Starting hash index tree inner IO type. No more than the {@link InlineUtils#MAX_BINARY_TUPLE_INLINE_SIZE}. */
     short T_HASH_INDEX_INNER_IO_START = 10_000;
 
-    /** Starting hash index tree meta IO type. No more than the {@link InlineUtils#MAX_BINARY_TUPLE_INLINE_SIZE}. */
+    /** Starting hash index tree leaf IO type. No more than the {@link InlineUtils#MAX_BINARY_TUPLE_INLINE_SIZE}. */
     short T_HASH_INDEX_LEAF_IO_START = 15_000;
 
-    // TODO: IGNITE-17671 Need to implement for sorted indexes
-
     /** Sorted index tree meta IO type. */
     short T_SORTED_INDEX_META_IO = 105;
 
-    /** Sorted index tree inner IO type. */
-    short T_SORTED_INDEX_INNER_IO = 106;
+    /** Starting sorted index tree inner IO type. No more than the {@link InlineUtils#MAX_BINARY_TUPLE_INLINE_SIZE}. */
+    short T_SORTED_INDEX_INNER_IO_START = 20_000;
 
-    /** Sorted index tree meta IO type. */
-    short T_SORTED_INDEX_LEAF_IO = 107;
+    /** Starting sorted index tree leaf IO type. No more than the {@link InlineUtils#MAX_BINARY_TUPLE_INLINE_SIZE}. */
+    short T_SORTED_INDEX_LEAF_IO_START = 25_000;
 }
diff --git a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/InlineUtils.java b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/InlineUtils.java
index 4bf94bcffe..716f7dbb1f 100644
--- a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/InlineUtils.java
+++ b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/InlineUtils.java
@@ -202,7 +202,7 @@ public class InlineUtils {
      * @param indexColumnsSize Index columns size in bytes.
      * @param inlineSize Inline size in bytes.
      */
-    public static boolean isFullyInlined(int indexColumnsSize, int inlineSize) {
+    public static boolean canFullyInline(int indexColumnsSize, int inlineSize) {
         return indexColumnsSize <= inlineSize + PARTITIONLESS_LINK_SIZE_BYTES;
     }
 }
diff --git a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/hash/InsertHashIndexRowInvokeClosure.java b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/hash/InsertHashIndexRowInvokeClosure.java
index 80a86ca356..d27ebe5d56 100644
--- a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/hash/InsertHashIndexRowInvokeClosure.java
+++ b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/hash/InsertHashIndexRowInvokeClosure.java
@@ -18,7 +18,7 @@
 package org.apache.ignite.internal.storage.pagememory.index.hash;
 
 import static org.apache.ignite.internal.pagememory.util.PageIdUtils.NULL_LINK;
-import static org.apache.ignite.internal.storage.pagememory.index.InlineUtils.isFullyInlined;
+import static org.apache.ignite.internal.storage.pagememory.index.InlineUtils.canFullyInline;
 
 import org.apache.ignite.internal.pagememory.tree.IgniteTree.InvokeClosure;
 import org.apache.ignite.internal.pagememory.tree.IgniteTree.OperationType;
@@ -67,7 +67,7 @@ class InsertHashIndexRowInvokeClosure implements InvokeClosure<HashIndexRow> {
             return;
         }
 
-        if (!isFullyInlined(hashIndexRow.indexColumns().valueSize(), inlineSize)) {
+        if (!canFullyInline(hashIndexRow.indexColumns().valueSize(), inlineSize)) {
             freeList.insertDataRow(hashIndexRow.indexColumns());
         }
     }
diff --git a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/hash/io/HashIndexTreeIo.java b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/hash/io/HashIndexTreeIo.java
index c515c18444..85687562e8 100644
--- a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/hash/io/HashIndexTreeIo.java
+++ b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/hash/io/HashIndexTreeIo.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.storage.pagememory.index.hash.io;
 
+import static java.nio.ByteOrder.LITTLE_ENDIAN;
 import static org.apache.ignite.internal.pagememory.util.PageIdUtils.NULL_LINK;
 import static org.apache.ignite.internal.pagememory.util.PageUtils.getBytes;
 import static org.apache.ignite.internal.pagememory.util.PageUtils.getInt;
@@ -29,7 +30,7 @@ import static org.apache.ignite.internal.pagememory.util.PageUtils.putShort;
 import static org.apache.ignite.internal.pagememory.util.PartitionlessLinks.PARTITIONLESS_LINK_SIZE_BYTES;
 import static org.apache.ignite.internal.pagememory.util.PartitionlessLinks.readPartitionless;
 import static org.apache.ignite.internal.pagememory.util.PartitionlessLinks.writePartitionless;
-import static org.apache.ignite.internal.storage.pagememory.index.InlineUtils.isFullyInlined;
+import static org.apache.ignite.internal.storage.pagememory.index.InlineUtils.canFullyInline;
 import static org.apache.ignite.internal.util.GridUnsafe.wrapPointer;
 
 import java.nio.ByteBuffer;
@@ -64,10 +65,10 @@ public interface HashIndexTreeIo {
     /** Item size without index columns in bytes. */
     int ITEM_SIZE_WITHOUT_COLUMNS = Integer.BYTES // Index columns hash.
             + Short.SIZE // Inlined index columns size.
-            + PARTITIONLESS_LINK_SIZE_BYTES // Index column link.
+            + PARTITIONLESS_LINK_SIZE_BYTES // Index columns link.
             + 2 * Long.BYTES; // Row ID.
 
-    /** Index columns are not fully inlined, the index column size is {@link #indexColumnsInlineSize()}. */
+    /** Special value that is written to the Index Key, indicating the value has not been fully inlined into it. */
     short NOT_FULLY_INLINE = -1;
 
     /** Offset of the index columns hash (4 bytes). */
@@ -140,7 +141,7 @@ public interface HashIndexTreeIo {
 
         IndexColumns indexColumns = row.indexColumns();
 
-        if (isFullyInlined(indexColumns.valueSize(), indexColumnsInlineSize())) {
+        if (canFullyInline(indexColumns.valueSize(), indexColumnsInlineSize())) {
             assert indexColumns.link() == NULL_LINK : "Index columns are completely inline, they should not be in FreeList";
 
             putShort(pageAddr + off, SIZE_OFFSET, (short) indexColumns.valueSize());
@@ -149,7 +150,9 @@ public interface HashIndexTreeIo {
         } else {
             putShort(pageAddr + off, SIZE_OFFSET, NOT_FULLY_INLINE);
 
-            putByteBuffer(pageAddr + off, TUPLE_OFFSET, indexColumns.valueBuffer().rewind().duplicate().limit(indexColumnsInlineSize()));
+            ByteBuffer bufferToWrite = indexColumns.valueBuffer().rewind().duplicate().limit(indexColumnsInlineSize());
+
+            putByteBuffer(pageAddr + off, TUPLE_OFFSET, bufferToWrite);
 
             writePartitionless(pageAddr + off + linkOffset(), indexColumns.link());
         }
@@ -185,11 +188,7 @@ public interface HashIndexTreeIo {
 
         int indexColumnsSize = getShort(pageAddr + off, SIZE_OFFSET);
 
-        if (indexColumnsSize != NOT_FULLY_INLINE) {
-            ByteBuffer indexColumnsBuffer = wrapPointer(pageAddr + off + TUPLE_OFFSET, indexColumnsSize);
-
-            cmp = indexColumnsBuffer.compareTo(row.indexColumns().valueBuffer().rewind());
-        } else {
+        if (indexColumnsSize == NOT_FULLY_INLINE) {
             indexColumnsSize = indexColumnsInlineSize();
 
             ByteBuffer indexColumnsBuffer = wrapPointer(pageAddr + off + TUPLE_OFFSET, indexColumnsSize);
@@ -207,6 +206,10 @@ public interface HashIndexTreeIo {
             dataPageReader.traverse(link, compareIndexColumnsValue, row.indexColumns().valueBuffer().rewind().duplicate());
 
             cmp = compareIndexColumnsValue.compareResult();
+        } else {
+            ByteBuffer indexColumnsBuffer = wrapPointer(pageAddr + off + TUPLE_OFFSET, indexColumnsSize);
+
+            cmp = indexColumnsBuffer.compareTo(row.indexColumns().valueBuffer().rewind());
         }
 
         if (cmp != 0) {
@@ -244,28 +247,27 @@ public interface HashIndexTreeIo {
 
         int indexColumnsSize = getShort(pageAddr + off, SIZE_OFFSET);
 
-        ByteBuffer indexColumnsBuffer;
+        byte[] indexColumnsBytes;
 
         long link;
 
-        if (indexColumnsSize != NOT_FULLY_INLINE) {
-            indexColumnsBuffer = ByteBuffer.wrap(getBytes(pageAddr + off, TUPLE_OFFSET, indexColumnsSize));
-
-            link = NULL_LINK;
-        } else {
+        if (indexColumnsSize == NOT_FULLY_INLINE) {
             link = readPartitionless(partitionId, pageAddr + off, linkOffset());
 
             ReadIndexColumnsValue indexColumnsTraversal = new ReadIndexColumnsValue();
 
             dataPageReader.traverse(link, indexColumnsTraversal, null);
 
-            indexColumnsBuffer = ByteBuffer.wrap(indexColumnsTraversal.result());
+            indexColumnsBytes = indexColumnsTraversal.result();
+        } else {
+            indexColumnsBytes = getBytes(pageAddr + off, TUPLE_OFFSET, indexColumnsSize);
+
+            link = NULL_LINK;
         }
 
-        IndexColumns indexColumns = new IndexColumns(partitionId, link, indexColumnsBuffer);
+        IndexColumns indexColumns = new IndexColumns(partitionId, link, ByteBuffer.wrap(indexColumnsBytes).order(LITTLE_ENDIAN));
 
         long rowIdMsb = getLong(pageAddr + off, rowIdMsbOffset());
-
         long rowIdLsb = getLong(pageAddr + off, rowIdLsbOffset());
 
         RowId rowId = new RowId(partitionId, rowIdMsb, rowIdLsb);
diff --git a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/sorted/InsertSortedIndexRowInvokeClosure.java b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/sorted/InsertSortedIndexRowInvokeClosure.java
index 789123c5d8..7774b8dae3 100644
--- a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/sorted/InsertSortedIndexRowInvokeClosure.java
+++ b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/sorted/InsertSortedIndexRowInvokeClosure.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.storage.pagememory.index.sorted;
 
 import static org.apache.ignite.internal.pagememory.util.PageIdUtils.NULL_LINK;
+import static org.apache.ignite.internal.storage.pagememory.index.InlineUtils.canFullyInline;
 
 import org.apache.ignite.internal.pagememory.tree.IgniteTree.InvokeClosure;
 import org.apache.ignite.internal.pagememory.tree.IgniteTree.OperationType;
@@ -32,7 +33,7 @@ import org.jetbrains.annotations.Nullable;
  */
 class InsertSortedIndexRowInvokeClosure implements InvokeClosure<SortedIndexRow> {
     /** Sorted index row instance for insertion. */
-    private final SortedIndexRow hashIndexRow;
+    private final SortedIndexRow sortedIndexRow;
 
     /** Free list to insert data into in case of necessity. */
     private final IndexColumnsFreeList freeList;
@@ -40,17 +41,22 @@ class InsertSortedIndexRowInvokeClosure implements InvokeClosure<SortedIndexRow>
     /** Operation type, either {@link OperationType#PUT} or {@link OperationType#NOOP} depending on the tree state. */
     private OperationType operationType = OperationType.PUT;
 
+    /** Inline size in bytes. */
+    private final int inlineSize;
+
     /**
      * Constructor.
      *
      * @param sortedIndexRow Sorted index row instance for insertion.
      * @param freeList Free list to insert data into in case of necessity.
+     * @param inlineSize Inline size in bytes.
      */
-    public InsertSortedIndexRowInvokeClosure(SortedIndexRow sortedIndexRow, IndexColumnsFreeList freeList) {
+    public InsertSortedIndexRowInvokeClosure(SortedIndexRow sortedIndexRow, IndexColumnsFreeList freeList, int inlineSize) {
         assert sortedIndexRow.indexColumns().link() == NULL_LINK;
 
-        this.hashIndexRow = sortedIndexRow;
+        this.sortedIndexRow = sortedIndexRow;
         this.freeList = freeList;
+        this.inlineSize = inlineSize;
     }
 
     @Override
@@ -61,12 +67,14 @@ class InsertSortedIndexRowInvokeClosure implements InvokeClosure<SortedIndexRow>
             return;
         }
 
-        freeList.insertDataRow(hashIndexRow.indexColumns());
+        if (!canFullyInline(sortedIndexRow.indexColumns().valueSize(), inlineSize)) {
+            freeList.insertDataRow(sortedIndexRow.indexColumns());
+        }
     }
 
     @Override
     public @Nullable SortedIndexRow newRow() {
-        return hashIndexRow;
+        return sortedIndexRow;
     }
 
     @Override
diff --git a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/sorted/PageMemorySortedIndexStorage.java b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/sorted/PageMemorySortedIndexStorage.java
index 9decc68e50..11cfa44a5a 100644
--- a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/sorted/PageMemorySortedIndexStorage.java
+++ b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/sorted/PageMemorySortedIndexStorage.java
@@ -99,7 +99,7 @@ public class PageMemorySortedIndexStorage implements SortedIndexStorage {
         try {
             SortedIndexRow sortedIndexRow = toSortedIndexRow(row.indexColumns(), row.rowId());
 
-            var insert = new InsertSortedIndexRowInvokeClosure(sortedIndexRow, freeList);
+            var insert = new InsertSortedIndexRowInvokeClosure(sortedIndexRow, freeList, sortedIndexTree.inlineSize());
 
             sortedIndexTree.invoke(sortedIndexRow, null, insert);
         } catch (IgniteInternalCheckedException e) {
diff --git a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/sorted/SortedIndexTree.java b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/sorted/SortedIndexTree.java
index 4903c794e6..2217081f09 100644
--- a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/sorted/SortedIndexTree.java
+++ b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/sorted/SortedIndexTree.java
@@ -17,6 +17,8 @@
 
 package org.apache.ignite.internal.storage.pagememory.index.sorted;
 
+import static org.apache.ignite.internal.storage.pagememory.index.sorted.io.SortedIndexTreeIo.ITEM_SIZE_WITHOUT_COLUMNS;
+
 import java.util.concurrent.atomic.AtomicLong;
 import org.apache.ignite.internal.pagememory.PageMemory;
 import org.apache.ignite.internal.pagememory.datapage.DataPageReader;
@@ -26,6 +28,8 @@ import org.apache.ignite.internal.pagememory.tree.io.BplusIo;
 import org.apache.ignite.internal.pagememory.util.PageLockListener;
 import org.apache.ignite.internal.schema.BinaryTuple;
 import org.apache.ignite.internal.storage.index.BinaryTupleComparator;
+import org.apache.ignite.internal.storage.index.SortedIndexDescriptor;
+import org.apache.ignite.internal.storage.pagememory.index.InlineUtils;
 import org.apache.ignite.internal.storage.pagememory.index.sorted.io.SortedIndexTreeInnerIo;
 import org.apache.ignite.internal.storage.pagememory.index.sorted.io.SortedIndexTreeIo;
 import org.apache.ignite.internal.storage.pagememory.index.sorted.io.SortedIndexTreeLeafIo;
@@ -43,6 +47,9 @@ public class SortedIndexTree extends BplusTree<SortedIndexRowKey, SortedIndexRow
     /** Comparator of index columns {@link BinaryTuple}s. */
     private final BinaryTupleComparator binaryTupleComparator;
 
+    /** Inline size in bytes. */
+    private final int inlineSize;
+
     /**
      * Constructor.
      *
@@ -54,8 +61,8 @@ public class SortedIndexTree extends BplusTree<SortedIndexRowKey, SortedIndexRow
      * @param globalRmvId Remove ID.
      * @param metaPageId Meta page ID.
      * @param reuseList Reuse list.
+     * @param indexDescriptor Index descriptor.
      * @param initNew {@code True} if new tree should be created.
-     * @param binaryTupleComparator Comparator of index columns {@link BinaryTuple}s.
      * @throws IgniteInternalCheckedException If failed.
      */
     public SortedIndexTree(
@@ -67,18 +74,29 @@ public class SortedIndexTree extends BplusTree<SortedIndexRowKey, SortedIndexRow
             AtomicLong globalRmvId,
             long metaPageId,
             @Nullable ReuseList reuseList,
-            boolean initNew,
-            BinaryTupleComparator binaryTupleComparator
+            SortedIndexDescriptor indexDescriptor,
+            boolean initNew
     ) throws IgniteInternalCheckedException {
         super("SortedIndexTree_" + grpId, grpId, grpName, partId, pageMem, lockLsnr, globalRmvId, metaPageId, reuseList);
 
-        setIos(SortedIndexTreeInnerIo.VERSIONS, SortedIndexTreeLeafIo.VERSIONS, SortedIndexTreeMetaIo.VERSIONS);
+        inlineSize = initNew ? InlineUtils.binaryTupleInlineSize(pageSize(), ITEM_SIZE_WITHOUT_COLUMNS, indexDescriptor)
+                : readInlineSizeFromMetaIo();
+
+        setIos(
+                SortedIndexTreeInnerIo.VERSIONS.get(inlineSize),
+                SortedIndexTreeLeafIo.VERSIONS.get(inlineSize),
+                SortedIndexTreeMetaIo.VERSIONS
+        );
 
         dataPageReader = new DataPageReader(pageMem, grpId, statisticsHolder());
 
-        this.binaryTupleComparator = binaryTupleComparator;
+        binaryTupleComparator = new BinaryTupleComparator(indexDescriptor);
 
         initTree(initNew);
+
+        if (initNew) {
+            writeInlineSizeToMetaIo(inlineSize);
+        }
     }
 
     /**
@@ -109,4 +127,41 @@ public class SortedIndexTree extends BplusTree<SortedIndexRowKey, SortedIndexRow
 
         return sortedIndexTreeIo.getRow(dataPageReader, partId, pageAddr, idx);
     }
+
+    /**
+     * Returns inline size in bytes.
+     */
+    public int inlineSize() {
+        return inlineSize;
+    }
+
+    private int readInlineSizeFromMetaIo() throws IgniteInternalCheckedException {
+        Integer inlineSize = read(
+                metaPageId,
+                (groupId, pageId, page, pageAddr, io, arg, intArg, statHolder) -> ((SortedIndexTreeMetaIo) io).getInlineSize(pageAddr),
+                null,
+                0,
+                -1
+        );
+
+        assert inlineSize != -1;
+
+        return inlineSize;
+    }
+
+    private void writeInlineSizeToMetaIo(int inlineSize) throws IgniteInternalCheckedException {
+        Boolean result = write(
+                metaPageId,
+                (groupId, pageId, page, pageAddr, io, arg, intArg, statHolder) -> {
+                    ((SortedIndexTreeMetaIo) io).setInlineSize(pageAddr, inlineSize);
+
+                    return Boolean.TRUE;
+                },
+                0,
+                Boolean.FALSE,
+                statisticsHolder()
+        );
+
+        assert result == Boolean.TRUE : result;
+    }
 }
diff --git a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/sorted/io/SortedIndexTreeInnerIo.java b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/sorted/io/SortedIndexTreeInnerIo.java
index 73a8f41849..9385b484cf 100644
--- a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/sorted/io/SortedIndexTreeInnerIo.java
+++ b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/sorted/io/SortedIndexTreeInnerIo.java
@@ -17,11 +17,18 @@
 
 package org.apache.ignite.internal.storage.pagememory.index.sorted.io;
 
+import static java.util.stream.Collectors.toUnmodifiableList;
+import static org.apache.ignite.internal.storage.pagememory.index.IndexPageTypes.T_SORTED_INDEX_INNER_IO_START;
+import static org.apache.ignite.internal.storage.pagememory.index.InlineUtils.MAX_BINARY_TUPLE_INLINE_SIZE;
+
+import java.util.List;
+import java.util.stream.IntStream;
 import org.apache.ignite.internal.pagememory.io.IoVersions;
 import org.apache.ignite.internal.pagememory.tree.BplusTree;
 import org.apache.ignite.internal.pagememory.tree.io.BplusInnerIo;
 import org.apache.ignite.internal.pagememory.tree.io.BplusIo;
-import org.apache.ignite.internal.storage.pagememory.index.IndexPageTypes;
+import org.apache.ignite.internal.schema.BinaryTuple;
+import org.apache.ignite.internal.storage.pagememory.index.InlineUtils;
 import org.apache.ignite.internal.storage.pagememory.index.sorted.SortedIndexRowKey;
 import org.apache.ignite.internal.storage.pagememory.index.sorted.SortedIndexTree;
 import org.apache.ignite.lang.IgniteInternalCheckedException;
@@ -30,16 +37,19 @@ import org.apache.ignite.lang.IgniteInternalCheckedException;
  * {@link BplusInnerIo} implementation for {@link SortedIndexTree}.
  */
 public class SortedIndexTreeInnerIo extends BplusInnerIo<SortedIndexRowKey> implements SortedIndexTreeIo {
-    /** I/O versions. */
-    public static final IoVersions<SortedIndexTreeInnerIo> VERSIONS = new IoVersions<>(new SortedIndexTreeInnerIo(1));
+    /** I/O versions for each {@link BinaryTuple} inline size up to the {@link InlineUtils#MAX_BINARY_TUPLE_INLINE_SIZE}. */
+    public static final List<IoVersions<SortedIndexTreeInnerIo>> VERSIONS = IntStream.rangeClosed(0, MAX_BINARY_TUPLE_INLINE_SIZE)
+            .mapToObj(inlineSize -> new IoVersions<>(new SortedIndexTreeInnerIo(1, inlineSize)))
+            .collect(toUnmodifiableList());
 
     /**
      * Constructor.
      *
      * @param ver Page format version.
+     * @param inlineSize Inline size in bytes.
      */
-    protected SortedIndexTreeInnerIo(int ver) {
-        super(IndexPageTypes.T_SORTED_INDEX_INNER_IO, ver, true, SIZE_IN_BYTES);
+    private SortedIndexTreeInnerIo(int ver, int inlineSize) {
+        super(T_SORTED_INDEX_INNER_IO_START + inlineSize, ver, true, ITEM_SIZE_WITHOUT_COLUMNS + inlineSize);
     }
 
     @Override
diff --git a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/sorted/io/SortedIndexTreeIo.java b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/sorted/io/SortedIndexTreeIo.java
index 47345bb7f5..86a7bc6c85 100644
--- a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/sorted/io/SortedIndexTreeIo.java
+++ b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/sorted/io/SortedIndexTreeIo.java
@@ -18,11 +18,18 @@
 package org.apache.ignite.internal.storage.pagememory.index.sorted.io;
 
 import static java.nio.ByteOrder.LITTLE_ENDIAN;
+import static org.apache.ignite.internal.pagememory.util.PageIdUtils.NULL_LINK;
+import static org.apache.ignite.internal.pagememory.util.PageUtils.getBytes;
 import static org.apache.ignite.internal.pagememory.util.PageUtils.getLong;
+import static org.apache.ignite.internal.pagememory.util.PageUtils.getShort;
+import static org.apache.ignite.internal.pagememory.util.PageUtils.putByteBuffer;
 import static org.apache.ignite.internal.pagememory.util.PageUtils.putLong;
+import static org.apache.ignite.internal.pagememory.util.PageUtils.putShort;
 import static org.apache.ignite.internal.pagememory.util.PartitionlessLinks.PARTITIONLESS_LINK_SIZE_BYTES;
 import static org.apache.ignite.internal.pagememory.util.PartitionlessLinks.readPartitionless;
 import static org.apache.ignite.internal.pagememory.util.PartitionlessLinks.writePartitionless;
+import static org.apache.ignite.internal.storage.pagememory.index.InlineUtils.canFullyInline;
+import static org.apache.ignite.internal.util.GridUnsafe.wrapPointer;
 
 import java.nio.ByteBuffer;
 import java.util.UUID;
@@ -48,17 +55,47 @@ import org.apache.ignite.lang.IgniteInternalCheckedException;
  * </ul>
  */
 public interface SortedIndexTreeIo {
-    /** Offset of the index column link (6 bytes). */
-    int INDEX_COLUMNS_LINK_OFFSET = 0;
+    /** Item size without index columns in bytes. */
+    int ITEM_SIZE_WITHOUT_COLUMNS = Short.SIZE // Inlined index columns size.
+            + PARTITIONLESS_LINK_SIZE_BYTES // Index columns link.
+            + 2 * Long.BYTES; // Row ID.
 
-    /** Offset of rowId's most significant bits, 8 bytes. */
-    int ROW_ID_MSB_OFFSET = INDEX_COLUMNS_LINK_OFFSET + PARTITIONLESS_LINK_SIZE_BYTES;
+    /** Special value that is written to the Index Key, indicating the value has not been fully inlined into it. */
+    short NOT_FULLY_INLINE = -1;
 
-    /** Offset of rowId's least significant bits, 8 bytes. */
-    int ROW_ID_LSB_OFFSET = ROW_ID_MSB_OFFSET + Long.BYTES;
+    /** Offset of the index columns size (2 bytes). */
+    int SIZE_OFFSET = 0;
 
-    /** Payload size in bytes. */
-    int SIZE_IN_BYTES = ROW_ID_LSB_OFFSET + Long.BYTES;
+    /** Offset of the index columns tuple (N bytes). */
+    int TUPLE_OFFSET = SIZE_OFFSET + Short.BYTES;
+
+    /**
+     * Returns offset of the index columns link (6 bytes).
+     */
+    default int linkOffset() {
+        return TUPLE_OFFSET + indexColumnsInlineSize();
+    }
+
+    /**
+     * Returns offset of rowId's the most significant bits (8 bytes).
+     */
+    default int rowIdMsbOffset() {
+        return linkOffset() + PARTITIONLESS_LINK_SIZE_BYTES;
+    }
+
+    /**
+     * Returns offset of rowId's least significant bits (8 bytes).
+     */
+    default int rowIdLsbOffset() {
+        return rowIdMsbOffset() + Long.BYTES;
+    }
+
+    /**
+     * Returns item size in bytes.
+     *
+     * @see BplusIo#getItemSize()
+     */
+    int getItemSize();
 
     /**
      * Returns an offset of the element inside the page.
@@ -76,7 +113,7 @@ public interface SortedIndexTreeIo {
         int dstOffset = offset(dstIdx);
         int srcOffset = offset(srcIdx);
 
-        PageUtils.copyMemory(srcPageAddr, srcOffset, dstPageAddr, dstOffset, SIZE_IN_BYTES);
+        PageUtils.copyMemory(srcPageAddr, srcOffset, dstPageAddr, dstOffset, getItemSize());
     }
 
     /**
@@ -84,17 +121,33 @@ public interface SortedIndexTreeIo {
      *
      * @see BplusIo#storeByOffset(long, int, Object)
      */
-    default void storeByOffset(long pageAddr, int off, SortedIndexRowKey rowKey) {
+    default void storeByOffset(long pageAddr, final int off, SortedIndexRowKey rowKey) {
         assert rowKey instanceof SortedIndexRow;
 
-        SortedIndexRow sortedIndexRow = (SortedIndexRow) rowKey;
+        SortedIndexRow row = (SortedIndexRow) rowKey;
+
+        IndexColumns indexColumns = row.indexColumns();
+
+        if (canFullyInline(indexColumns.valueSize(), indexColumnsInlineSize())) {
+            assert indexColumns.link() == NULL_LINK : "Index columns are completely inline, they should not be in FreeList";
 
-        writePartitionless(pageAddr + off + INDEX_COLUMNS_LINK_OFFSET, sortedIndexRow.indexColumns().link());
+            putShort(pageAddr + off, SIZE_OFFSET, (short) indexColumns.valueSize());
 
-        RowId rowId = sortedIndexRow.rowId();
+            putByteBuffer(pageAddr + off, TUPLE_OFFSET, indexColumns.valueBuffer().rewind());
+        } else {
+            putShort(pageAddr + off, SIZE_OFFSET, NOT_FULLY_INLINE);
 
-        putLong(pageAddr, off + ROW_ID_MSB_OFFSET, rowId.mostSignificantBits());
-        putLong(pageAddr, off + ROW_ID_LSB_OFFSET, rowId.leastSignificantBits());
+            ByteBuffer bufferToWrite = indexColumns.valueBuffer().rewind().duplicate().limit(indexColumnsInlineSize());
+
+            putByteBuffer(pageAddr + off, TUPLE_OFFSET, bufferToWrite);
+
+            writePartitionless(pageAddr + off + linkOffset(), indexColumns.link());
+        }
+
+        RowId rowId = row.rowId();
+
+        putLong(pageAddr + off, rowIdMsbOffset(), rowId.mostSignificantBits());
+        putLong(pageAddr + off, rowIdLsbOffset(), rowId.leastSignificantBits());
     }
 
     /**
@@ -117,19 +170,28 @@ public interface SortedIndexTreeIo {
             int idx,
             SortedIndexRowKey rowKey
     ) throws IgniteInternalCheckedException {
-        int off = offset(idx);
+        final int off = offset(idx);
 
-        long link = readPartitionless(partitionId, pageAddr, off + INDEX_COLUMNS_LINK_OFFSET);
+        int indexColumnsSize = getShort(pageAddr + off, SIZE_OFFSET);
 
-        //TODO Add in-place compare in IGNITE-17671
-        ReadIndexColumnsValue indexColumnsTraversal = new ReadIndexColumnsValue();
+        ByteBuffer firstBinaryTupleBuffer;
 
-        dataPageReader.traverse(link, indexColumnsTraversal, null);
+        if (indexColumnsSize == NOT_FULLY_INLINE) {
+            // TODO: IGNITE-17325 Use a comparator for inlined tuple
+            long link = readPartitionless(partitionId, pageAddr + off, linkOffset());
+
+            ReadIndexColumnsValue indexColumnsTraversal = new ReadIndexColumnsValue();
+
+            dataPageReader.traverse(link, indexColumnsTraversal, null);
+
+            firstBinaryTupleBuffer = ByteBuffer.wrap(indexColumnsTraversal.result());
+        } else {
+            firstBinaryTupleBuffer = wrapPointer(pageAddr + off + TUPLE_OFFSET, indexColumnsSize);
+        }
 
-        ByteBuffer firstBinaryTupleBuffer = ByteBuffer.wrap(indexColumnsTraversal.result()).order(LITTLE_ENDIAN);
         ByteBuffer secondBinaryTupleBuffer = rowKey.indexColumns().valueBuffer();
 
-        int cmp = binaryTupleComparator.compare(firstBinaryTupleBuffer, secondBinaryTupleBuffer);
+        int cmp = binaryTupleComparator.compare(firstBinaryTupleBuffer.order(LITTLE_ENDIAN), secondBinaryTupleBuffer);
 
         if (cmp != 0) {
             return cmp;
@@ -137,19 +199,19 @@ public interface SortedIndexTreeIo {
 
         assert rowKey instanceof SortedIndexRow : rowKey;
 
-        SortedIndexRow sortedIndexRow = (SortedIndexRow) rowKey;
+        SortedIndexRow row = (SortedIndexRow) rowKey;
 
-        long rowIdMsb = getLong(pageAddr, off + ROW_ID_MSB_OFFSET);
+        long rowIdMsb = getLong(pageAddr + off, rowIdMsbOffset());
 
-        cmp = Long.compare(rowIdMsb, sortedIndexRow.rowId().mostSignificantBits());
+        cmp = Long.compare(rowIdMsb, row.rowId().mostSignificantBits());
 
         if (cmp != 0) {
             return cmp;
         }
 
-        long rowIdLsb = getLong(pageAddr, off + ROW_ID_LSB_OFFSET);
+        long rowIdLsb = getLong(pageAddr + off, rowIdLsbOffset());
 
-        return Long.compare(rowIdLsb, sortedIndexRow.rowId().leastSignificantBits());
+        return Long.compare(rowIdLsb, row.rowId().leastSignificantBits());
     }
 
     /**
@@ -164,23 +226,42 @@ public interface SortedIndexTreeIo {
      */
     default SortedIndexRow getRow(DataPageReader dataPageReader, int partitionId, long pageAddr, int idx)
             throws IgniteInternalCheckedException {
-        int off = offset(idx);
+        final int off = offset(idx);
 
-        long link = readPartitionless(partitionId, pageAddr, off + INDEX_COLUMNS_LINK_OFFSET);
+        int indexColumnsSize = getShort(pageAddr + off, SIZE_OFFSET);
 
-        ReadIndexColumnsValue indexColumnsTraversal = new ReadIndexColumnsValue();
+        ByteBuffer indexColumnsBuffer;
 
-        dataPageReader.traverse(link, indexColumnsTraversal, null);
+        long link;
 
-        ByteBuffer indexColumnsBuffer = ByteBuffer.wrap(indexColumnsTraversal.result()).order(LITTLE_ENDIAN);
+        if (indexColumnsSize == NOT_FULLY_INLINE) {
+            link = readPartitionless(partitionId, pageAddr + off, linkOffset());
 
-        IndexColumns indexColumns = new IndexColumns(partitionId, link, indexColumnsBuffer);
+            ReadIndexColumnsValue indexColumnsTraversal = new ReadIndexColumnsValue();
 
-        long rowIdMsb = getLong(pageAddr, off + ROW_ID_MSB_OFFSET);
-        long rowIdLsb = getLong(pageAddr, off + ROW_ID_LSB_OFFSET);
+            dataPageReader.traverse(link, indexColumnsTraversal, null);
+
+            indexColumnsBuffer = ByteBuffer.wrap(indexColumnsTraversal.result());
+        } else {
+            indexColumnsBuffer = ByteBuffer.wrap(getBytes(pageAddr + off, TUPLE_OFFSET, indexColumnsSize));
+
+            link = NULL_LINK;
+        }
+
+        IndexColumns indexColumns = new IndexColumns(partitionId, link, indexColumnsBuffer.order(LITTLE_ENDIAN));
+
+        long rowIdMsb = getLong(pageAddr + off, rowIdMsbOffset());
+        long rowIdLsb = getLong(pageAddr + off, rowIdLsbOffset());
 
         RowId rowId = new RowId(partitionId, rowIdMsb, rowIdLsb);
 
         return new SortedIndexRow(indexColumns, rowId);
     }
+
+    /**
+     * Returns the inline size for index columns in bytes.
+     */
+    default int indexColumnsInlineSize() {
+        return getItemSize() - ITEM_SIZE_WITHOUT_COLUMNS;
+    }
 }
diff --git a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/sorted/io/SortedIndexTreeLeafIo.java b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/sorted/io/SortedIndexTreeLeafIo.java
index 87a8bce44d..31bb799003 100644
--- a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/sorted/io/SortedIndexTreeLeafIo.java
+++ b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/sorted/io/SortedIndexTreeLeafIo.java
@@ -17,11 +17,18 @@
 
 package org.apache.ignite.internal.storage.pagememory.index.sorted.io;
 
+import static java.util.stream.Collectors.toUnmodifiableList;
+import static org.apache.ignite.internal.storage.pagememory.index.IndexPageTypes.T_SORTED_INDEX_LEAF_IO_START;
+import static org.apache.ignite.internal.storage.pagememory.index.InlineUtils.MAX_BINARY_TUPLE_INLINE_SIZE;
+
+import java.util.List;
+import java.util.stream.IntStream;
 import org.apache.ignite.internal.pagememory.io.IoVersions;
 import org.apache.ignite.internal.pagememory.tree.BplusTree;
 import org.apache.ignite.internal.pagememory.tree.io.BplusIo;
 import org.apache.ignite.internal.pagememory.tree.io.BplusLeafIo;
-import org.apache.ignite.internal.storage.pagememory.index.IndexPageTypes;
+import org.apache.ignite.internal.schema.BinaryTuple;
+import org.apache.ignite.internal.storage.pagememory.index.InlineUtils;
 import org.apache.ignite.internal.storage.pagememory.index.sorted.SortedIndexRowKey;
 import org.apache.ignite.internal.storage.pagememory.index.sorted.SortedIndexTree;
 import org.apache.ignite.lang.IgniteInternalCheckedException;
@@ -30,16 +37,19 @@ import org.apache.ignite.lang.IgniteInternalCheckedException;
  * {@link BplusLeafIo} implementation for {@link SortedIndexTree}.
  */
 public class SortedIndexTreeLeafIo extends BplusLeafIo<SortedIndexRowKey> implements SortedIndexTreeIo {
-    /** I/O versions. */
-    public static final IoVersions<SortedIndexTreeLeafIo> VERSIONS = new IoVersions<>(new SortedIndexTreeLeafIo(1));
+    /** I/O versions for each {@link BinaryTuple} inline size up to the {@link InlineUtils#MAX_BINARY_TUPLE_INLINE_SIZE}. */
+    public static final List<IoVersions<SortedIndexTreeLeafIo>> VERSIONS = IntStream.rangeClosed(0, MAX_BINARY_TUPLE_INLINE_SIZE)
+            .mapToObj(inlineSize -> new IoVersions<>(new SortedIndexTreeLeafIo(1, inlineSize)))
+            .collect(toUnmodifiableList());
 
     /**
      * Constructor.
      *
      * @param ver Page format version.
+     * @param inlineSize Inline size in bytes.
      */
-    protected SortedIndexTreeLeafIo(int ver) {
-        super(IndexPageTypes.T_SORTED_INDEX_LEAF_IO, ver, SIZE_IN_BYTES);
+    private SortedIndexTreeLeafIo(int ver, int inlineSize) {
+        super(T_SORTED_INDEX_LEAF_IO_START + inlineSize, ver, ITEM_SIZE_WITHOUT_COLUMNS + inlineSize);
     }
 
     @Override
diff --git a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/sorted/io/SortedIndexTreeMetaIo.java b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/sorted/io/SortedIndexTreeMetaIo.java
index 17a244191f..8d3fe19ff0 100644
--- a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/sorted/io/SortedIndexTreeMetaIo.java
+++ b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/sorted/io/SortedIndexTreeMetaIo.java
@@ -17,9 +17,13 @@
 
 package org.apache.ignite.internal.storage.pagememory.index.sorted.io;
 
+import static org.apache.ignite.internal.pagememory.util.PageUtils.getInt;
+import static org.apache.ignite.internal.pagememory.util.PageUtils.putInt;
+import static org.apache.ignite.internal.storage.pagememory.index.IndexPageTypes.T_SORTED_INDEX_META_IO;
+import static org.apache.ignite.internal.storage.pagememory.index.InlineUtils.MAX_BINARY_TUPLE_INLINE_SIZE;
+
 import org.apache.ignite.internal.pagememory.io.IoVersions;
 import org.apache.ignite.internal.pagememory.tree.io.BplusMetaIo;
-import org.apache.ignite.internal.storage.pagememory.index.IndexPageTypes;
 import org.apache.ignite.internal.storage.pagememory.index.sorted.SortedIndexTree;
 
 /**
@@ -29,12 +33,38 @@ public class SortedIndexTreeMetaIo extends BplusMetaIo {
     /** I/O versions. */
     public static final IoVersions<SortedIndexTreeMetaIo> VERSIONS = new IoVersions<>(new SortedIndexTreeMetaIo(1));
 
+    /** Offset of the inline size in bytes. */
+    private static final int INLINE_SIZE_OFFSET = COMMON_META_END;
+
     /**
      * Constructor.
      *
      * @param ver Page format version.
      */
     protected SortedIndexTreeMetaIo(int ver) {
-        super(IndexPageTypes.T_SORTED_INDEX_META_IO, ver);
+        super(T_SORTED_INDEX_META_IO, ver);
+    }
+
+    /**
+     * Returns the inline size in bytes.
+     *
+     * @param pageAddr Page address.
+     */
+    public int getInlineSize(long pageAddr) {
+        return getInt(pageAddr, INLINE_SIZE_OFFSET);
+    }
+
+    /**
+     * Sets the inline size.
+     *
+     * @param pageAddr Page address.
+     * @param inlineSize Inline size in bytes.
+     */
+    public void setInlineSize(long pageAddr, int inlineSize) {
+        assertPageType(pageAddr);
+
+        assert inlineSize > 0 && inlineSize <= MAX_BINARY_TUPLE_INLINE_SIZE : inlineSize;
+
+        putInt(pageAddr, INLINE_SIZE_OFFSET, inlineSize);
     }
 }
diff --git a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/AbstractPageMemoryMvPartitionStorage.java b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/AbstractPageMemoryMvPartitionStorage.java
index b79bb33555..eaf33041c3 100644
--- a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/AbstractPageMemoryMvPartitionStorage.java
+++ b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/AbstractPageMemoryMvPartitionStorage.java
@@ -49,7 +49,6 @@ import org.apache.ignite.internal.storage.ReadResult;
 import org.apache.ignite.internal.storage.RowId;
 import org.apache.ignite.internal.storage.StorageException;
 import org.apache.ignite.internal.storage.TxIdMismatchException;
-import org.apache.ignite.internal.storage.index.BinaryTupleComparator;
 import org.apache.ignite.internal.storage.index.HashIndexDescriptor;
 import org.apache.ignite.internal.storage.index.SortedIndexDescriptor;
 import org.apache.ignite.internal.storage.pagememory.AbstractPageMemoryTableStorage;
@@ -245,8 +244,8 @@ public abstract class AbstractPageMemoryMvPartitionStorage implements MvPartitio
                     new AtomicLong(),
                     metaPageId,
                     rowVersionFreeList,
-                    initNew,
-                    new BinaryTupleComparator(indexDescriptor)
+                    indexDescriptor,
+                    initNew
             );
 
             if (initNew) {
diff --git a/modules/storage-page-memory/src/test/java/org/apache/ignite/internal/storage/pagememory/index/AbstractPageMemoryHashIndexStorageTest.java b/modules/storage-page-memory/src/test/java/org/apache/ignite/internal/storage/pagememory/index/AbstractPageMemoryHashIndexStorageTest.java
index 351bace528..207171b22b 100644
--- a/modules/storage-page-memory/src/test/java/org/apache/ignite/internal/storage/pagememory/index/AbstractPageMemoryHashIndexStorageTest.java
+++ b/modules/storage-page-memory/src/test/java/org/apache/ignite/internal/storage/pagememory/index/AbstractPageMemoryHashIndexStorageTest.java
@@ -17,14 +17,13 @@
 
 package org.apache.ignite.internal.storage.pagememory.index;
 
-import static java.util.stream.Collectors.joining;
 import static org.apache.ignite.internal.storage.pagememory.index.InlineUtils.MAX_BINARY_TUPLE_INLINE_SIZE;
+import static org.apache.ignite.internal.testframework.IgniteTestUtils.randomString;
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.hamcrest.Matchers.contains;
 import static org.hamcrest.Matchers.empty;
 
-import java.util.concurrent.ThreadLocalRandom;
-import java.util.stream.IntStream;
+import java.util.Random;
 import org.apache.ignite.internal.pagememory.PageMemory;
 import org.apache.ignite.internal.schema.configuration.TablesConfiguration;
 import org.apache.ignite.internal.storage.RowId;
@@ -42,6 +41,8 @@ import org.junit.jupiter.api.Test;
 abstract class AbstractPageMemoryHashIndexStorageTest extends AbstractHashIndexStorageTest {
     protected BasePageMemoryStorageEngineConfiguration<?, ?> baseEngineConfig;
 
+    private final Random random = new Random();
+
     /**
      * Initializes the internal structures needed for tests.
      *
@@ -63,22 +64,10 @@ abstract class AbstractPageMemoryHashIndexStorageTest extends AbstractHashIndexS
     public void testDestroy() {
     }
 
-    @Test
-    void testWithHashCollisionStrings() {
-        IndexRow indexRow0 = createIndexRow(1, "foo0", new RowId(TEST_PARTITION));
-        IndexRow indexRow1 = createIndexRow(1, "foo1", new RowId(TEST_PARTITION));
-
-        put(indexRow0);
-        put(indexRow1);
-
-        assertThat(getAll(indexRow0), contains(indexRow0.rowId()));
-        assertThat(getAll(indexRow1), contains(indexRow1.rowId()));
-    }
-
     @Test
     void testWithStringsLargerThanMaximumInlineSize() {
-        IndexRow indexRow0 = createIndexRow(1, randomString(MAX_BINARY_TUPLE_INLINE_SIZE), new RowId(TEST_PARTITION));
-        IndexRow indexRow1 = createIndexRow(1, randomString(MAX_BINARY_TUPLE_INLINE_SIZE), new RowId(TEST_PARTITION));
+        IndexRow indexRow0 = createIndexRow(1, randomString(random, MAX_BINARY_TUPLE_INLINE_SIZE), new RowId(TEST_PARTITION));
+        IndexRow indexRow1 = createIndexRow(1, randomString(random, MAX_BINARY_TUPLE_INLINE_SIZE), new RowId(TEST_PARTITION));
 
         put(indexRow0);
         put(indexRow1);
@@ -91,8 +80,8 @@ abstract class AbstractPageMemoryHashIndexStorageTest extends AbstractHashIndexS
 
     @Test
     void testFragmentedIndexColumns() {
-        IndexRow indexRow0 = createIndexRow(1, randomString(baseEngineConfig.pageSize().value() * 2), new RowId(TEST_PARTITION));
-        IndexRow indexRow1 = createIndexRow(1, randomString(baseEngineConfig.pageSize().value() * 2), new RowId(TEST_PARTITION));
+        IndexRow indexRow0 = createIndexRow(1, randomString(random, baseEngineConfig.pageSize().value() * 2), new RowId(TEST_PARTITION));
+        IndexRow indexRow1 = createIndexRow(1, randomString(random, baseEngineConfig.pageSize().value() * 2), new RowId(TEST_PARTITION));
 
         put(indexRow0);
         put(indexRow1);
@@ -102,13 +91,4 @@ abstract class AbstractPageMemoryHashIndexStorageTest extends AbstractHashIndexS
 
         assertThat(getAll(createIndexRow(1, "foo", new RowId(TEST_PARTITION))), empty());
     }
-
-    private static String randomString(int size) {
-        ThreadLocalRandom random = ThreadLocalRandom.current();
-
-        return IntStream.range(0, size)
-                .map(i -> 'A' + random.nextInt('Z' - 'A'))
-                .mapToObj(i -> String.valueOf((char) i))
-                .collect(joining());
-    }
 }
diff --git a/modules/storage-page-memory/src/test/java/org/apache/ignite/internal/storage/pagememory/index/AbstractPageMemorySortedIndexStorageTest.java b/modules/storage-page-memory/src/test/java/org/apache/ignite/internal/storage/pagememory/index/AbstractPageMemorySortedIndexStorageTest.java
new file mode 100644
index 0000000000..d2073e8189
--- /dev/null
+++ b/modules/storage-page-memory/src/test/java/org/apache/ignite/internal/storage/pagememory/index/AbstractPageMemorySortedIndexStorageTest.java
@@ -0,0 +1,123 @@
+/*
+ * 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.index;
+
+import static org.apache.ignite.internal.storage.pagememory.index.InlineUtils.MAX_BINARY_TUPLE_INLINE_SIZE;
+import static org.apache.ignite.internal.testframework.IgniteTestUtils.randomString;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.empty;
+
+import java.util.Random;
+import org.apache.ignite.internal.pagememory.PageMemory;
+import org.apache.ignite.internal.schema.configuration.TablesConfiguration;
+import org.apache.ignite.internal.schema.testutils.builder.SchemaBuilders;
+import org.apache.ignite.internal.schema.testutils.definition.ColumnType.ColumnTypeSpec;
+import org.apache.ignite.internal.schema.testutils.definition.index.SortedIndexDefinition;
+import org.apache.ignite.internal.storage.RowId;
+import org.apache.ignite.internal.storage.engine.MvTableStorage;
+import org.apache.ignite.internal.storage.index.AbstractSortedIndexStorageTest;
+import org.apache.ignite.internal.storage.index.IndexRow;
+import org.apache.ignite.internal.storage.index.SortedIndexStorage;
+import org.apache.ignite.internal.storage.index.impl.BinaryTupleRowSerializer;
+import org.apache.ignite.internal.storage.pagememory.configuration.schema.BasePageMemoryStorageEngineConfiguration;
+import org.junit.jupiter.api.Test;
+
+/**
+ * Base class for testing {@link SortedIndexStorage} based on {@link PageMemory}.
+ */
+abstract class AbstractPageMemorySortedIndexStorageTest extends AbstractSortedIndexStorageTest {
+    protected BasePageMemoryStorageEngineConfiguration<?, ?> baseEngineConfig;
+
+    private final Random random = new Random();
+
+    /**
+     * Initializes the internal structures needed for tests.
+     *
+     * <p>This method *MUST* always be called in either subclass' constructor or setUp method.
+     */
+    protected final void initialize(
+            MvTableStorage tableStorage,
+            TablesConfiguration tablesCfg,
+            BasePageMemoryStorageEngineConfiguration<?, ?> baseEngineConfig
+    ) {
+        this.baseEngineConfig = baseEngineConfig;
+
+        initialize(tableStorage, tablesCfg);
+    }
+
+    @Test
+    void testWithStringsLargerThanMaximumInlineSize() throws Exception {
+        SortedIndexDefinition indexDefinition = SchemaBuilders.sortedIndex("TEST_INDEX")
+                .addIndexColumn(ColumnTypeSpec.INT32.name()).asc().done()
+                .addIndexColumn(ColumnTypeSpec.STRING.name()).asc().done()
+                .build();
+
+        SortedIndexStorage index = createIndexStorage(indexDefinition);
+
+        var serializer = new BinaryTupleRowSerializer(index.indexDescriptor());
+
+        IndexRow indexRow0 = createIndexRow(serializer, new RowId(TEST_PARTITION), 10, randomString(random, MAX_BINARY_TUPLE_INLINE_SIZE));
+        IndexRow indexRow1 = createIndexRow(serializer, new RowId(TEST_PARTITION), 10, randomString(random, MAX_BINARY_TUPLE_INLINE_SIZE));
+        IndexRow indexRow2 = createIndexRow(serializer, new RowId(TEST_PARTITION), 20, randomString(random, MAX_BINARY_TUPLE_INLINE_SIZE));
+        IndexRow indexRow3 = createIndexRow(serializer, new RowId(TEST_PARTITION), 20, randomString(random, MAX_BINARY_TUPLE_INLINE_SIZE));
+
+        put(index, indexRow0);
+        put(index, indexRow1);
+        put(index, indexRow2);
+
+        assertThat(get(index, indexRow0.indexColumns()), containsInAnyOrder(indexRow0.rowId()));
+        assertThat(get(index, indexRow1.indexColumns()), containsInAnyOrder(indexRow1.rowId()));
+        assertThat(get(index, indexRow2.indexColumns()), containsInAnyOrder(indexRow2.rowId()));
+
+        assertThat(get(index, indexRow3.indexColumns()), empty());
+    }
+
+    @Test
+    void testFragmentedIndexColumns() throws Exception {
+        SortedIndexDefinition indexDefinition = SchemaBuilders.sortedIndex("TEST_INDEX")
+                .addIndexColumn(ColumnTypeSpec.INT32.name()).asc().done()
+                .addIndexColumn(ColumnTypeSpec.STRING.name()).asc().done()
+                .build();
+
+        SortedIndexStorage index = createIndexStorage(indexDefinition);
+
+        var serializer = new BinaryTupleRowSerializer(index.indexDescriptor());
+
+        int pageSize = baseEngineConfig.pageSize().value();
+
+        IndexRow indexRow0 = createIndexRow(serializer, new RowId(TEST_PARTITION), 10, randomString(random, pageSize * 2));
+        IndexRow indexRow1 = createIndexRow(serializer, new RowId(TEST_PARTITION), 10, randomString(random, pageSize * 2));
+        IndexRow indexRow2 = createIndexRow(serializer, new RowId(TEST_PARTITION), 20, randomString(random, pageSize * 2));
+        IndexRow indexRow3 = createIndexRow(serializer, new RowId(TEST_PARTITION), 20, randomString(random, pageSize * 2));
+
+        put(index, indexRow0);
+        put(index, indexRow1);
+        put(index, indexRow2);
+
+        assertThat(get(index, indexRow0.indexColumns()), containsInAnyOrder(indexRow0.rowId()));
+        assertThat(get(index, indexRow1.indexColumns()), containsInAnyOrder(indexRow1.rowId()));
+        assertThat(get(index, indexRow2.indexColumns()), containsInAnyOrder(indexRow2.rowId()));
+
+        assertThat(get(index, indexRow3.indexColumns()), empty());
+    }
+
+    private static IndexRow createIndexRow(BinaryTupleRowSerializer serializer, RowId rowId, Object... objects) {
+        return serializer.serializeRow(objects, rowId);
+    }
+}
diff --git a/modules/storage-page-memory/src/test/java/org/apache/ignite/internal/storage/pagememory/index/PersistentPageMemorySortedIndexStorageTest.java b/modules/storage-page-memory/src/test/java/org/apache/ignite/internal/storage/pagememory/index/PersistentPageMemorySortedIndexStorageTest.java
index 655808ce6f..9168e5ba91 100644
--- a/modules/storage-page-memory/src/test/java/org/apache/ignite/internal/storage/pagememory/index/PersistentPageMemorySortedIndexStorageTest.java
+++ b/modules/storage-page-memory/src/test/java/org/apache/ignite/internal/storage/pagememory/index/PersistentPageMemorySortedIndexStorageTest.java
@@ -22,7 +22,6 @@ import org.apache.ignite.internal.configuration.testframework.ConfigurationExten
 import org.apache.ignite.internal.configuration.testframework.InjectConfiguration;
 import org.apache.ignite.internal.pagememory.io.PageIoRegistry;
 import org.apache.ignite.internal.schema.configuration.TablesConfiguration;
-import org.apache.ignite.internal.storage.index.AbstractSortedIndexStorageTest;
 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.PersistentPageMemoryStorageEngineConfiguration;
@@ -37,7 +36,7 @@ import org.junit.jupiter.api.extension.ExtendWith;
  * Sorted index test implementation for persistent page memory storage.
  */
 @ExtendWith({ConfigurationExtension.class, WorkDirectoryExtension.class})
-class PersistentPageMemorySortedIndexStorageTest extends AbstractSortedIndexStorageTest {
+class PersistentPageMemorySortedIndexStorageTest extends AbstractPageMemorySortedIndexStorageTest {
     private PersistentPageMemoryStorageEngine engine;
 
     private PersistentPageMemoryTableStorage table;
@@ -64,7 +63,7 @@ class PersistentPageMemorySortedIndexStorageTest extends AbstractSortedIndexStor
 
         table.start();
 
-        initialize(table, tablesConfig);
+        initialize(table, tablesConfig, engineConfig);
     }
 
     @AfterEach
diff --git a/modules/storage-page-memory/src/test/java/org/apache/ignite/internal/storage/pagememory/index/VolatilePageMemorySortedIndexStorageTest.java b/modules/storage-page-memory/src/test/java/org/apache/ignite/internal/storage/pagememory/index/VolatilePageMemorySortedIndexStorageTest.java
index a341f4e71a..c31aa12556 100644
--- a/modules/storage-page-memory/src/test/java/org/apache/ignite/internal/storage/pagememory/index/VolatilePageMemorySortedIndexStorageTest.java
+++ b/modules/storage-page-memory/src/test/java/org/apache/ignite/internal/storage/pagememory/index/VolatilePageMemorySortedIndexStorageTest.java
@@ -21,7 +21,6 @@ import org.apache.ignite.internal.configuration.testframework.ConfigurationExten
 import org.apache.ignite.internal.configuration.testframework.InjectConfiguration;
 import org.apache.ignite.internal.pagememory.io.PageIoRegistry;
 import org.apache.ignite.internal.schema.configuration.TablesConfiguration;
-import org.apache.ignite.internal.storage.index.AbstractSortedIndexStorageTest;
 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.VolatilePageMemoryStorageEngineConfiguration;
@@ -34,7 +33,7 @@ import org.junit.jupiter.api.extension.ExtendWith;
  * Sorted index test implementation for volatile page memory storage.
  */
 @ExtendWith(ConfigurationExtension.class)
-class VolatilePageMemorySortedIndexStorageTest extends AbstractSortedIndexStorageTest {
+class VolatilePageMemorySortedIndexStorageTest extends AbstractPageMemorySortedIndexStorageTest {
     private VolatilePageMemoryStorageEngine engine;
 
     private VolatilePageMemoryTableStorage table;
@@ -60,7 +59,7 @@ class VolatilePageMemorySortedIndexStorageTest extends AbstractSortedIndexStorag
 
         table.start();
 
-        initialize(table, tablesConfig);
+        initialize(table, tablesConfig, engineConfig);
     }
 
     @AfterEach