You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@ignite.apache.org by GitBox <gi...@apache.org> on 2022/12/09 07:27:42 UTC

[GitHub] [ignite-3] ibessonov commented on a diff in pull request #1407: IGNITE-18322 Define scan contract SortedIndexStorage

ibessonov commented on code in PR #1407:
URL: https://github.com/apache/ignite-3/pull/1407#discussion_r1043260865


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/tree/BplusTree.java:
##########
@@ -6222,6 +6250,72 @@ public void close() {
         }
     }
 
+    /**
+     * Class for getting the next row.
+     */
+    private final class GetNext extends Get {
+        @Nullable
+        private T nextRow;
+
+        private GetNext(L row, boolean includeRow) {
+            super(row, false);
+
+            shift = includeRow ? -1 : 1;
+        }
+
+        @Override
+        boolean found(BplusIo<L> io, long pageAddr, int idx, int lvl) {
+            // Must never be called because we always have a shift.
+            throw new IllegalStateException();
+        }
+
+        @Override
+        boolean notFound(BplusIo<L> io, long pageAddr, int idx, int lvl) throws IgniteInternalCheckedException {
+            if (lvl != 0) {
+                return false;
+            }
+
+            int cnt = io.getCount(pageAddr);
+
+            if (cnt == 0) {
+                // Empty tree.
+                assert io.getForward(pageAddr, partId) == 0L;
+            } else {
+                assert io.isLeaf() : io;
+                assert cnt > 0 : cnt;
+                assert idx >= 0 || idx == -1 : idx;
+                assert cnt >= idx : "cnt=" + cnt + ", idx=" + idx;
+
+                checkDestroyed();
+
+                if (idx == -1) {
+                    idx = findNextRowIdx(pageAddr, io, cnt);
+                }
+
+                if (cnt - idx > 0) {

Review Comment:
   Why not `idx < cnt`? Very non-intuitive construction



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/tree/BplusTree.java:
##########
@@ -1572,6 +1572,34 @@ public final T findOne(L row) throws IgniteInternalCheckedException {
         return findOne(row, null, null);
     }
 
+    /**
+     * Searches for the next row from the passed from the arguments.

Review Comment:
   ```suggestion
        * Searches for the next row from the passed argument.
   ```



##########
modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/AbstractSortedIndexStorageTest.java:
##########
@@ -579,6 +585,401 @@ void testNullValues(ColumnDefinition columnDefinition) throws Exception {
         }
     }
 
+    /**
+     * Checks simple scenarios for a scanning cursor.
+     */
+    @Test
+    void testScanSimple() {
+        SortedIndexDefinition indexDefinition = SchemaBuilders.sortedIndex("TEST_IDX")
+                .addIndexColumn(ColumnType.INT32.typeSpec().name()).asc().done()
+                .build();
+
+        SortedIndexStorage indexStorage = createIndexStorage(indexDefinition);
+
+        BinaryTupleRowSerializer serializer = new BinaryTupleRowSerializer(indexStorage.indexDescriptor());
+
+        for (int i = 0; i < 5; i++) {
+            put(indexStorage, serializer.serializeRow(new Object[]{i}, new RowId(TEST_PARTITION)));
+        }
+
+        // Checking without borders.
+        assertThat(
+                scan(indexStorage, null, null, 0).stream().map(objects -> objects[0]).collect(toList()),
+                contains(0, 1, 2, 3, 4)
+        );
+
+        // Let's check without borders.
+        assertThat(
+                indexStorage.scan(
+                                serializer.serializeRowPrefix(0),
+                                serializer.serializeRowPrefix(4),
+                                (GREATER_OR_EQUAL | LESS_OR_EQUAL)
+                        ).stream()
+                        .map(indexRow -> serializer.deserializeColumns(indexRow)[0])
+                        .collect(toList()),
+                contains(0, 1, 2, 3, 4)
+        );
+
+        assertThat(
+                indexStorage.scan(
+                                serializer.serializeRowPrefix(0),
+                                serializer.serializeRowPrefix(4),
+                                (GREATER_OR_EQUAL | LESS)
+                        ).stream()
+                        .map(indexRow -> serializer.deserializeColumns(indexRow)[0])
+                        .collect(toList()),
+                contains(0, 1, 2, 3)

Review Comment:
   What if this scan returned `(0, 1, 2, 3, 4)`? `contains` is still true, right? We need a match.
   Same applies to all assertions in the test



##########
modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/AbstractSortedIndexStorageTest.java:
##########
@@ -579,6 +585,401 @@ void testNullValues(ColumnDefinition columnDefinition) throws Exception {
         }
     }
 
+    /**
+     * Checks simple scenarios for a scanning cursor.
+     */
+    @Test
+    void testScanSimple() {
+        SortedIndexDefinition indexDefinition = SchemaBuilders.sortedIndex("TEST_IDX")
+                .addIndexColumn(ColumnType.INT32.typeSpec().name()).asc().done()
+                .build();
+
+        SortedIndexStorage indexStorage = createIndexStorage(indexDefinition);
+
+        BinaryTupleRowSerializer serializer = new BinaryTupleRowSerializer(indexStorage.indexDescriptor());
+
+        for (int i = 0; i < 5; i++) {
+            put(indexStorage, serializer.serializeRow(new Object[]{i}, new RowId(TEST_PARTITION)));
+        }
+
+        // Checking without borders.
+        assertThat(
+                scan(indexStorage, null, null, 0).stream().map(objects -> objects[0]).collect(toList()),
+                contains(0, 1, 2, 3, 4)
+        );
+
+        // Let's check without borders.
+        assertThat(
+                indexStorage.scan(
+                                serializer.serializeRowPrefix(0),
+                                serializer.serializeRowPrefix(4),
+                                (GREATER_OR_EQUAL | LESS_OR_EQUAL)
+                        ).stream()
+                        .map(indexRow -> serializer.deserializeColumns(indexRow)[0])
+                        .collect(toList()),
+                contains(0, 1, 2, 3, 4)
+        );
+
+        assertThat(
+                indexStorage.scan(
+                                serializer.serializeRowPrefix(0),
+                                serializer.serializeRowPrefix(4),
+                                (GREATER_OR_EQUAL | LESS)
+                        ).stream()
+                        .map(indexRow -> serializer.deserializeColumns(indexRow)[0])
+                        .collect(toList()),
+                contains(0, 1, 2, 3)
+        );
+
+        assertThat(
+                indexStorage.scan(
+                                serializer.serializeRowPrefix(0),
+                                serializer.serializeRowPrefix(4),
+                                (GREATER | LESS_OR_EQUAL)
+                        ).stream()
+                        .map(indexRow -> serializer.deserializeColumns(indexRow)[0])
+                        .collect(toList()),
+                contains(1, 2, 3, 4)
+        );
+
+        assertThat(
+                indexStorage.scan(
+                                serializer.serializeRowPrefix(0),
+                                serializer.serializeRowPrefix(4),
+                                (GREATER | LESS)
+                        ).stream()
+                        .map(indexRow -> serializer.deserializeColumns(indexRow)[0])
+                        .collect(toList()),
+                contains(1, 2, 3)
+        );
+
+        // Let's check only with the lower bound.
+        assertThat(
+                indexStorage.scan(
+                                serializer.serializeRowPrefix(1),
+                                null,
+                                (GREATER_OR_EQUAL | LESS_OR_EQUAL)
+                        ).stream()
+                        .map(indexRow -> serializer.deserializeColumns(indexRow)[0])
+                        .collect(toList()),
+                contains(1, 2, 3, 4)
+        );
+
+        assertThat(
+                indexStorage.scan(
+                                serializer.serializeRowPrefix(1),
+                                null,
+                                (GREATER_OR_EQUAL | LESS)
+                        ).stream()
+                        .map(indexRow -> serializer.deserializeColumns(indexRow)[0])
+                        .collect(toList()),
+                contains(1, 2, 3, 4)
+        );
+
+        assertThat(
+                indexStorage.scan(
+                                serializer.serializeRowPrefix(1),
+                                null,
+                                (GREATER | LESS_OR_EQUAL)
+                        ).stream()
+                        .map(indexRow -> serializer.deserializeColumns(indexRow)[0])
+                        .collect(toList()),
+                contains(2, 3, 4)
+        );
+
+        assertThat(
+                indexStorage.scan(
+                                serializer.serializeRowPrefix(1),
+                                null,
+                                (GREATER | LESS)
+                        ).stream()
+                        .map(indexRow -> serializer.deserializeColumns(indexRow)[0])
+                        .collect(toList()),
+                contains(2, 3, 4)
+        );
+
+        // Let's check only with the upper bound.
+        assertThat(
+                indexStorage.scan(
+                                null,
+                                serializer.serializeRowPrefix(3),
+                                (GREATER_OR_EQUAL | LESS_OR_EQUAL)
+                        ).stream()
+                        .map(indexRow -> serializer.deserializeColumns(indexRow)[0])
+                        .collect(toList()),
+                contains(0, 1, 2, 3)
+        );
+
+        assertThat(
+                indexStorage.scan(
+                                null,
+                                serializer.serializeRowPrefix(3),
+                                (GREATER_OR_EQUAL | LESS)
+                        ).stream()
+                        .map(indexRow -> serializer.deserializeColumns(indexRow)[0])
+                        .collect(toList()),
+                contains(0, 1, 2)
+        );
+
+        assertThat(
+                indexStorage.scan(
+                                null,
+                                serializer.serializeRowPrefix(3),
+                                (GREATER | LESS_OR_EQUAL)
+                        ).stream()
+                        .map(indexRow -> serializer.deserializeColumns(indexRow)[0])
+                        .collect(toList()),
+                contains(0, 1, 2, 3)
+        );
+
+        assertThat(
+                indexStorage.scan(
+                                null,
+                                serializer.serializeRowPrefix(3),
+                                (GREATER | LESS)
+                        ).stream()
+                        .map(indexRow -> serializer.deserializeColumns(indexRow)[0])
+                        .collect(toList()),
+                contains(0, 1, 2)
+        );
+    }
+
+    @Test
+    void testScanContractForEmptyIndex() {
+        SortedIndexDefinition indexDefinition = SchemaBuilders.sortedIndex("TEST_IDX")
+                .addIndexColumn(ColumnType.INT32.typeSpec().name()).asc().done()
+                .build();
+
+        SortedIndexStorage indexStorage = createIndexStorage(indexDefinition);
+
+        BinaryTupleRowSerializer serializer = new BinaryTupleRowSerializer(indexStorage.indexDescriptor());
+
+        Cursor<IndexRow> scanBeforeHashNext = indexStorage.scan(null, null, 0);
+
+        Cursor<IndexRow> scanAfterHasNext = indexStorage.scan(null, null, 0);
+
+        assertFalse(scanAfterHasNext.hasNext());
+
+        Cursor<IndexRow> scanWithoutHasNext = indexStorage.scan(null, null, 0);
+
+        put(indexStorage, serializer.serializeRow(new Object[]{0}, new RowId(TEST_PARTITION)));
+
+        assertTrue(scanBeforeHashNext.hasNext());
+        assertFalse(scanAfterHasNext.hasNext());
+
+        assertEquals(0, serializer.deserializeColumns(scanBeforeHashNext.next())[0]);
+
+        assertThrows(NoSuchElementException.class, scanAfterHasNext::next);
+        assertEquals(0, serializer.deserializeColumns(scanAfterHasNext.next())[0]);
+
+        assertEquals(0, serializer.deserializeColumns(scanWithoutHasNext.next())[0]);
+
+        assertFalse(scanBeforeHashNext.hasNext());
+        assertFalse(scanAfterHasNext.hasNext());
+        assertFalse(scanWithoutHasNext.hasNext());
+
+        assertThrows(NoSuchElementException.class, scanBeforeHashNext::next);
+        assertThrows(NoSuchElementException.class, scanAfterHasNext::next);
+        assertThrows(NoSuchElementException.class, scanWithoutHasNext::next);
+    }
+
+    @Test
+    void testScanContractAddRowsOnly() {
+        SortedIndexDefinition indexDefinition = SchemaBuilders.sortedIndex("TEST_IDX")
+                .addIndexColumn(ColumnType.INT32.typeSpec().name()).asc().done()
+                .build();
+
+        SortedIndexStorage indexStorage = createIndexStorage(indexDefinition);
+
+        BinaryTupleRowSerializer serializer = new BinaryTupleRowSerializer(indexStorage.indexDescriptor());
+
+        RowId rowId0 = new RowId(TEST_PARTITION, 0, 0);
+        RowId rowId1 = new RowId(TEST_PARTITION, 0, 1);
+        RowId rowId2 = new RowId(TEST_PARTITION, 1, 0);
+
+        Cursor<IndexRow> scan = indexStorage.scan(null, null, 0);
+
+        put(indexStorage, serializer.serializeRow(new Object[]{0}, rowId1));
+
+        IndexRow nextRow = scan.next();
+
+        assertEquals(0, serializer.deserializeColumns(nextRow)[0]);
+        assertEquals(rowId1, nextRow.rowId());
+
+        put(indexStorage, serializer.serializeRow(new Object[]{0}, rowId0));
+        put(indexStorage, serializer.serializeRow(new Object[]{0}, rowId2));
+
+        assertTrue(scan.hasNext());
+
+        nextRow = scan.next();
+
+        assertEquals(0, serializer.deserializeColumns(nextRow)[0]);
+        assertEquals(rowId2, nextRow.rowId());
+
+        put(indexStorage, serializer.serializeRow(new Object[]{1}, rowId0));
+        put(indexStorage, serializer.serializeRow(new Object[]{-1}, rowId0));
+
+        assertTrue(scan.hasNext());
+
+        nextRow = scan.next();
+
+        assertEquals(1, serializer.deserializeColumns(nextRow)[0]);
+        assertEquals(rowId0, nextRow.rowId());
+
+        assertFalse(scan.hasNext());
+        assertThrows(NoSuchElementException.class, scan::next);
+    }
+
+    @Test
+    void testScanContractForFinishCursor() {
+        SortedIndexDefinition indexDefinition = SchemaBuilders.sortedIndex("TEST_IDX")
+                .addIndexColumn(ColumnType.INT32.typeSpec().name()).asc().done()
+                .build();
+
+        SortedIndexStorage indexStorage = createIndexStorage(indexDefinition);
+
+        BinaryTupleRowSerializer serializer = new BinaryTupleRowSerializer(indexStorage.indexDescriptor());
+
+        Cursor<IndexRow> scan = indexStorage.scan(null, null, 0);
+
+        assertFalse(scan.hasNext());
+        assertThrows(NoSuchElementException.class, scan::next);
+
+        RowId rowId0 = new RowId(TEST_PARTITION, 0, 0);
+        RowId rowId1 = new RowId(TEST_PARTITION, 0, 1);
+
+        put(indexStorage, serializer.serializeRow(new Object[]{0}, rowId0));
+
+        assertTrue(scan.hasNext());
+
+        IndexRow nextRow = scan.next();
+
+        assertEquals(0, serializer.deserializeColumns(nextRow)[0]);
+        assertEquals(rowId0, nextRow.rowId());
+
+        assertFalse(scan.hasNext());
+        assertThrows(NoSuchElementException.class, scan::next);
+
+        put(indexStorage, serializer.serializeRow(new Object[]{0}, rowId0));
+
+        assertFalse(scan.hasNext());

Review Comment:
   You know what, at each step it would be cool ta have a comment that shows the state of the index and the state of the cursor. Like `0 1 2 | 3 4` or something, otherwise it might be complicated to remember everything.



##########
modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/AbstractSortedIndexStorageTest.java:
##########
@@ -579,6 +585,401 @@ void testNullValues(ColumnDefinition columnDefinition) throws Exception {
         }
     }
 
+    /**
+     * Checks simple scenarios for a scanning cursor.
+     */
+    @Test
+    void testScanSimple() {
+        SortedIndexDefinition indexDefinition = SchemaBuilders.sortedIndex("TEST_IDX")
+                .addIndexColumn(ColumnType.INT32.typeSpec().name()).asc().done()
+                .build();
+
+        SortedIndexStorage indexStorage = createIndexStorage(indexDefinition);
+
+        BinaryTupleRowSerializer serializer = new BinaryTupleRowSerializer(indexStorage.indexDescriptor());
+
+        for (int i = 0; i < 5; i++) {
+            put(indexStorage, serializer.serializeRow(new Object[]{i}, new RowId(TEST_PARTITION)));
+        }
+
+        // Checking without borders.
+        assertThat(
+                scan(indexStorage, null, null, 0).stream().map(objects -> objects[0]).collect(toList()),
+                contains(0, 1, 2, 3, 4)
+        );
+
+        // Let's check without borders.
+        assertThat(
+                indexStorage.scan(
+                                serializer.serializeRowPrefix(0),
+                                serializer.serializeRowPrefix(4),
+                                (GREATER_OR_EQUAL | LESS_OR_EQUAL)
+                        ).stream()
+                        .map(indexRow -> serializer.deserializeColumns(indexRow)[0])
+                        .collect(toList()),
+                contains(0, 1, 2, 3, 4)
+        );
+
+        assertThat(
+                indexStorage.scan(
+                                serializer.serializeRowPrefix(0),
+                                serializer.serializeRowPrefix(4),
+                                (GREATER_OR_EQUAL | LESS)
+                        ).stream()
+                        .map(indexRow -> serializer.deserializeColumns(indexRow)[0])
+                        .collect(toList()),
+                contains(0, 1, 2, 3)
+        );
+
+        assertThat(
+                indexStorage.scan(
+                                serializer.serializeRowPrefix(0),
+                                serializer.serializeRowPrefix(4),
+                                (GREATER | LESS_OR_EQUAL)
+                        ).stream()
+                        .map(indexRow -> serializer.deserializeColumns(indexRow)[0])
+                        .collect(toList()),
+                contains(1, 2, 3, 4)
+        );
+
+        assertThat(
+                indexStorage.scan(
+                                serializer.serializeRowPrefix(0),
+                                serializer.serializeRowPrefix(4),
+                                (GREATER | LESS)
+                        ).stream()
+                        .map(indexRow -> serializer.deserializeColumns(indexRow)[0])
+                        .collect(toList()),
+                contains(1, 2, 3)
+        );
+
+        // Let's check only with the lower bound.
+        assertThat(
+                indexStorage.scan(
+                                serializer.serializeRowPrefix(1),
+                                null,
+                                (GREATER_OR_EQUAL | LESS_OR_EQUAL)
+                        ).stream()
+                        .map(indexRow -> serializer.deserializeColumns(indexRow)[0])
+                        .collect(toList()),
+                contains(1, 2, 3, 4)
+        );
+
+        assertThat(
+                indexStorage.scan(
+                                serializer.serializeRowPrefix(1),
+                                null,
+                                (GREATER_OR_EQUAL | LESS)
+                        ).stream()
+                        .map(indexRow -> serializer.deserializeColumns(indexRow)[0])
+                        .collect(toList()),
+                contains(1, 2, 3, 4)
+        );
+
+        assertThat(
+                indexStorage.scan(
+                                serializer.serializeRowPrefix(1),
+                                null,
+                                (GREATER | LESS_OR_EQUAL)
+                        ).stream()
+                        .map(indexRow -> serializer.deserializeColumns(indexRow)[0])
+                        .collect(toList()),
+                contains(2, 3, 4)
+        );
+
+        assertThat(
+                indexStorage.scan(
+                                serializer.serializeRowPrefix(1),
+                                null,
+                                (GREATER | LESS)
+                        ).stream()
+                        .map(indexRow -> serializer.deserializeColumns(indexRow)[0])
+                        .collect(toList()),
+                contains(2, 3, 4)
+        );
+
+        // Let's check only with the upper bound.
+        assertThat(
+                indexStorage.scan(
+                                null,
+                                serializer.serializeRowPrefix(3),
+                                (GREATER_OR_EQUAL | LESS_OR_EQUAL)
+                        ).stream()
+                        .map(indexRow -> serializer.deserializeColumns(indexRow)[0])
+                        .collect(toList()),
+                contains(0, 1, 2, 3)
+        );
+
+        assertThat(
+                indexStorage.scan(
+                                null,
+                                serializer.serializeRowPrefix(3),
+                                (GREATER_OR_EQUAL | LESS)
+                        ).stream()
+                        .map(indexRow -> serializer.deserializeColumns(indexRow)[0])
+                        .collect(toList()),
+                contains(0, 1, 2)
+        );
+
+        assertThat(
+                indexStorage.scan(
+                                null,
+                                serializer.serializeRowPrefix(3),
+                                (GREATER | LESS_OR_EQUAL)
+                        ).stream()
+                        .map(indexRow -> serializer.deserializeColumns(indexRow)[0])
+                        .collect(toList()),
+                contains(0, 1, 2, 3)
+        );
+
+        assertThat(
+                indexStorage.scan(
+                                null,
+                                serializer.serializeRowPrefix(3),
+                                (GREATER | LESS)
+                        ).stream()
+                        .map(indexRow -> serializer.deserializeColumns(indexRow)[0])
+                        .collect(toList()),
+                contains(0, 1, 2)
+        );
+    }
+
+    @Test
+    void testScanContractForEmptyIndex() {
+        SortedIndexDefinition indexDefinition = SchemaBuilders.sortedIndex("TEST_IDX")
+                .addIndexColumn(ColumnType.INT32.typeSpec().name()).asc().done()
+                .build();
+
+        SortedIndexStorage indexStorage = createIndexStorage(indexDefinition);
+
+        BinaryTupleRowSerializer serializer = new BinaryTupleRowSerializer(indexStorage.indexDescriptor());
+
+        Cursor<IndexRow> scanBeforeHashNext = indexStorage.scan(null, null, 0);
+
+        Cursor<IndexRow> scanAfterHasNext = indexStorage.scan(null, null, 0);
+
+        assertFalse(scanAfterHasNext.hasNext());
+
+        Cursor<IndexRow> scanWithoutHasNext = indexStorage.scan(null, null, 0);
+
+        put(indexStorage, serializer.serializeRow(new Object[]{0}, new RowId(TEST_PARTITION)));
+
+        assertTrue(scanBeforeHashNext.hasNext());
+        assertFalse(scanAfterHasNext.hasNext());
+
+        assertEquals(0, serializer.deserializeColumns(scanBeforeHashNext.next())[0]);
+
+        assertThrows(NoSuchElementException.class, scanAfterHasNext::next);
+        assertEquals(0, serializer.deserializeColumns(scanAfterHasNext.next())[0]);
+
+        assertEquals(0, serializer.deserializeColumns(scanWithoutHasNext.next())[0]);
+
+        assertFalse(scanBeforeHashNext.hasNext());
+        assertFalse(scanAfterHasNext.hasNext());
+        assertFalse(scanWithoutHasNext.hasNext());
+
+        assertThrows(NoSuchElementException.class, scanBeforeHashNext::next);
+        assertThrows(NoSuchElementException.class, scanAfterHasNext::next);
+        assertThrows(NoSuchElementException.class, scanWithoutHasNext::next);
+    }
+
+    @Test
+    void testScanContractAddRowsOnly() {
+        SortedIndexDefinition indexDefinition = SchemaBuilders.sortedIndex("TEST_IDX")
+                .addIndexColumn(ColumnType.INT32.typeSpec().name()).asc().done()
+                .build();
+
+        SortedIndexStorage indexStorage = createIndexStorage(indexDefinition);
+
+        BinaryTupleRowSerializer serializer = new BinaryTupleRowSerializer(indexStorage.indexDescriptor());
+
+        RowId rowId0 = new RowId(TEST_PARTITION, 0, 0);
+        RowId rowId1 = new RowId(TEST_PARTITION, 0, 1);
+        RowId rowId2 = new RowId(TEST_PARTITION, 1, 0);
+
+        Cursor<IndexRow> scan = indexStorage.scan(null, null, 0);
+
+        put(indexStorage, serializer.serializeRow(new Object[]{0}, rowId1));
+
+        IndexRow nextRow = scan.next();
+
+        assertEquals(0, serializer.deserializeColumns(nextRow)[0]);
+        assertEquals(rowId1, nextRow.rowId());
+
+        put(indexStorage, serializer.serializeRow(new Object[]{0}, rowId0));
+        put(indexStorage, serializer.serializeRow(new Object[]{0}, rowId2));
+
+        assertTrue(scan.hasNext());
+
+        nextRow = scan.next();
+
+        assertEquals(0, serializer.deserializeColumns(nextRow)[0]);
+        assertEquals(rowId2, nextRow.rowId());
+
+        put(indexStorage, serializer.serializeRow(new Object[]{1}, rowId0));
+        put(indexStorage, serializer.serializeRow(new Object[]{-1}, rowId0));
+
+        assertTrue(scan.hasNext());
+
+        nextRow = scan.next();
+
+        assertEquals(1, serializer.deserializeColumns(nextRow)[0]);
+        assertEquals(rowId0, nextRow.rowId());
+
+        assertFalse(scan.hasNext());
+        assertThrows(NoSuchElementException.class, scan::next);
+    }
+
+    @Test
+    void testScanContractForFinishCursor() {
+        SortedIndexDefinition indexDefinition = SchemaBuilders.sortedIndex("TEST_IDX")
+                .addIndexColumn(ColumnType.INT32.typeSpec().name()).asc().done()
+                .build();
+
+        SortedIndexStorage indexStorage = createIndexStorage(indexDefinition);
+
+        BinaryTupleRowSerializer serializer = new BinaryTupleRowSerializer(indexStorage.indexDescriptor());
+
+        Cursor<IndexRow> scan = indexStorage.scan(null, null, 0);
+
+        assertFalse(scan.hasNext());
+        assertThrows(NoSuchElementException.class, scan::next);
+
+        RowId rowId0 = new RowId(TEST_PARTITION, 0, 0);
+        RowId rowId1 = new RowId(TEST_PARTITION, 0, 1);
+
+        put(indexStorage, serializer.serializeRow(new Object[]{0}, rowId0));
+
+        assertTrue(scan.hasNext());
+
+        IndexRow nextRow = scan.next();
+
+        assertEquals(0, serializer.deserializeColumns(nextRow)[0]);
+        assertEquals(rowId0, nextRow.rowId());
+
+        assertFalse(scan.hasNext());
+        assertThrows(NoSuchElementException.class, scan::next);
+
+        put(indexStorage, serializer.serializeRow(new Object[]{0}, rowId0));
+
+        assertFalse(scan.hasNext());
+        assertThrows(NoSuchElementException.class, scan::next);
+
+        put(indexStorage, serializer.serializeRow(new Object[]{0}, rowId1));
+
+        assertTrue(scan.hasNext());
+
+        nextRow = scan.next();
+
+        assertEquals(0, serializer.deserializeColumns(nextRow)[0]);
+        assertEquals(rowId1, nextRow.rowId());
+
+        assertFalse(scan.hasNext());
+        assertThrows(NoSuchElementException.class, scan::next);
+
+        put(indexStorage, serializer.serializeRow(new Object[]{1}, rowId0));
+
+        assertTrue(scan.hasNext());
+
+        nextRow = scan.next();
+
+        assertEquals(1, serializer.deserializeColumns(nextRow)[0]);
+        assertEquals(rowId0, nextRow.rowId());
+
+        assertFalse(scan.hasNext());
+        assertThrows(NoSuchElementException.class, scan::next);
+
+        put(indexStorage, serializer.serializeRow(new Object[]{-1}, rowId0));
+
+        assertFalse(scan.hasNext());
+        assertThrows(NoSuchElementException.class, scan::next);
+    }
+
+    @Test
+    void testScanContractNextMethodOnly() {
+        SortedIndexDefinition indexDefinition = SchemaBuilders.sortedIndex("TEST_IDX")
+                .addIndexColumn(ColumnType.INT32.typeSpec().name()).asc().done()
+                .build();
+
+        SortedIndexStorage indexStorage = createIndexStorage(indexDefinition);
+
+        BinaryTupleRowSerializer serializer = new BinaryTupleRowSerializer(indexStorage.indexDescriptor());
+
+        RowId rowId0 = new RowId(TEST_PARTITION, 0, 0);
+        RowId rowId1 = new RowId(TEST_PARTITION, 0, 1);
+        RowId rowId2 = new RowId(TEST_PARTITION, 0, 1);
+
+        Cursor<IndexRow> scan = indexStorage.scan(null, null, 0);
+
+        put(indexStorage, serializer.serializeRow(new Object[]{0}, rowId0));
+
+        IndexRow nextRow = scan.next();
+
+        assertEquals(0, serializer.deserializeColumns(nextRow)[0]);
+        assertEquals(rowId0, nextRow.rowId());
+
+        put(indexStorage, serializer.serializeRow(new Object[]{0}, rowId1));
+
+        nextRow = scan.next();
+
+        assertEquals(0, serializer.deserializeColumns(nextRow)[0]);
+        assertEquals(rowId1, nextRow.rowId());
+
+        put(indexStorage, serializer.serializeRow(new Object[]{1}, rowId2));
+        put(indexStorage, serializer.serializeRow(new Object[]{-1}, rowId2));
+
+        nextRow = scan.next();
+
+        assertEquals(1, serializer.deserializeColumns(nextRow)[0]);
+        assertEquals(rowId2, nextRow.rowId());
+
+        assertThrows(NoSuchElementException.class, scan::next);
+    }
+
+    @Test
+    void testScanContractRemoveRowsOnly() {
+        SortedIndexDefinition indexDefinition = SchemaBuilders.sortedIndex("TEST_IDX")
+                .addIndexColumn(ColumnType.INT32.typeSpec().name()).asc().done()
+                .build();
+
+        SortedIndexStorage indexStorage = createIndexStorage(indexDefinition);
+
+        BinaryTupleRowSerializer serializer = new BinaryTupleRowSerializer(indexStorage.indexDescriptor());
+
+        RowId rowId0 = new RowId(TEST_PARTITION, 0, 0);
+        RowId rowId1 = new RowId(TEST_PARTITION, 0, 1);
+
+        put(indexStorage, serializer.serializeRow(new Object[]{0}, rowId0));
+        put(indexStorage, serializer.serializeRow(new Object[]{0}, rowId1));
+        put(indexStorage, serializer.serializeRow(new Object[]{1}, rowId0));
+        put(indexStorage, serializer.serializeRow(new Object[]{2}, rowId1));
+
+        Cursor<IndexRow> scan = indexStorage.scan(null, null, 0);
+
+        assertTrue(scan.hasNext());
+
+        remove(indexStorage, serializer.serializeRow(new Object[]{0}, rowId0));
+
+        IndexRow nextRow = scan.next();
+
+        assertEquals(0, serializer.deserializeColumns(nextRow)[0]);
+        assertEquals(rowId0, nextRow.rowId());
+
+        remove(indexStorage, serializer.serializeRow(new Object[]{0}, rowId1));
+
+        assertTrue(scan.hasNext());
+
+        nextRow = scan.next();
+
+        assertEquals(1, serializer.deserializeColumns(nextRow)[0]);
+        assertEquals(rowId0, nextRow.rowId());
+
+        remove(indexStorage, serializer.serializeRow(new Object[]{2}, rowId1));
+
+        assertFalse(scan.hasNext());
+        assertThrows(NoSuchElementException.class, scan::next);
+    }
+

Review Comment:
   We need more tests. What if current position is removed but the new entry is inserted into "the same" place? Or n the place strictly before, for example. Combinations of insertions and deletions should be checked, at least some of them.



##########
modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/AbstractSortedIndexStorageTest.java:
##########
@@ -579,6 +585,401 @@ void testNullValues(ColumnDefinition columnDefinition) throws Exception {
         }
     }
 
+    /**
+     * Checks simple scenarios for a scanning cursor.
+     */
+    @Test
+    void testScanSimple() {
+        SortedIndexDefinition indexDefinition = SchemaBuilders.sortedIndex("TEST_IDX")
+                .addIndexColumn(ColumnType.INT32.typeSpec().name()).asc().done()
+                .build();
+
+        SortedIndexStorage indexStorage = createIndexStorage(indexDefinition);
+
+        BinaryTupleRowSerializer serializer = new BinaryTupleRowSerializer(indexStorage.indexDescriptor());
+
+        for (int i = 0; i < 5; i++) {
+            put(indexStorage, serializer.serializeRow(new Object[]{i}, new RowId(TEST_PARTITION)));
+        }
+
+        // Checking without borders.
+        assertThat(
+                scan(indexStorage, null, null, 0).stream().map(objects -> objects[0]).collect(toList()),

Review Comment:
   I feel like pattern `.stream().map(objects -> objects[0]).collect(toList())` can be extracted into a method and code will become more clear



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/tree/BplusTree.java:
##########
@@ -6222,6 +6250,72 @@ public void close() {
         }
     }
 
+    /**
+     * Class for getting the next row.
+     */
+    private final class GetNext extends Get {
+        @Nullable
+        private T nextRow;
+
+        private GetNext(L row, boolean includeRow) {
+            super(row, false);
+
+            shift = includeRow ? -1 : 1;
+        }
+
+        @Override
+        boolean found(BplusIo<L> io, long pageAddr, int idx, int lvl) {
+            // Must never be called because we always have a shift.
+            throw new IllegalStateException();
+        }
+
+        @Override
+        boolean notFound(BplusIo<L> io, long pageAddr, int idx, int lvl) throws IgniteInternalCheckedException {
+            if (lvl != 0) {
+                return false;
+            }
+
+            int cnt = io.getCount(pageAddr);
+
+            if (cnt == 0) {
+                // Empty tree.
+                assert io.getForward(pageAddr, partId) == 0L;
+            } else {
+                assert io.isLeaf() : io;
+                assert cnt > 0 : cnt;
+                assert idx >= 0 || idx == -1 : idx;
+                assert cnt >= idx : "cnt=" + cnt + ", idx=" + idx;
+
+                checkDestroyed();
+
+                if (idx == -1) {
+                    idx = findNextRowIdx(pageAddr, io, cnt);
+                }
+
+                if (cnt - idx > 0) {
+                    nextRow = getRow(io, pageAddr, idx);
+                }
+            }
+
+            return true;
+        }
+
+        int findNextRowIdx(long pageAddr, BplusIo<L> io, int cnt) throws IgniteInternalCheckedException {
+            // Compare with the first row on the page.
+            int cmp = compare(0, io, pageAddr, 0, row);

Review Comment:
   Why is this required? Won't `findInsertionPoint` do its job?



##########
modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/AbstractSortedIndexStorageTest.java:
##########
@@ -579,6 +585,401 @@ void testNullValues(ColumnDefinition columnDefinition) throws Exception {
         }
     }
 
+    /**
+     * Checks simple scenarios for a scanning cursor.
+     */
+    @Test
+    void testScanSimple() {
+        SortedIndexDefinition indexDefinition = SchemaBuilders.sortedIndex("TEST_IDX")
+                .addIndexColumn(ColumnType.INT32.typeSpec().name()).asc().done()
+                .build();
+
+        SortedIndexStorage indexStorage = createIndexStorage(indexDefinition);
+
+        BinaryTupleRowSerializer serializer = new BinaryTupleRowSerializer(indexStorage.indexDescriptor());
+
+        for (int i = 0; i < 5; i++) {
+            put(indexStorage, serializer.serializeRow(new Object[]{i}, new RowId(TEST_PARTITION)));
+        }
+
+        // Checking without borders.
+        assertThat(
+                scan(indexStorage, null, null, 0).stream().map(objects -> objects[0]).collect(toList()),
+                contains(0, 1, 2, 3, 4)
+        );
+
+        // Let's check without borders.
+        assertThat(
+                indexStorage.scan(
+                                serializer.serializeRowPrefix(0),
+                                serializer.serializeRowPrefix(4),
+                                (GREATER_OR_EQUAL | LESS_OR_EQUAL)
+                        ).stream()
+                        .map(indexRow -> serializer.deserializeColumns(indexRow)[0])
+                        .collect(toList()),
+                contains(0, 1, 2, 3, 4)
+        );
+
+        assertThat(
+                indexStorage.scan(
+                                serializer.serializeRowPrefix(0),
+                                serializer.serializeRowPrefix(4),
+                                (GREATER_OR_EQUAL | LESS)
+                        ).stream()
+                        .map(indexRow -> serializer.deserializeColumns(indexRow)[0])
+                        .collect(toList()),
+                contains(0, 1, 2, 3)
+        );
+
+        assertThat(
+                indexStorage.scan(
+                                serializer.serializeRowPrefix(0),
+                                serializer.serializeRowPrefix(4),
+                                (GREATER | LESS_OR_EQUAL)
+                        ).stream()
+                        .map(indexRow -> serializer.deserializeColumns(indexRow)[0])
+                        .collect(toList()),
+                contains(1, 2, 3, 4)
+        );
+
+        assertThat(
+                indexStorage.scan(
+                                serializer.serializeRowPrefix(0),
+                                serializer.serializeRowPrefix(4),
+                                (GREATER | LESS)
+                        ).stream()
+                        .map(indexRow -> serializer.deserializeColumns(indexRow)[0])
+                        .collect(toList()),
+                contains(1, 2, 3)
+        );
+
+        // Let's check only with the lower bound.
+        assertThat(
+                indexStorage.scan(
+                                serializer.serializeRowPrefix(1),
+                                null,
+                                (GREATER_OR_EQUAL | LESS_OR_EQUAL)
+                        ).stream()
+                        .map(indexRow -> serializer.deserializeColumns(indexRow)[0])
+                        .collect(toList()),
+                contains(1, 2, 3, 4)
+        );
+
+        assertThat(
+                indexStorage.scan(
+                                serializer.serializeRowPrefix(1),
+                                null,
+                                (GREATER_OR_EQUAL | LESS)
+                        ).stream()
+                        .map(indexRow -> serializer.deserializeColumns(indexRow)[0])
+                        .collect(toList()),
+                contains(1, 2, 3, 4)
+        );
+
+        assertThat(
+                indexStorage.scan(
+                                serializer.serializeRowPrefix(1),
+                                null,
+                                (GREATER | LESS_OR_EQUAL)
+                        ).stream()
+                        .map(indexRow -> serializer.deserializeColumns(indexRow)[0])
+                        .collect(toList()),
+                contains(2, 3, 4)
+        );
+
+        assertThat(
+                indexStorage.scan(
+                                serializer.serializeRowPrefix(1),
+                                null,
+                                (GREATER | LESS)
+                        ).stream()
+                        .map(indexRow -> serializer.deserializeColumns(indexRow)[0])
+                        .collect(toList()),
+                contains(2, 3, 4)
+        );
+
+        // Let's check only with the upper bound.
+        assertThat(
+                indexStorage.scan(
+                                null,
+                                serializer.serializeRowPrefix(3),
+                                (GREATER_OR_EQUAL | LESS_OR_EQUAL)
+                        ).stream()
+                        .map(indexRow -> serializer.deserializeColumns(indexRow)[0])
+                        .collect(toList()),
+                contains(0, 1, 2, 3)
+        );
+
+        assertThat(
+                indexStorage.scan(
+                                null,
+                                serializer.serializeRowPrefix(3),
+                                (GREATER_OR_EQUAL | LESS)
+                        ).stream()
+                        .map(indexRow -> serializer.deserializeColumns(indexRow)[0])
+                        .collect(toList()),
+                contains(0, 1, 2)
+        );
+
+        assertThat(
+                indexStorage.scan(
+                                null,
+                                serializer.serializeRowPrefix(3),
+                                (GREATER | LESS_OR_EQUAL)
+                        ).stream()
+                        .map(indexRow -> serializer.deserializeColumns(indexRow)[0])
+                        .collect(toList()),
+                contains(0, 1, 2, 3)
+        );
+
+        assertThat(
+                indexStorage.scan(
+                                null,
+                                serializer.serializeRowPrefix(3),
+                                (GREATER | LESS)
+                        ).stream()
+                        .map(indexRow -> serializer.deserializeColumns(indexRow)[0])
+                        .collect(toList()),
+                contains(0, 1, 2)
+        );
+    }
+
+    @Test
+    void testScanContractForEmptyIndex() {
+        SortedIndexDefinition indexDefinition = SchemaBuilders.sortedIndex("TEST_IDX")
+                .addIndexColumn(ColumnType.INT32.typeSpec().name()).asc().done()
+                .build();
+
+        SortedIndexStorage indexStorage = createIndexStorage(indexDefinition);
+
+        BinaryTupleRowSerializer serializer = new BinaryTupleRowSerializer(indexStorage.indexDescriptor());
+
+        Cursor<IndexRow> scanBeforeHashNext = indexStorage.scan(null, null, 0);

Review Comment:
   Can you clarify this naming convention? `hasNext` is a operation on cursor. I guess you check it in relation to `put`, but put itself is not mentioned in the name. As a result, I don't understand what you mean with these names.
   And it's not scan, it's cursor, right? You can't _scan_ after calling `hasNext`, because cursor is not yet created, etc. You see how confused I am



##########
modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/index/RocksDbSortedIndexStorage.java:
##########
@@ -167,24 +167,71 @@ private Cursor<ByteBuffer> createScanCursor(byte @Nullable [] lowerBound, byte @
 
         RocksIterator it = indexCf.newIterator(options);
 
-        if (lowerBound == null) {
-            it.seek(partitionStorage.partitionStartPrefix());
-        } else {
-            it.seek(lowerBound);
-        }
+        return new Cursor<>() {
+            @Nullable
+            private Boolean hasNext;
 
-        return new RocksIteratorAdapter<>(it) {
-            @Override
-            protected ByteBuffer decodeEntry(byte[] key, byte[] value) {
-                return ByteBuffer.wrap(key).order(ORDER);
-            }
+            @Nullable
+            private byte[] key;
 
             @Override
             public void close() {
-                super.close();
+                it.close();
 
                 RocksUtils.closeAll(options, upperBoundSlice);
             }
+
+            @Override
+            public boolean hasNext() {
+                advanceIfNeeded();
+
+                return hasNext;
+            }
+
+            @Override
+            public ByteBuffer next() {
+                advanceIfNeeded();
+
+                boolean hasNext = this.hasNext;
+
+                this.hasNext = null;
+
+                if (!hasNext) {
+                    throw new NoSuchElementException();
+                }
+
+                return ByteBuffer.wrap(key).order(ORDER);
+            }
+
+            private void advanceIfNeeded() throws StorageException {
+                if (hasNext == null) {
+                    try {
+                        it.refresh();
+                    } catch (RocksDBException e) {
+                        throw new StorageException("Error refreshing an iterator", e);
+                    }
+
+                    if (key == null) {
+                        it.seek(lowerBound == null ? partitionStorage.partitionStartPrefix() : lowerBound);
+                    } else {
+                        it.seekForPrev(key);
+
+                        it.next();

Review Comment:
   Technically, iterator may not be valid in this case and calling `next` is unsafe. You should add a test where index has a single value and it's deleted while iterator is "standing" on it.
   It'll most likely fail.



##########
modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/AbstractSortedIndexStorageTest.java:
##########
@@ -579,6 +585,401 @@ void testNullValues(ColumnDefinition columnDefinition) throws Exception {
         }
     }
 
+    /**
+     * Checks simple scenarios for a scanning cursor.
+     */
+    @Test
+    void testScanSimple() {
+        SortedIndexDefinition indexDefinition = SchemaBuilders.sortedIndex("TEST_IDX")
+                .addIndexColumn(ColumnType.INT32.typeSpec().name()).asc().done()
+                .build();
+
+        SortedIndexStorage indexStorage = createIndexStorage(indexDefinition);
+
+        BinaryTupleRowSerializer serializer = new BinaryTupleRowSerializer(indexStorage.indexDescriptor());
+
+        for (int i = 0; i < 5; i++) {
+            put(indexStorage, serializer.serializeRow(new Object[]{i}, new RowId(TEST_PARTITION)));
+        }
+
+        // Checking without borders.
+        assertThat(
+                scan(indexStorage, null, null, 0).stream().map(objects -> objects[0]).collect(toList()),
+                contains(0, 1, 2, 3, 4)
+        );
+
+        // Let's check without borders.
+        assertThat(
+                indexStorage.scan(
+                                serializer.serializeRowPrefix(0),
+                                serializer.serializeRowPrefix(4),
+                                (GREATER_OR_EQUAL | LESS_OR_EQUAL)
+                        ).stream()
+                        .map(indexRow -> serializer.deserializeColumns(indexRow)[0])
+                        .collect(toList()),
+                contains(0, 1, 2, 3, 4)
+        );
+
+        assertThat(
+                indexStorage.scan(
+                                serializer.serializeRowPrefix(0),
+                                serializer.serializeRowPrefix(4),
+                                (GREATER_OR_EQUAL | LESS)
+                        ).stream()
+                        .map(indexRow -> serializer.deserializeColumns(indexRow)[0])
+                        .collect(toList()),
+                contains(0, 1, 2, 3)
+        );
+
+        assertThat(
+                indexStorage.scan(
+                                serializer.serializeRowPrefix(0),
+                                serializer.serializeRowPrefix(4),
+                                (GREATER | LESS_OR_EQUAL)
+                        ).stream()
+                        .map(indexRow -> serializer.deserializeColumns(indexRow)[0])
+                        .collect(toList()),
+                contains(1, 2, 3, 4)
+        );
+
+        assertThat(
+                indexStorage.scan(
+                                serializer.serializeRowPrefix(0),
+                                serializer.serializeRowPrefix(4),
+                                (GREATER | LESS)
+                        ).stream()
+                        .map(indexRow -> serializer.deserializeColumns(indexRow)[0])
+                        .collect(toList()),
+                contains(1, 2, 3)
+        );
+
+        // Let's check only with the lower bound.
+        assertThat(
+                indexStorage.scan(
+                                serializer.serializeRowPrefix(1),
+                                null,
+                                (GREATER_OR_EQUAL | LESS_OR_EQUAL)
+                        ).stream()
+                        .map(indexRow -> serializer.deserializeColumns(indexRow)[0])
+                        .collect(toList()),
+                contains(1, 2, 3, 4)
+        );
+
+        assertThat(
+                indexStorage.scan(
+                                serializer.serializeRowPrefix(1),
+                                null,
+                                (GREATER_OR_EQUAL | LESS)
+                        ).stream()
+                        .map(indexRow -> serializer.deserializeColumns(indexRow)[0])
+                        .collect(toList()),
+                contains(1, 2, 3, 4)
+        );
+
+        assertThat(
+                indexStorage.scan(
+                                serializer.serializeRowPrefix(1),
+                                null,
+                                (GREATER | LESS_OR_EQUAL)
+                        ).stream()
+                        .map(indexRow -> serializer.deserializeColumns(indexRow)[0])
+                        .collect(toList()),
+                contains(2, 3, 4)
+        );
+
+        assertThat(
+                indexStorage.scan(
+                                serializer.serializeRowPrefix(1),
+                                null,
+                                (GREATER | LESS)
+                        ).stream()
+                        .map(indexRow -> serializer.deserializeColumns(indexRow)[0])
+                        .collect(toList()),
+                contains(2, 3, 4)
+        );
+
+        // Let's check only with the upper bound.
+        assertThat(
+                indexStorage.scan(
+                                null,
+                                serializer.serializeRowPrefix(3),
+                                (GREATER_OR_EQUAL | LESS_OR_EQUAL)
+                        ).stream()
+                        .map(indexRow -> serializer.deserializeColumns(indexRow)[0])
+                        .collect(toList()),
+                contains(0, 1, 2, 3)
+        );
+
+        assertThat(
+                indexStorage.scan(
+                                null,
+                                serializer.serializeRowPrefix(3),
+                                (GREATER_OR_EQUAL | LESS)
+                        ).stream()
+                        .map(indexRow -> serializer.deserializeColumns(indexRow)[0])
+                        .collect(toList()),
+                contains(0, 1, 2)
+        );
+
+        assertThat(
+                indexStorage.scan(
+                                null,
+                                serializer.serializeRowPrefix(3),
+                                (GREATER | LESS_OR_EQUAL)
+                        ).stream()
+                        .map(indexRow -> serializer.deserializeColumns(indexRow)[0])
+                        .collect(toList()),
+                contains(0, 1, 2, 3)
+        );
+
+        assertThat(
+                indexStorage.scan(
+                                null,
+                                serializer.serializeRowPrefix(3),
+                                (GREATER | LESS)
+                        ).stream()
+                        .map(indexRow -> serializer.deserializeColumns(indexRow)[0])
+                        .collect(toList()),
+                contains(0, 1, 2)
+        );
+    }
+
+    @Test
+    void testScanContractForEmptyIndex() {
+        SortedIndexDefinition indexDefinition = SchemaBuilders.sortedIndex("TEST_IDX")

Review Comment:
   Can it become 3 separate tests? I'm sure we can shorten the header part somehow



##########
modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/impl/TestSortedIndexStorage.java:
##########
@@ -45,7 +44,9 @@
  * Test implementation of MV sorted index storage.
  */
 public class TestSortedIndexStorage implements SortedIndexStorage {
-    private final ConcurrentNavigableMap<ByteBuffer, Set<RowId>> index;
+    private static final Object NULL = new Object();
+
+    private final ConcurrentNavigableMap<ByteBuffer, NavigableMap<RowId, Object>> index;

Review Comment:
   I guess `NavigableSet` should be used instead of the map. What do you store in those values?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org