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/03/21 15:00:25 UTC

[GitHub] [ignite-3] ibessonov opened a new pull request #739: IGNITE-16697 MV storage methods and reference implementation, not tested yet.

ibessonov opened a new pull request #739:
URL: https://github.com/apache/ignite-3/pull/739


   […ted yet
   
   Signed-off-by: ibessonov <be...@gmail.com>](https://issues.apache.org/jira/browse/IGNITE-16697)


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



[GitHub] [ignite-3] AMashenkov commented on a change in pull request #739: IGNITE-16697 MV storage methods and reference implementation

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #739:
URL: https://github.com/apache/ignite-3/pull/739#discussion_r833028962



##########
File path: modules/storage-api/src/main/java/org/apache/ignite/internal/storage/PartitionStorage.java
##########
@@ -128,6 +143,66 @@
      */
     Cursor<DataRow> scan(Predicate<SearchRow> filter) throws StorageException;
 
+    /**
+     * Scans the partition and returns a cursor of values in at the given timestamp.
+     *
+     * @param keyFilter Key filter. Binary rows passed to the filter may or may not have a value, filter should only check keys.
+     * @param timestamp Timestamp

Review comment:
       ```suggestion
        * @param timestamp Timestamp.
   ```




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



[GitHub] [ignite-3] AMashenkov commented on a change in pull request #739: IGNITE-16697 MV storage methods and reference implementation

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #739:
URL: https://github.com/apache/ignite-3/pull/739#discussion_r833002638



##########
File path: modules/storage-api/src/main/java/org/apache/ignite/internal/storage/index/SortedIndexStorage.java
##########
@@ -58,12 +64,63 @@
      */
     void remove(IndexRow row);
 
+    /** Exclude lower bound. */
+    byte GREATER = 0;
+
+    /** Include lower bound. */
+    byte GREATER_OR_EQUAL = 1;
+
+    /** Exclude upper bound. */
+    byte LESS = 0;
+
+    /** Include upper bound. */
+    byte LESS_OR_EQUAL = 1 << 1;
+
+    byte FORWARD = 0;
+
+    byte BACKWARDS = 1 << 2;
+
     /**
      * Returns a range of index values between the lower bound (inclusive) and the upper bound (inclusive).
      */
     // TODO: add options https://issues.apache.org/jira/browse/IGNITE-16059
     Cursor<IndexRow> range(IndexRowPrefix lowerBound, IndexRowPrefix upperBound);
 
+    /**
+     * Returns a range of index values between the lower bound and the upper bound, consistent with the passed timestamp.
+     *
+     * @param lowerBound Lower bound. Exclusivity is controlled by a {@link #GREATER_OR_EQUAL} or {@link #GREATER} flag.
+     *      {@code null} means unbounded.
+     * @param upperBound Upper bound. Exclusivity is controlled by a {@link #LESS} or {@link #LESS_OR_EQUAL} flag.
+     *      {@code null} means unbounded.
+     * @param flags Control flags. {@link #GREATER} | {@link #LESS} | {@link #FORWARD} by default. Other available values
+     *      are {@link #GREATER_OR_EQUAL}, {@link #LESS_OR_EQUAL} and {@link #BACKWARDS}.
+     * @param timestamp Timestamp value for consistent multiversioned index scan.
+     * @param columnsProjection Bit set with column indexes to return. {@code null} means returning all available columns.
+     * @param partitionFilter Partition filter predicate. {@code null} means returning data from all partitions.
+     * @return Cursor with fetched index rows.
+     * @throws IllegalArgumentException If backwards flag is passed and backwards iteration is not supported by the storage.

Review comment:
       Should we add smth like `UnsupportedBackwardIndexScanException` exception class for this, 
   and\or `boolean isBackwardScanSupported()` method?




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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #739: IGNITE-16697 MV storage methods and reference implementation

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #739:
URL: https://github.com/apache/ignite-3/pull/739#discussion_r840405509



##########
File path: modules/storage-api/src/test/java/org/apache/ignite/internal/storage/AbstractMvPartitionStorageTest.java
##########
@@ -0,0 +1,206 @@
+/*
+ * 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;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.UUID;
+import java.util.function.Predicate;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.internal.util.Cursor;
+import org.junit.jupiter.api.Test;
+
+/**
+ * Base test for MV partition storages.
+ */
+public abstract class AbstractMvPartitionStorageTest extends BaseMvStoragesTest {
+    /**
+     * Creates a storage instance for testing.
+     */
+    protected abstract MvPartitionStorage partitionStorage();
+
+    /**
+     * Tests that reads and scan from empty storage return empty results.
+     */
+    @Test
+    public void testEmpty() throws Exception {
+        MvPartitionStorage pk = partitionStorage();
+
+        BinaryRow binaryKey = binaryKey(new TestKey(10, "foo"));
+
+        // Read.
+        assertNull(pk.read(binaryKey, null));
+        assertNull(pk.read(binaryKey, Timestamp.nextVersion()));
+
+        // Scan.
+        assertEquals(List.of(), convert(pk.scan(row -> true, null)));
+        assertEquals(List.of(), convert(pk.scan(row -> true, Timestamp.nextVersion())));
+    }
+
+    /**
+     * Tests basic invariants of {@link MvPartitionStorage#addWrite(BinaryRow, UUID)}.
+     */
+    @Test
+    public void testAddWrite() throws Exception {
+        MvPartitionStorage pk = partitionStorage();
+
+        TestKey key = new TestKey(10, "foo");
+        TestValue value = new TestValue(20, "bar");
+
+        BinaryRow binaryRow = binaryRow(key, value);
+
+        pk.addWrite(binaryRow, UUID.randomUUID());
+
+        // Attempt to write from another transaction.
+        assertThrows(TxIdMismatchException.class, () -> pk.addWrite(binaryRow, UUID.randomUUID()));
+
+        // Read without timestamp returns uncommited row.
+        assertEquals(value, value(pk.read(binaryKey(key), null)));
+
+        // Read with timestamp returns null.
+        assertNull(pk.read(binaryKey(key), Timestamp.nextVersion()));
+    }
+
+    /**
+     * Tests basic invariants of {@link MvPartitionStorage#abortWrite(BinaryRow)}.
+     */
+    @Test
+    public void testAbortWrite() throws Exception {
+        MvPartitionStorage pk = partitionStorage();
+
+        TestKey key = new TestKey(10, "foo");
+        TestValue value = new TestValue(20, "bar");
+
+        pk.addWrite(binaryRow(key, value), UUID.randomUUID());
+
+        pk.abortWrite(binaryKey(key));
+
+        // Aborted row can't be read.
+        assertNull(pk.read(binaryKey(key), null));
+    }
+
+    /**
+     * Tests basic invariants of {@link MvPartitionStorage#commitWrite(BinaryRow, Timestamp)}.
+     */
+    @Test
+    public void testCommitWrite() throws Exception {
+        MvPartitionStorage pk = partitionStorage();
+
+        TestKey key = new TestKey(10, "foo");
+        TestValue value = new TestValue(20, "bar");
+
+        BinaryRow binaryRow = binaryRow(key, value);
+
+        pk.addWrite(binaryRow, UUID.randomUUID());
+
+        Timestamp tsBefore = Timestamp.nextVersion();
+
+        Timestamp tsExact = Timestamp.nextVersion();
+        pk.commitWrite(binaryRow, tsExact);
+
+        Timestamp tsAfter = Timestamp.nextVersion();
+
+        // Row is invisible at the time before writing.
+        assertNull(pk.read(binaryRow, tsBefore));
+
+        // Row is valid at the time during and after writing.
+        assertEquals(value, value(pk.read(binaryRow, null)));
+        assertEquals(value, value(pk.read(binaryRow, tsExact)));
+        assertEquals(value, value(pk.read(binaryRow, tsAfter)));
+
+        TestValue newValue = new TestValue(30, "duh");
+
+        pk.addWrite(binaryRow(key, newValue), UUID.randomUUID());
+
+        // Same checks, but now there are two different versions.
+        assertNull(pk.read(binaryRow, tsBefore));
+
+        assertEquals(newValue, value(pk.read(binaryRow, null)));
+
+        assertEquals(value, value(pk.read(binaryRow, tsExact)));
+        assertEquals(value, value(pk.read(binaryRow, tsAfter)));
+        assertEquals(value, value(pk.read(binaryRow, Timestamp.nextVersion())));
+
+        pk.commitWrite(binaryKey(key), Timestamp.nextVersion());
+
+        assertEquals(newValue, value(pk.read(binaryRow, null)));
+        assertEquals(newValue, value(pk.read(binaryRow, Timestamp.nextVersion())));
+    }
+
+    /**
+     * Tests basic invariants of {@link MvPartitionStorage#scan(Predicate, Timestamp)}.
+     */
+    @Test
+    public void testScan() throws Exception {
+        MvPartitionStorage pk = partitionStorage();
+
+        TestKey key1 = new TestKey(1, "1");
+        TestValue value1 = new TestValue(10, "xxx");
+
+        TestKey key2 = new TestKey(2, "2");
+        TestValue value2 = new TestValue(20, "yyy");
+
+        pk.addWrite(binaryRow(key1, value1), UUID.randomUUID());
+        pk.addWrite(binaryRow(key2, value2), UUID.randomUUID());
+
+        // Scan with and without filters.
+        assertEquals(List.of(value1, value2), convert(pk.scan(row -> true, null)));
+        assertEquals(List.of(value1), convert(pk.scan(row -> key(row).intKey == 1, null)));
+        assertEquals(List.of(value2), convert(pk.scan(row -> key(row).intKey == 2, null)));
+
+        Timestamp ts1 = Timestamp.nextVersion();
+
+        Timestamp ts2 = Timestamp.nextVersion();
+        pk.commitWrite(binaryKey(key1), ts2);
+
+        Timestamp ts3 = Timestamp.nextVersion();
+
+        Timestamp ts4 = Timestamp.nextVersion();
+        pk.commitWrite(binaryKey(key2), ts4);
+
+        Timestamp ts5 = Timestamp.nextVersion();
+
+        // Full scan with various timestamp values.
+        assertEquals(List.of(), convert(pk.scan(row -> true, ts1)));
+
+        assertEquals(List.of(value1), convert(pk.scan(row -> true, ts2)));
+        assertEquals(List.of(value1), convert(pk.scan(row -> true, ts3)));
+
+        assertEquals(List.of(value1, value2), convert(pk.scan(row -> true, ts4)));
+        assertEquals(List.of(value1, value2), convert(pk.scan(row -> true, ts5)));
+    }
+
+    private List<TestValue> convert(Cursor<BinaryRow> cursor) throws Exception {
+        try (cursor) {
+            List<TestValue> list = StreamSupport.stream(cursor.spliterator(), false)

Review comment:
       Thank you!




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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #739: IGNITE-16697 MV storage methods and reference implementation

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #739:
URL: https://github.com/apache/ignite-3/pull/739#discussion_r840408048



##########
File path: modules/storage-api/src/test/java/org/apache/ignite/internal/storage/AbstractMvPartitionStorageTest.java
##########
@@ -0,0 +1,206 @@
+/*
+ * 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;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.UUID;
+import java.util.function.Predicate;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.internal.util.Cursor;
+import org.junit.jupiter.api.Test;
+
+/**
+ * Base test for MV partition storages.
+ */
+public abstract class AbstractMvPartitionStorageTest extends BaseMvStoragesTest {
+    /**
+     * Creates a storage instance for testing.
+     */
+    protected abstract MvPartitionStorage partitionStorage();
+
+    /**
+     * Tests that reads and scan from empty storage return empty results.
+     */
+    @Test
+    public void testEmpty() throws Exception {
+        MvPartitionStorage pk = partitionStorage();
+
+        BinaryRow binaryKey = binaryKey(new TestKey(10, "foo"));
+
+        // Read.
+        assertNull(pk.read(binaryKey, null));
+        assertNull(pk.read(binaryKey, Timestamp.nextVersion()));
+
+        // Scan.
+        assertEquals(List.of(), convert(pk.scan(row -> true, null)));
+        assertEquals(List.of(), convert(pk.scan(row -> true, Timestamp.nextVersion())));
+    }
+
+    /**
+     * Tests basic invariants of {@link MvPartitionStorage#addWrite(BinaryRow, UUID)}.
+     */
+    @Test
+    public void testAddWrite() throws Exception {
+        MvPartitionStorage pk = partitionStorage();
+
+        TestKey key = new TestKey(10, "foo");
+        TestValue value = new TestValue(20, "bar");
+
+        BinaryRow binaryRow = binaryRow(key, value);
+
+        pk.addWrite(binaryRow, UUID.randomUUID());
+
+        // Attempt to write from another transaction.
+        assertThrows(TxIdMismatchException.class, () -> pk.addWrite(binaryRow, UUID.randomUUID()));
+
+        // Read without timestamp returns uncommited row.
+        assertEquals(value, value(pk.read(binaryKey(key), null)));
+
+        // Read with timestamp returns null.
+        assertNull(pk.read(binaryKey(key), Timestamp.nextVersion()));

Review comment:
       It's tested in another test




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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #739: IGNITE-16697 MV storage methods and reference implementation

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #739:
URL: https://github.com/apache/ignite-3/pull/739#discussion_r840402832



##########
File path: modules/storage-api/src/test/java/org/apache/ignite/internal/storage/AbstractSortedIndexMvStorageTest.java
##########
@@ -0,0 +1,238 @@
+/*
+ * 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;
+
+import static org.apache.ignite.internal.storage.index.SortedIndexMvStorage.BACKWARDS;
+import static org.apache.ignite.internal.storage.index.SortedIndexMvStorage.FORWARD;
+import static org.apache.ignite.internal.storage.index.SortedIndexMvStorage.GREATER_OR_EQUAL;
+import static org.apache.ignite.internal.storage.index.SortedIndexMvStorage.LESS_OR_EQUAL;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+import java.util.List;
+import java.util.UUID;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+import org.apache.ignite.configuration.schemas.table.ColumnChange;
+import org.apache.ignite.configuration.schemas.table.SortedIndexChange;
+import org.apache.ignite.configuration.schemas.table.SortedIndexConfigurationSchema;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.configuration.testframework.ConfigurationExtension;
+import org.apache.ignite.internal.configuration.testframework.InjectConfiguration;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.schema.marshaller.MarshallerException;
+import org.apache.ignite.internal.schema.row.Row;
+import org.apache.ignite.internal.storage.index.IndexRowPrefix;
+import org.apache.ignite.internal.storage.index.SortedIndexMvStorage;
+import org.apache.ignite.internal.storage.index.SortedIndexMvStorage.IndexRowEx;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.internal.util.Cursor;
+import org.apache.ignite.lang.IgniteException;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+/**
+ * Base test for MV index storages.
+ */
+@ExtendWith(ConfigurationExtension.class)
+public abstract class AbstractSortedIndexMvStorageTest extends BaseMvStoragesTest {
+    protected static final String INDEX1 = "asc_asc";
+    protected static final String INDEX2 = "asc_desc";
+
+    protected TableConfiguration tableCfg;
+
+    @BeforeEach
+    void setUp(@InjectConfiguration(polymorphicExtensions = SortedIndexConfigurationSchema.class) TableConfiguration tableCfg) {
+        tableCfg.change(tableChange -> tableChange
+                .changePartitions(1)
+                .changePrimaryKey(pk -> pk.changeColumns("intKey", "strKey"))
+                .changeColumns(columns -> columns
+                        .create("intKey", column("intKey", "INT32"))
+                        .create("strKey", column("strKey", "STRING"))
+                        .create("intVal", column("intVal", "INT32"))
+                        .create("strVal", column("strVal", "STRING"))
+                )
+                .changeIndices(indexes -> indexes
+                        .create(INDEX1, idx -> idx.convert(SortedIndexChange.class).changeColumns(idxColumns -> idxColumns
+                                .create("strVal", c -> c.changeName("strVal").changeAsc(true))
+                                .create("intVal", c -> c.changeName("intVal").changeAsc(true))
+                        ))
+                        .create(INDEX2, idx -> idx.convert(SortedIndexChange.class).changeColumns(idxColumns -> idxColumns
+                                .create("strVal", c -> c.changeName("strVal").changeAsc(true))
+                                .create("intVal", c -> c.changeName("intVal").changeAsc(false))
+                        ))
+                )
+        ).join();
+
+        this.tableCfg = tableCfg;
+    }
+
+    private static Consumer<ColumnChange> column(String name, String typeName) {
+        return c -> c.changeName(name).changeNullable(false).changeType(type -> type.changeType(typeName));
+    }
+
+    /**
+     * Creates a storage instance for testing.
+     */
+    protected abstract MvPartitionStorage partitionStorage();
+
+    /**
+     * Creates a storage instanc efor testing.
+     */
+    protected abstract SortedIndexMvStorage createIndexStorage(String name, TableView tableCfg);
+
+    @Test
+    public void testEmpty() throws Exception {
+        SortedIndexMvStorage index1 = createIndexStorage(INDEX1, tableCfg.value());
+        SortedIndexMvStorage index2 = createIndexStorage(INDEX2, tableCfg.value());
+
+        assertEquals(List.of(), convert(index1.scan(null, null, (byte) 0, null, null)));
+        assertEquals(List.of(), convert(index2.scan(null, null, (byte) 0, null, null)));
+    }
+
+    @Test
+    public void testBounds() throws Exception {
+        SortedIndexMvStorage index1 = createIndexStorage(INDEX1, tableCfg.value());
+        SortedIndexMvStorage index2 = createIndexStorage(INDEX2, tableCfg.value());
+
+        TestValue val9010 = new TestValue(90, "10");
+        TestValue val8010 = new TestValue(80, "10");
+        TestValue val9020 = new TestValue(90, "20");
+        TestValue val8020 = new TestValue(80, "20");
+
+        insert(new TestKey(1, "1"), val9010, null);
+        insert(new TestKey(2, "2"), val8010, null);
+        insert(new TestKey(3, "3"), val9020, null);
+        insert(new TestKey(4, "4"), val8020, null);
+
+        // Test without bounds.
+        assertEquals(List.of(val8010, val9010, val8020, val9020), convert(index1.scan(
+                null, null, FORWARD, null, null
+        )));
+
+        assertEquals(List.of(val9020, val8020, val9010, val8010), convert(index1.scan(
+                null, null, BACKWARDS, null, null
+        )));
+
+        assertEquals(List.of(val9010, val8010, val9020, val8020), convert(index2.scan(
+                null, null, FORWARD, null, null
+        )));
+
+        assertEquals(List.of(val8020, val9020, val8010, val9010), convert(index2.scan(
+                null, null, BACKWARDS, null, null
+        )));
+
+        // Lower bound exclusive.
+        assertEquals(List.of(val8020, val9020), convert(index1.scan(
+                prefix("10"), null, FORWARD, null, null
+        )));
+
+        assertEquals(List.of(val9020, val8020), convert(index1.scan(
+                prefix("10"), null, BACKWARDS, null, null
+        )));
+
+        assertEquals(List.of(val9020, val8020), convert(index2.scan(
+                prefix("10"), null, FORWARD, null, null
+        )));
+
+        assertEquals(List.of(val8020, val9020), convert(index2.scan(
+                prefix("10"), null, BACKWARDS, null, null
+        )));
+
+        // Lower bound inclusive.
+        assertEquals(List.of(val8010, val9010, val8020, val9020), convert(index1.scan(
+                prefix("10"), null, GREATER_OR_EQUAL | FORWARD, null, null
+        )));
+
+        assertEquals(List.of(val9020, val8020, val9010, val8010), convert(index1.scan(
+                prefix("10"), null, GREATER_OR_EQUAL | BACKWARDS, null, null
+        )));
+
+        assertEquals(List.of(val9010, val8010, val9020, val8020), convert(index2.scan(
+                prefix("10"), null, GREATER_OR_EQUAL | FORWARD, null, null
+        )));
+
+        assertEquals(List.of(val8020, val9020, val8010, val9010), convert(index2.scan(
+                prefix("10"), null, GREATER_OR_EQUAL | BACKWARDS, null, null
+        )));
+
+        // Upper bound exclusive.
+        assertEquals(List.of(val8010, val9010), convert(index1.scan(
+                null, prefix("20"), FORWARD, null, null
+        )));
+
+        assertEquals(List.of(val9010, val8010), convert(index1.scan(
+                null, prefix("20"), BACKWARDS, null, null
+        )));
+
+        assertEquals(List.of(val9010, val8010), convert(index2.scan(
+                null, prefix("20"), FORWARD, null, null
+        )));
+
+        assertEquals(List.of(val8010, val9010), convert(index2.scan(
+                null, prefix("20"), BACKWARDS, null, null
+        )));
+
+        // Lower bound inclusive.
+        assertEquals(List.of(val8010, val9010, val8020, val9020), convert(index1.scan(
+                null, prefix("20"), LESS_OR_EQUAL | FORWARD, null, null
+        )));
+
+        assertEquals(List.of(val9020, val8020, val9010, val8010), convert(index1.scan(
+                null, prefix("20"), LESS_OR_EQUAL | BACKWARDS, null, null
+        )));
+
+        assertEquals(List.of(val9010, val8010, val9020, val8020), convert(index2.scan(
+                null, prefix("20"), LESS_OR_EQUAL | FORWARD, null, null
+        )));
+
+        assertEquals(List.of(val8020, val9020, val8010, val9010), convert(index2.scan(
+                null, prefix("20"), LESS_OR_EQUAL | BACKWARDS, null, null
+        )));
+    }
+
+    protected void insert(TestKey key, TestValue value, Timestamp ts) {
+        MvPartitionStorage pk = partitionStorage();
+
+        BinaryRow binaryRow = binaryRow(key, value);
+
+        pk.addWrite(binaryRow, UUID.randomUUID());
+
+        pk.commitWrite(binaryRow, ts == null ? Timestamp.nextVersion() : ts);
+    }
+
+    protected IndexRowPrefix prefix(String val) {
+        return () -> new Object[]{val};
+    }
+
+    protected List<TestValue> convert(Cursor<IndexRowEx> cursor) throws Exception {
+        try (cursor) {
+            return StreamSupport.stream(cursor.spliterator(), false)

Review comment:
       We should wait for your PR to me merged, seems like a valuable addition




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



[GitHub] [ignite-3] AMashenkov commented on a change in pull request #739: IGNITE-16697 MV storage methods and reference implementation

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #739:
URL: https://github.com/apache/ignite-3/pull/739#discussion_r833002638



##########
File path: modules/storage-api/src/main/java/org/apache/ignite/internal/storage/index/SortedIndexStorage.java
##########
@@ -58,12 +64,63 @@
      */
     void remove(IndexRow row);
 
+    /** Exclude lower bound. */
+    byte GREATER = 0;
+
+    /** Include lower bound. */
+    byte GREATER_OR_EQUAL = 1;
+
+    /** Exclude upper bound. */
+    byte LESS = 0;
+
+    /** Include upper bound. */
+    byte LESS_OR_EQUAL = 1 << 1;
+
+    byte FORWARD = 0;
+
+    byte BACKWARDS = 1 << 2;
+
     /**
      * Returns a range of index values between the lower bound (inclusive) and the upper bound (inclusive).
      */
     // TODO: add options https://issues.apache.org/jira/browse/IGNITE-16059
     Cursor<IndexRow> range(IndexRowPrefix lowerBound, IndexRowPrefix upperBound);
 
+    /**
+     * Returns a range of index values between the lower bound and the upper bound, consistent with the passed timestamp.
+     *
+     * @param lowerBound Lower bound. Exclusivity is controlled by a {@link #GREATER_OR_EQUAL} or {@link #GREATER} flag.
+     *      {@code null} means unbounded.
+     * @param upperBound Upper bound. Exclusivity is controlled by a {@link #LESS} or {@link #LESS_OR_EQUAL} flag.
+     *      {@code null} means unbounded.
+     * @param flags Control flags. {@link #GREATER} | {@link #LESS} | {@link #FORWARD} by default. Other available values
+     *      are {@link #GREATER_OR_EQUAL}, {@link #LESS_OR_EQUAL} and {@link #BACKWARDS}.
+     * @param timestamp Timestamp value for consistent multiversioned index scan.
+     * @param columnsProjection Bit set with column indexes to return. {@code null} means returning all available columns.
+     * @param partitionFilter Partition filter predicate. {@code null} means returning data from all partitions.
+     * @return Cursor with fetched index rows.
+     * @throws IllegalArgumentException If backwards flag is passed and backwards iteration is not supported by the storage.

Review comment:
       Let's add smth like UnsupportedBackwardIndexScanException exception for this




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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #739: IGNITE-16697 MV storage methods and reference implementation

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #739:
URL: https://github.com/apache/ignite-3/pull/739#discussion_r840402387



##########
File path: modules/storage-api/src/test/java/org/apache/ignite/internal/storage/AbstractSortedIndexMvStorageTest.java
##########
@@ -0,0 +1,238 @@
+/*
+ * 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;
+
+import static org.apache.ignite.internal.storage.index.SortedIndexMvStorage.BACKWARDS;
+import static org.apache.ignite.internal.storage.index.SortedIndexMvStorage.FORWARD;
+import static org.apache.ignite.internal.storage.index.SortedIndexMvStorage.GREATER_OR_EQUAL;
+import static org.apache.ignite.internal.storage.index.SortedIndexMvStorage.LESS_OR_EQUAL;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+import java.util.List;
+import java.util.UUID;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+import org.apache.ignite.configuration.schemas.table.ColumnChange;
+import org.apache.ignite.configuration.schemas.table.SortedIndexChange;
+import org.apache.ignite.configuration.schemas.table.SortedIndexConfigurationSchema;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.configuration.testframework.ConfigurationExtension;
+import org.apache.ignite.internal.configuration.testframework.InjectConfiguration;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.schema.marshaller.MarshallerException;
+import org.apache.ignite.internal.schema.row.Row;
+import org.apache.ignite.internal.storage.index.IndexRowPrefix;
+import org.apache.ignite.internal.storage.index.SortedIndexMvStorage;
+import org.apache.ignite.internal.storage.index.SortedIndexMvStorage.IndexRowEx;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.internal.util.Cursor;
+import org.apache.ignite.lang.IgniteException;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+/**
+ * Base test for MV index storages.
+ */
+@ExtendWith(ConfigurationExtension.class)
+public abstract class AbstractSortedIndexMvStorageTest extends BaseMvStoragesTest {
+    protected static final String INDEX1 = "asc_asc";
+    protected static final String INDEX2 = "asc_desc";
+
+    protected TableConfiguration tableCfg;
+
+    @BeforeEach
+    void setUp(@InjectConfiguration(polymorphicExtensions = SortedIndexConfigurationSchema.class) TableConfiguration tableCfg) {
+        tableCfg.change(tableChange -> tableChange
+                .changePartitions(1)
+                .changePrimaryKey(pk -> pk.changeColumns("intKey", "strKey"))
+                .changeColumns(columns -> columns
+                        .create("intKey", column("intKey", "INT32"))
+                        .create("strKey", column("strKey", "STRING"))
+                        .create("intVal", column("intVal", "INT32"))
+                        .create("strVal", column("strVal", "STRING"))
+                )
+                .changeIndices(indexes -> indexes
+                        .create(INDEX1, idx -> idx.convert(SortedIndexChange.class).changeColumns(idxColumns -> idxColumns
+                                .create("strVal", c -> c.changeName("strVal").changeAsc(true))
+                                .create("intVal", c -> c.changeName("intVal").changeAsc(true))
+                        ))
+                        .create(INDEX2, idx -> idx.convert(SortedIndexChange.class).changeColumns(idxColumns -> idxColumns
+                                .create("strVal", c -> c.changeName("strVal").changeAsc(true))
+                                .create("intVal", c -> c.changeName("intVal").changeAsc(false))
+                        ))
+                )
+        ).join();
+
+        this.tableCfg = tableCfg;
+    }
+
+    private static Consumer<ColumnChange> column(String name, String typeName) {
+        return c -> c.changeName(name).changeNullable(false).changeType(type -> type.changeType(typeName));
+    }
+
+    /**
+     * Creates a storage instance for testing.
+     */
+    protected abstract MvPartitionStorage partitionStorage();
+
+    /**
+     * Creates a storage instanc efor testing.
+     */
+    protected abstract SortedIndexMvStorage createIndexStorage(String name, TableView tableCfg);
+
+    @Test
+    public void testEmpty() throws Exception {
+        SortedIndexMvStorage index1 = createIndexStorage(INDEX1, tableCfg.value());
+        SortedIndexMvStorage index2 = createIndexStorage(INDEX2, tableCfg.value());

Review comment:
       They have different orderings, just to be more confident




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



[GitHub] [ignite-3] sashapolo commented on a change in pull request #739: IGNITE-16697 MV storage methods and reference implementation

Posted by GitBox <gi...@apache.org>.
sashapolo commented on a change in pull request #739:
URL: https://github.com/apache/ignite-3/pull/739#discussion_r840312887



##########
File path: modules/storage-api/src/test/java/org/apache/ignite/internal/storage/AbstractSortedIndexMvStorageTest.java
##########
@@ -0,0 +1,238 @@
+/*
+ * 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;
+
+import static org.apache.ignite.internal.storage.index.SortedIndexMvStorage.BACKWARDS;
+import static org.apache.ignite.internal.storage.index.SortedIndexMvStorage.FORWARD;
+import static org.apache.ignite.internal.storage.index.SortedIndexMvStorage.GREATER_OR_EQUAL;
+import static org.apache.ignite.internal.storage.index.SortedIndexMvStorage.LESS_OR_EQUAL;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+import java.util.List;
+import java.util.UUID;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+import org.apache.ignite.configuration.schemas.table.ColumnChange;
+import org.apache.ignite.configuration.schemas.table.SortedIndexChange;
+import org.apache.ignite.configuration.schemas.table.SortedIndexConfigurationSchema;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.configuration.testframework.ConfigurationExtension;
+import org.apache.ignite.internal.configuration.testframework.InjectConfiguration;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.schema.marshaller.MarshallerException;
+import org.apache.ignite.internal.schema.row.Row;
+import org.apache.ignite.internal.storage.index.IndexRowPrefix;
+import org.apache.ignite.internal.storage.index.SortedIndexMvStorage;
+import org.apache.ignite.internal.storage.index.SortedIndexMvStorage.IndexRowEx;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.internal.util.Cursor;
+import org.apache.ignite.lang.IgniteException;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+/**
+ * Base test for MV index storages.
+ */
+@ExtendWith(ConfigurationExtension.class)
+public abstract class AbstractSortedIndexMvStorageTest extends BaseMvStoragesTest {
+    protected static final String INDEX1 = "asc_asc";
+    protected static final String INDEX2 = "asc_desc";
+
+    protected TableConfiguration tableCfg;
+
+    @BeforeEach
+    void setUp(@InjectConfiguration(polymorphicExtensions = SortedIndexConfigurationSchema.class) TableConfiguration tableCfg) {
+        tableCfg.change(tableChange -> tableChange
+                .changePartitions(1)
+                .changePrimaryKey(pk -> pk.changeColumns("intKey", "strKey"))
+                .changeColumns(columns -> columns
+                        .create("intKey", column("intKey", "INT32"))
+                        .create("strKey", column("strKey", "STRING"))
+                        .create("intVal", column("intVal", "INT32"))
+                        .create("strVal", column("strVal", "STRING"))
+                )
+                .changeIndices(indexes -> indexes
+                        .create(INDEX1, idx -> idx.convert(SortedIndexChange.class).changeColumns(idxColumns -> idxColumns
+                                .create("strVal", c -> c.changeName("strVal").changeAsc(true))
+                                .create("intVal", c -> c.changeName("intVal").changeAsc(true))
+                        ))
+                        .create(INDEX2, idx -> idx.convert(SortedIndexChange.class).changeColumns(idxColumns -> idxColumns
+                                .create("strVal", c -> c.changeName("strVal").changeAsc(true))
+                                .create("intVal", c -> c.changeName("intVal").changeAsc(false))
+                        ))
+                )
+        ).join();
+
+        this.tableCfg = tableCfg;
+    }
+
+    private static Consumer<ColumnChange> column(String name, String typeName) {
+        return c -> c.changeName(name).changeNullable(false).changeType(type -> type.changeType(typeName));
+    }
+
+    /**
+     * Creates a storage instance for testing.
+     */
+    protected abstract MvPartitionStorage partitionStorage();
+
+    /**
+     * Creates a storage instanc efor testing.
+     */
+    protected abstract SortedIndexMvStorage createIndexStorage(String name, TableView tableCfg);
+
+    @Test
+    public void testEmpty() throws Exception {
+        SortedIndexMvStorage index1 = createIndexStorage(INDEX1, tableCfg.value());
+        SortedIndexMvStorage index2 = createIndexStorage(INDEX2, tableCfg.value());

Review comment:
       why do you need two indices here?




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



[GitHub] [ignite-3] tledkov-gridgain commented on a change in pull request #739: IGNITE-16697 MV storage methods and reference implementation

Posted by GitBox <gi...@apache.org>.
tledkov-gridgain commented on a change in pull request #739:
URL: https://github.com/apache/ignite-3/pull/739#discussion_r833051149



##########
File path: modules/storage-api/src/main/java/org/apache/ignite/internal/storage/PartitionStorage.java
##########
@@ -128,6 +143,66 @@
      */
     Cursor<DataRow> scan(Predicate<SearchRow> filter) throws StorageException;
 
+    /**
+     * Scans the partition and returns a cursor of values in at the given timestamp.
+     *
+     * @param keyFilter Key filter. Binary rows passed to the filter may or may not have a value, filter should only check keys.
+     * @param timestamp Timestamp
+     * @return Cursor.
+     */
+    default Cursor<BinaryRow> scan(Predicate<BinaryRow> keyFilter, @Nullable Timestamp timestamp) {
+        throw new UnsupportedOperationException("scan");
+    }
+
+    /**
+     * Exception class that describes the situation where two independant transactions attempting to write values for the same key.
+     */
+    class TxIdMismatchException extends RuntimeException {
+    }

Review comment:
       Why does not inherit `IgniteException`?




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



[GitHub] [ignite-3] sashapolo commented on a change in pull request #739: IGNITE-16697 MV storage methods and reference implementation

Posted by GitBox <gi...@apache.org>.
sashapolo commented on a change in pull request #739:
URL: https://github.com/apache/ignite-3/pull/739#discussion_r839616044



##########
File path: modules/storage-api/src/main/java/org/apache/ignite/internal/storage/MvPartitionStorage.java
##########
@@ -0,0 +1,96 @@
+/*
+ * 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;
+
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Predicate;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.internal.util.Cursor;
+import org.apache.ignite.lang.IgniteException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Multi-versioned partition storage.
+ * POC version, that represents a combination between a replicated TX-aware MV storage and physical MV storage. Their API are very similar,
+ * althouh there are very important differences that will be addressed in the future.
+ */
+public interface MvPartitionStorage {
+    /**
+     * Exception class that describes the situation where two independant transactions attempting to write values for the same key.

Review comment:
       ```suggestion
        * Exception class that describes the situation when two independent transactions attempt to write values for the same key.
   ```

##########
File path: modules/storage-api/src/main/java/org/apache/ignite/internal/storage/MvPartitionStorage.java
##########
@@ -0,0 +1,96 @@
+/*
+ * 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;
+
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Predicate;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.internal.util.Cursor;
+import org.apache.ignite.lang.IgniteException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Multi-versioned partition storage.
+ * POC version, that represents a combination between a replicated TX-aware MV storage and physical MV storage. Their API are very similar,
+ * althouh there are very important differences that will be addressed in the future.
+ */
+public interface MvPartitionStorage {
+    /**
+     * Exception class that describes the situation where two independant transactions attempting to write values for the same key.
+     */
+    class TxIdMismatchException extends IgniteException {

Review comment:
       I would prefer extracting this class into an upper level.

##########
File path: modules/storage-api/src/main/java/org/apache/ignite/internal/storage/MvPartitionStorage.java
##########
@@ -0,0 +1,96 @@
+/*
+ * 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;
+
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Predicate;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.internal.util.Cursor;
+import org.apache.ignite.lang.IgniteException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Multi-versioned partition storage.
+ * POC version, that represents a combination between a replicated TX-aware MV storage and physical MV storage. Their API are very similar,
+ * althouh there are very important differences that will be addressed in the future.
+ */
+public interface MvPartitionStorage {
+    /**
+     * Exception class that describes the situation where two independant transactions attempting to write values for the same key.
+     */
+    class TxIdMismatchException extends IgniteException {
+    }
+
+    /**
+     * Reads the value from the storage as it was at the given timestamp.
+     *
+     * @param key Key.
+     * @param timestamp Timestamp.
+     * @return Binary row that corresponds to the key or {@code null} if value is not found.
+     */
+    @Nullable
+    BinaryRow read(BinaryRow key, @Nullable Timestamp timestamp);
+
+    /**
+     * Creates uncommited version, assigned to the passed transaction id..

Review comment:
       ```suggestion
        * Creates an uncommitted version, assigned to the given transaction id.
   ```

##########
File path: modules/storage-api/src/main/java/org/apache/ignite/internal/storage/MvPartitionStorage.java
##########
@@ -0,0 +1,96 @@
+/*
+ * 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;
+
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Predicate;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.internal.util.Cursor;
+import org.apache.ignite.lang.IgniteException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Multi-versioned partition storage.
+ * POC version, that represents a combination between a replicated TX-aware MV storage and physical MV storage. Their API are very similar,
+ * althouh there are very important differences that will be addressed in the future.
+ */
+public interface MvPartitionStorage {
+    /**
+     * Exception class that describes the situation where two independant transactions attempting to write values for the same key.
+     */
+    class TxIdMismatchException extends IgniteException {
+    }
+
+    /**
+     * Reads the value from the storage as it was at the given timestamp.

Review comment:
       You should also specify, what a `null` timestamp means

##########
File path: modules/storage-api/src/main/java/org/apache/ignite/internal/storage/MvPartitionStorage.java
##########
@@ -0,0 +1,96 @@
+/*
+ * 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;
+
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Predicate;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.internal.util.Cursor;
+import org.apache.ignite.lang.IgniteException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Multi-versioned partition storage.
+ * POC version, that represents a combination between a replicated TX-aware MV storage and physical MV storage. Their API are very similar,
+ * althouh there are very important differences that will be addressed in the future.

Review comment:
       ```suggestion
    * although there are very important differences that will be addressed in the future.
   ```

##########
File path: modules/storage-api/src/main/java/org/apache/ignite/internal/storage/MvPartitionStorage.java
##########
@@ -0,0 +1,96 @@
+/*
+ * 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;
+
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Predicate;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.internal.util.Cursor;
+import org.apache.ignite.lang.IgniteException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Multi-versioned partition storage.
+ * POC version, that represents a combination between a replicated TX-aware MV storage and physical MV storage. Their API are very similar,
+ * althouh there are very important differences that will be addressed in the future.
+ */
+public interface MvPartitionStorage {
+    /**
+     * Exception class that describes the situation where two independant transactions attempting to write values for the same key.
+     */
+    class TxIdMismatchException extends IgniteException {
+    }
+
+    /**
+     * Reads the value from the storage as it was at the given timestamp.
+     *
+     * @param key Key.
+     * @param timestamp Timestamp.
+     * @return Binary row that corresponds to the key or {@code null} if value is not found.
+     */
+    @Nullable
+    BinaryRow read(BinaryRow key, @Nullable Timestamp timestamp);
+
+    /**
+     * Creates uncommited version, assigned to the passed transaction id..
+     *
+     * @param row Binary row to update. Key only row means value removal.
+     * @param txId Transaction id.
+     * @throws TxIdMismatchException If there's another pending update associated with different transaction id.
+     * @throws StorageException If failed to write data to the storage.
+     */
+    void addWrite(BinaryRow row, UUID txId) throws TxIdMismatchException, StorageException;
+
+    /**
+     * Aborts a pending update of the ongoing uncommited transaction. Invoked during rollback.
+     *
+     * @param key Key.
+     * @throws StorageException If failed to write data to the storage.
+     */
+    void abortWrite(BinaryRow key) throws StorageException;
+
+    /**
+     * Commits a pending update of the ongoing transaction. Invoked during commit. Commited value will be versioned by the given timestamp.
+     *
+     * @param key Key.
+     * @param timestamp Timestamp to associate with commited value.
+     * @throws StorageException If failed to write data to the storage.
+     */
+    void commitWrite(BinaryRow key, Timestamp timestamp) throws StorageException;
+
+    /**
+     * Removes data associated with old timestamps.
+     *
+     * @param from Start of hashes range to process. Inclusive.
+     * @param to End of hashes range to process. Inclusive.
+     * @param timestamp Timestamp to remove all the data with a lesser timestamp.
+     * @return Future for the operation.
+     */
+    CompletableFuture<?> cleanup(int from, int to, Timestamp timestamp);

Review comment:
       IDEA tells me that this method is never used, which means it is not covered by tests

##########
File path: modules/storage-api/src/main/java/org/apache/ignite/internal/storage/MvPartitionStorage.java
##########
@@ -0,0 +1,96 @@
+/*
+ * 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;
+
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Predicate;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.internal.util.Cursor;
+import org.apache.ignite.lang.IgniteException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Multi-versioned partition storage.
+ * POC version, that represents a combination between a replicated TX-aware MV storage and physical MV storage. Their API are very similar,
+ * althouh there are very important differences that will be addressed in the future.
+ */
+public interface MvPartitionStorage {
+    /**
+     * Exception class that describes the situation where two independant transactions attempting to write values for the same key.
+     */
+    class TxIdMismatchException extends IgniteException {
+    }
+
+    /**
+     * Reads the value from the storage as it was at the given timestamp.
+     *
+     * @param key Key.
+     * @param timestamp Timestamp.
+     * @return Binary row that corresponds to the key or {@code null} if value is not found.
+     */
+    @Nullable
+    BinaryRow read(BinaryRow key, @Nullable Timestamp timestamp);
+
+    /**
+     * Creates uncommited version, assigned to the passed transaction id..
+     *
+     * @param row Binary row to update. Key only row means value removal.
+     * @param txId Transaction id.
+     * @throws TxIdMismatchException If there's another pending update associated with different transaction id.
+     * @throws StorageException If failed to write data to the storage.
+     */
+    void addWrite(BinaryRow row, UUID txId) throws TxIdMismatchException, StorageException;
+
+    /**
+     * Aborts a pending update of the ongoing uncommited transaction. Invoked during rollback.
+     *
+     * @param key Key.
+     * @throws StorageException If failed to write data to the storage.
+     */
+    void abortWrite(BinaryRow key) throws StorageException;
+
+    /**
+     * Commits a pending update of the ongoing transaction. Invoked during commit. Commited value will be versioned by the given timestamp.
+     *
+     * @param key Key.
+     * @param timestamp Timestamp to associate with commited value.
+     * @throws StorageException If failed to write data to the storage.
+     */
+    void commitWrite(BinaryRow key, Timestamp timestamp) throws StorageException;
+
+    /**
+     * Removes data associated with old timestamps.
+     *
+     * @param from Start of hashes range to process. Inclusive.

Review comment:
       What are these "hashes ranges"? How would one obtain a hash range?

##########
File path: modules/storage-api/src/main/java/org/apache/ignite/internal/storage/index/SortedIndexDescriptor.java
##########
@@ -166,7 +166,7 @@ public SortedIndexDescriptor(String name, TableView tableConfig) {
     /**
      * Creates a {@link SchemaDescriptor} from a list of index key columns.
      */
-    private static SchemaDescriptor createSchemaDescriptor(List<ColumnView> indexKeyColumnViews) {
+    public static SchemaDescriptor createSchemaDescriptor(List<ColumnView> indexKeyColumnViews) {

Review comment:
       Looks like this change is not needed

##########
File path: modules/storage-api/src/main/java/org/apache/ignite/internal/storage/index/SortedIndexMvStorage.java
##########
@@ -0,0 +1,95 @@
+/*
+ * 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.index;
+
+import java.util.function.IntPredicate;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.internal.util.Cursor;
+import org.intellij.lang.annotations.MagicConstant;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Storage for a sorted index.
+ * POC version, that represents a combination between a replicated TX-aware MV storage and physical MV storage. Real future implementation
+ * will be defined later. Things to notice here: TX-aware implementation should have a projection bitset instead of full row reading.
+ * Physical storage API will be enriched with append/remove methods, like in reference implementation.
+ */
+public interface SortedIndexMvStorage {
+    /** Exclude lower bound. */
+    int GREATER = 0;

Review comment:
       Again, IDEA tells me that both `GREATER` and `LESS` constants are never used, which means they are not covered by tests

##########
File path: modules/storage-api/src/main/java/org/apache/ignite/internal/storage/index/SortedIndexMvStorage.java
##########
@@ -0,0 +1,95 @@
+/*
+ * 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.index;
+
+import java.util.function.IntPredicate;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.internal.util.Cursor;
+import org.intellij.lang.annotations.MagicConstant;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Storage for a sorted index.
+ * POC version, that represents a combination between a replicated TX-aware MV storage and physical MV storage. Real future implementation
+ * will be defined later. Things to notice here: TX-aware implementation should have a projection bitset instead of full row reading.
+ * Physical storage API will be enriched with append/remove methods, like in reference implementation.
+ */
+public interface SortedIndexMvStorage {
+    /** Exclude lower bound. */
+    int GREATER = 0;
+
+    /** Include lower bound. */
+    int GREATER_OR_EQUAL = 1;
+
+    /** Exclude upper bound. */
+    int LESS = 0;
+
+    /** Include upper bound. */
+    int LESS_OR_EQUAL = 1 << 1;
+
+    /** Forward scan. */
+    int FORWARD = 0;
+
+    /** Backwards scan. */
+    int BACKWARDS = 1 << 2;
+
+    /**
+     * The sole purpose of this class is to avoid massive refactoring while changing the original IndexRow.
+     */
+    interface IndexRowEx {
+        /**
+         * Key-only binary row if index-only scan is supported, full binary row otherwise.
+         */
+        BinaryRow row();
+
+        /**
+         * Returns indexed column value.
+         *
+         * @param idx PK column index.
+         * @return Indexed column value.
+         */
+        Object value(int idx);
+    }
+
+    boolean supportsBackwardsScan();
+
+    boolean supportsIndexOnlyScan();
+
+    /**
+     * Returns a range of index values between the lower bound and the upper bound, consistent with the passed timestamp.
+     *
+     * @param lowerBound Lower bound. Exclusivity is controlled by a {@link #GREATER_OR_EQUAL} or {@link #GREATER} flag.
+     *      {@code null} means unbounded.
+     * @param upperBound Upper bound. Exclusivity is controlled by a {@link #LESS} or {@link #LESS_OR_EQUAL} flag.
+     *      {@code null} means unbounded.
+     * @param flags Control flags. {@link #GREATER} | {@link #LESS} | {@link #FORWARD} by default. Other available values
+     *      are {@link #GREATER_OR_EQUAL}, {@link #LESS_OR_EQUAL} and {@link #BACKWARDS}.
+     * @param timestamp Timestamp value for consistent multiversioned index scan.

Review comment:
       ```suggestion
        * @param timestamp Timestamp value for consistent multi-versioned index scan.
   ```

##########
File path: modules/storage-api/src/main/java/org/apache/ignite/internal/storage/index/SortedIndexMvStorage.java
##########
@@ -0,0 +1,95 @@
+/*
+ * 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.index;
+
+import java.util.function.IntPredicate;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.internal.util.Cursor;
+import org.intellij.lang.annotations.MagicConstant;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Storage for a sorted index.
+ * POC version, that represents a combination between a replicated TX-aware MV storage and physical MV storage. Real future implementation
+ * will be defined later. Things to notice here: TX-aware implementation should have a projection bitset instead of full row reading.
+ * Physical storage API will be enriched with append/remove methods, like in reference implementation.
+ */
+public interface SortedIndexMvStorage {
+    /** Exclude lower bound. */
+    int GREATER = 0;
+
+    /** Include lower bound. */
+    int GREATER_OR_EQUAL = 1;
+
+    /** Exclude upper bound. */
+    int LESS = 0;
+
+    /** Include upper bound. */
+    int LESS_OR_EQUAL = 1 << 1;
+
+    /** Forward scan. */
+    int FORWARD = 0;
+
+    /** Backwards scan. */
+    int BACKWARDS = 1 << 2;
+
+    /**
+     * The sole purpose of this class is to avoid massive refactoring while changing the original IndexRow.
+     */
+    interface IndexRowEx {
+        /**
+         * Key-only binary row if index-only scan is supported, full binary row otherwise.
+         */
+        BinaryRow row();
+
+        /**
+         * Returns indexed column value.
+         *
+         * @param idx PK column index.
+         * @return Indexed column value.
+         */
+        Object value(int idx);
+    }
+
+    boolean supportsBackwardsScan();
+
+    boolean supportsIndexOnlyScan();
+
+    /**
+     * Returns a range of index values between the lower bound and the upper bound, consistent with the passed timestamp.

Review comment:
       What does "consistent" mean here?

##########
File path: modules/storage-api/src/test/java/org/apache/ignite/internal/storage/BaseMvStoragesTest.java
##########
@@ -0,0 +1,180 @@
+/*
+ * 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;
+
+import java.util.Locale;
+import java.util.Objects;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.schema.Column;
+import org.apache.ignite.internal.schema.NativeTypes;
+import org.apache.ignite.internal.schema.SchemaDescriptor;
+import org.apache.ignite.internal.schema.marshaller.KvMarshaller;
+import org.apache.ignite.internal.schema.marshaller.MarshallerException;
+import org.apache.ignite.internal.schema.marshaller.MarshallerFactory;
+import org.apache.ignite.internal.schema.marshaller.reflection.ReflectionMarshallerFactory;
+import org.apache.ignite.internal.schema.row.Row;
+import org.apache.ignite.lang.IgniteException;
+import org.jetbrains.annotations.Nullable;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+
+/**
+ * Base test for MV storages, contains pojo classes, their descriptor and a marshaller instance.
+ */
+public class BaseMvStoragesTest {
+    /** Default reflection marshaller factory. */
+    protected static MarshallerFactory marshallerFactory;
+
+    /** Schema descriptor for tests. */
+    protected static SchemaDescriptor schemaDescriptor;
+
+    /** Key-value marshaller for tests. */
+    protected static KvMarshaller<TestKey, TestValue> kvMarshaller;
+
+    @BeforeAll
+    static void beforeAll() {
+        marshallerFactory = new ReflectionMarshallerFactory();
+
+        schemaDescriptor = new SchemaDescriptor(1, new Column[]{
+                new Column("intKey".toUpperCase(Locale.ROOT), NativeTypes.INT32, false),
+                new Column("strKey".toUpperCase(Locale.ROOT), NativeTypes.STRING, false),
+        }, new Column[]{
+                new Column("intVal".toUpperCase(Locale.ROOT), NativeTypes.INT32, false),
+                new Column("strVal".toUpperCase(Locale.ROOT), NativeTypes.STRING, false),
+        });
+
+        kvMarshaller = marshallerFactory.create(schemaDescriptor, TestKey.class, TestValue.class);
+    }
+
+    @AfterAll
+    static void afterAll() {
+        kvMarshaller = null;
+        schemaDescriptor = null;
+        marshallerFactory = null;
+    }
+
+    protected BinaryRow binaryKey(TestKey key) {

Review comment:
       Looks like all these methods can be `static`

##########
File path: modules/storage-api/src/test/java/org/apache/ignite/internal/storage/basic/TestMvPartitionStorage.java
##########
@@ -0,0 +1,172 @@
+/*
+ * 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.basic;
+
+import java.nio.ByteBuffer;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.Predicate;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.storage.MvPartitionStorage;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.internal.util.Cursor;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Test implementation of MV partition storage.
+ */
+public class TestMvPartitionStorage implements MvPartitionStorage {
+    private static final VersionChain NULL = new VersionChain(null, null, null, null);
+
+    private final ConcurrentHashMap<ByteBuffer, VersionChain> map = new ConcurrentHashMap<>();
+
+    private final List<TestSortedIndexMvStorage> indexes;
+
+    public TestMvPartitionStorage(List<TestSortedIndexMvStorage> indexes) {
+        this.indexes = indexes;
+    }
+
+    private static class VersionChain {
+        final BinaryRow row;
+        final Timestamp begin;
+        final UUID txId;
+        final VersionChain next;
+
+        VersionChain(BinaryRow row, Timestamp begin, UUID txId, VersionChain next) {
+            this.row = row;
+            this.begin = begin;
+            this.txId = txId;
+            this.next = next;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public void addWrite(BinaryRow row, UUID txId) throws TxIdMismatchException {
+        //TODO Make it idempotent?
+        map.compute(row.keySlice(), (keyBuf, versionChain) -> {
+            if (versionChain != null && versionChain.begin == null && !txId.equals(versionChain.txId)) {
+                throw new TxIdMismatchException();
+            }
+
+            return new VersionChain(row, null, txId, versionChain);
+        });
+
+        for (TestSortedIndexMvStorage index : indexes) {
+            index.append(row);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public void abortWrite(BinaryRow key) {
+        map.merge(key.keySlice(), NULL, (versionChain, ignored) -> {
+            assert versionChain != null;
+            assert versionChain.begin == null && versionChain.txId != null;
+
+            BinaryRow aborted = versionChain.row;
+
+            for (TestSortedIndexMvStorage index : indexes) {
+                abortWrite(versionChain.next, aborted, index);
+            }
+
+            return versionChain.next;
+        });
+    }
+
+    private void abortWrite(VersionChain head, BinaryRow aborted, TestSortedIndexMvStorage index) {
+        for (VersionChain cur = head; cur != null; cur = cur.next) {
+            if (index.matches(aborted, cur.row)) {
+                return;
+            }
+        }
+
+        index.remove(aborted);
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public void commitWrite(BinaryRow key, Timestamp timestamp) {
+        map.compute(key.keySlice(), (keyBuf, versionChain) -> {
+            assert versionChain != null;
+            assert versionChain.begin == null && versionChain.txId != null;
+
+            return new VersionChain(versionChain.row, timestamp, null, versionChain.next);
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    @Nullable
+    public BinaryRow read(BinaryRow key, @Nullable Timestamp timestamp) {
+        VersionChain versionChain = map.get(key.keySlice());
+
+        return read(versionChain, timestamp);
+    }
+
+    @Nullable
+    private BinaryRow read(VersionChain versionChain, @Nullable Timestamp timestamp) {
+        if (versionChain == null) {
+            return null;
+        }
+
+        if (timestamp == null) {
+            return versionChain.row;
+        }
+
+        VersionChain cur = versionChain;
+
+        if (cur.begin == null) {
+            cur = cur.next;
+        }
+
+        while (cur != null) {
+            if (timestamp.compareTo(cur.begin) >= 0) {
+                BinaryRow row = cur.row;
+
+                return row.hasValue() ? row : null;
+            }
+
+            cur = cur.next;
+        }
+
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public Cursor<BinaryRow> scan(Predicate<BinaryRow> keyFilter, @Nullable Timestamp timestamp) {
+        Iterator<BinaryRow> iterator = map.values().stream()
+                .map(versionChain -> read(versionChain, timestamp))
+                .filter(Objects::nonNull)
+                .filter(keyFilter::test)

Review comment:
       ```suggestion
                   .filter(keyFilter)
   ```

##########
File path: modules/storage-api/src/test/java/org/apache/ignite/internal/storage/basic/TestMvPartitionStorage.java
##########
@@ -0,0 +1,172 @@
+/*
+ * 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.basic;
+
+import java.nio.ByteBuffer;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.Predicate;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.storage.MvPartitionStorage;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.internal.util.Cursor;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Test implementation of MV partition storage.
+ */
+public class TestMvPartitionStorage implements MvPartitionStorage {
+    private static final VersionChain NULL = new VersionChain(null, null, null, null);
+
+    private final ConcurrentHashMap<ByteBuffer, VersionChain> map = new ConcurrentHashMap<>();
+
+    private final List<TestSortedIndexMvStorage> indexes;
+
+    public TestMvPartitionStorage(List<TestSortedIndexMvStorage> indexes) {
+        this.indexes = indexes;
+    }
+
+    private static class VersionChain {
+        final BinaryRow row;
+        final Timestamp begin;
+        final UUID txId;
+        final VersionChain next;
+
+        VersionChain(BinaryRow row, Timestamp begin, UUID txId, VersionChain next) {
+            this.row = row;
+            this.begin = begin;
+            this.txId = txId;
+            this.next = next;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public void addWrite(BinaryRow row, UUID txId) throws TxIdMismatchException {
+        //TODO Make it idempotent?
+        map.compute(row.keySlice(), (keyBuf, versionChain) -> {
+            if (versionChain != null && versionChain.begin == null && !txId.equals(versionChain.txId)) {
+                throw new TxIdMismatchException();
+            }
+
+            return new VersionChain(row, null, txId, versionChain);
+        });
+
+        for (TestSortedIndexMvStorage index : indexes) {
+            index.append(row);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public void abortWrite(BinaryRow key) {
+        map.merge(key.keySlice(), NULL, (versionChain, ignored) -> {
+            assert versionChain != null;
+            assert versionChain.begin == null && versionChain.txId != null;
+
+            BinaryRow aborted = versionChain.row;
+
+            for (TestSortedIndexMvStorage index : indexes) {
+                abortWrite(versionChain.next, aborted, index);
+            }
+
+            return versionChain.next;
+        });
+    }
+
+    private void abortWrite(VersionChain head, BinaryRow aborted, TestSortedIndexMvStorage index) {
+        for (VersionChain cur = head; cur != null; cur = cur.next) {
+            if (index.matches(aborted, cur.row)) {
+                return;
+            }
+        }
+
+        index.remove(aborted);
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public void commitWrite(BinaryRow key, Timestamp timestamp) {
+        map.compute(key.keySlice(), (keyBuf, versionChain) -> {
+            assert versionChain != null;

Review comment:
       Should we actually throw an exception here? Should we also specify it in the interface?

##########
File path: modules/storage-api/src/test/java/org/apache/ignite/internal/storage/basic/TestSortedIndexMvStorage.java
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.basic;
+
+import java.util.Iterator;
+import java.util.Map;
+import java.util.NavigableSet;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.function.IntPredicate;
+import java.util.function.ToIntFunction;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.ColumnView;
+import org.apache.ignite.configuration.schemas.table.IndexColumnView;
+import org.apache.ignite.configuration.schemas.table.SortedIndexView;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.schema.NativeType;
+import org.apache.ignite.internal.schema.SchemaDescriptor;
+import org.apache.ignite.internal.schema.configuration.SchemaConfigurationConverter;
+import org.apache.ignite.internal.schema.configuration.SchemaDescriptorConverter;
+import org.apache.ignite.internal.schema.row.Row;
+import org.apache.ignite.internal.storage.index.IndexRowPrefix;
+import org.apache.ignite.internal.storage.index.PrefixComparator;
+import org.apache.ignite.internal.storage.index.SortedIndexMvStorage;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.internal.util.Cursor;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Test implementation of MV sorted index storage.
+ */
+public class TestSortedIndexMvStorage implements SortedIndexMvStorage {
+    private final TableView tableCfg;

Review comment:
       this field is not used

##########
File path: modules/storage-api/src/test/java/org/apache/ignite/internal/storage/basic/TestSortedIndexMvStorage.java
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.basic;
+
+import java.util.Iterator;
+import java.util.Map;
+import java.util.NavigableSet;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.function.IntPredicate;
+import java.util.function.ToIntFunction;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.ColumnView;
+import org.apache.ignite.configuration.schemas.table.IndexColumnView;
+import org.apache.ignite.configuration.schemas.table.SortedIndexView;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.schema.NativeType;
+import org.apache.ignite.internal.schema.SchemaDescriptor;
+import org.apache.ignite.internal.schema.configuration.SchemaConfigurationConverter;
+import org.apache.ignite.internal.schema.configuration.SchemaDescriptorConverter;
+import org.apache.ignite.internal.schema.row.Row;
+import org.apache.ignite.internal.storage.index.IndexRowPrefix;
+import org.apache.ignite.internal.storage.index.PrefixComparator;
+import org.apache.ignite.internal.storage.index.SortedIndexMvStorage;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.internal.util.Cursor;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Test implementation of MV sorted index storage.
+ */
+public class TestSortedIndexMvStorage implements SortedIndexMvStorage {
+    private final TableView tableCfg;
+
+    private final NavigableSet<BinaryRow> index;
+
+    private final SchemaDescriptor descriptor;
+    private final Map<Integer, TestMvPartitionStorage> pk;
+
+    private final int partitions;
+
+    private final IndexColumnView[] indexColumns;
+
+    private final int[] columnIndexes;
+
+    private final NativeType[] nativeTypes;
+
+    protected TestSortedIndexMvStorage(

Review comment:
       why `protected`?

##########
File path: modules/storage-api/src/main/java/org/apache/ignite/internal/storage/MvPartitionStorage.java
##########
@@ -0,0 +1,96 @@
+/*
+ * 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;
+
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Predicate;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.internal.util.Cursor;
+import org.apache.ignite.lang.IgniteException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Multi-versioned partition storage.
+ * POC version, that represents a combination between a replicated TX-aware MV storage and physical MV storage. Their API are very similar,
+ * althouh there are very important differences that will be addressed in the future.
+ */
+public interface MvPartitionStorage {
+    /**
+     * Exception class that describes the situation where two independant transactions attempting to write values for the same key.
+     */
+    class TxIdMismatchException extends IgniteException {
+    }
+
+    /**
+     * Reads the value from the storage as it was at the given timestamp.
+     *
+     * @param key Key.
+     * @param timestamp Timestamp.
+     * @return Binary row that corresponds to the key or {@code null} if value is not found.
+     */
+    @Nullable
+    BinaryRow read(BinaryRow key, @Nullable Timestamp timestamp);
+
+    /**
+     * Creates uncommited version, assigned to the passed transaction id..
+     *
+     * @param row Binary row to update. Key only row means value removal.
+     * @param txId Transaction id.
+     * @throws TxIdMismatchException If there's another pending update associated with different transaction id.
+     * @throws StorageException If failed to write data to the storage.
+     */
+    void addWrite(BinaryRow row, UUID txId) throws TxIdMismatchException, StorageException;
+
+    /**
+     * Aborts a pending update of the ongoing uncommited transaction. Invoked during rollback.
+     *
+     * @param key Key.
+     * @throws StorageException If failed to write data to the storage.
+     */
+    void abortWrite(BinaryRow key) throws StorageException;
+
+    /**
+     * Commits a pending update of the ongoing transaction. Invoked during commit. Commited value will be versioned by the given timestamp.

Review comment:
       ```suggestion
        * Commits a pending update of the ongoing transaction. Invoked during commit. Committed value will be versioned by the given timestamp.
   ```

##########
File path: modules/storage-api/src/main/java/org/apache/ignite/internal/storage/index/PrefixComparator.java
##########
@@ -76,7 +74,18 @@ int compare(BinaryRow binaryRow) {
      * Compares a particular column of a {@code row} with the given value.
      */
     private static int compare(Column column, Row row, @Nullable Object value) {
-        boolean nullRow = row.hasNullValue(column.schemaIndex(), column.type().spec());
+        int schemaIndex = column.schemaIndex();
+
+        NativeTypeSpec typeSpec = column.type().spec();
+
+        return compareColumns(row, schemaIndex, typeSpec, value);
+    }
+
+    /**
+     * Compares a particular column of a {@code row} with the given value.
+     */
+    public static int compareColumns(Row row, int schemaIndex, NativeTypeSpec typeSpec, Object value) {

Review comment:
       This is also a huge abstraction leak, can we get rid of it somehow?

##########
File path: modules/storage-api/src/main/java/org/apache/ignite/internal/storage/MvPartitionStorage.java
##########
@@ -0,0 +1,96 @@
+/*
+ * 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;
+
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Predicate;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.internal.util.Cursor;
+import org.apache.ignite.lang.IgniteException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Multi-versioned partition storage.
+ * POC version, that represents a combination between a replicated TX-aware MV storage and physical MV storage. Their API are very similar,
+ * althouh there are very important differences that will be addressed in the future.
+ */
+public interface MvPartitionStorage {
+    /**
+     * Exception class that describes the situation where two independant transactions attempting to write values for the same key.
+     */
+    class TxIdMismatchException extends IgniteException {
+    }
+
+    /**
+     * Reads the value from the storage as it was at the given timestamp.
+     *
+     * @param key Key.
+     * @param timestamp Timestamp.
+     * @return Binary row that corresponds to the key or {@code null} if value is not found.
+     */
+    @Nullable
+    BinaryRow read(BinaryRow key, @Nullable Timestamp timestamp);
+
+    /**
+     * Creates uncommited version, assigned to the passed transaction id..
+     *
+     * @param row Binary row to update. Key only row means value removal.
+     * @param txId Transaction id.
+     * @throws TxIdMismatchException If there's another pending update associated with different transaction id.
+     * @throws StorageException If failed to write data to the storage.
+     */
+    void addWrite(BinaryRow row, UUID txId) throws TxIdMismatchException, StorageException;
+
+    /**
+     * Aborts a pending update of the ongoing uncommited transaction. Invoked during rollback.

Review comment:
       ```suggestion
        * Aborts a pending update of the ongoing uncommitted transaction. Invoked during rollback.
   ```

##########
File path: modules/storage-api/src/main/java/org/apache/ignite/internal/storage/index/PrefixComparator.java
##########
@@ -76,7 +74,18 @@ int compare(BinaryRow binaryRow) {
      * Compares a particular column of a {@code row} with the given value.
      */
     private static int compare(Column column, Row row, @Nullable Object value) {
-        boolean nullRow = row.hasNullValue(column.schemaIndex(), column.type().spec());
+        int schemaIndex = column.schemaIndex();
+
+        NativeTypeSpec typeSpec = column.type().spec();
+
+        return compareColumns(row, schemaIndex, typeSpec, value);
+    }
+
+    /**
+     * Compares a particular column of a {@code row} with the given value.
+     */
+    public static int compareColumns(Row row, int schemaIndex, NativeTypeSpec typeSpec, Object value) {

Review comment:
       ```suggestion
       public static int compareColumns(Row row, int schemaIndex, NativeTypeSpec typeSpec, @Nullable Object value) {
   ```

##########
File path: modules/storage-api/src/main/java/org/apache/ignite/internal/storage/index/SortedIndexMvStorage.java
##########
@@ -0,0 +1,95 @@
+/*
+ * 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.index;
+
+import java.util.function.IntPredicate;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.internal.util.Cursor;
+import org.intellij.lang.annotations.MagicConstant;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Storage for a sorted index.
+ * POC version, that represents a combination between a replicated TX-aware MV storage and physical MV storage. Real future implementation
+ * will be defined later. Things to notice here: TX-aware implementation should have a projection bitset instead of full row reading.
+ * Physical storage API will be enriched with append/remove methods, like in reference implementation.
+ */
+public interface SortedIndexMvStorage {
+    /** Exclude lower bound. */
+    int GREATER = 0;
+
+    /** Include lower bound. */
+    int GREATER_OR_EQUAL = 1;
+
+    /** Exclude upper bound. */
+    int LESS = 0;
+
+    /** Include upper bound. */
+    int LESS_OR_EQUAL = 1 << 1;
+
+    /** Forward scan. */
+    int FORWARD = 0;
+
+    /** Backwards scan. */
+    int BACKWARDS = 1 << 2;
+
+    /**
+     * The sole purpose of this class is to avoid massive refactoring while changing the original IndexRow.
+     */
+    interface IndexRowEx {
+        /**
+         * Key-only binary row if index-only scan is supported, full binary row otherwise.
+         */
+        BinaryRow row();
+
+        /**
+         * Returns indexed column value.
+         *
+         * @param idx PK column index.
+         * @return Indexed column value.
+         */
+        Object value(int idx);
+    }
+
+    boolean supportsBackwardsScan();

Review comment:
       why are these methods needed?

##########
File path: modules/storage-api/src/main/java/org/apache/ignite/internal/storage/MvPartitionStorage.java
##########
@@ -0,0 +1,96 @@
+/*
+ * 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;
+
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Predicate;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.internal.util.Cursor;
+import org.apache.ignite.lang.IgniteException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Multi-versioned partition storage.
+ * POC version, that represents a combination between a replicated TX-aware MV storage and physical MV storage. Their API are very similar,
+ * althouh there are very important differences that will be addressed in the future.
+ */
+public interface MvPartitionStorage {
+    /**
+     * Exception class that describes the situation where two independant transactions attempting to write values for the same key.
+     */
+    class TxIdMismatchException extends IgniteException {
+    }
+
+    /**
+     * Reads the value from the storage as it was at the given timestamp.
+     *
+     * @param key Key.
+     * @param timestamp Timestamp.
+     * @return Binary row that corresponds to the key or {@code null} if value is not found.
+     */
+    @Nullable
+    BinaryRow read(BinaryRow key, @Nullable Timestamp timestamp);
+
+    /**
+     * Creates uncommited version, assigned to the passed transaction id..
+     *
+     * @param row Binary row to update. Key only row means value removal.
+     * @param txId Transaction id.
+     * @throws TxIdMismatchException If there's another pending update associated with different transaction id.
+     * @throws StorageException If failed to write data to the storage.
+     */
+    void addWrite(BinaryRow row, UUID txId) throws TxIdMismatchException, StorageException;
+
+    /**
+     * Aborts a pending update of the ongoing uncommited transaction. Invoked during rollback.
+     *
+     * @param key Key.
+     * @throws StorageException If failed to write data to the storage.
+     */
+    void abortWrite(BinaryRow key) throws StorageException;
+
+    /**
+     * Commits a pending update of the ongoing transaction. Invoked during commit. Commited value will be versioned by the given timestamp.
+     *
+     * @param key Key.
+     * @param timestamp Timestamp to associate with commited value.
+     * @throws StorageException If failed to write data to the storage.
+     */
+    void commitWrite(BinaryRow key, Timestamp timestamp) throws StorageException;
+
+    /**
+     * Removes data associated with old timestamps.
+     *
+     * @param from Start of hashes range to process. Inclusive.
+     * @param to End of hashes range to process. Inclusive.
+     * @param timestamp Timestamp to remove all the data with a lesser timestamp.

Review comment:
       ```suggestion
        * @param timestamp Timestamp to remove all data with a smaller timestamp.
   ```

##########
File path: modules/storage-api/src/main/java/org/apache/ignite/internal/storage/MvPartitionStorage.java
##########
@@ -0,0 +1,96 @@
+/*
+ * 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;
+
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Predicate;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.internal.util.Cursor;
+import org.apache.ignite.lang.IgniteException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Multi-versioned partition storage.
+ * POC version, that represents a combination between a replicated TX-aware MV storage and physical MV storage. Their API are very similar,
+ * althouh there are very important differences that will be addressed in the future.
+ */
+public interface MvPartitionStorage {
+    /**
+     * Exception class that describes the situation where two independant transactions attempting to write values for the same key.
+     */
+    class TxIdMismatchException extends IgniteException {
+    }
+
+    /**
+     * Reads the value from the storage as it was at the given timestamp.
+     *
+     * @param key Key.
+     * @param timestamp Timestamp.
+     * @return Binary row that corresponds to the key or {@code null} if value is not found.
+     */
+    @Nullable
+    BinaryRow read(BinaryRow key, @Nullable Timestamp timestamp);
+
+    /**
+     * Creates uncommited version, assigned to the passed transaction id..
+     *
+     * @param row Binary row to update. Key only row means value removal.
+     * @param txId Transaction id.
+     * @throws TxIdMismatchException If there's another pending update associated with different transaction id.
+     * @throws StorageException If failed to write data to the storage.
+     */
+    void addWrite(BinaryRow row, UUID txId) throws TxIdMismatchException, StorageException;
+
+    /**
+     * Aborts a pending update of the ongoing uncommited transaction. Invoked during rollback.
+     *
+     * @param key Key.
+     * @throws StorageException If failed to write data to the storage.
+     */
+    void abortWrite(BinaryRow key) throws StorageException;
+
+    /**
+     * Commits a pending update of the ongoing transaction. Invoked during commit. Commited value will be versioned by the given timestamp.
+     *
+     * @param key Key.
+     * @param timestamp Timestamp to associate with commited value.

Review comment:
       ```suggestion
        * @param timestamp Timestamp to associate with committed value.
   ```

##########
File path: modules/storage-api/src/test/java/org/apache/ignite/internal/storage/BaseMvStoragesTest.java
##########
@@ -0,0 +1,180 @@
+/*
+ * 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;
+
+import java.util.Locale;
+import java.util.Objects;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.schema.Column;
+import org.apache.ignite.internal.schema.NativeTypes;
+import org.apache.ignite.internal.schema.SchemaDescriptor;
+import org.apache.ignite.internal.schema.marshaller.KvMarshaller;
+import org.apache.ignite.internal.schema.marshaller.MarshallerException;
+import org.apache.ignite.internal.schema.marshaller.MarshallerFactory;
+import org.apache.ignite.internal.schema.marshaller.reflection.ReflectionMarshallerFactory;
+import org.apache.ignite.internal.schema.row.Row;
+import org.apache.ignite.lang.IgniteException;
+import org.jetbrains.annotations.Nullable;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+
+/**
+ * Base test for MV storages, contains pojo classes, their descriptor and a marshaller instance.
+ */
+public class BaseMvStoragesTest {
+    /** Default reflection marshaller factory. */
+    protected static MarshallerFactory marshallerFactory;
+
+    /** Schema descriptor for tests. */
+    protected static SchemaDescriptor schemaDescriptor;
+
+    /** Key-value marshaller for tests. */
+    protected static KvMarshaller<TestKey, TestValue> kvMarshaller;
+
+    @BeforeAll
+    static void beforeAll() {
+        marshallerFactory = new ReflectionMarshallerFactory();
+
+        schemaDescriptor = new SchemaDescriptor(1, new Column[]{
+                new Column("intKey".toUpperCase(Locale.ROOT), NativeTypes.INT32, false),
+                new Column("strKey".toUpperCase(Locale.ROOT), NativeTypes.STRING, false),
+        }, new Column[]{
+                new Column("intVal".toUpperCase(Locale.ROOT), NativeTypes.INT32, false),
+                new Column("strVal".toUpperCase(Locale.ROOT), NativeTypes.STRING, false),
+        });
+
+        kvMarshaller = marshallerFactory.create(schemaDescriptor, TestKey.class, TestValue.class);
+    }
+
+    @AfterAll
+    static void afterAll() {
+        kvMarshaller = null;
+        schemaDescriptor = null;
+        marshallerFactory = null;
+    }
+
+    protected BinaryRow binaryKey(TestKey key) {
+        try {
+            return kvMarshaller.marshal(key);
+        } catch (MarshallerException e) {
+            throw new IgniteException(e);
+        }
+    }
+
+    protected BinaryRow binaryRow(TestKey key, TestValue value) {
+        try {
+            return kvMarshaller.marshal(key, value);
+        } catch (MarshallerException e) {
+            throw new IgniteException(e);
+        }
+    }
+
+    @Nullable
+    protected TestKey key(BinaryRow binaryRow) {
+        try {
+            return kvMarshaller.unmarshalKey(new Row(schemaDescriptor, binaryRow));
+        } catch (MarshallerException e) {
+            throw new IgniteException(e);
+        }
+    }
+
+    @Nullable
+    protected TestValue value(BinaryRow binaryRow) {
+        try {
+            return kvMarshaller.unmarshalValue(new Row(schemaDescriptor, binaryRow));
+        } catch (MarshallerException e) {
+            throw new IgniteException(e);
+        }
+    }
+
+    /**
+     * Test pojo key.
+     */
+    protected static class TestKey {
+        public int intKey;
+
+        public String strKey;
+
+        public TestKey() {
+        }
+
+        public TestKey(int intKey, String strKey) {
+            this.intKey = intKey;
+            this.strKey = strKey;
+        }
+
+        @Override
+        public boolean equals(Object o) {
+            if (this == o) {
+                return true;
+            }
+            if (o == null || getClass() != o.getClass()) {
+                return false;
+            }
+            TestKey testKey = (TestKey) o;
+            return intKey == testKey.intKey && Objects.equals(strKey, testKey.strKey);
+        }
+
+        @Override
+        public int hashCode() {
+            return Objects.hash(intKey, strKey);
+        }
+    }
+
+    /**
+     * Test pojo value.
+     */
+    protected static class TestValue implements Comparable<TestValue> {
+        public Integer intVal;
+
+        public String strVal;
+
+        public TestValue() {
+        }
+
+        public TestValue(Integer intVal, String strVal) {
+            this.intVal = intVal;
+            this.strVal = strVal;
+        }
+
+        @Override
+        public int compareTo(TestValue o) {
+            //TODO Compare nuppable values.

Review comment:
       ```suggestion
               //TODO Compare nullable values.
   ```

##########
File path: modules/storage-api/src/test/java/org/apache/ignite/internal/storage/BaseMvStoragesTest.java
##########
@@ -0,0 +1,180 @@
+/*
+ * 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;
+
+import java.util.Locale;
+import java.util.Objects;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.schema.Column;
+import org.apache.ignite.internal.schema.NativeTypes;
+import org.apache.ignite.internal.schema.SchemaDescriptor;
+import org.apache.ignite.internal.schema.marshaller.KvMarshaller;
+import org.apache.ignite.internal.schema.marshaller.MarshallerException;
+import org.apache.ignite.internal.schema.marshaller.MarshallerFactory;
+import org.apache.ignite.internal.schema.marshaller.reflection.ReflectionMarshallerFactory;
+import org.apache.ignite.internal.schema.row.Row;
+import org.apache.ignite.lang.IgniteException;
+import org.jetbrains.annotations.Nullable;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+
+/**
+ * Base test for MV storages, contains pojo classes, their descriptor and a marshaller instance.
+ */
+public class BaseMvStoragesTest {
+    /** Default reflection marshaller factory. */
+    protected static MarshallerFactory marshallerFactory;
+
+    /** Schema descriptor for tests. */
+    protected static SchemaDescriptor schemaDescriptor;
+
+    /** Key-value marshaller for tests. */
+    protected static KvMarshaller<TestKey, TestValue> kvMarshaller;
+
+    @BeforeAll
+    static void beforeAll() {
+        marshallerFactory = new ReflectionMarshallerFactory();
+
+        schemaDescriptor = new SchemaDescriptor(1, new Column[]{
+                new Column("intKey".toUpperCase(Locale.ROOT), NativeTypes.INT32, false),
+                new Column("strKey".toUpperCase(Locale.ROOT), NativeTypes.STRING, false),
+        }, new Column[]{
+                new Column("intVal".toUpperCase(Locale.ROOT), NativeTypes.INT32, false),
+                new Column("strVal".toUpperCase(Locale.ROOT), NativeTypes.STRING, false),
+        });
+
+        kvMarshaller = marshallerFactory.create(schemaDescriptor, TestKey.class, TestValue.class);
+    }
+
+    @AfterAll
+    static void afterAll() {
+        kvMarshaller = null;
+        schemaDescriptor = null;
+        marshallerFactory = null;
+    }
+
+    protected BinaryRow binaryKey(TestKey key) {
+        try {
+            return kvMarshaller.marshal(key);
+        } catch (MarshallerException e) {
+            throw new IgniteException(e);
+        }
+    }
+
+    protected BinaryRow binaryRow(TestKey key, TestValue value) {
+        try {
+            return kvMarshaller.marshal(key, value);
+        } catch (MarshallerException e) {
+            throw new IgniteException(e);
+        }
+    }
+
+    @Nullable
+    protected TestKey key(BinaryRow binaryRow) {
+        try {
+            return kvMarshaller.unmarshalKey(new Row(schemaDescriptor, binaryRow));
+        } catch (MarshallerException e) {
+            throw new IgniteException(e);
+        }
+    }
+
+    @Nullable
+    protected TestValue value(BinaryRow binaryRow) {
+        try {
+            return kvMarshaller.unmarshalValue(new Row(schemaDescriptor, binaryRow));
+        } catch (MarshallerException e) {
+            throw new IgniteException(e);
+        }
+    }
+
+    /**
+     * Test pojo key.
+     */
+    protected static class TestKey {
+        public int intKey;
+
+        public String strKey;
+
+        public TestKey() {
+        }
+
+        public TestKey(int intKey, String strKey) {
+            this.intKey = intKey;
+            this.strKey = strKey;
+        }
+
+        @Override
+        public boolean equals(Object o) {
+            if (this == o) {
+                return true;
+            }
+            if (o == null || getClass() != o.getClass()) {
+                return false;
+            }
+            TestKey testKey = (TestKey) o;
+            return intKey == testKey.intKey && Objects.equals(strKey, testKey.strKey);
+        }
+
+        @Override
+        public int hashCode() {
+            return Objects.hash(intKey, strKey);
+        }
+    }
+
+    /**
+     * Test pojo value.
+     */
+    protected static class TestValue implements Comparable<TestValue> {
+        public Integer intVal;
+
+        public String strVal;
+
+        public TestValue() {
+        }
+
+        public TestValue(Integer intVal, String strVal) {
+            this.intVal = intVal;
+            this.strVal = strVal;
+        }
+
+        @Override
+        public int compareTo(TestValue o) {
+            //TODO Compare nuppable values.

Review comment:
       do we need a ticket?

##########
File path: modules/storage-api/src/test/java/org/apache/ignite/internal/storage/basic/TestMvPartitionStorage.java
##########
@@ -0,0 +1,172 @@
+/*
+ * 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.basic;
+
+import java.nio.ByteBuffer;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.Predicate;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.storage.MvPartitionStorage;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.internal.util.Cursor;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Test implementation of MV partition storage.
+ */
+public class TestMvPartitionStorage implements MvPartitionStorage {
+    private static final VersionChain NULL = new VersionChain(null, null, null, null);
+
+    private final ConcurrentHashMap<ByteBuffer, VersionChain> map = new ConcurrentHashMap<>();
+
+    private final List<TestSortedIndexMvStorage> indexes;
+
+    public TestMvPartitionStorage(List<TestSortedIndexMvStorage> indexes) {
+        this.indexes = indexes;
+    }
+
+    private static class VersionChain {
+        final BinaryRow row;
+        final Timestamp begin;
+        final UUID txId;
+        final VersionChain next;
+
+        VersionChain(BinaryRow row, Timestamp begin, UUID txId, VersionChain next) {

Review comment:
       Looks like all these fields and parameters should be marked as `@Nullable`

##########
File path: modules/storage-api/src/test/java/org/apache/ignite/internal/storage/basic/TestSortedIndexMvStorage.java
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.basic;
+
+import java.util.Iterator;
+import java.util.Map;
+import java.util.NavigableSet;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.function.IntPredicate;
+import java.util.function.ToIntFunction;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.ColumnView;
+import org.apache.ignite.configuration.schemas.table.IndexColumnView;
+import org.apache.ignite.configuration.schemas.table.SortedIndexView;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.schema.NativeType;
+import org.apache.ignite.internal.schema.SchemaDescriptor;
+import org.apache.ignite.internal.schema.configuration.SchemaConfigurationConverter;
+import org.apache.ignite.internal.schema.configuration.SchemaDescriptorConverter;
+import org.apache.ignite.internal.schema.row.Row;
+import org.apache.ignite.internal.storage.index.IndexRowPrefix;
+import org.apache.ignite.internal.storage.index.PrefixComparator;
+import org.apache.ignite.internal.storage.index.SortedIndexMvStorage;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.internal.util.Cursor;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Test implementation of MV sorted index storage.
+ */
+public class TestSortedIndexMvStorage implements SortedIndexMvStorage {
+    private final TableView tableCfg;
+
+    private final NavigableSet<BinaryRow> index;
+
+    private final SchemaDescriptor descriptor;
+    private final Map<Integer, TestMvPartitionStorage> pk;

Review comment:
       missing linebreak

##########
File path: modules/storage-api/src/test/java/org/apache/ignite/internal/storage/basic/TestSortedIndexMvStorage.java
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.basic;
+
+import java.util.Iterator;
+import java.util.Map;
+import java.util.NavigableSet;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.function.IntPredicate;
+import java.util.function.ToIntFunction;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.ColumnView;
+import org.apache.ignite.configuration.schemas.table.IndexColumnView;
+import org.apache.ignite.configuration.schemas.table.SortedIndexView;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.schema.NativeType;
+import org.apache.ignite.internal.schema.SchemaDescriptor;
+import org.apache.ignite.internal.schema.configuration.SchemaConfigurationConverter;
+import org.apache.ignite.internal.schema.configuration.SchemaDescriptorConverter;
+import org.apache.ignite.internal.schema.row.Row;
+import org.apache.ignite.internal.storage.index.IndexRowPrefix;
+import org.apache.ignite.internal.storage.index.PrefixComparator;
+import org.apache.ignite.internal.storage.index.SortedIndexMvStorage;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.internal.util.Cursor;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Test implementation of MV sorted index storage.
+ */
+public class TestSortedIndexMvStorage implements SortedIndexMvStorage {
+    private final TableView tableCfg;
+
+    private final NavigableSet<BinaryRow> index;
+
+    private final SchemaDescriptor descriptor;
+    private final Map<Integer, TestMvPartitionStorage> pk;
+
+    private final int partitions;
+
+    private final IndexColumnView[] indexColumns;
+
+    private final int[] columnIndexes;
+
+    private final NativeType[] nativeTypes;
+
+    protected TestSortedIndexMvStorage(
+            String name,
+            TableView tableCfg,
+            SchemaDescriptor descriptor,
+            Map<Integer, TestMvPartitionStorage> pk
+    ) {
+        this.tableCfg = tableCfg;
+
+        this.descriptor = descriptor;
+
+        this.pk = pk;
+
+        partitions = tableCfg.partitions();
+
+        index = new ConcurrentSkipListSet<>((l, r) -> {
+            int cmp = compareColumns(l, r);
+
+            if (cmp != 0) {
+                return cmp;
+            }
+
+            return l.keySlice().compareTo(r.keySlice());
+        });
+
+        // Init columns.
+        NamedListView<? extends ColumnView> tblColumns = tableCfg.columns();
+
+        TableIndexView idxCfg = tableCfg.indices().get(name);
+
+        assert idxCfg instanceof SortedIndexView;
+
+        SortedIndexView sortedIdxCfg = (SortedIndexView) idxCfg;
+
+        NamedListView<? extends IndexColumnView> columns = sortedIdxCfg.columns();
+
+        int length = columns.size();
+
+        this.indexColumns = new IndexColumnView[length];
+        this.columnIndexes = new int[length];
+        this.nativeTypes = new NativeType[length];
+
+        for (int i = 0; i < length; i++) {
+            IndexColumnView idxColumn = columns.get(i);
+
+            indexColumns[i] = idxColumn;
+
+            int columnIndex = tblColumns.namedListKeys().indexOf(idxColumn.name());
+
+            columnIndexes[i] = columnIndex;
+
+            nativeTypes[i] = SchemaDescriptorConverter.convert(SchemaConfigurationConverter.convert(tblColumns.get(columnIndex).type()));
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public boolean supportsBackwardsScan() {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public boolean supportsIndexOnlyScan() {
+        return false;
+    }
+
+    private int compareColumns(BinaryRow l, BinaryRow r) {
+        Row leftRow = new Row(descriptor, l);

Review comment:
       Can we reuse `BinaryRowComparator` logic here? For example, there's a `binaryRowComparator` method there...

##########
File path: modules/storage-api/src/test/java/org/apache/ignite/internal/storage/basic/TestSortedIndexMvStorage.java
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.basic;
+
+import java.util.Iterator;
+import java.util.Map;
+import java.util.NavigableSet;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.function.IntPredicate;
+import java.util.function.ToIntFunction;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.ColumnView;
+import org.apache.ignite.configuration.schemas.table.IndexColumnView;
+import org.apache.ignite.configuration.schemas.table.SortedIndexView;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.schema.NativeType;
+import org.apache.ignite.internal.schema.SchemaDescriptor;
+import org.apache.ignite.internal.schema.configuration.SchemaConfigurationConverter;
+import org.apache.ignite.internal.schema.configuration.SchemaDescriptorConverter;
+import org.apache.ignite.internal.schema.row.Row;
+import org.apache.ignite.internal.storage.index.IndexRowPrefix;
+import org.apache.ignite.internal.storage.index.PrefixComparator;
+import org.apache.ignite.internal.storage.index.SortedIndexMvStorage;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.internal.util.Cursor;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Test implementation of MV sorted index storage.
+ */
+public class TestSortedIndexMvStorage implements SortedIndexMvStorage {
+    private final TableView tableCfg;
+
+    private final NavigableSet<BinaryRow> index;
+
+    private final SchemaDescriptor descriptor;
+    private final Map<Integer, TestMvPartitionStorage> pk;
+
+    private final int partitions;
+
+    private final IndexColumnView[] indexColumns;
+
+    private final int[] columnIndexes;
+
+    private final NativeType[] nativeTypes;
+
+    protected TestSortedIndexMvStorage(
+            String name,
+            TableView tableCfg,
+            SchemaDescriptor descriptor,
+            Map<Integer, TestMvPartitionStorage> pk
+    ) {
+        this.tableCfg = tableCfg;
+
+        this.descriptor = descriptor;
+
+        this.pk = pk;
+
+        partitions = tableCfg.partitions();
+
+        index = new ConcurrentSkipListSet<>((l, r) -> {
+            int cmp = compareColumns(l, r);
+
+            if (cmp != 0) {
+                return cmp;
+            }
+
+            return l.keySlice().compareTo(r.keySlice());
+        });
+
+        // Init columns.
+        NamedListView<? extends ColumnView> tblColumns = tableCfg.columns();
+
+        TableIndexView idxCfg = tableCfg.indices().get(name);
+
+        assert idxCfg instanceof SortedIndexView;
+
+        SortedIndexView sortedIdxCfg = (SortedIndexView) idxCfg;
+
+        NamedListView<? extends IndexColumnView> columns = sortedIdxCfg.columns();
+
+        int length = columns.size();
+
+        this.indexColumns = new IndexColumnView[length];
+        this.columnIndexes = new int[length];
+        this.nativeTypes = new NativeType[length];
+
+        for (int i = 0; i < length; i++) {
+            IndexColumnView idxColumn = columns.get(i);
+
+            indexColumns[i] = idxColumn;
+
+            int columnIndex = tblColumns.namedListKeys().indexOf(idxColumn.name());
+
+            columnIndexes[i] = columnIndex;
+
+            nativeTypes[i] = SchemaDescriptorConverter.convert(SchemaConfigurationConverter.convert(tblColumns.get(columnIndex).type()));
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public boolean supportsBackwardsScan() {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public boolean supportsIndexOnlyScan() {
+        return false;
+    }
+
+    private int compareColumns(BinaryRow l, BinaryRow r) {
+        Row leftRow = new Row(descriptor, l);
+        Row rightRow = new Row(descriptor, r);
+
+        for (int i = 0; i < indexColumns.length; i++) {
+            int columnIndex = columnIndexes[i];
+
+            int cmp = PrefixComparator.compareColumns(leftRow, columnIndex, nativeTypes[i].spec(), rightRow.value(columnIndex));
+
+            if (cmp != 0) {
+                return indexColumns[i].asc() ? cmp : -cmp;
+            }
+        }
+
+        return 0;
+    }
+
+    public void append(BinaryRow row) {
+        index.add(row);
+    }
+
+    public void remove(BinaryRow row) {
+        index.remove(row);
+    }
+
+    public boolean matches(BinaryRow aborted, BinaryRow existing) {
+        return compareColumns(aborted, existing) == 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public Cursor<IndexRowEx> scan(
+            @Nullable IndexRowPrefix lowerBound,
+            @Nullable IndexRowPrefix upperBound,
+            int flags,
+            Timestamp timestamp,
+            @Nullable IntPredicate partitionFilter
+    ) {
+        boolean includeLower = (flags & GREATER_OR_EQUAL) != 0;
+        boolean includeUpper = (flags & LESS_OR_EQUAL) != 0;
+
+        NavigableSet<BinaryRow> index = this.index;
+        int direction = 1;
+
+        // Swap bounds and flip index for backwards scan.
+        if ((flags & BACKWARDS) != 0) {
+            index = index.descendingSet();
+            direction = -1;
+
+            boolean tempBoolean = includeLower;
+            includeLower = includeUpper;
+            includeUpper = tempBoolean;
+
+            IndexRowPrefix tempBound = lowerBound;
+            lowerBound = upperBound;
+            upperBound = tempBound;
+        }
+
+        ToIntFunction<BinaryRow> lowerCmp = lowerBound == null ? row -> 1 : boundComparator(lowerBound, direction, includeLower ? 0 : -1);
+        ToIntFunction<BinaryRow> upperCmp = upperBound == null ? row -> -1 : boundComparator(upperBound, direction, includeUpper ? 0 : 1);
+
+        Iterator<IndexRowEx> iterator = index.stream()
+                .dropWhile(binaryRow -> {
+                    return lowerCmp.applyAsInt(binaryRow) < 0;

Review comment:
       can be written shorter:
   ```
   .dropWhile(binaryRow -> lowerCmp.applyAsInt(binaryRow) < 0)
   .takeWhile(binaryRow -> upperCmp.applyAsInt(binaryRow) <= 0)
   ```

##########
File path: modules/storage-api/src/test/java/org/apache/ignite/internal/storage/basic/TestSortedIndexMvStorage.java
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.basic;
+
+import java.util.Iterator;
+import java.util.Map;
+import java.util.NavigableSet;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.function.IntPredicate;
+import java.util.function.ToIntFunction;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.ColumnView;
+import org.apache.ignite.configuration.schemas.table.IndexColumnView;
+import org.apache.ignite.configuration.schemas.table.SortedIndexView;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.schema.NativeType;
+import org.apache.ignite.internal.schema.SchemaDescriptor;
+import org.apache.ignite.internal.schema.configuration.SchemaConfigurationConverter;
+import org.apache.ignite.internal.schema.configuration.SchemaDescriptorConverter;
+import org.apache.ignite.internal.schema.row.Row;
+import org.apache.ignite.internal.storage.index.IndexRowPrefix;
+import org.apache.ignite.internal.storage.index.PrefixComparator;
+import org.apache.ignite.internal.storage.index.SortedIndexMvStorage;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.internal.util.Cursor;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Test implementation of MV sorted index storage.
+ */
+public class TestSortedIndexMvStorage implements SortedIndexMvStorage {
+    private final TableView tableCfg;
+
+    private final NavigableSet<BinaryRow> index;
+
+    private final SchemaDescriptor descriptor;
+    private final Map<Integer, TestMvPartitionStorage> pk;
+
+    private final int partitions;
+
+    private final IndexColumnView[] indexColumns;
+
+    private final int[] columnIndexes;
+
+    private final NativeType[] nativeTypes;
+
+    protected TestSortedIndexMvStorage(
+            String name,
+            TableView tableCfg,
+            SchemaDescriptor descriptor,
+            Map<Integer, TestMvPartitionStorage> pk
+    ) {
+        this.tableCfg = tableCfg;
+
+        this.descriptor = descriptor;
+
+        this.pk = pk;
+
+        partitions = tableCfg.partitions();
+
+        index = new ConcurrentSkipListSet<>((l, r) -> {
+            int cmp = compareColumns(l, r);
+
+            if (cmp != 0) {
+                return cmp;
+            }
+
+            return l.keySlice().compareTo(r.keySlice());
+        });
+
+        // Init columns.
+        NamedListView<? extends ColumnView> tblColumns = tableCfg.columns();
+
+        TableIndexView idxCfg = tableCfg.indices().get(name);
+
+        assert idxCfg instanceof SortedIndexView;
+
+        SortedIndexView sortedIdxCfg = (SortedIndexView) idxCfg;
+
+        NamedListView<? extends IndexColumnView> columns = sortedIdxCfg.columns();
+
+        int length = columns.size();
+
+        this.indexColumns = new IndexColumnView[length];

Review comment:
       Do you really need all these fields? Can they be replaced with a `SortedIndexDescriptor`?

##########
File path: modules/storage-api/src/test/java/org/apache/ignite/internal/storage/basic/TestSortedIndexMvStorage.java
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.basic;
+
+import java.util.Iterator;
+import java.util.Map;
+import java.util.NavigableSet;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.function.IntPredicate;
+import java.util.function.ToIntFunction;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.ColumnView;
+import org.apache.ignite.configuration.schemas.table.IndexColumnView;
+import org.apache.ignite.configuration.schemas.table.SortedIndexView;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.schema.NativeType;
+import org.apache.ignite.internal.schema.SchemaDescriptor;
+import org.apache.ignite.internal.schema.configuration.SchemaConfigurationConverter;
+import org.apache.ignite.internal.schema.configuration.SchemaDescriptorConverter;
+import org.apache.ignite.internal.schema.row.Row;
+import org.apache.ignite.internal.storage.index.IndexRowPrefix;
+import org.apache.ignite.internal.storage.index.PrefixComparator;
+import org.apache.ignite.internal.storage.index.SortedIndexMvStorage;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.internal.util.Cursor;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Test implementation of MV sorted index storage.
+ */
+public class TestSortedIndexMvStorage implements SortedIndexMvStorage {
+    private final TableView tableCfg;
+
+    private final NavigableSet<BinaryRow> index;
+
+    private final SchemaDescriptor descriptor;
+    private final Map<Integer, TestMvPartitionStorage> pk;
+
+    private final int partitions;
+
+    private final IndexColumnView[] indexColumns;
+
+    private final int[] columnIndexes;
+
+    private final NativeType[] nativeTypes;
+
+    protected TestSortedIndexMvStorage(
+            String name,
+            TableView tableCfg,
+            SchemaDescriptor descriptor,
+            Map<Integer, TestMvPartitionStorage> pk
+    ) {
+        this.tableCfg = tableCfg;
+
+        this.descriptor = descriptor;
+
+        this.pk = pk;
+
+        partitions = tableCfg.partitions();
+
+        index = new ConcurrentSkipListSet<>((l, r) -> {
+            int cmp = compareColumns(l, r);

Review comment:
       can be replaced with `((Comparator<BinaryRow>) this::compareColumns).thenComparing(BinaryRow::keySlice)`




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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #739: IGNITE-16697 MV storage methods and reference implementation

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #739:
URL: https://github.com/apache/ignite-3/pull/739#discussion_r833041672



##########
File path: modules/storage-api/src/main/java/org/apache/ignite/internal/storage/index/SortedIndexStorage.java
##########
@@ -58,12 +64,63 @@
      */
     void remove(IndexRow row);
 
+    /** Exclude lower bound. */
+    byte GREATER = 0;
+
+    /** Include lower bound. */
+    byte GREATER_OR_EQUAL = 1;
+
+    /** Exclude upper bound. */
+    byte LESS = 0;
+
+    /** Include upper bound. */
+    byte LESS_OR_EQUAL = 1 << 1;
+
+    byte FORWARD = 0;
+
+    byte BACKWARDS = 1 << 2;
+
     /**
      * Returns a range of index values between the lower bound (inclusive) and the upper bound (inclusive).
      */
     // TODO: add options https://issues.apache.org/jira/browse/IGNITE-16059
     Cursor<IndexRow> range(IndexRowPrefix lowerBound, IndexRowPrefix upperBound);
 
+    /**
+     * Returns a range of index values between the lower bound and the upper bound, consistent with the passed timestamp.
+     *
+     * @param lowerBound Lower bound. Exclusivity is controlled by a {@link #GREATER_OR_EQUAL} or {@link #GREATER} flag.
+     *      {@code null} means unbounded.
+     * @param upperBound Upper bound. Exclusivity is controlled by a {@link #LESS} or {@link #LESS_OR_EQUAL} flag.
+     *      {@code null} means unbounded.
+     * @param flags Control flags. {@link #GREATER} | {@link #LESS} | {@link #FORWARD} by default. Other available values
+     *      are {@link #GREATER_OR_EQUAL}, {@link #LESS_OR_EQUAL} and {@link #BACKWARDS}.
+     * @param timestamp Timestamp value for consistent multiversioned index scan.
+     * @param columnsProjection Bit set with column indexes to return. {@code null} means returning all available columns.
+     * @param partitionFilter Partition filter predicate. {@code null} means returning data from all partitions.
+     * @return Cursor with fetched index rows.
+     * @throws IllegalArgumentException If backwards flag is passed and backwards iteration is not supported by the storage.

Review comment:
       Good idea, I'll do it

##########
File path: modules/storage-api/src/main/java/org/apache/ignite/internal/storage/index/SortedIndexStorage.java
##########
@@ -58,12 +64,63 @@
      */
     void remove(IndexRow row);
 
+    /** Exclude lower bound. */
+    byte GREATER = 0;
+
+    /** Include lower bound. */
+    byte GREATER_OR_EQUAL = 1;
+
+    /** Exclude upper bound. */
+    byte LESS = 0;
+
+    /** Include upper bound. */
+    byte LESS_OR_EQUAL = 1 << 1;
+
+    byte FORWARD = 0;
+
+    byte BACKWARDS = 1 << 2;
+
     /**
      * Returns a range of index values between the lower bound (inclusive) and the upper bound (inclusive).
      */
     // TODO: add options https://issues.apache.org/jira/browse/IGNITE-16059
     Cursor<IndexRow> range(IndexRowPrefix lowerBound, IndexRowPrefix upperBound);
 
+    /**
+     * Returns a range of index values between the lower bound and the upper bound, consistent with the passed timestamp.
+     *
+     * @param lowerBound Lower bound. Exclusivity is controlled by a {@link #GREATER_OR_EQUAL} or {@link #GREATER} flag.
+     *      {@code null} means unbounded.
+     * @param upperBound Upper bound. Exclusivity is controlled by a {@link #LESS} or {@link #LESS_OR_EQUAL} flag.
+     *      {@code null} means unbounded.
+     * @param flags Control flags. {@link #GREATER} | {@link #LESS} | {@link #FORWARD} by default. Other available values
+     *      are {@link #GREATER_OR_EQUAL}, {@link #LESS_OR_EQUAL} and {@link #BACKWARDS}.
+     * @param timestamp Timestamp value for consistent multiversioned index scan.
+     * @param columnsProjection Bit set with column indexes to return. {@code null} means returning all available columns.
+     * @param partitionFilter Partition filter predicate. {@code null} means returning data from all partitions.
+     * @return Cursor with fetched index rows.
+     * @throws IllegalArgumentException If backwards flag is passed and backwards iteration is not supported by the storage.
+     */
+    default Cursor<IndexRowEx> scan(
+            @Nullable IndexRowPrefix lowerBound,
+            @Nullable IndexRowPrefix upperBound,
+            @MagicConstant(flagsFromClass = SortedIndexStorage.class) byte flags,
+            Timestamp timestamp,
+            @Nullable BitSet columnsProjection,
+            @Nullable IntPredicate partitionFilter
+    ) {
+        throw new UnsupportedOperationException("scan");
+    }
+
+    /**
+     * The sole purpose of this class is to avoid massive refactoring while changing the original IndexRow.
+     */
+    interface IndexRowEx {

Review comment:
       I'm not planning to fix this in current PR, it's too much work. I'll link a new ticket

##########
File path: modules/storage-api/src/main/java/org/apache/ignite/internal/storage/index/SortedIndexDescriptor.java
##########
@@ -166,7 +166,7 @@ public SortedIndexDescriptor(String name, TableView tableConfig) {
     /**
      * Creates a {@link SchemaDescriptor} from a list of index key columns.
      */
-    private static SchemaDescriptor createSchemaDescriptor(List<ColumnView> indexKeyColumnViews) {
+    public static SchemaDescriptor createSchemaDescriptor(List<ColumnView> indexKeyColumnViews) {

Review comment:
       I guess it's an artifact, I made it private again

##########
File path: modules/storage-api/src/test/java/org/apache/ignite/internal/storage/basic/TestMvPartitionStorage.java
##########
@@ -0,0 +1,172 @@
+/*
+ * 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.basic;
+
+import java.nio.ByteBuffer;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.Predicate;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.storage.MvPartitionStorage;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.internal.util.Cursor;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Test implementation of MV partition storage.
+ */
+public class TestMvPartitionStorage implements MvPartitionStorage {
+    private static final VersionChain NULL = new VersionChain(null, null, null, null);
+
+    private final ConcurrentHashMap<ByteBuffer, VersionChain> map = new ConcurrentHashMap<>();
+
+    private final List<TestSortedIndexMvStorage> indexes;
+
+    public TestMvPartitionStorage(List<TestSortedIndexMvStorage> indexes) {
+        this.indexes = indexes;
+    }
+
+    private static class VersionChain {
+        final BinaryRow row;
+        final Timestamp begin;
+        final UUID txId;
+        final VersionChain next;
+
+        VersionChain(BinaryRow row, Timestamp begin, UUID txId, VersionChain next) {
+            this.row = row;
+            this.begin = begin;
+            this.txId = txId;
+            this.next = next;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public void addWrite(BinaryRow row, UUID txId) throws TxIdMismatchException {
+        //TODO Make it idempotent?
+        map.compute(row.keySlice(), (keyBuf, versionChain) -> {
+            if (versionChain != null && versionChain.begin == null && !txId.equals(versionChain.txId)) {
+                throw new TxIdMismatchException();
+            }
+
+            return new VersionChain(row, null, txId, versionChain);
+        });
+
+        for (TestSortedIndexMvStorage index : indexes) {
+            index.append(row);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public void abortWrite(BinaryRow key) {
+        map.merge(key.keySlice(), NULL, (versionChain, ignored) -> {
+            assert versionChain != null;
+            assert versionChain.begin == null && versionChain.txId != null;
+
+            BinaryRow aborted = versionChain.row;
+
+            for (TestSortedIndexMvStorage index : indexes) {
+                abortWrite(versionChain.next, aborted, index);
+            }
+
+            return versionChain.next;
+        });
+    }
+
+    private void abortWrite(VersionChain head, BinaryRow aborted, TestSortedIndexMvStorage index) {
+        for (VersionChain cur = head; cur != null; cur = cur.next) {
+            if (index.matches(aborted, cur.row)) {
+                return;
+            }
+        }
+
+        index.remove(aborted);
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public void commitWrite(BinaryRow key, Timestamp timestamp) {
+        map.compute(key.keySlice(), (keyBuf, versionChain) -> {
+            assert versionChain != null;

Review comment:
       This invariant must be guaranteed by external lock manager in the future

##########
File path: modules/storage-api/src/test/java/org/apache/ignite/internal/storage/basic/TestSortedIndexMvStorage.java
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.basic;
+
+import java.util.Iterator;
+import java.util.Map;
+import java.util.NavigableSet;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.function.IntPredicate;
+import java.util.function.ToIntFunction;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.ColumnView;
+import org.apache.ignite.configuration.schemas.table.IndexColumnView;
+import org.apache.ignite.configuration.schemas.table.SortedIndexView;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.schema.NativeType;
+import org.apache.ignite.internal.schema.SchemaDescriptor;
+import org.apache.ignite.internal.schema.configuration.SchemaConfigurationConverter;
+import org.apache.ignite.internal.schema.configuration.SchemaDescriptorConverter;
+import org.apache.ignite.internal.schema.row.Row;
+import org.apache.ignite.internal.storage.index.IndexRowPrefix;
+import org.apache.ignite.internal.storage.index.PrefixComparator;
+import org.apache.ignite.internal.storage.index.SortedIndexMvStorage;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.internal.util.Cursor;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Test implementation of MV sorted index storage.
+ */
+public class TestSortedIndexMvStorage implements SortedIndexMvStorage {
+    private final TableView tableCfg;
+
+    private final NavigableSet<BinaryRow> index;
+
+    private final SchemaDescriptor descriptor;
+    private final Map<Integer, TestMvPartitionStorage> pk;
+
+    private final int partitions;
+
+    private final IndexColumnView[] indexColumns;
+
+    private final int[] columnIndexes;
+
+    private final NativeType[] nativeTypes;
+
+    protected TestSortedIndexMvStorage(
+            String name,
+            TableView tableCfg,
+            SchemaDescriptor descriptor,
+            Map<Integer, TestMvPartitionStorage> pk
+    ) {
+        this.tableCfg = tableCfg;
+
+        this.descriptor = descriptor;
+
+        this.pk = pk;
+
+        partitions = tableCfg.partitions();
+
+        index = new ConcurrentSkipListSet<>((l, r) -> {
+            int cmp = compareColumns(l, r);
+
+            if (cmp != 0) {
+                return cmp;
+            }
+
+            return l.keySlice().compareTo(r.keySlice());
+        });
+
+        // Init columns.
+        NamedListView<? extends ColumnView> tblColumns = tableCfg.columns();
+
+        TableIndexView idxCfg = tableCfg.indices().get(name);
+
+        assert idxCfg instanceof SortedIndexView;
+
+        SortedIndexView sortedIdxCfg = (SortedIndexView) idxCfg;
+
+        NamedListView<? extends IndexColumnView> columns = sortedIdxCfg.columns();
+
+        int length = columns.size();
+
+        this.indexColumns = new IndexColumnView[length];
+        this.columnIndexes = new int[length];
+        this.nativeTypes = new NativeType[length];
+
+        for (int i = 0; i < length; i++) {
+            IndexColumnView idxColumn = columns.get(i);
+
+            indexColumns[i] = idxColumn;
+
+            int columnIndex = tblColumns.namedListKeys().indexOf(idxColumn.name());
+
+            columnIndexes[i] = columnIndex;
+
+            nativeTypes[i] = SchemaDescriptorConverter.convert(SchemaConfigurationConverter.convert(tblColumns.get(columnIndex).type()));
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public boolean supportsBackwardsScan() {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public boolean supportsIndexOnlyScan() {
+        return false;
+    }
+
+    private int compareColumns(BinaryRow l, BinaryRow r) {
+        Row leftRow = new Row(descriptor, l);
+        Row rightRow = new Row(descriptor, r);
+
+        for (int i = 0; i < indexColumns.length; i++) {
+            int columnIndex = columnIndexes[i];
+
+            int cmp = PrefixComparator.compareColumns(leftRow, columnIndex, nativeTypes[i].spec(), rightRow.value(columnIndex));
+
+            if (cmp != 0) {
+                return indexColumns[i].asc() ? cmp : -cmp;
+            }
+        }
+
+        return 0;
+    }
+
+    public void append(BinaryRow row) {
+        index.add(row);
+    }
+
+    public void remove(BinaryRow row) {
+        index.remove(row);
+    }
+
+    public boolean matches(BinaryRow aborted, BinaryRow existing) {
+        return compareColumns(aborted, existing) == 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public Cursor<IndexRowEx> scan(
+            @Nullable IndexRowPrefix lowerBound,
+            @Nullable IndexRowPrefix upperBound,
+            int flags,
+            Timestamp timestamp,
+            @Nullable IntPredicate partitionFilter
+    ) {
+        boolean includeLower = (flags & GREATER_OR_EQUAL) != 0;
+        boolean includeUpper = (flags & LESS_OR_EQUAL) != 0;
+
+        NavigableSet<BinaryRow> index = this.index;
+        int direction = 1;
+
+        // Swap bounds and flip index for backwards scan.
+        if ((flags & BACKWARDS) != 0) {
+            index = index.descendingSet();
+            direction = -1;
+
+            boolean tempBoolean = includeLower;
+            includeLower = includeUpper;
+            includeUpper = tempBoolean;
+
+            IndexRowPrefix tempBound = lowerBound;
+            lowerBound = upperBound;
+            upperBound = tempBound;
+        }
+
+        ToIntFunction<BinaryRow> lowerCmp = lowerBound == null ? row -> 1 : boundComparator(lowerBound, direction, includeLower ? 0 : -1);
+        ToIntFunction<BinaryRow> upperCmp = upperBound == null ? row -> -1 : boundComparator(upperBound, direction, includeUpper ? 0 : 1);
+
+        Iterator<IndexRowEx> iterator = index.stream()
+                .dropWhile(binaryRow -> {
+                    return lowerCmp.applyAsInt(binaryRow) < 0;

Review comment:
       Ok

##########
File path: modules/storage-api/src/test/java/org/apache/ignite/internal/storage/basic/TestSortedIndexMvStorage.java
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.basic;
+
+import java.util.Iterator;
+import java.util.Map;
+import java.util.NavigableSet;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.function.IntPredicate;
+import java.util.function.ToIntFunction;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.ColumnView;
+import org.apache.ignite.configuration.schemas.table.IndexColumnView;
+import org.apache.ignite.configuration.schemas.table.SortedIndexView;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.schema.NativeType;
+import org.apache.ignite.internal.schema.SchemaDescriptor;
+import org.apache.ignite.internal.schema.configuration.SchemaConfigurationConverter;
+import org.apache.ignite.internal.schema.configuration.SchemaDescriptorConverter;
+import org.apache.ignite.internal.schema.row.Row;
+import org.apache.ignite.internal.storage.index.IndexRowPrefix;
+import org.apache.ignite.internal.storage.index.PrefixComparator;
+import org.apache.ignite.internal.storage.index.SortedIndexMvStorage;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.internal.util.Cursor;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Test implementation of MV sorted index storage.
+ */
+public class TestSortedIndexMvStorage implements SortedIndexMvStorage {
+    private final TableView tableCfg;
+
+    private final NavigableSet<BinaryRow> index;
+
+    private final SchemaDescriptor descriptor;
+    private final Map<Integer, TestMvPartitionStorage> pk;
+
+    private final int partitions;
+
+    private final IndexColumnView[] indexColumns;
+
+    private final int[] columnIndexes;
+
+    private final NativeType[] nativeTypes;
+
+    protected TestSortedIndexMvStorage(
+            String name,
+            TableView tableCfg,
+            SchemaDescriptor descriptor,
+            Map<Integer, TestMvPartitionStorage> pk
+    ) {
+        this.tableCfg = tableCfg;
+
+        this.descriptor = descriptor;
+
+        this.pk = pk;
+
+        partitions = tableCfg.partitions();
+
+        index = new ConcurrentSkipListSet<>((l, r) -> {
+            int cmp = compareColumns(l, r);
+
+            if (cmp != 0) {
+                return cmp;
+            }
+
+            return l.keySlice().compareTo(r.keySlice());
+        });
+
+        // Init columns.
+        NamedListView<? extends ColumnView> tblColumns = tableCfg.columns();
+
+        TableIndexView idxCfg = tableCfg.indices().get(name);
+
+        assert idxCfg instanceof SortedIndexView;
+
+        SortedIndexView sortedIdxCfg = (SortedIndexView) idxCfg;
+
+        NamedListView<? extends IndexColumnView> columns = sortedIdxCfg.columns();
+
+        int length = columns.size();
+
+        this.indexColumns = new IndexColumnView[length];
+        this.columnIndexes = new int[length];
+        this.nativeTypes = new NativeType[length];
+
+        for (int i = 0; i < length; i++) {
+            IndexColumnView idxColumn = columns.get(i);
+
+            indexColumns[i] = idxColumn;
+
+            int columnIndex = tblColumns.namedListKeys().indexOf(idxColumn.name());
+
+            columnIndexes[i] = columnIndex;
+
+            nativeTypes[i] = SchemaDescriptorConverter.convert(SchemaConfigurationConverter.convert(tblColumns.get(columnIndex).type()));
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public boolean supportsBackwardsScan() {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public boolean supportsIndexOnlyScan() {
+        return false;
+    }
+
+    private int compareColumns(BinaryRow l, BinaryRow r) {
+        Row leftRow = new Row(descriptor, l);

Review comment:
       It's weird that we have two almost identical comparison methods

##########
File path: modules/storage-api/src/test/java/org/apache/ignite/internal/storage/basic/TestSortedIndexMvStorage.java
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.basic;
+
+import java.util.Iterator;
+import java.util.Map;
+import java.util.NavigableSet;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.function.IntPredicate;
+import java.util.function.ToIntFunction;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.ColumnView;
+import org.apache.ignite.configuration.schemas.table.IndexColumnView;
+import org.apache.ignite.configuration.schemas.table.SortedIndexView;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.schema.NativeType;
+import org.apache.ignite.internal.schema.SchemaDescriptor;
+import org.apache.ignite.internal.schema.configuration.SchemaConfigurationConverter;
+import org.apache.ignite.internal.schema.configuration.SchemaDescriptorConverter;
+import org.apache.ignite.internal.schema.row.Row;
+import org.apache.ignite.internal.storage.index.IndexRowPrefix;
+import org.apache.ignite.internal.storage.index.PrefixComparator;
+import org.apache.ignite.internal.storage.index.SortedIndexMvStorage;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.internal.util.Cursor;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Test implementation of MV sorted index storage.
+ */
+public class TestSortedIndexMvStorage implements SortedIndexMvStorage {
+    private final TableView tableCfg;
+
+    private final NavigableSet<BinaryRow> index;
+
+    private final SchemaDescriptor descriptor;
+    private final Map<Integer, TestMvPartitionStorage> pk;
+
+    private final int partitions;
+
+    private final IndexColumnView[] indexColumns;
+
+    private final int[] columnIndexes;
+
+    private final NativeType[] nativeTypes;
+
+    protected TestSortedIndexMvStorage(
+            String name,
+            TableView tableCfg,
+            SchemaDescriptor descriptor,
+            Map<Integer, TestMvPartitionStorage> pk
+    ) {
+        this.tableCfg = tableCfg;
+
+        this.descriptor = descriptor;
+
+        this.pk = pk;
+
+        partitions = tableCfg.partitions();
+
+        index = new ConcurrentSkipListSet<>((l, r) -> {
+            int cmp = compareColumns(l, r);
+
+            if (cmp != 0) {
+                return cmp;
+            }
+
+            return l.keySlice().compareTo(r.keySlice());
+        });
+
+        // Init columns.
+        NamedListView<? extends ColumnView> tblColumns = tableCfg.columns();
+
+        TableIndexView idxCfg = tableCfg.indices().get(name);
+
+        assert idxCfg instanceof SortedIndexView;
+
+        SortedIndexView sortedIdxCfg = (SortedIndexView) idxCfg;
+
+        NamedListView<? extends IndexColumnView> columns = sortedIdxCfg.columns();
+
+        int length = columns.size();
+
+        this.indexColumns = new IndexColumnView[length];
+        this.columnIndexes = new int[length];
+        this.nativeTypes = new NativeType[length];
+
+        for (int i = 0; i < length; i++) {
+            IndexColumnView idxColumn = columns.get(i);
+
+            indexColumns[i] = idxColumn;
+
+            int columnIndex = tblColumns.namedListKeys().indexOf(idxColumn.name());
+
+            columnIndexes[i] = columnIndex;
+
+            nativeTypes[i] = SchemaDescriptorConverter.convert(SchemaConfigurationConverter.convert(tblColumns.get(columnIndex).type()));
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public boolean supportsBackwardsScan() {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public boolean supportsIndexOnlyScan() {
+        return false;
+    }
+
+    private int compareColumns(BinaryRow l, BinaryRow r) {
+        Row leftRow = new Row(descriptor, l);

Review comment:
       Again, there's a mismatch with columns indexes, as far as I know

##########
File path: modules/storage-api/src/main/java/org/apache/ignite/internal/storage/MvPartitionStorage.java
##########
@@ -0,0 +1,96 @@
+/*
+ * 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;
+
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Predicate;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.internal.util.Cursor;
+import org.apache.ignite.lang.IgniteException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Multi-versioned partition storage.
+ * POC version, that represents a combination between a replicated TX-aware MV storage and physical MV storage. Their API are very similar,
+ * althouh there are very important differences that will be addressed in the future.
+ */
+public interface MvPartitionStorage {
+    /**
+     * Exception class that describes the situation where two independant transactions attempting to write values for the same key.
+     */
+    class TxIdMismatchException extends IgniteException {
+    }
+
+    /**
+     * Reads the value from the storage as it was at the given timestamp.

Review comment:
       Ok

##########
File path: modules/storage-api/src/main/java/org/apache/ignite/internal/storage/PartitionStorage.java
##########
@@ -128,6 +143,66 @@
      */
     Cursor<DataRow> scan(Predicate<SearchRow> filter) throws StorageException;
 
+    /**
+     * Scans the partition and returns a cursor of values in at the given timestamp.
+     *
+     * @param keyFilter Key filter. Binary rows passed to the filter may or may not have a value, filter should only check keys.
+     * @param timestamp Timestamp
+     * @return Cursor.
+     */
+    default Cursor<BinaryRow> scan(Predicate<BinaryRow> keyFilter, @Nullable Timestamp timestamp) {
+        throw new UnsupportedOperationException("scan");
+    }
+
+    /**
+     * Exception class that describes the situation where two independant transactions attempting to write values for the same key.
+     */
+    class TxIdMismatchException extends RuntimeException {
+    }

Review comment:
       All inner classes in interfaces are implicitly static

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/Cursor.java
##########
@@ -25,5 +25,38 @@
  * @param <T> Type of elements.
  */
 public interface Cursor<T> extends Iterator<T>, Iterable<T>, AutoCloseable {
+    /**
+     * Creates an iterator based cursor.
+     *
+     * @param it Iterator.
+     * @param <T> Type of elements in iterator.
+     * @return Cursor.
+     */
+    static <T> Cursor<T> fromIterator(Iterator<? extends T> it) {
+        return new Cursor<T>() {
+            /** {@inheritDoc} */
+            @Override
+            public void close() throws Exception {
+                // No-op.
+            }
 
+            /** {@inheritDoc} */
+            @Override
+            public Iterator<T> iterator() {
+                return this;
+            }

Review comment:
       I'll do it, thank you!

##########
File path: modules/storage-api/src/main/java/org/apache/ignite/internal/storage/PartitionStorage.java
##########
@@ -128,6 +143,66 @@
      */
     Cursor<DataRow> scan(Predicate<SearchRow> filter) throws StorageException;
 
+    /**
+     * Scans the partition and returns a cursor of values in at the given timestamp.
+     *
+     * @param keyFilter Key filter. Binary rows passed to the filter may or may not have a value, filter should only check keys.
+     * @param timestamp Timestamp
+     * @return Cursor.
+     */
+    default Cursor<BinaryRow> scan(Predicate<BinaryRow> keyFilter, @Nullable Timestamp timestamp) {
+        throw new UnsupportedOperationException("scan");
+    }
+
+    /**
+     * Exception class that describes the situation where two independant transactions attempting to write values for the same key.
+     */
+    class TxIdMismatchException extends RuntimeException {
+    }

Review comment:
       My mistake

##########
File path: modules/storage-api/src/main/java/org/apache/ignite/internal/storage/index/SortedIndexMvStorage.java
##########
@@ -0,0 +1,95 @@
+/*
+ * 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.index;
+
+import java.util.function.IntPredicate;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.internal.util.Cursor;
+import org.intellij.lang.annotations.MagicConstant;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Storage for a sorted index.
+ * POC version, that represents a combination between a replicated TX-aware MV storage and physical MV storage. Real future implementation
+ * will be defined later. Things to notice here: TX-aware implementation should have a projection bitset instead of full row reading.
+ * Physical storage API will be enriched with append/remove methods, like in reference implementation.
+ */
+public interface SortedIndexMvStorage {
+    /** Exclude lower bound. */
+    int GREATER = 0;

Review comment:
       These are default values, I didn't use them in tests explicitly, maybe I should fix that

##########
File path: modules/storage-api/src/main/java/org/apache/ignite/internal/storage/index/SortedIndexMvStorage.java
##########
@@ -0,0 +1,95 @@
+/*
+ * 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.index;
+
+import java.util.function.IntPredicate;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.internal.util.Cursor;
+import org.intellij.lang.annotations.MagicConstant;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Storage for a sorted index.
+ * POC version, that represents a combination between a replicated TX-aware MV storage and physical MV storage. Real future implementation
+ * will be defined later. Things to notice here: TX-aware implementation should have a projection bitset instead of full row reading.
+ * Physical storage API will be enriched with append/remove methods, like in reference implementation.
+ */
+public interface SortedIndexMvStorage {
+    /** Exclude lower bound. */
+    int GREATER = 0;
+
+    /** Include lower bound. */
+    int GREATER_OR_EQUAL = 1;
+
+    /** Exclude upper bound. */
+    int LESS = 0;
+
+    /** Include upper bound. */
+    int LESS_OR_EQUAL = 1 << 1;
+
+    /** Forward scan. */
+    int FORWARD = 0;
+
+    /** Backwards scan. */
+    int BACKWARDS = 1 << 2;
+
+    /**
+     * The sole purpose of this class is to avoid massive refactoring while changing the original IndexRow.
+     */
+    interface IndexRowEx {
+        /**
+         * Key-only binary row if index-only scan is supported, full binary row otherwise.
+         */
+        BinaryRow row();
+
+        /**
+         * Returns indexed column value.
+         *
+         * @param idx PK column index.
+         * @return Indexed column value.
+         */
+        Object value(int idx);
+    }
+
+    boolean supportsBackwardsScan();
+
+    boolean supportsIndexOnlyScan();
+
+    /**
+     * Returns a range of index values between the lower bound and the upper bound, consistent with the passed timestamp.

Review comment:
       It means the state of the data as it was at the given time

##########
File path: modules/storage-api/src/test/java/org/apache/ignite/internal/storage/basic/TestMvPartitionStorage.java
##########
@@ -0,0 +1,172 @@
+/*
+ * 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.basic;
+
+import java.nio.ByteBuffer;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.Predicate;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.storage.MvPartitionStorage;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.internal.util.Cursor;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Test implementation of MV partition storage.
+ */
+public class TestMvPartitionStorage implements MvPartitionStorage {
+    private static final VersionChain NULL = new VersionChain(null, null, null, null);
+
+    private final ConcurrentHashMap<ByteBuffer, VersionChain> map = new ConcurrentHashMap<>();
+
+    private final List<TestSortedIndexMvStorage> indexes;
+
+    public TestMvPartitionStorage(List<TestSortedIndexMvStorage> indexes) {
+        this.indexes = indexes;
+    }
+
+    private static class VersionChain {
+        final BinaryRow row;
+        final Timestamp begin;
+        final UUID txId;
+        final VersionChain next;
+
+        VersionChain(BinaryRow row, Timestamp begin, UUID txId, VersionChain next) {

Review comment:
       Ok

##########
File path: modules/storage-api/src/main/java/org/apache/ignite/internal/storage/MvPartitionStorage.java
##########
@@ -0,0 +1,96 @@
+/*
+ * 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;
+
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Predicate;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.internal.util.Cursor;
+import org.apache.ignite.lang.IgniteException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Multi-versioned partition storage.
+ * POC version, that represents a combination between a replicated TX-aware MV storage and physical MV storage. Their API are very similar,
+ * althouh there are very important differences that will be addressed in the future.
+ */
+public interface MvPartitionStorage {
+    /**
+     * Exception class that describes the situation where two independant transactions attempting to write values for the same key.
+     */
+    class TxIdMismatchException extends IgniteException {
+    }
+
+    /**
+     * Reads the value from the storage as it was at the given timestamp.
+     *
+     * @param key Key.
+     * @param timestamp Timestamp.
+     * @return Binary row that corresponds to the key or {@code null} if value is not found.
+     */
+    @Nullable
+    BinaryRow read(BinaryRow key, @Nullable Timestamp timestamp);
+
+    /**
+     * Creates uncommited version, assigned to the passed transaction id..
+     *
+     * @param row Binary row to update. Key only row means value removal.
+     * @param txId Transaction id.
+     * @throws TxIdMismatchException If there's another pending update associated with different transaction id.
+     * @throws StorageException If failed to write data to the storage.
+     */
+    void addWrite(BinaryRow row, UUID txId) throws TxIdMismatchException, StorageException;
+
+    /**
+     * Aborts a pending update of the ongoing uncommited transaction. Invoked during rollback.
+     *
+     * @param key Key.
+     * @throws StorageException If failed to write data to the storage.
+     */
+    void abortWrite(BinaryRow key) throws StorageException;
+
+    /**
+     * Commits a pending update of the ongoing transaction. Invoked during commit. Commited value will be versioned by the given timestamp.
+     *
+     * @param key Key.
+     * @param timestamp Timestamp to associate with commited value.
+     * @throws StorageException If failed to write data to the storage.
+     */
+    void commitWrite(BinaryRow key, Timestamp timestamp) throws StorageException;
+
+    /**
+     * Removes data associated with old timestamps.
+     *
+     * @param from Start of hashes range to process. Inclusive.
+     * @param to End of hashes range to process. Inclusive.
+     * @param timestamp Timestamp to remove all the data with a lesser timestamp.
+     * @return Future for the operation.
+     */
+    CompletableFuture<?> cleanup(int from, int to, Timestamp timestamp);

Review comment:
       That's right, it's neither implemented nor ever used

##########
File path: modules/storage-api/src/test/java/org/apache/ignite/internal/storage/basic/TestSortedIndexMvStorage.java
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.basic;
+
+import java.util.Iterator;
+import java.util.Map;
+import java.util.NavigableSet;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.function.IntPredicate;
+import java.util.function.ToIntFunction;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.ColumnView;
+import org.apache.ignite.configuration.schemas.table.IndexColumnView;
+import org.apache.ignite.configuration.schemas.table.SortedIndexView;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.schema.NativeType;
+import org.apache.ignite.internal.schema.SchemaDescriptor;
+import org.apache.ignite.internal.schema.configuration.SchemaConfigurationConverter;
+import org.apache.ignite.internal.schema.configuration.SchemaDescriptorConverter;
+import org.apache.ignite.internal.schema.row.Row;
+import org.apache.ignite.internal.storage.index.IndexRowPrefix;
+import org.apache.ignite.internal.storage.index.PrefixComparator;
+import org.apache.ignite.internal.storage.index.SortedIndexMvStorage;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.internal.util.Cursor;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Test implementation of MV sorted index storage.
+ */
+public class TestSortedIndexMvStorage implements SortedIndexMvStorage {
+    private final TableView tableCfg;
+
+    private final NavigableSet<BinaryRow> index;
+
+    private final SchemaDescriptor descriptor;
+    private final Map<Integer, TestMvPartitionStorage> pk;
+
+    private final int partitions;
+
+    private final IndexColumnView[] indexColumns;
+
+    private final int[] columnIndexes;
+
+    private final NativeType[] nativeTypes;
+
+    protected TestSortedIndexMvStorage(

Review comment:
       There are no reasons for this

##########
File path: modules/storage-api/src/main/java/org/apache/ignite/internal/storage/index/SortedIndexMvStorage.java
##########
@@ -0,0 +1,95 @@
+/*
+ * 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.index;
+
+import java.util.function.IntPredicate;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.internal.util.Cursor;
+import org.intellij.lang.annotations.MagicConstant;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Storage for a sorted index.
+ * POC version, that represents a combination between a replicated TX-aware MV storage and physical MV storage. Real future implementation
+ * will be defined later. Things to notice here: TX-aware implementation should have a projection bitset instead of full row reading.
+ * Physical storage API will be enriched with append/remove methods, like in reference implementation.
+ */
+public interface SortedIndexMvStorage {
+    /** Exclude lower bound. */
+    int GREATER = 0;
+
+    /** Include lower bound. */
+    int GREATER_OR_EQUAL = 1;
+
+    /** Exclude upper bound. */
+    int LESS = 0;
+
+    /** Include upper bound. */
+    int LESS_OR_EQUAL = 1 << 1;
+
+    /** Forward scan. */
+    int FORWARD = 0;
+
+    /** Backwards scan. */
+    int BACKWARDS = 1 << 2;
+
+    /**
+     * The sole purpose of this class is to avoid massive refactoring while changing the original IndexRow.
+     */
+    interface IndexRowEx {
+        /**
+         * Key-only binary row if index-only scan is supported, full binary row otherwise.
+         */
+        BinaryRow row();
+
+        /**
+         * Returns indexed column value.
+         *
+         * @param idx PK column index.
+         * @return Indexed column value.
+         */
+        Object value(int idx);
+    }
+
+    boolean supportsBackwardsScan();

Review comment:
       Guys asked for them, there's a chance that we won't support backwards scan in B+Tree, for example, but who knows

##########
File path: modules/storage-api/src/main/java/org/apache/ignite/internal/storage/MvPartitionStorage.java
##########
@@ -0,0 +1,96 @@
+/*
+ * 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;
+
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Predicate;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.internal.util.Cursor;
+import org.apache.ignite.lang.IgniteException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Multi-versioned partition storage.
+ * POC version, that represents a combination between a replicated TX-aware MV storage and physical MV storage. Their API are very similar,
+ * althouh there are very important differences that will be addressed in the future.
+ */
+public interface MvPartitionStorage {
+    /**
+     * Exception class that describes the situation where two independant transactions attempting to write values for the same key.
+     */
+    class TxIdMismatchException extends IgniteException {

Review comment:
       Sure

##########
File path: modules/storage-api/src/test/java/org/apache/ignite/internal/storage/BaseMvStoragesTest.java
##########
@@ -0,0 +1,180 @@
+/*
+ * 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;
+
+import java.util.Locale;
+import java.util.Objects;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.schema.Column;
+import org.apache.ignite.internal.schema.NativeTypes;
+import org.apache.ignite.internal.schema.SchemaDescriptor;
+import org.apache.ignite.internal.schema.marshaller.KvMarshaller;
+import org.apache.ignite.internal.schema.marshaller.MarshallerException;
+import org.apache.ignite.internal.schema.marshaller.MarshallerFactory;
+import org.apache.ignite.internal.schema.marshaller.reflection.ReflectionMarshallerFactory;
+import org.apache.ignite.internal.schema.row.Row;
+import org.apache.ignite.lang.IgniteException;
+import org.jetbrains.annotations.Nullable;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+
+/**
+ * Base test for MV storages, contains pojo classes, their descriptor and a marshaller instance.
+ */
+public class BaseMvStoragesTest {
+    /** Default reflection marshaller factory. */
+    protected static MarshallerFactory marshallerFactory;
+
+    /** Schema descriptor for tests. */
+    protected static SchemaDescriptor schemaDescriptor;
+
+    /** Key-value marshaller for tests. */
+    protected static KvMarshaller<TestKey, TestValue> kvMarshaller;
+
+    @BeforeAll
+    static void beforeAll() {
+        marshallerFactory = new ReflectionMarshallerFactory();
+
+        schemaDescriptor = new SchemaDescriptor(1, new Column[]{
+                new Column("intKey".toUpperCase(Locale.ROOT), NativeTypes.INT32, false),
+                new Column("strKey".toUpperCase(Locale.ROOT), NativeTypes.STRING, false),
+        }, new Column[]{
+                new Column("intVal".toUpperCase(Locale.ROOT), NativeTypes.INT32, false),
+                new Column("strVal".toUpperCase(Locale.ROOT), NativeTypes.STRING, false),
+        });
+
+        kvMarshaller = marshallerFactory.create(schemaDescriptor, TestKey.class, TestValue.class);
+    }
+
+    @AfterAll
+    static void afterAll() {
+        kvMarshaller = null;
+        schemaDescriptor = null;
+        marshallerFactory = null;
+    }
+
+    protected BinaryRow binaryKey(TestKey key) {

Review comment:
       Correct, I missed that somehow

##########
File path: modules/storage-api/src/test/java/org/apache/ignite/internal/storage/basic/TestSortedIndexMvStorage.java
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.basic;
+
+import java.util.Iterator;
+import java.util.Map;
+import java.util.NavigableSet;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.function.IntPredicate;
+import java.util.function.ToIntFunction;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.ColumnView;
+import org.apache.ignite.configuration.schemas.table.IndexColumnView;
+import org.apache.ignite.configuration.schemas.table.SortedIndexView;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.schema.NativeType;
+import org.apache.ignite.internal.schema.SchemaDescriptor;
+import org.apache.ignite.internal.schema.configuration.SchemaConfigurationConverter;
+import org.apache.ignite.internal.schema.configuration.SchemaDescriptorConverter;
+import org.apache.ignite.internal.schema.row.Row;
+import org.apache.ignite.internal.storage.index.IndexRowPrefix;
+import org.apache.ignite.internal.storage.index.PrefixComparator;
+import org.apache.ignite.internal.storage.index.SortedIndexMvStorage;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.internal.util.Cursor;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Test implementation of MV sorted index storage.
+ */
+public class TestSortedIndexMvStorage implements SortedIndexMvStorage {
+    private final TableView tableCfg;
+
+    private final NavigableSet<BinaryRow> index;
+
+    private final SchemaDescriptor descriptor;
+    private final Map<Integer, TestMvPartitionStorage> pk;
+
+    private final int partitions;
+
+    private final IndexColumnView[] indexColumns;
+
+    private final int[] columnIndexes;
+
+    private final NativeType[] nativeTypes;
+
+    protected TestSortedIndexMvStorage(
+            String name,
+            TableView tableCfg,
+            SchemaDescriptor descriptor,
+            Map<Integer, TestMvPartitionStorage> pk
+    ) {
+        this.tableCfg = tableCfg;
+
+        this.descriptor = descriptor;
+
+        this.pk = pk;
+
+        partitions = tableCfg.partitions();
+
+        index = new ConcurrentSkipListSet<>((l, r) -> {
+            int cmp = compareColumns(l, r);
+
+            if (cmp != 0) {
+                return cmp;
+            }
+
+            return l.keySlice().compareTo(r.keySlice());
+        });
+
+        // Init columns.
+        NamedListView<? extends ColumnView> tblColumns = tableCfg.columns();
+
+        TableIndexView idxCfg = tableCfg.indices().get(name);
+
+        assert idxCfg instanceof SortedIndexView;
+
+        SortedIndexView sortedIdxCfg = (SortedIndexView) idxCfg;
+
+        NamedListView<? extends IndexColumnView> columns = sortedIdxCfg.columns();
+
+        int length = columns.size();
+
+        this.indexColumns = new IndexColumnView[length];

Review comment:
       There must have been a reason for this, I think SortedIndexDescriptor uses different indexes for columns. This code addresses the original BinaryRow, not the "compressed" one that only has the required columns

##########
File path: modules/storage-api/src/test/java/org/apache/ignite/internal/storage/basic/TestSortedIndexMvStorage.java
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.basic;
+
+import java.util.Iterator;
+import java.util.Map;
+import java.util.NavigableSet;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.function.IntPredicate;
+import java.util.function.ToIntFunction;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.ColumnView;
+import org.apache.ignite.configuration.schemas.table.IndexColumnView;
+import org.apache.ignite.configuration.schemas.table.SortedIndexView;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.schema.NativeType;
+import org.apache.ignite.internal.schema.SchemaDescriptor;
+import org.apache.ignite.internal.schema.configuration.SchemaConfigurationConverter;
+import org.apache.ignite.internal.schema.configuration.SchemaDescriptorConverter;
+import org.apache.ignite.internal.schema.row.Row;
+import org.apache.ignite.internal.storage.index.IndexRowPrefix;
+import org.apache.ignite.internal.storage.index.PrefixComparator;
+import org.apache.ignite.internal.storage.index.SortedIndexMvStorage;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.internal.util.Cursor;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Test implementation of MV sorted index storage.
+ */
+public class TestSortedIndexMvStorage implements SortedIndexMvStorage {
+    private final TableView tableCfg;

Review comment:
       Indeed

##########
File path: modules/storage-api/src/test/java/org/apache/ignite/internal/storage/basic/TestSortedIndexMvStorage.java
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.basic;
+
+import java.util.Iterator;
+import java.util.Map;
+import java.util.NavigableSet;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.function.IntPredicate;
+import java.util.function.ToIntFunction;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.ColumnView;
+import org.apache.ignite.configuration.schemas.table.IndexColumnView;
+import org.apache.ignite.configuration.schemas.table.SortedIndexView;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.schema.NativeType;
+import org.apache.ignite.internal.schema.SchemaDescriptor;
+import org.apache.ignite.internal.schema.configuration.SchemaConfigurationConverter;
+import org.apache.ignite.internal.schema.configuration.SchemaDescriptorConverter;
+import org.apache.ignite.internal.schema.row.Row;
+import org.apache.ignite.internal.storage.index.IndexRowPrefix;
+import org.apache.ignite.internal.storage.index.PrefixComparator;
+import org.apache.ignite.internal.storage.index.SortedIndexMvStorage;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.internal.util.Cursor;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Test implementation of MV sorted index storage.
+ */
+public class TestSortedIndexMvStorage implements SortedIndexMvStorage {
+    private final TableView tableCfg;
+
+    private final NavigableSet<BinaryRow> index;
+
+    private final SchemaDescriptor descriptor;
+    private final Map<Integer, TestMvPartitionStorage> pk;
+
+    private final int partitions;
+
+    private final IndexColumnView[] indexColumns;
+
+    private final int[] columnIndexes;
+
+    private final NativeType[] nativeTypes;
+
+    protected TestSortedIndexMvStorage(
+            String name,
+            TableView tableCfg,
+            SchemaDescriptor descriptor,
+            Map<Integer, TestMvPartitionStorage> pk
+    ) {
+        this.tableCfg = tableCfg;
+
+        this.descriptor = descriptor;
+
+        this.pk = pk;
+
+        partitions = tableCfg.partitions();
+
+        index = new ConcurrentSkipListSet<>((l, r) -> {
+            int cmp = compareColumns(l, r);

Review comment:
       Thank you

##########
File path: modules/storage-api/src/main/java/org/apache/ignite/internal/storage/MvPartitionStorage.java
##########
@@ -0,0 +1,96 @@
+/*
+ * 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;
+
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Predicate;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.internal.util.Cursor;
+import org.apache.ignite.lang.IgniteException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Multi-versioned partition storage.
+ * POC version, that represents a combination between a replicated TX-aware MV storage and physical MV storage. Their API are very similar,
+ * althouh there are very important differences that will be addressed in the future.
+ */
+public interface MvPartitionStorage {
+    /**
+     * Exception class that describes the situation where two independant transactions attempting to write values for the same key.
+     */
+    class TxIdMismatchException extends IgniteException {
+    }
+
+    /**
+     * Reads the value from the storage as it was at the given timestamp.
+     *
+     * @param key Key.
+     * @param timestamp Timestamp.
+     * @return Binary row that corresponds to the key or {@code null} if value is not found.
+     */
+    @Nullable
+    BinaryRow read(BinaryRow key, @Nullable Timestamp timestamp);
+
+    /**
+     * Creates uncommited version, assigned to the passed transaction id..
+     *
+     * @param row Binary row to update. Key only row means value removal.
+     * @param txId Transaction id.
+     * @throws TxIdMismatchException If there's another pending update associated with different transaction id.
+     * @throws StorageException If failed to write data to the storage.
+     */
+    void addWrite(BinaryRow row, UUID txId) throws TxIdMismatchException, StorageException;
+
+    /**
+     * Aborts a pending update of the ongoing uncommited transaction. Invoked during rollback.
+     *
+     * @param key Key.
+     * @throws StorageException If failed to write data to the storage.
+     */
+    void abortWrite(BinaryRow key) throws StorageException;
+
+    /**
+     * Commits a pending update of the ongoing transaction. Invoked during commit. Commited value will be versioned by the given timestamp.
+     *
+     * @param key Key.
+     * @param timestamp Timestamp to associate with commited value.
+     * @throws StorageException If failed to write data to the storage.
+     */
+    void commitWrite(BinaryRow key, Timestamp timestamp) throws StorageException;
+
+    /**
+     * Removes data associated with old timestamps.
+     *
+     * @param from Start of hashes range to process. Inclusive.

Review comment:
       You know what, I'll drop this method completely. Things change too quickly, this signature makes no sense most likely




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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #739: IGNITE-16697 MV storage methods and reference implementation

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #739:
URL: https://github.com/apache/ignite-3/pull/739#discussion_r840324707



##########
File path: modules/storage-api/src/test/java/org/apache/ignite/internal/storage/BaseMvStoragesTest.java
##########
@@ -0,0 +1,180 @@
+/*
+ * 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;
+
+import java.util.Locale;
+import java.util.Objects;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.schema.Column;
+import org.apache.ignite.internal.schema.NativeTypes;
+import org.apache.ignite.internal.schema.SchemaDescriptor;
+import org.apache.ignite.internal.schema.marshaller.KvMarshaller;
+import org.apache.ignite.internal.schema.marshaller.MarshallerException;
+import org.apache.ignite.internal.schema.marshaller.MarshallerFactory;
+import org.apache.ignite.internal.schema.marshaller.reflection.ReflectionMarshallerFactory;
+import org.apache.ignite.internal.schema.row.Row;
+import org.apache.ignite.lang.IgniteException;
+import org.jetbrains.annotations.Nullable;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+
+/**
+ * Base test for MV storages, contains pojo classes, their descriptor and a marshaller instance.
+ */
+public class BaseMvStoragesTest {
+    /** Default reflection marshaller factory. */
+    protected static MarshallerFactory marshallerFactory;
+
+    /** Schema descriptor for tests. */
+    protected static SchemaDescriptor schemaDescriptor;
+
+    /** Key-value marshaller for tests. */
+    protected static KvMarshaller<TestKey, TestValue> kvMarshaller;
+
+    @BeforeAll
+    static void beforeAll() {
+        marshallerFactory = new ReflectionMarshallerFactory();
+
+        schemaDescriptor = new SchemaDescriptor(1, new Column[]{
+                new Column("intKey".toUpperCase(Locale.ROOT), NativeTypes.INT32, false),
+                new Column("strKey".toUpperCase(Locale.ROOT), NativeTypes.STRING, false),
+        }, new Column[]{
+                new Column("intVal".toUpperCase(Locale.ROOT), NativeTypes.INT32, false),
+                new Column("strVal".toUpperCase(Locale.ROOT), NativeTypes.STRING, false),
+        });
+
+        kvMarshaller = marshallerFactory.create(schemaDescriptor, TestKey.class, TestValue.class);
+    }
+
+    @AfterAll
+    static void afterAll() {
+        kvMarshaller = null;
+        schemaDescriptor = null;
+        marshallerFactory = null;
+    }
+
+    protected BinaryRow binaryKey(TestKey key) {
+        try {
+            return kvMarshaller.marshal(key);
+        } catch (MarshallerException e) {
+            throw new IgniteException(e);
+        }
+    }
+
+    protected BinaryRow binaryRow(TestKey key, TestValue value) {
+        try {
+            return kvMarshaller.marshal(key, value);
+        } catch (MarshallerException e) {
+            throw new IgniteException(e);
+        }
+    }
+
+    @Nullable
+    protected TestKey key(BinaryRow binaryRow) {
+        try {
+            return kvMarshaller.unmarshalKey(new Row(schemaDescriptor, binaryRow));
+        } catch (MarshallerException e) {
+            throw new IgniteException(e);
+        }
+    }
+
+    @Nullable
+    protected TestValue value(BinaryRow binaryRow) {
+        try {
+            return kvMarshaller.unmarshalValue(new Row(schemaDescriptor, binaryRow));
+        } catch (MarshallerException e) {
+            throw new IgniteException(e);
+        }
+    }
+
+    /**
+     * Test pojo key.
+     */
+    protected static class TestKey {
+        public int intKey;
+
+        public String strKey;
+
+        public TestKey() {
+        }
+
+        public TestKey(int intKey, String strKey) {
+            this.intKey = intKey;
+            this.strKey = strKey;
+        }
+
+        @Override
+        public boolean equals(Object o) {
+            if (this == o) {
+                return true;
+            }
+            if (o == null || getClass() != o.getClass()) {
+                return false;
+            }
+            TestKey testKey = (TestKey) o;
+            return intKey == testKey.intKey && Objects.equals(strKey, testKey.strKey);
+        }
+
+        @Override
+        public int hashCode() {
+            return Objects.hash(intKey, strKey);
+        }
+    }
+
+    /**
+     * Test pojo value.
+     */
+    protected static class TestValue implements Comparable<TestValue> {
+        public Integer intVal;
+
+        public String strVal;
+
+        public TestValue() {
+        }
+
+        public TestValue(Integer intVal, String strVal) {
+            this.intVal = intVal;
+            this.strVal = strVal;
+        }
+
+        @Override
+        public int compareTo(TestValue o) {
+            //TODO Compare nuppable values.

Review comment:
       No, nulls are not used in tests at this moment




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



[GitHub] [ignite-3] AMashenkov commented on a change in pull request #739: IGNITE-16697 MV storage methods and reference implementation

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #739:
URL: https://github.com/apache/ignite-3/pull/739#discussion_r833030337



##########
File path: modules/storage-api/src/main/java/org/apache/ignite/internal/storage/PartitionStorage.java
##########
@@ -128,6 +143,66 @@
      */
     Cursor<DataRow> scan(Predicate<SearchRow> filter) throws StorageException;
 
+    /**
+     * Scans the partition and returns a cursor of values in at the given timestamp.
+     *
+     * @param keyFilter Key filter. Binary rows passed to the filter may or may not have a value, filter should only check keys.
+     * @param timestamp Timestamp
+     * @return Cursor.
+     */
+    default Cursor<BinaryRow> scan(Predicate<BinaryRow> keyFilter, @Nullable Timestamp timestamp) {
+        throw new UnsupportedOperationException("scan");
+    }
+
+    /**
+     * Exception class that describes the situation where two independant transactions attempting to write values for the same key.
+     */
+    class TxIdMismatchException extends RuntimeException {
+    }

Review comment:
       Let's move to a separate high-level class (or make is static at least).




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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #739: IGNITE-16697 MV storage methods and reference implementation

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #739:
URL: https://github.com/apache/ignite-3/pull/739#discussion_r840402997



##########
File path: modules/storage-api/src/test/java/org/apache/ignite/internal/storage/AbstractSortedIndexMvStorageTest.java
##########
@@ -0,0 +1,238 @@
+/*
+ * 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;
+
+import static org.apache.ignite.internal.storage.index.SortedIndexMvStorage.BACKWARDS;
+import static org.apache.ignite.internal.storage.index.SortedIndexMvStorage.FORWARD;
+import static org.apache.ignite.internal.storage.index.SortedIndexMvStorage.GREATER_OR_EQUAL;
+import static org.apache.ignite.internal.storage.index.SortedIndexMvStorage.LESS_OR_EQUAL;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+import java.util.List;
+import java.util.UUID;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+import org.apache.ignite.configuration.schemas.table.ColumnChange;
+import org.apache.ignite.configuration.schemas.table.SortedIndexChange;
+import org.apache.ignite.configuration.schemas.table.SortedIndexConfigurationSchema;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.configuration.testframework.ConfigurationExtension;
+import org.apache.ignite.internal.configuration.testframework.InjectConfiguration;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.schema.marshaller.MarshallerException;
+import org.apache.ignite.internal.schema.row.Row;
+import org.apache.ignite.internal.storage.index.IndexRowPrefix;
+import org.apache.ignite.internal.storage.index.SortedIndexMvStorage;
+import org.apache.ignite.internal.storage.index.SortedIndexMvStorage.IndexRowEx;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.internal.util.Cursor;
+import org.apache.ignite.lang.IgniteException;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+/**
+ * Base test for MV index storages.
+ */
+@ExtendWith(ConfigurationExtension.class)
+public abstract class AbstractSortedIndexMvStorageTest extends BaseMvStoragesTest {
+    protected static final String INDEX1 = "asc_asc";
+    protected static final String INDEX2 = "asc_desc";
+
+    protected TableConfiguration tableCfg;
+
+    @BeforeEach
+    void setUp(@InjectConfiguration(polymorphicExtensions = SortedIndexConfigurationSchema.class) TableConfiguration tableCfg) {
+        tableCfg.change(tableChange -> tableChange
+                .changePartitions(1)
+                .changePrimaryKey(pk -> pk.changeColumns("intKey", "strKey"))
+                .changeColumns(columns -> columns
+                        .create("intKey", column("intKey", "INT32"))
+                        .create("strKey", column("strKey", "STRING"))
+                        .create("intVal", column("intVal", "INT32"))
+                        .create("strVal", column("strVal", "STRING"))
+                )
+                .changeIndices(indexes -> indexes
+                        .create(INDEX1, idx -> idx.convert(SortedIndexChange.class).changeColumns(idxColumns -> idxColumns
+                                .create("strVal", c -> c.changeName("strVal").changeAsc(true))
+                                .create("intVal", c -> c.changeName("intVal").changeAsc(true))
+                        ))
+                        .create(INDEX2, idx -> idx.convert(SortedIndexChange.class).changeColumns(idxColumns -> idxColumns
+                                .create("strVal", c -> c.changeName("strVal").changeAsc(true))
+                                .create("intVal", c -> c.changeName("intVal").changeAsc(false))
+                        ))
+                )
+        ).join();
+
+        this.tableCfg = tableCfg;
+    }
+
+    private static Consumer<ColumnChange> column(String name, String typeName) {
+        return c -> c.changeName(name).changeNullable(false).changeType(type -> type.changeType(typeName));
+    }
+
+    /**
+     * Creates a storage instance for testing.
+     */
+    protected abstract MvPartitionStorage partitionStorage();
+
+    /**
+     * Creates a storage instanc efor testing.
+     */
+    protected abstract SortedIndexMvStorage createIndexStorage(String name, TableView tableCfg);
+
+    @Test
+    public void testEmpty() throws Exception {
+        SortedIndexMvStorage index1 = createIndexStorage(INDEX1, tableCfg.value());
+        SortedIndexMvStorage index2 = createIndexStorage(INDEX2, tableCfg.value());
+
+        assertEquals(List.of(), convert(index1.scan(null, null, (byte) 0, null, null)));
+        assertEquals(List.of(), convert(index2.scan(null, null, (byte) 0, null, null)));
+    }
+
+    @Test
+    public void testBounds() throws Exception {
+        SortedIndexMvStorage index1 = createIndexStorage(INDEX1, tableCfg.value());
+        SortedIndexMvStorage index2 = createIndexStorage(INDEX2, tableCfg.value());
+
+        TestValue val9010 = new TestValue(90, "10");
+        TestValue val8010 = new TestValue(80, "10");
+        TestValue val9020 = new TestValue(90, "20");
+        TestValue val8020 = new TestValue(80, "20");
+
+        insert(new TestKey(1, "1"), val9010, null);
+        insert(new TestKey(2, "2"), val8010, null);
+        insert(new TestKey(3, "3"), val9020, null);
+        insert(new TestKey(4, "4"), val8020, null);
+
+        // Test without bounds.
+        assertEquals(List.of(val8010, val9010, val8020, val9020), convert(index1.scan(
+                null, null, FORWARD, null, null
+        )));
+
+        assertEquals(List.of(val9020, val8020, val9010, val8010), convert(index1.scan(
+                null, null, BACKWARDS, null, null
+        )));
+
+        assertEquals(List.of(val9010, val8010, val9020, val8020), convert(index2.scan(
+                null, null, FORWARD, null, null
+        )));
+
+        assertEquals(List.of(val8020, val9020, val8010, val9010), convert(index2.scan(
+                null, null, BACKWARDS, null, null
+        )));
+
+        // Lower bound exclusive.
+        assertEquals(List.of(val8020, val9020), convert(index1.scan(
+                prefix("10"), null, FORWARD, null, null
+        )));
+
+        assertEquals(List.of(val9020, val8020), convert(index1.scan(
+                prefix("10"), null, BACKWARDS, null, null
+        )));
+
+        assertEquals(List.of(val9020, val8020), convert(index2.scan(
+                prefix("10"), null, FORWARD, null, null
+        )));
+
+        assertEquals(List.of(val8020, val9020), convert(index2.scan(
+                prefix("10"), null, BACKWARDS, null, null
+        )));
+
+        // Lower bound inclusive.
+        assertEquals(List.of(val8010, val9010, val8020, val9020), convert(index1.scan(
+                prefix("10"), null, GREATER_OR_EQUAL | FORWARD, null, null
+        )));
+
+        assertEquals(List.of(val9020, val8020, val9010, val8010), convert(index1.scan(
+                prefix("10"), null, GREATER_OR_EQUAL | BACKWARDS, null, null
+        )));
+
+        assertEquals(List.of(val9010, val8010, val9020, val8020), convert(index2.scan(
+                prefix("10"), null, GREATER_OR_EQUAL | FORWARD, null, null
+        )));
+
+        assertEquals(List.of(val8020, val9020, val8010, val9010), convert(index2.scan(
+                prefix("10"), null, GREATER_OR_EQUAL | BACKWARDS, null, null
+        )));
+
+        // Upper bound exclusive.
+        assertEquals(List.of(val8010, val9010), convert(index1.scan(
+                null, prefix("20"), FORWARD, null, null
+        )));
+
+        assertEquals(List.of(val9010, val8010), convert(index1.scan(
+                null, prefix("20"), BACKWARDS, null, null
+        )));
+
+        assertEquals(List.of(val9010, val8010), convert(index2.scan(
+                null, prefix("20"), FORWARD, null, null
+        )));
+
+        assertEquals(List.of(val8010, val9010), convert(index2.scan(
+                null, prefix("20"), BACKWARDS, null, null
+        )));
+
+        // Lower bound inclusive.

Review comment:
       Probably, let me see




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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #739: IGNITE-16697 MV storage methods and reference implementation

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #739:
URL: https://github.com/apache/ignite-3/pull/739#discussion_r840403702



##########
File path: modules/storage-api/src/test/java/org/apache/ignite/internal/storage/AbstractSortedIndexMvStorageTest.java
##########
@@ -0,0 +1,238 @@
+/*
+ * 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;
+
+import static org.apache.ignite.internal.storage.index.SortedIndexMvStorage.BACKWARDS;
+import static org.apache.ignite.internal.storage.index.SortedIndexMvStorage.FORWARD;
+import static org.apache.ignite.internal.storage.index.SortedIndexMvStorage.GREATER_OR_EQUAL;
+import static org.apache.ignite.internal.storage.index.SortedIndexMvStorage.LESS_OR_EQUAL;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+import java.util.List;
+import java.util.UUID;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+import org.apache.ignite.configuration.schemas.table.ColumnChange;
+import org.apache.ignite.configuration.schemas.table.SortedIndexChange;
+import org.apache.ignite.configuration.schemas.table.SortedIndexConfigurationSchema;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.configuration.testframework.ConfigurationExtension;
+import org.apache.ignite.internal.configuration.testframework.InjectConfiguration;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.schema.marshaller.MarshallerException;
+import org.apache.ignite.internal.schema.row.Row;
+import org.apache.ignite.internal.storage.index.IndexRowPrefix;
+import org.apache.ignite.internal.storage.index.SortedIndexMvStorage;
+import org.apache.ignite.internal.storage.index.SortedIndexMvStorage.IndexRowEx;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.internal.util.Cursor;
+import org.apache.ignite.lang.IgniteException;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+/**
+ * Base test for MV index storages.
+ */
+@ExtendWith(ConfigurationExtension.class)
+public abstract class AbstractSortedIndexMvStorageTest extends BaseMvStoragesTest {
+    protected static final String INDEX1 = "asc_asc";
+    protected static final String INDEX2 = "asc_desc";
+
+    protected TableConfiguration tableCfg;
+
+    @BeforeEach
+    void setUp(@InjectConfiguration(polymorphicExtensions = SortedIndexConfigurationSchema.class) TableConfiguration tableCfg) {
+        tableCfg.change(tableChange -> tableChange
+                .changePartitions(1)
+                .changePrimaryKey(pk -> pk.changeColumns("intKey", "strKey"))
+                .changeColumns(columns -> columns
+                        .create("intKey", column("intKey", "INT32"))
+                        .create("strKey", column("strKey", "STRING"))
+                        .create("intVal", column("intVal", "INT32"))
+                        .create("strVal", column("strVal", "STRING"))
+                )
+                .changeIndices(indexes -> indexes
+                        .create(INDEX1, idx -> idx.convert(SortedIndexChange.class).changeColumns(idxColumns -> idxColumns
+                                .create("strVal", c -> c.changeName("strVal").changeAsc(true))
+                                .create("intVal", c -> c.changeName("intVal").changeAsc(true))
+                        ))
+                        .create(INDEX2, idx -> idx.convert(SortedIndexChange.class).changeColumns(idxColumns -> idxColumns
+                                .create("strVal", c -> c.changeName("strVal").changeAsc(true))
+                                .create("intVal", c -> c.changeName("intVal").changeAsc(false))
+                        ))
+                )
+        ).join();
+
+        this.tableCfg = tableCfg;
+    }
+
+    private static Consumer<ColumnChange> column(String name, String typeName) {
+        return c -> c.changeName(name).changeNullable(false).changeType(type -> type.changeType(typeName));
+    }
+
+    /**
+     * Creates a storage instance for testing.
+     */
+    protected abstract MvPartitionStorage partitionStorage();
+
+    /**
+     * Creates a storage instanc efor testing.
+     */
+    protected abstract SortedIndexMvStorage createIndexStorage(String name, TableView tableCfg);
+
+    @Test
+    public void testEmpty() throws Exception {
+        SortedIndexMvStorage index1 = createIndexStorage(INDEX1, tableCfg.value());
+        SortedIndexMvStorage index2 = createIndexStorage(INDEX2, tableCfg.value());
+
+        assertEquals(List.of(), convert(index1.scan(null, null, (byte) 0, null, null)));
+        assertEquals(List.of(), convert(index2.scan(null, null, (byte) 0, null, null)));
+    }
+
+    @Test
+    public void testBounds() throws Exception {

Review comment:
       Yes, I'll add more tests soon




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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #739: IGNITE-16697 MV storage methods and reference implementation

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #739:
URL: https://github.com/apache/ignite-3/pull/739#discussion_r840405743



##########
File path: modules/storage-api/src/test/java/org/apache/ignite/internal/storage/AbstractMvPartitionStorageTest.java
##########
@@ -0,0 +1,206 @@
+/*
+ * 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;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.UUID;
+import java.util.function.Predicate;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.internal.util.Cursor;
+import org.junit.jupiter.api.Test;
+
+/**
+ * Base test for MV partition storages.
+ */
+public abstract class AbstractMvPartitionStorageTest extends BaseMvStoragesTest {
+    /**
+     * Creates a storage instance for testing.
+     */
+    protected abstract MvPartitionStorage partitionStorage();
+
+    /**
+     * Tests that reads and scan from empty storage return empty results.
+     */
+    @Test
+    public void testEmpty() throws Exception {

Review comment:
       Why? It's, like, 6 lines of code
   I'll remove "pk.read" stuff, it's already present in other tests.




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



[GitHub] [ignite-3] AMashenkov commented on a change in pull request #739: IGNITE-16697 MV storage methods and reference implementation

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #739:
URL: https://github.com/apache/ignite-3/pull/739#discussion_r833002638



##########
File path: modules/storage-api/src/main/java/org/apache/ignite/internal/storage/index/SortedIndexStorage.java
##########
@@ -58,12 +64,63 @@
      */
     void remove(IndexRow row);
 
+    /** Exclude lower bound. */
+    byte GREATER = 0;
+
+    /** Include lower bound. */
+    byte GREATER_OR_EQUAL = 1;
+
+    /** Exclude upper bound. */
+    byte LESS = 0;
+
+    /** Include upper bound. */
+    byte LESS_OR_EQUAL = 1 << 1;
+
+    byte FORWARD = 0;
+
+    byte BACKWARDS = 1 << 2;
+
     /**
      * Returns a range of index values between the lower bound (inclusive) and the upper bound (inclusive).
      */
     // TODO: add options https://issues.apache.org/jira/browse/IGNITE-16059
     Cursor<IndexRow> range(IndexRowPrefix lowerBound, IndexRowPrefix upperBound);
 
+    /**
+     * Returns a range of index values between the lower bound and the upper bound, consistent with the passed timestamp.
+     *
+     * @param lowerBound Lower bound. Exclusivity is controlled by a {@link #GREATER_OR_EQUAL} or {@link #GREATER} flag.
+     *      {@code null} means unbounded.
+     * @param upperBound Upper bound. Exclusivity is controlled by a {@link #LESS} or {@link #LESS_OR_EQUAL} flag.
+     *      {@code null} means unbounded.
+     * @param flags Control flags. {@link #GREATER} | {@link #LESS} | {@link #FORWARD} by default. Other available values
+     *      are {@link #GREATER_OR_EQUAL}, {@link #LESS_OR_EQUAL} and {@link #BACKWARDS}.
+     * @param timestamp Timestamp value for consistent multiversioned index scan.
+     * @param columnsProjection Bit set with column indexes to return. {@code null} means returning all available columns.
+     * @param partitionFilter Partition filter predicate. {@code null} means returning data from all partitions.
+     * @return Cursor with fetched index rows.
+     * @throws IllegalArgumentException If backwards flag is passed and backwards iteration is not supported by the storage.

Review comment:
       Should we add smth like UnsupportedBackwardIndexScanException exception for this




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



[GitHub] [ignite-3] AMashenkov commented on a change in pull request #739: IGNITE-16697 MV storage methods and reference implementation

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #739:
URL: https://github.com/apache/ignite-3/pull/739#discussion_r832995381



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/Cursor.java
##########
@@ -25,5 +25,38 @@
  * @param <T> Type of elements.
  */
 public interface Cursor<T> extends Iterator<T>, Iterable<T>, AutoCloseable {
+    /**
+     * Creates an iterator based cursor.
+     *
+     * @param it Iterator.
+     * @param <T> Type of elements in iterator.
+     * @return Cursor.
+     */
+    static <T> Cursor<T> fromIterator(Iterator<? extends T> it) {
+        return new Cursor<T>() {
+            /** {@inheritDoc} */
+            @Override
+            public void close() throws Exception {
+                // No-op.
+            }
 
+            /** {@inheritDoc} */
+            @Override
+            public Iterator<T> iterator() {
+                return this;
+            }

Review comment:
       I guess, `Cursor extends Iterable` to make Cursor usage easier, e.g. iteration via 'for' loop.
   Does it make sense to make this implementation as a default in the Cursor interface?
   
   I suggest just move method "as is" from static nested class to outer interface.
   




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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #739: IGNITE-16697 MV storage methods and reference implementation

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #739:
URL: https://github.com/apache/ignite-3/pull/739#discussion_r840404039



##########
File path: modules/storage-api/src/test/java/org/apache/ignite/internal/storage/AbstractSortedIndexMvStorageTest.java
##########
@@ -0,0 +1,238 @@
+/*
+ * 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;
+
+import static org.apache.ignite.internal.storage.index.SortedIndexMvStorage.BACKWARDS;
+import static org.apache.ignite.internal.storage.index.SortedIndexMvStorage.FORWARD;
+import static org.apache.ignite.internal.storage.index.SortedIndexMvStorage.GREATER_OR_EQUAL;
+import static org.apache.ignite.internal.storage.index.SortedIndexMvStorage.LESS_OR_EQUAL;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+import java.util.List;
+import java.util.UUID;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+import org.apache.ignite.configuration.schemas.table.ColumnChange;
+import org.apache.ignite.configuration.schemas.table.SortedIndexChange;
+import org.apache.ignite.configuration.schemas.table.SortedIndexConfigurationSchema;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.configuration.testframework.ConfigurationExtension;
+import org.apache.ignite.internal.configuration.testframework.InjectConfiguration;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.schema.marshaller.MarshallerException;
+import org.apache.ignite.internal.schema.row.Row;
+import org.apache.ignite.internal.storage.index.IndexRowPrefix;
+import org.apache.ignite.internal.storage.index.SortedIndexMvStorage;
+import org.apache.ignite.internal.storage.index.SortedIndexMvStorage.IndexRowEx;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.internal.util.Cursor;
+import org.apache.ignite.lang.IgniteException;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+/**
+ * Base test for MV index storages.
+ */
+@ExtendWith(ConfigurationExtension.class)
+public abstract class AbstractSortedIndexMvStorageTest extends BaseMvStoragesTest {
+    protected static final String INDEX1 = "asc_asc";
+    protected static final String INDEX2 = "asc_desc";
+
+    protected TableConfiguration tableCfg;
+
+    @BeforeEach
+    void setUp(@InjectConfiguration(polymorphicExtensions = SortedIndexConfigurationSchema.class) TableConfiguration tableCfg) {
+        tableCfg.change(tableChange -> tableChange
+                .changePartitions(1)
+                .changePrimaryKey(pk -> pk.changeColumns("intKey", "strKey"))
+                .changeColumns(columns -> columns
+                        .create("intKey", column("intKey", "INT32"))
+                        .create("strKey", column("strKey", "STRING"))
+                        .create("intVal", column("intVal", "INT32"))
+                        .create("strVal", column("strVal", "STRING"))
+                )
+                .changeIndices(indexes -> indexes
+                        .create(INDEX1, idx -> idx.convert(SortedIndexChange.class).changeColumns(idxColumns -> idxColumns
+                                .create("strVal", c -> c.changeName("strVal").changeAsc(true))
+                                .create("intVal", c -> c.changeName("intVal").changeAsc(true))
+                        ))
+                        .create(INDEX2, idx -> idx.convert(SortedIndexChange.class).changeColumns(idxColumns -> idxColumns
+                                .create("strVal", c -> c.changeName("strVal").changeAsc(true))
+                                .create("intVal", c -> c.changeName("intVal").changeAsc(false))
+                        ))
+                )
+        ).join();
+
+        this.tableCfg = tableCfg;
+    }
+
+    private static Consumer<ColumnChange> column(String name, String typeName) {
+        return c -> c.changeName(name).changeNullable(false).changeType(type -> type.changeType(typeName));
+    }
+
+    /**
+     * Creates a storage instance for testing.
+     */
+    protected abstract MvPartitionStorage partitionStorage();
+
+    /**
+     * Creates a storage instanc efor testing.
+     */
+    protected abstract SortedIndexMvStorage createIndexStorage(String name, TableView tableCfg);
+
+    @Test
+    public void testEmpty() throws Exception {
+        SortedIndexMvStorage index1 = createIndexStorage(INDEX1, tableCfg.value());
+        SortedIndexMvStorage index2 = createIndexStorage(INDEX2, tableCfg.value());
+
+        assertEquals(List.of(), convert(index1.scan(null, null, (byte) 0, null, null)));
+        assertEquals(List.of(), convert(index2.scan(null, null, (byte) 0, null, null)));
+    }
+
+    @Test
+    public void testBounds() throws Exception {

Review comment:
       But I'll ignore partition filter for now




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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #739: IGNITE-16697 MV storage methods and reference implementation

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #739:
URL: https://github.com/apache/ignite-3/pull/739#discussion_r840407283



##########
File path: modules/storage-api/src/test/java/org/apache/ignite/internal/storage/AbstractMvPartitionStorageTest.java
##########
@@ -0,0 +1,206 @@
+/*
+ * 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;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.UUID;
+import java.util.function.Predicate;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.internal.util.Cursor;
+import org.junit.jupiter.api.Test;
+
+/**
+ * Base test for MV partition storages.
+ */
+public abstract class AbstractMvPartitionStorageTest extends BaseMvStoragesTest {
+    /**
+     * Creates a storage instance for testing.
+     */
+    protected abstract MvPartitionStorage partitionStorage();
+
+    /**
+     * Tests that reads and scan from empty storage return empty results.
+     */
+    @Test
+    public void testEmpty() throws Exception {
+        MvPartitionStorage pk = partitionStorage();
+
+        BinaryRow binaryKey = binaryKey(new TestKey(10, "foo"));
+
+        // Read.
+        assertNull(pk.read(binaryKey, null));
+        assertNull(pk.read(binaryKey, Timestamp.nextVersion()));
+
+        // Scan.
+        assertEquals(List.of(), convert(pk.scan(row -> true, null)));
+        assertEquals(List.of(), convert(pk.scan(row -> true, Timestamp.nextVersion())));
+    }
+
+    /**
+     * Tests basic invariants of {@link MvPartitionStorage#addWrite(BinaryRow, UUID)}.
+     */
+    @Test
+    public void testAddWrite() throws Exception {
+        MvPartitionStorage pk = partitionStorage();
+
+        TestKey key = new TestKey(10, "foo");
+        TestValue value = new TestValue(20, "bar");
+
+        BinaryRow binaryRow = binaryRow(key, value);
+
+        pk.addWrite(binaryRow, UUID.randomUUID());
+
+        // Attempt to write from another transaction.
+        assertThrows(TxIdMismatchException.class, () -> pk.addWrite(binaryRow, UUID.randomUUID()));
+
+        // Read without timestamp returns uncommited row.
+        assertEquals(value, value(pk.read(binaryKey(key), null)));
+
+        // Read with timestamp returns null.
+        assertNull(pk.read(binaryKey(key), Timestamp.nextVersion()));
+    }
+
+    /**
+     * Tests basic invariants of {@link MvPartitionStorage#abortWrite(BinaryRow)}.
+     */
+    @Test
+    public void testAbortWrite() throws Exception {
+        MvPartitionStorage pk = partitionStorage();
+
+        TestKey key = new TestKey(10, "foo");
+        TestValue value = new TestValue(20, "bar");
+
+        pk.addWrite(binaryRow(key, value), UUID.randomUUID());
+
+        pk.abortWrite(binaryKey(key));
+
+        // Aborted row can't be read.
+        assertNull(pk.read(binaryKey(key), null));
+    }
+
+    /**
+     * Tests basic invariants of {@link MvPartitionStorage#commitWrite(BinaryRow, Timestamp)}.
+     */
+    @Test
+    public void testCommitWrite() throws Exception {
+        MvPartitionStorage pk = partitionStorage();
+
+        TestKey key = new TestKey(10, "foo");
+        TestValue value = new TestValue(20, "bar");
+
+        BinaryRow binaryRow = binaryRow(key, value);
+
+        pk.addWrite(binaryRow, UUID.randomUUID());
+
+        Timestamp tsBefore = Timestamp.nextVersion();
+
+        Timestamp tsExact = Timestamp.nextVersion();
+        pk.commitWrite(binaryRow, tsExact);
+
+        Timestamp tsAfter = Timestamp.nextVersion();
+
+        // Row is invisible at the time before writing.
+        assertNull(pk.read(binaryRow, tsBefore));
+
+        // Row is valid at the time during and after writing.
+        assertEquals(value, value(pk.read(binaryRow, null)));
+        assertEquals(value, value(pk.read(binaryRow, tsExact)));
+        assertEquals(value, value(pk.read(binaryRow, tsAfter)));
+
+        TestValue newValue = new TestValue(30, "duh");
+
+        pk.addWrite(binaryRow(key, newValue), UUID.randomUUID());
+
+        // Same checks, but now there are two different versions.
+        assertNull(pk.read(binaryRow, tsBefore));
+
+        assertEquals(newValue, value(pk.read(binaryRow, null)));
+
+        assertEquals(value, value(pk.read(binaryRow, tsExact)));
+        assertEquals(value, value(pk.read(binaryRow, tsAfter)));
+        assertEquals(value, value(pk.read(binaryRow, Timestamp.nextVersion())));
+
+        pk.commitWrite(binaryKey(key), Timestamp.nextVersion());
+
+        assertEquals(newValue, value(pk.read(binaryRow, null)));

Review comment:
       There is a test for that




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



[GitHub] [ignite-3] tledkov-gridgain commented on a change in pull request #739: IGNITE-16697 MV storage methods and reference implementation

Posted by GitBox <gi...@apache.org>.
tledkov-gridgain commented on a change in pull request #739:
URL: https://github.com/apache/ignite-3/pull/739#discussion_r833051149



##########
File path: modules/storage-api/src/main/java/org/apache/ignite/internal/storage/PartitionStorage.java
##########
@@ -128,6 +143,66 @@
      */
     Cursor<DataRow> scan(Predicate<SearchRow> filter) throws StorageException;
 
+    /**
+     * Scans the partition and returns a cursor of values in at the given timestamp.
+     *
+     * @param keyFilter Key filter. Binary rows passed to the filter may or may not have a value, filter should only check keys.
+     * @param timestamp Timestamp
+     * @return Cursor.
+     */
+    default Cursor<BinaryRow> scan(Predicate<BinaryRow> keyFilter, @Nullable Timestamp timestamp) {
+        throw new UnsupportedOperationException("scan");
+    }
+
+    /**
+     * Exception class that describes the situation where two independant transactions attempting to write values for the same key.
+     */
+    class TxIdMismatchException extends RuntimeException {
+    }

Review comment:
       Why does not inherit `IgniteException` or `IgniteInternalException`?




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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #739: IGNITE-16697 MV storage methods and reference implementation

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #739:
URL: https://github.com/apache/ignite-3/pull/739#discussion_r840407098



##########
File path: modules/storage-api/src/test/java/org/apache/ignite/internal/storage/AbstractMvPartitionStorageTest.java
##########
@@ -0,0 +1,206 @@
+/*
+ * 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;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.UUID;
+import java.util.function.Predicate;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.internal.util.Cursor;
+import org.junit.jupiter.api.Test;
+
+/**
+ * Base test for MV partition storages.
+ */
+public abstract class AbstractMvPartitionStorageTest extends BaseMvStoragesTest {
+    /**
+     * Creates a storage instance for testing.
+     */
+    protected abstract MvPartitionStorage partitionStorage();
+
+    /**
+     * Tests that reads and scan from empty storage return empty results.
+     */
+    @Test
+    public void testEmpty() throws Exception {
+        MvPartitionStorage pk = partitionStorage();
+
+        BinaryRow binaryKey = binaryKey(new TestKey(10, "foo"));
+
+        // Read.
+        assertNull(pk.read(binaryKey, null));
+        assertNull(pk.read(binaryKey, Timestamp.nextVersion()));
+
+        // Scan.
+        assertEquals(List.of(), convert(pk.scan(row -> true, null)));
+        assertEquals(List.of(), convert(pk.scan(row -> true, Timestamp.nextVersion())));
+    }
+
+    /**
+     * Tests basic invariants of {@link MvPartitionStorage#addWrite(BinaryRow, UUID)}.
+     */
+    @Test
+    public void testAddWrite() throws Exception {
+        MvPartitionStorage pk = partitionStorage();
+
+        TestKey key = new TestKey(10, "foo");
+        TestValue value = new TestValue(20, "bar");
+
+        BinaryRow binaryRow = binaryRow(key, value);
+
+        pk.addWrite(binaryRow, UUID.randomUUID());
+
+        // Attempt to write from another transaction.
+        assertThrows(TxIdMismatchException.class, () -> pk.addWrite(binaryRow, UUID.randomUUID()));
+
+        // Read without timestamp returns uncommited row.
+        assertEquals(value, value(pk.read(binaryKey(key), null)));
+
+        // Read with timestamp returns null.
+        assertNull(pk.read(binaryKey(key), Timestamp.nextVersion()));
+    }
+
+    /**
+     * Tests basic invariants of {@link MvPartitionStorage#abortWrite(BinaryRow)}.
+     */
+    @Test
+    public void testAbortWrite() throws Exception {

Review comment:
       It should be tested in tests for indexes




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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #739: IGNITE-16697 MV storage methods and reference implementation

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #739:
URL: https://github.com/apache/ignite-3/pull/739#discussion_r840405743



##########
File path: modules/storage-api/src/test/java/org/apache/ignite/internal/storage/AbstractMvPartitionStorageTest.java
##########
@@ -0,0 +1,206 @@
+/*
+ * 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;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.UUID;
+import java.util.function.Predicate;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.internal.util.Cursor;
+import org.junit.jupiter.api.Test;
+
+/**
+ * Base test for MV partition storages.
+ */
+public abstract class AbstractMvPartitionStorageTest extends BaseMvStoragesTest {
+    /**
+     * Creates a storage instance for testing.
+     */
+    protected abstract MvPartitionStorage partitionStorage();
+
+    /**
+     * Tests that reads and scan from empty storage return empty results.
+     */
+    @Test
+    public void testEmpty() throws Exception {

Review comment:
       Why? It's, like, 6 lines of code




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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #739: IGNITE-16697 MV storage methods and reference implementation

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #739:
URL: https://github.com/apache/ignite-3/pull/739#discussion_r840404166



##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -138,6 +139,7 @@ public void start() throws StorageException {
         DBOptions dbOptions = new DBOptions()
                 .setCreateIfMissing(true)
                 .setCreateMissingColumnFamilies(true)
+                .setWalRecoveryMode(WALRecoveryMode.SkipAnyCorruptedRecords)

Review comment:
       Oopsie




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



[GitHub] [ignite-3] sashapolo commented on a change in pull request #739: IGNITE-16697 MV storage methods and reference implementation

Posted by GitBox <gi...@apache.org>.
sashapolo commented on a change in pull request #739:
URL: https://github.com/apache/ignite-3/pull/739#discussion_r840320473



##########
File path: modules/storage-api/src/test/java/org/apache/ignite/internal/storage/AbstractSortedIndexMvStorageTest.java
##########
@@ -0,0 +1,238 @@
+/*
+ * 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;
+
+import static org.apache.ignite.internal.storage.index.SortedIndexMvStorage.BACKWARDS;
+import static org.apache.ignite.internal.storage.index.SortedIndexMvStorage.FORWARD;
+import static org.apache.ignite.internal.storage.index.SortedIndexMvStorage.GREATER_OR_EQUAL;
+import static org.apache.ignite.internal.storage.index.SortedIndexMvStorage.LESS_OR_EQUAL;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+import java.util.List;
+import java.util.UUID;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+import org.apache.ignite.configuration.schemas.table.ColumnChange;
+import org.apache.ignite.configuration.schemas.table.SortedIndexChange;
+import org.apache.ignite.configuration.schemas.table.SortedIndexConfigurationSchema;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.configuration.testframework.ConfigurationExtension;
+import org.apache.ignite.internal.configuration.testframework.InjectConfiguration;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.schema.marshaller.MarshallerException;
+import org.apache.ignite.internal.schema.row.Row;
+import org.apache.ignite.internal.storage.index.IndexRowPrefix;
+import org.apache.ignite.internal.storage.index.SortedIndexMvStorage;
+import org.apache.ignite.internal.storage.index.SortedIndexMvStorage.IndexRowEx;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.internal.util.Cursor;
+import org.apache.ignite.lang.IgniteException;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+/**
+ * Base test for MV index storages.
+ */
+@ExtendWith(ConfigurationExtension.class)
+public abstract class AbstractSortedIndexMvStorageTest extends BaseMvStoragesTest {
+    protected static final String INDEX1 = "asc_asc";
+    protected static final String INDEX2 = "asc_desc";
+
+    protected TableConfiguration tableCfg;
+
+    @BeforeEach
+    void setUp(@InjectConfiguration(polymorphicExtensions = SortedIndexConfigurationSchema.class) TableConfiguration tableCfg) {
+        tableCfg.change(tableChange -> tableChange
+                .changePartitions(1)
+                .changePrimaryKey(pk -> pk.changeColumns("intKey", "strKey"))
+                .changeColumns(columns -> columns
+                        .create("intKey", column("intKey", "INT32"))
+                        .create("strKey", column("strKey", "STRING"))
+                        .create("intVal", column("intVal", "INT32"))
+                        .create("strVal", column("strVal", "STRING"))
+                )
+                .changeIndices(indexes -> indexes
+                        .create(INDEX1, idx -> idx.convert(SortedIndexChange.class).changeColumns(idxColumns -> idxColumns
+                                .create("strVal", c -> c.changeName("strVal").changeAsc(true))
+                                .create("intVal", c -> c.changeName("intVal").changeAsc(true))
+                        ))
+                        .create(INDEX2, idx -> idx.convert(SortedIndexChange.class).changeColumns(idxColumns -> idxColumns
+                                .create("strVal", c -> c.changeName("strVal").changeAsc(true))
+                                .create("intVal", c -> c.changeName("intVal").changeAsc(false))
+                        ))
+                )
+        ).join();
+
+        this.tableCfg = tableCfg;
+    }
+
+    private static Consumer<ColumnChange> column(String name, String typeName) {
+        return c -> c.changeName(name).changeNullable(false).changeType(type -> type.changeType(typeName));
+    }
+
+    /**
+     * Creates a storage instance for testing.
+     */
+    protected abstract MvPartitionStorage partitionStorage();
+
+    /**
+     * Creates a storage instanc efor testing.
+     */
+    protected abstract SortedIndexMvStorage createIndexStorage(String name, TableView tableCfg);
+
+    @Test
+    public void testEmpty() throws Exception {
+        SortedIndexMvStorage index1 = createIndexStorage(INDEX1, tableCfg.value());
+        SortedIndexMvStorage index2 = createIndexStorage(INDEX2, tableCfg.value());
+
+        assertEquals(List.of(), convert(index1.scan(null, null, (byte) 0, null, null)));
+        assertEquals(List.of(), convert(index2.scan(null, null, (byte) 0, null, null)));
+    }
+
+    @Test
+    public void testBounds() throws Exception {
+        SortedIndexMvStorage index1 = createIndexStorage(INDEX1, tableCfg.value());
+        SortedIndexMvStorage index2 = createIndexStorage(INDEX2, tableCfg.value());
+
+        TestValue val9010 = new TestValue(90, "10");
+        TestValue val8010 = new TestValue(80, "10");
+        TestValue val9020 = new TestValue(90, "20");
+        TestValue val8020 = new TestValue(80, "20");
+
+        insert(new TestKey(1, "1"), val9010, null);
+        insert(new TestKey(2, "2"), val8010, null);
+        insert(new TestKey(3, "3"), val9020, null);
+        insert(new TestKey(4, "4"), val8020, null);
+
+        // Test without bounds.
+        assertEquals(List.of(val8010, val9010, val8020, val9020), convert(index1.scan(
+                null, null, FORWARD, null, null
+        )));
+
+        assertEquals(List.of(val9020, val8020, val9010, val8010), convert(index1.scan(
+                null, null, BACKWARDS, null, null
+        )));
+
+        assertEquals(List.of(val9010, val8010, val9020, val8020), convert(index2.scan(
+                null, null, FORWARD, null, null
+        )));
+
+        assertEquals(List.of(val8020, val9020, val8010, val9010), convert(index2.scan(
+                null, null, BACKWARDS, null, null
+        )));
+
+        // Lower bound exclusive.
+        assertEquals(List.of(val8020, val9020), convert(index1.scan(
+                prefix("10"), null, FORWARD, null, null
+        )));
+
+        assertEquals(List.of(val9020, val8020), convert(index1.scan(
+                prefix("10"), null, BACKWARDS, null, null
+        )));
+
+        assertEquals(List.of(val9020, val8020), convert(index2.scan(
+                prefix("10"), null, FORWARD, null, null
+        )));
+
+        assertEquals(List.of(val8020, val9020), convert(index2.scan(
+                prefix("10"), null, BACKWARDS, null, null
+        )));
+
+        // Lower bound inclusive.
+        assertEquals(List.of(val8010, val9010, val8020, val9020), convert(index1.scan(
+                prefix("10"), null, GREATER_OR_EQUAL | FORWARD, null, null
+        )));
+
+        assertEquals(List.of(val9020, val8020, val9010, val8010), convert(index1.scan(
+                prefix("10"), null, GREATER_OR_EQUAL | BACKWARDS, null, null
+        )));
+
+        assertEquals(List.of(val9010, val8010, val9020, val8020), convert(index2.scan(
+                prefix("10"), null, GREATER_OR_EQUAL | FORWARD, null, null
+        )));
+
+        assertEquals(List.of(val8020, val9020, val8010, val9010), convert(index2.scan(
+                prefix("10"), null, GREATER_OR_EQUAL | BACKWARDS, null, null
+        )));
+
+        // Upper bound exclusive.
+        assertEquals(List.of(val8010, val9010), convert(index1.scan(
+                null, prefix("20"), FORWARD, null, null
+        )));
+
+        assertEquals(List.of(val9010, val8010), convert(index1.scan(
+                null, prefix("20"), BACKWARDS, null, null
+        )));
+
+        assertEquals(List.of(val9010, val8010), convert(index2.scan(
+                null, prefix("20"), FORWARD, null, null
+        )));
+
+        assertEquals(List.of(val8010, val9010), convert(index2.scan(
+                null, prefix("20"), BACKWARDS, null, null
+        )));
+
+        // Lower bound inclusive.

Review comment:
       Don't you test upper inclusive bound here?

##########
File path: modules/storage-api/src/test/java/org/apache/ignite/internal/storage/AbstractSortedIndexMvStorageTest.java
##########
@@ -0,0 +1,238 @@
+/*
+ * 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;
+
+import static org.apache.ignite.internal.storage.index.SortedIndexMvStorage.BACKWARDS;
+import static org.apache.ignite.internal.storage.index.SortedIndexMvStorage.FORWARD;
+import static org.apache.ignite.internal.storage.index.SortedIndexMvStorage.GREATER_OR_EQUAL;
+import static org.apache.ignite.internal.storage.index.SortedIndexMvStorage.LESS_OR_EQUAL;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+import java.util.List;
+import java.util.UUID;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+import org.apache.ignite.configuration.schemas.table.ColumnChange;
+import org.apache.ignite.configuration.schemas.table.SortedIndexChange;
+import org.apache.ignite.configuration.schemas.table.SortedIndexConfigurationSchema;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.configuration.testframework.ConfigurationExtension;
+import org.apache.ignite.internal.configuration.testframework.InjectConfiguration;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.schema.marshaller.MarshallerException;
+import org.apache.ignite.internal.schema.row.Row;
+import org.apache.ignite.internal.storage.index.IndexRowPrefix;
+import org.apache.ignite.internal.storage.index.SortedIndexMvStorage;
+import org.apache.ignite.internal.storage.index.SortedIndexMvStorage.IndexRowEx;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.internal.util.Cursor;
+import org.apache.ignite.lang.IgniteException;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+/**
+ * Base test for MV index storages.
+ */
+@ExtendWith(ConfigurationExtension.class)
+public abstract class AbstractSortedIndexMvStorageTest extends BaseMvStoragesTest {
+    protected static final String INDEX1 = "asc_asc";
+    protected static final String INDEX2 = "asc_desc";
+
+    protected TableConfiguration tableCfg;
+
+    @BeforeEach
+    void setUp(@InjectConfiguration(polymorphicExtensions = SortedIndexConfigurationSchema.class) TableConfiguration tableCfg) {
+        tableCfg.change(tableChange -> tableChange
+                .changePartitions(1)
+                .changePrimaryKey(pk -> pk.changeColumns("intKey", "strKey"))
+                .changeColumns(columns -> columns
+                        .create("intKey", column("intKey", "INT32"))
+                        .create("strKey", column("strKey", "STRING"))
+                        .create("intVal", column("intVal", "INT32"))
+                        .create("strVal", column("strVal", "STRING"))
+                )
+                .changeIndices(indexes -> indexes
+                        .create(INDEX1, idx -> idx.convert(SortedIndexChange.class).changeColumns(idxColumns -> idxColumns
+                                .create("strVal", c -> c.changeName("strVal").changeAsc(true))
+                                .create("intVal", c -> c.changeName("intVal").changeAsc(true))
+                        ))
+                        .create(INDEX2, idx -> idx.convert(SortedIndexChange.class).changeColumns(idxColumns -> idxColumns
+                                .create("strVal", c -> c.changeName("strVal").changeAsc(true))
+                                .create("intVal", c -> c.changeName("intVal").changeAsc(false))
+                        ))
+                )
+        ).join();
+
+        this.tableCfg = tableCfg;
+    }
+
+    private static Consumer<ColumnChange> column(String name, String typeName) {
+        return c -> c.changeName(name).changeNullable(false).changeType(type -> type.changeType(typeName));
+    }
+
+    /**
+     * Creates a storage instance for testing.
+     */
+    protected abstract MvPartitionStorage partitionStorage();
+
+    /**
+     * Creates a storage instanc efor testing.
+     */
+    protected abstract SortedIndexMvStorage createIndexStorage(String name, TableView tableCfg);
+
+    @Test
+    public void testEmpty() throws Exception {
+        SortedIndexMvStorage index1 = createIndexStorage(INDEX1, tableCfg.value());
+        SortedIndexMvStorage index2 = createIndexStorage(INDEX2, tableCfg.value());
+
+        assertEquals(List.of(), convert(index1.scan(null, null, (byte) 0, null, null)));
+        assertEquals(List.of(), convert(index2.scan(null, null, (byte) 0, null, null)));
+    }
+
+    @Test
+    public void testBounds() throws Exception {
+        SortedIndexMvStorage index1 = createIndexStorage(INDEX1, tableCfg.value());
+        SortedIndexMvStorage index2 = createIndexStorage(INDEX2, tableCfg.value());
+
+        TestValue val9010 = new TestValue(90, "10");
+        TestValue val8010 = new TestValue(80, "10");
+        TestValue val9020 = new TestValue(90, "20");
+        TestValue val8020 = new TestValue(80, "20");
+
+        insert(new TestKey(1, "1"), val9010, null);
+        insert(new TestKey(2, "2"), val8010, null);
+        insert(new TestKey(3, "3"), val9020, null);
+        insert(new TestKey(4, "4"), val8020, null);
+
+        // Test without bounds.
+        assertEquals(List.of(val8010, val9010, val8020, val9020), convert(index1.scan(
+                null, null, FORWARD, null, null
+        )));
+
+        assertEquals(List.of(val9020, val8020, val9010, val8010), convert(index1.scan(
+                null, null, BACKWARDS, null, null
+        )));
+
+        assertEquals(List.of(val9010, val8010, val9020, val8020), convert(index2.scan(
+                null, null, FORWARD, null, null
+        )));
+
+        assertEquals(List.of(val8020, val9020, val8010, val9010), convert(index2.scan(
+                null, null, BACKWARDS, null, null
+        )));
+
+        // Lower bound exclusive.
+        assertEquals(List.of(val8020, val9020), convert(index1.scan(
+                prefix("10"), null, FORWARD, null, null
+        )));
+
+        assertEquals(List.of(val9020, val8020), convert(index1.scan(
+                prefix("10"), null, BACKWARDS, null, null
+        )));
+
+        assertEquals(List.of(val9020, val8020), convert(index2.scan(
+                prefix("10"), null, FORWARD, null, null
+        )));
+
+        assertEquals(List.of(val8020, val9020), convert(index2.scan(
+                prefix("10"), null, BACKWARDS, null, null
+        )));
+
+        // Lower bound inclusive.
+        assertEquals(List.of(val8010, val9010, val8020, val9020), convert(index1.scan(
+                prefix("10"), null, GREATER_OR_EQUAL | FORWARD, null, null
+        )));
+
+        assertEquals(List.of(val9020, val8020, val9010, val8010), convert(index1.scan(
+                prefix("10"), null, GREATER_OR_EQUAL | BACKWARDS, null, null
+        )));
+
+        assertEquals(List.of(val9010, val8010, val9020, val8020), convert(index2.scan(
+                prefix("10"), null, GREATER_OR_EQUAL | FORWARD, null, null
+        )));
+
+        assertEquals(List.of(val8020, val9020, val8010, val9010), convert(index2.scan(
+                prefix("10"), null, GREATER_OR_EQUAL | BACKWARDS, null, null
+        )));
+
+        // Upper bound exclusive.
+        assertEquals(List.of(val8010, val9010), convert(index1.scan(
+                null, prefix("20"), FORWARD, null, null
+        )));
+
+        assertEquals(List.of(val9010, val8010), convert(index1.scan(
+                null, prefix("20"), BACKWARDS, null, null
+        )));
+
+        assertEquals(List.of(val9010, val8010), convert(index2.scan(
+                null, prefix("20"), FORWARD, null, null
+        )));
+
+        assertEquals(List.of(val8010, val9010), convert(index2.scan(
+                null, prefix("20"), BACKWARDS, null, null
+        )));
+
+        // Lower bound inclusive.
+        assertEquals(List.of(val8010, val9010, val8020, val9020), convert(index1.scan(
+                null, prefix("20"), LESS_OR_EQUAL | FORWARD, null, null
+        )));
+
+        assertEquals(List.of(val9020, val8020, val9010, val8010), convert(index1.scan(
+                null, prefix("20"), LESS_OR_EQUAL | BACKWARDS, null, null
+        )));
+
+        assertEquals(List.of(val9010, val8010, val9020, val8020), convert(index2.scan(
+                null, prefix("20"), LESS_OR_EQUAL | FORWARD, null, null
+        )));
+
+        assertEquals(List.of(val8020, val9020, val8010, val9010), convert(index2.scan(
+                null, prefix("20"), LESS_OR_EQUAL | BACKWARDS, null, null
+        )));
+    }
+
+    protected void insert(TestKey key, TestValue value, Timestamp ts) {
+        MvPartitionStorage pk = partitionStorage();
+
+        BinaryRow binaryRow = binaryRow(key, value);
+
+        pk.addWrite(binaryRow, UUID.randomUUID());
+
+        pk.commitWrite(binaryRow, ts == null ? Timestamp.nextVersion() : ts);
+    }
+
+    protected IndexRowPrefix prefix(String val) {
+        return () -> new Object[]{val};
+    }
+
+    protected List<TestValue> convert(Cursor<IndexRowEx> cursor) throws Exception {
+        try (cursor) {
+            return StreamSupport.stream(cursor.spliterator(), false)

Review comment:
       I suggest adding a `default Stream stream()` method to the `Cursor` interface. I've already done this in my other PR, and it proved to be useful

##########
File path: modules/storage-api/src/test/java/org/apache/ignite/internal/storage/AbstractMvPartitionStorageTest.java
##########
@@ -0,0 +1,206 @@
+/*
+ * 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;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.UUID;
+import java.util.function.Predicate;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.internal.util.Cursor;
+import org.junit.jupiter.api.Test;
+
+/**
+ * Base test for MV partition storages.
+ */
+public abstract class AbstractMvPartitionStorageTest extends BaseMvStoragesTest {
+    /**
+     * Creates a storage instance for testing.
+     */
+    protected abstract MvPartitionStorage partitionStorage();
+
+    /**
+     * Tests that reads and scan from empty storage return empty results.
+     */
+    @Test
+    public void testEmpty() throws Exception {
+        MvPartitionStorage pk = partitionStorage();
+
+        BinaryRow binaryKey = binaryKey(new TestKey(10, "foo"));
+
+        // Read.
+        assertNull(pk.read(binaryKey, null));
+        assertNull(pk.read(binaryKey, Timestamp.nextVersion()));
+
+        // Scan.
+        assertEquals(List.of(), convert(pk.scan(row -> true, null)));
+        assertEquals(List.of(), convert(pk.scan(row -> true, Timestamp.nextVersion())));
+    }
+
+    /**
+     * Tests basic invariants of {@link MvPartitionStorage#addWrite(BinaryRow, UUID)}.
+     */
+    @Test
+    public void testAddWrite() throws Exception {
+        MvPartitionStorage pk = partitionStorage();
+
+        TestKey key = new TestKey(10, "foo");
+        TestValue value = new TestValue(20, "bar");
+
+        BinaryRow binaryRow = binaryRow(key, value);
+
+        pk.addWrite(binaryRow, UUID.randomUUID());
+
+        // Attempt to write from another transaction.
+        assertThrows(TxIdMismatchException.class, () -> pk.addWrite(binaryRow, UUID.randomUUID()));
+
+        // Read without timestamp returns uncommited row.
+        assertEquals(value, value(pk.read(binaryKey(key), null)));
+
+        // Read with timestamp returns null.
+        assertNull(pk.read(binaryKey(key), Timestamp.nextVersion()));

Review comment:
       addWrite also influences the index, that is not tested

##########
File path: modules/storage-api/src/test/java/org/apache/ignite/internal/storage/AbstractSortedIndexMvStorageTest.java
##########
@@ -0,0 +1,238 @@
+/*
+ * 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;
+
+import static org.apache.ignite.internal.storage.index.SortedIndexMvStorage.BACKWARDS;
+import static org.apache.ignite.internal.storage.index.SortedIndexMvStorage.FORWARD;
+import static org.apache.ignite.internal.storage.index.SortedIndexMvStorage.GREATER_OR_EQUAL;
+import static org.apache.ignite.internal.storage.index.SortedIndexMvStorage.LESS_OR_EQUAL;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+import java.util.List;
+import java.util.UUID;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+import org.apache.ignite.configuration.schemas.table.ColumnChange;
+import org.apache.ignite.configuration.schemas.table.SortedIndexChange;
+import org.apache.ignite.configuration.schemas.table.SortedIndexConfigurationSchema;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.configuration.testframework.ConfigurationExtension;
+import org.apache.ignite.internal.configuration.testframework.InjectConfiguration;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.schema.marshaller.MarshallerException;
+import org.apache.ignite.internal.schema.row.Row;
+import org.apache.ignite.internal.storage.index.IndexRowPrefix;
+import org.apache.ignite.internal.storage.index.SortedIndexMvStorage;
+import org.apache.ignite.internal.storage.index.SortedIndexMvStorage.IndexRowEx;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.internal.util.Cursor;
+import org.apache.ignite.lang.IgniteException;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+/**
+ * Base test for MV index storages.
+ */
+@ExtendWith(ConfigurationExtension.class)
+public abstract class AbstractSortedIndexMvStorageTest extends BaseMvStoragesTest {
+    protected static final String INDEX1 = "asc_asc";
+    protected static final String INDEX2 = "asc_desc";
+
+    protected TableConfiguration tableCfg;
+
+    @BeforeEach
+    void setUp(@InjectConfiguration(polymorphicExtensions = SortedIndexConfigurationSchema.class) TableConfiguration tableCfg) {
+        tableCfg.change(tableChange -> tableChange
+                .changePartitions(1)
+                .changePrimaryKey(pk -> pk.changeColumns("intKey", "strKey"))
+                .changeColumns(columns -> columns
+                        .create("intKey", column("intKey", "INT32"))
+                        .create("strKey", column("strKey", "STRING"))
+                        .create("intVal", column("intVal", "INT32"))
+                        .create("strVal", column("strVal", "STRING"))
+                )
+                .changeIndices(indexes -> indexes
+                        .create(INDEX1, idx -> idx.convert(SortedIndexChange.class).changeColumns(idxColumns -> idxColumns
+                                .create("strVal", c -> c.changeName("strVal").changeAsc(true))
+                                .create("intVal", c -> c.changeName("intVal").changeAsc(true))
+                        ))
+                        .create(INDEX2, idx -> idx.convert(SortedIndexChange.class).changeColumns(idxColumns -> idxColumns
+                                .create("strVal", c -> c.changeName("strVal").changeAsc(true))
+                                .create("intVal", c -> c.changeName("intVal").changeAsc(false))
+                        ))
+                )
+        ).join();
+
+        this.tableCfg = tableCfg;
+    }
+
+    private static Consumer<ColumnChange> column(String name, String typeName) {
+        return c -> c.changeName(name).changeNullable(false).changeType(type -> type.changeType(typeName));
+    }
+
+    /**
+     * Creates a storage instance for testing.
+     */
+    protected abstract MvPartitionStorage partitionStorage();
+
+    /**
+     * Creates a storage instanc efor testing.
+     */
+    protected abstract SortedIndexMvStorage createIndexStorage(String name, TableView tableCfg);
+
+    @Test
+    public void testEmpty() throws Exception {
+        SortedIndexMvStorage index1 = createIndexStorage(INDEX1, tableCfg.value());
+        SortedIndexMvStorage index2 = createIndexStorage(INDEX2, tableCfg.value());

Review comment:
       why do you need to indices here?

##########
File path: modules/storage-api/src/test/java/org/apache/ignite/internal/storage/AbstractSortedIndexMvStorageTest.java
##########
@@ -0,0 +1,238 @@
+/*
+ * 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;
+
+import static org.apache.ignite.internal.storage.index.SortedIndexMvStorage.BACKWARDS;
+import static org.apache.ignite.internal.storage.index.SortedIndexMvStorage.FORWARD;
+import static org.apache.ignite.internal.storage.index.SortedIndexMvStorage.GREATER_OR_EQUAL;
+import static org.apache.ignite.internal.storage.index.SortedIndexMvStorage.LESS_OR_EQUAL;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+import java.util.List;
+import java.util.UUID;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+import org.apache.ignite.configuration.schemas.table.ColumnChange;
+import org.apache.ignite.configuration.schemas.table.SortedIndexChange;
+import org.apache.ignite.configuration.schemas.table.SortedIndexConfigurationSchema;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.configuration.testframework.ConfigurationExtension;
+import org.apache.ignite.internal.configuration.testframework.InjectConfiguration;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.schema.marshaller.MarshallerException;
+import org.apache.ignite.internal.schema.row.Row;
+import org.apache.ignite.internal.storage.index.IndexRowPrefix;
+import org.apache.ignite.internal.storage.index.SortedIndexMvStorage;
+import org.apache.ignite.internal.storage.index.SortedIndexMvStorage.IndexRowEx;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.internal.util.Cursor;
+import org.apache.ignite.lang.IgniteException;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+/**
+ * Base test for MV index storages.
+ */
+@ExtendWith(ConfigurationExtension.class)
+public abstract class AbstractSortedIndexMvStorageTest extends BaseMvStoragesTest {
+    protected static final String INDEX1 = "asc_asc";
+    protected static final String INDEX2 = "asc_desc";
+
+    protected TableConfiguration tableCfg;
+
+    @BeforeEach
+    void setUp(@InjectConfiguration(polymorphicExtensions = SortedIndexConfigurationSchema.class) TableConfiguration tableCfg) {
+        tableCfg.change(tableChange -> tableChange
+                .changePartitions(1)
+                .changePrimaryKey(pk -> pk.changeColumns("intKey", "strKey"))
+                .changeColumns(columns -> columns
+                        .create("intKey", column("intKey", "INT32"))
+                        .create("strKey", column("strKey", "STRING"))
+                        .create("intVal", column("intVal", "INT32"))
+                        .create("strVal", column("strVal", "STRING"))
+                )
+                .changeIndices(indexes -> indexes
+                        .create(INDEX1, idx -> idx.convert(SortedIndexChange.class).changeColumns(idxColumns -> idxColumns
+                                .create("strVal", c -> c.changeName("strVal").changeAsc(true))
+                                .create("intVal", c -> c.changeName("intVal").changeAsc(true))
+                        ))
+                        .create(INDEX2, idx -> idx.convert(SortedIndexChange.class).changeColumns(idxColumns -> idxColumns
+                                .create("strVal", c -> c.changeName("strVal").changeAsc(true))
+                                .create("intVal", c -> c.changeName("intVal").changeAsc(false))
+                        ))
+                )
+        ).join();
+
+        this.tableCfg = tableCfg;
+    }
+
+    private static Consumer<ColumnChange> column(String name, String typeName) {
+        return c -> c.changeName(name).changeNullable(false).changeType(type -> type.changeType(typeName));
+    }
+
+    /**
+     * Creates a storage instance for testing.
+     */
+    protected abstract MvPartitionStorage partitionStorage();
+
+    /**
+     * Creates a storage instanc efor testing.
+     */
+    protected abstract SortedIndexMvStorage createIndexStorage(String name, TableView tableCfg);
+
+    @Test
+    public void testEmpty() throws Exception {
+        SortedIndexMvStorage index1 = createIndexStorage(INDEX1, tableCfg.value());
+        SortedIndexMvStorage index2 = createIndexStorage(INDEX2, tableCfg.value());
+
+        assertEquals(List.of(), convert(index1.scan(null, null, (byte) 0, null, null)));
+        assertEquals(List.of(), convert(index2.scan(null, null, (byte) 0, null, null)));
+    }
+
+    @Test
+    public void testBounds() throws Exception {

Review comment:
       There are no tests  for the following scenarios:
   1. Non null timestamp 
   2. Non null partition filter
   3. Test that index scan returns correct values as described in the design document's example

##########
File path: modules/storage-api/src/test/java/org/apache/ignite/internal/storage/AbstractMvPartitionStorageTest.java
##########
@@ -0,0 +1,206 @@
+/*
+ * 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;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.UUID;
+import java.util.function.Predicate;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.internal.util.Cursor;
+import org.junit.jupiter.api.Test;
+
+/**
+ * Base test for MV partition storages.
+ */
+public abstract class AbstractMvPartitionStorageTest extends BaseMvStoragesTest {
+    /**
+     * Creates a storage instance for testing.
+     */
+    protected abstract MvPartitionStorage partitionStorage();
+
+    /**
+     * Tests that reads and scan from empty storage return empty results.
+     */
+    @Test
+    public void testEmpty() throws Exception {

Review comment:
       I would suggest to split this test into two

##########
File path: modules/storage-api/src/test/java/org/apache/ignite/internal/storage/AbstractMvPartitionStorageTest.java
##########
@@ -0,0 +1,206 @@
+/*
+ * 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;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.UUID;
+import java.util.function.Predicate;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.internal.util.Cursor;
+import org.junit.jupiter.api.Test;
+
+/**
+ * Base test for MV partition storages.
+ */
+public abstract class AbstractMvPartitionStorageTest extends BaseMvStoragesTest {
+    /**
+     * Creates a storage instance for testing.
+     */
+    protected abstract MvPartitionStorage partitionStorage();
+
+    /**
+     * Tests that reads and scan from empty storage return empty results.
+     */
+    @Test
+    public void testEmpty() throws Exception {
+        MvPartitionStorage pk = partitionStorage();
+
+        BinaryRow binaryKey = binaryKey(new TestKey(10, "foo"));
+
+        // Read.
+        assertNull(pk.read(binaryKey, null));
+        assertNull(pk.read(binaryKey, Timestamp.nextVersion()));
+
+        // Scan.
+        assertEquals(List.of(), convert(pk.scan(row -> true, null)));
+        assertEquals(List.of(), convert(pk.scan(row -> true, Timestamp.nextVersion())));
+    }
+
+    /**
+     * Tests basic invariants of {@link MvPartitionStorage#addWrite(BinaryRow, UUID)}.
+     */
+    @Test
+    public void testAddWrite() throws Exception {
+        MvPartitionStorage pk = partitionStorage();
+
+        TestKey key = new TestKey(10, "foo");
+        TestValue value = new TestValue(20, "bar");
+
+        BinaryRow binaryRow = binaryRow(key, value);
+
+        pk.addWrite(binaryRow, UUID.randomUUID());
+
+        // Attempt to write from another transaction.
+        assertThrows(TxIdMismatchException.class, () -> pk.addWrite(binaryRow, UUID.randomUUID()));
+
+        // Read without timestamp returns uncommited row.
+        assertEquals(value, value(pk.read(binaryKey(key), null)));
+
+        // Read with timestamp returns null.
+        assertNull(pk.read(binaryKey(key), Timestamp.nextVersion()));
+    }
+
+    /**
+     * Tests basic invariants of {@link MvPartitionStorage#abortWrite(BinaryRow)}.
+     */
+    @Test
+    public void testAbortWrite() throws Exception {
+        MvPartitionStorage pk = partitionStorage();
+
+        TestKey key = new TestKey(10, "foo");
+        TestValue value = new TestValue(20, "bar");
+
+        pk.addWrite(binaryRow(key, value), UUID.randomUUID());
+
+        pk.abortWrite(binaryKey(key));
+
+        // Aborted row can't be read.
+        assertNull(pk.read(binaryKey(key), null));
+    }
+
+    /**
+     * Tests basic invariants of {@link MvPartitionStorage#commitWrite(BinaryRow, Timestamp)}.
+     */
+    @Test
+    public void testCommitWrite() throws Exception {
+        MvPartitionStorage pk = partitionStorage();
+
+        TestKey key = new TestKey(10, "foo");
+        TestValue value = new TestValue(20, "bar");
+
+        BinaryRow binaryRow = binaryRow(key, value);
+
+        pk.addWrite(binaryRow, UUID.randomUUID());
+
+        Timestamp tsBefore = Timestamp.nextVersion();
+
+        Timestamp tsExact = Timestamp.nextVersion();
+        pk.commitWrite(binaryRow, tsExact);
+
+        Timestamp tsAfter = Timestamp.nextVersion();
+
+        // Row is invisible at the time before writing.
+        assertNull(pk.read(binaryRow, tsBefore));
+
+        // Row is valid at the time during and after writing.
+        assertEquals(value, value(pk.read(binaryRow, null)));
+        assertEquals(value, value(pk.read(binaryRow, tsExact)));
+        assertEquals(value, value(pk.read(binaryRow, tsAfter)));
+
+        TestValue newValue = new TestValue(30, "duh");
+
+        pk.addWrite(binaryRow(key, newValue), UUID.randomUUID());
+
+        // Same checks, but now there are two different versions.
+        assertNull(pk.read(binaryRow, tsBefore));
+
+        assertEquals(newValue, value(pk.read(binaryRow, null)));
+
+        assertEquals(value, value(pk.read(binaryRow, tsExact)));
+        assertEquals(value, value(pk.read(binaryRow, tsAfter)));
+        assertEquals(value, value(pk.read(binaryRow, Timestamp.nextVersion())));
+
+        pk.commitWrite(binaryKey(key), Timestamp.nextVersion());
+
+        assertEquals(newValue, value(pk.read(binaryRow, null)));
+        assertEquals(newValue, value(pk.read(binaryRow, Timestamp.nextVersion())));
+    }
+
+    /**
+     * Tests basic invariants of {@link MvPartitionStorage#scan(Predicate, Timestamp)}.
+     */
+    @Test
+    public void testScan() throws Exception {
+        MvPartitionStorage pk = partitionStorage();
+
+        TestKey key1 = new TestKey(1, "1");
+        TestValue value1 = new TestValue(10, "xxx");
+
+        TestKey key2 = new TestKey(2, "2");
+        TestValue value2 = new TestValue(20, "yyy");
+
+        pk.addWrite(binaryRow(key1, value1), UUID.randomUUID());
+        pk.addWrite(binaryRow(key2, value2), UUID.randomUUID());
+
+        // Scan with and without filters.
+        assertEquals(List.of(value1, value2), convert(pk.scan(row -> true, null)));
+        assertEquals(List.of(value1), convert(pk.scan(row -> key(row).intKey == 1, null)));
+        assertEquals(List.of(value2), convert(pk.scan(row -> key(row).intKey == 2, null)));
+
+        Timestamp ts1 = Timestamp.nextVersion();
+
+        Timestamp ts2 = Timestamp.nextVersion();
+        pk.commitWrite(binaryKey(key1), ts2);
+
+        Timestamp ts3 = Timestamp.nextVersion();
+
+        Timestamp ts4 = Timestamp.nextVersion();
+        pk.commitWrite(binaryKey(key2), ts4);
+
+        Timestamp ts5 = Timestamp.nextVersion();
+
+        // Full scan with various timestamp values.
+        assertEquals(List.of(), convert(pk.scan(row -> true, ts1)));
+
+        assertEquals(List.of(value1), convert(pk.scan(row -> true, ts2)));
+        assertEquals(List.of(value1), convert(pk.scan(row -> true, ts3)));
+
+        assertEquals(List.of(value1, value2), convert(pk.scan(row -> true, ts4)));
+        assertEquals(List.of(value1, value2), convert(pk.scan(row -> true, ts5)));
+    }
+
+    private List<TestValue> convert(Cursor<BinaryRow> cursor) throws Exception {
+        try (cursor) {
+            List<TestValue> list = StreamSupport.stream(cursor.spliterator(), false)

Review comment:
       can be written shorter, for example:
   ```
   return StreamSupport.stream(cursor.spliterator(), false)
           .map(this::value)
           .sorted(Comparator.nullsFirst(Comparator.naturalOrder()))
           .collect(Collectors.toList());
   ```        

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -138,6 +139,7 @@ public void start() throws StorageException {
         DBOptions dbOptions = new DBOptions()
                 .setCreateIfMissing(true)
                 .setCreateMissingColumnFamilies(true)
+                .setWalRecoveryMode(WALRecoveryMode.SkipAnyCorruptedRecords)

Review comment:
       Why is this needed?

##########
File path: modules/storage-api/src/test/java/org/apache/ignite/internal/storage/AbstractMvPartitionStorageTest.java
##########
@@ -0,0 +1,206 @@
+/*
+ * 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;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.UUID;
+import java.util.function.Predicate;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.internal.util.Cursor;
+import org.junit.jupiter.api.Test;
+
+/**
+ * Base test for MV partition storages.
+ */
+public abstract class AbstractMvPartitionStorageTest extends BaseMvStoragesTest {
+    /**
+     * Creates a storage instance for testing.
+     */
+    protected abstract MvPartitionStorage partitionStorage();
+
+    /**
+     * Tests that reads and scan from empty storage return empty results.
+     */
+    @Test
+    public void testEmpty() throws Exception {
+        MvPartitionStorage pk = partitionStorage();
+
+        BinaryRow binaryKey = binaryKey(new TestKey(10, "foo"));
+
+        // Read.
+        assertNull(pk.read(binaryKey, null));
+        assertNull(pk.read(binaryKey, Timestamp.nextVersion()));
+
+        // Scan.
+        assertEquals(List.of(), convert(pk.scan(row -> true, null)));
+        assertEquals(List.of(), convert(pk.scan(row -> true, Timestamp.nextVersion())));
+    }
+
+    /**
+     * Tests basic invariants of {@link MvPartitionStorage#addWrite(BinaryRow, UUID)}.
+     */
+    @Test
+    public void testAddWrite() throws Exception {
+        MvPartitionStorage pk = partitionStorage();
+
+        TestKey key = new TestKey(10, "foo");
+        TestValue value = new TestValue(20, "bar");
+
+        BinaryRow binaryRow = binaryRow(key, value);
+
+        pk.addWrite(binaryRow, UUID.randomUUID());
+
+        // Attempt to write from another transaction.
+        assertThrows(TxIdMismatchException.class, () -> pk.addWrite(binaryRow, UUID.randomUUID()));
+
+        // Read without timestamp returns uncommited row.
+        assertEquals(value, value(pk.read(binaryKey(key), null)));
+
+        // Read with timestamp returns null.
+        assertNull(pk.read(binaryKey(key), Timestamp.nextVersion()));
+    }
+
+    /**
+     * Tests basic invariants of {@link MvPartitionStorage#abortWrite(BinaryRow)}.
+     */
+    @Test
+    public void testAbortWrite() throws Exception {
+        MvPartitionStorage pk = partitionStorage();
+
+        TestKey key = new TestKey(10, "foo");
+        TestValue value = new TestValue(20, "bar");
+
+        pk.addWrite(binaryRow(key, value), UUID.randomUUID());
+
+        pk.abortWrite(binaryKey(key));
+
+        // Aborted row can't be read.
+        assertNull(pk.read(binaryKey(key), null));
+    }
+
+    /**
+     * Tests basic invariants of {@link MvPartitionStorage#commitWrite(BinaryRow, Timestamp)}.
+     */
+    @Test
+    public void testCommitWrite() throws Exception {
+        MvPartitionStorage pk = partitionStorage();
+
+        TestKey key = new TestKey(10, "foo");
+        TestValue value = new TestValue(20, "bar");
+
+        BinaryRow binaryRow = binaryRow(key, value);
+
+        pk.addWrite(binaryRow, UUID.randomUUID());
+
+        Timestamp tsBefore = Timestamp.nextVersion();
+
+        Timestamp tsExact = Timestamp.nextVersion();
+        pk.commitWrite(binaryRow, tsExact);
+
+        Timestamp tsAfter = Timestamp.nextVersion();
+
+        // Row is invisible at the time before writing.
+        assertNull(pk.read(binaryRow, tsBefore));
+
+        // Row is valid at the time during and after writing.
+        assertEquals(value, value(pk.read(binaryRow, null)));
+        assertEquals(value, value(pk.read(binaryRow, tsExact)));
+        assertEquals(value, value(pk.read(binaryRow, tsAfter)));
+
+        TestValue newValue = new TestValue(30, "duh");
+
+        pk.addWrite(binaryRow(key, newValue), UUID.randomUUID());
+
+        // Same checks, but now there are two different versions.
+        assertNull(pk.read(binaryRow, tsBefore));
+
+        assertEquals(newValue, value(pk.read(binaryRow, null)));
+
+        assertEquals(value, value(pk.read(binaryRow, tsExact)));
+        assertEquals(value, value(pk.read(binaryRow, tsAfter)));
+        assertEquals(value, value(pk.read(binaryRow, Timestamp.nextVersion())));
+
+        pk.commitWrite(binaryKey(key), Timestamp.nextVersion());
+
+        assertEquals(newValue, value(pk.read(binaryRow, null)));

Review comment:
       should we also check that previous value can still be extracted after a commit?

##########
File path: modules/storage-api/src/test/java/org/apache/ignite/internal/storage/AbstractMvPartitionStorageTest.java
##########
@@ -0,0 +1,206 @@
+/*
+ * 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;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.UUID;
+import java.util.function.Predicate;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.internal.util.Cursor;
+import org.junit.jupiter.api.Test;
+
+/**
+ * Base test for MV partition storages.
+ */
+public abstract class AbstractMvPartitionStorageTest extends BaseMvStoragesTest {
+    /**
+     * Creates a storage instance for testing.
+     */
+    protected abstract MvPartitionStorage partitionStorage();
+
+    /**
+     * Tests that reads and scan from empty storage return empty results.
+     */
+    @Test
+    public void testEmpty() throws Exception {
+        MvPartitionStorage pk = partitionStorage();
+
+        BinaryRow binaryKey = binaryKey(new TestKey(10, "foo"));
+
+        // Read.
+        assertNull(pk.read(binaryKey, null));
+        assertNull(pk.read(binaryKey, Timestamp.nextVersion()));
+
+        // Scan.
+        assertEquals(List.of(), convert(pk.scan(row -> true, null)));
+        assertEquals(List.of(), convert(pk.scan(row -> true, Timestamp.nextVersion())));
+    }
+
+    /**
+     * Tests basic invariants of {@link MvPartitionStorage#addWrite(BinaryRow, UUID)}.
+     */
+    @Test
+    public void testAddWrite() throws Exception {
+        MvPartitionStorage pk = partitionStorage();
+
+        TestKey key = new TestKey(10, "foo");
+        TestValue value = new TestValue(20, "bar");
+
+        BinaryRow binaryRow = binaryRow(key, value);
+
+        pk.addWrite(binaryRow, UUID.randomUUID());
+
+        // Attempt to write from another transaction.
+        assertThrows(TxIdMismatchException.class, () -> pk.addWrite(binaryRow, UUID.randomUUID()));
+
+        // Read without timestamp returns uncommited row.
+        assertEquals(value, value(pk.read(binaryKey(key), null)));
+
+        // Read with timestamp returns null.
+        assertNull(pk.read(binaryKey(key), Timestamp.nextVersion()));
+    }
+
+    /**
+     * Tests basic invariants of {@link MvPartitionStorage#abortWrite(BinaryRow)}.
+     */
+    @Test
+    public void testAbortWrite() throws Exception {

Review comment:
       `abortWrite` also influences the state of the indexes, that's not tested anywhere




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



[GitHub] [ignite-3] AMashenkov commented on a change in pull request #739: IGNITE-16697 MV storage methods and reference implementation

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #739:
URL: https://github.com/apache/ignite-3/pull/739#discussion_r833010502



##########
File path: modules/storage-api/src/test/java/org/apache/ignite/internal/storage/basic/TestMvSortedIndexStorage.java
##########
@@ -0,0 +1,253 @@
+/*
+ * 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.basic;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.Arrays;
+import java.util.BitSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.NavigableSet;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.function.IntPredicate;
+import java.util.function.ToIntFunction;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.schema.row.Row;
+import org.apache.ignite.internal.storage.index.IndexRowPrefix;
+import org.apache.ignite.internal.storage.index.PrefixComparator;
+import org.apache.ignite.internal.storage.index.SortedIndexDescriptor;
+import org.apache.ignite.internal.storage.index.SortedIndexDescriptor.ColumnDescriptor;
+import org.apache.ignite.internal.storage.index.SortedIndexStorage;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.internal.util.Cursor;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Test implementation of MV sorted index storage.
+ */
+public abstract class TestMvSortedIndexStorage implements SortedIndexStorage {
+    private final SortedIndexDescriptor descriptor;
+
+    private final NavigableSet<BinaryRow> index;
+
+    private final Map<Integer, TestMvPartitionStorage> pk;
+
+    private final int partitions;
+
+    protected TestMvSortedIndexStorage(SortedIndexDescriptor descriptor, Map<Integer, TestMvPartitionStorage> pk, int partitions) {
+        this.descriptor = descriptor;
+        this.pk = pk;
+        this.partitions = partitions;
+
+        index = new ConcurrentSkipListSet<>((l, r) -> {
+            int cmp = compareColumns(l, r);
+
+            if (cmp != 0) {
+                return cmp;
+            }
+
+            return l.keySlice().compareTo(r.keySlice());
+        });
+    }
+
+    @Override
+    public SortedIndexDescriptor indexDescriptor() {
+        return descriptor;
+    }
+
+    private int compareColumns(BinaryRow l, BinaryRow r) {
+        List<ColumnDescriptor> columnDescriptors = descriptor.indexRowColumns();
+
+        Row leftRow = new Row(descriptor.asSchemaDescriptor(), l);
+
+        Object[] leftTuple = convert(leftRow, columnDescriptors, null);
+
+        return new PrefixComparator(descriptor, () -> leftTuple).compare(r);
+    }
+
+    public void append(BinaryRow row) {
+        index.add(row);
+    }
+
+    public void remove(BinaryRow row) {
+        index.remove(row);
+    }
+
+    public boolean matches(BinaryRow aborted, BinaryRow existing) {
+        return compareColumns(aborted, existing) == 0;
+    }
+
+    @Override
+    public Cursor<IndexRowEx> scan(
+            @Nullable IndexRowPrefix lowerBound,
+            @Nullable IndexRowPrefix upperBound,
+            byte flags,
+            Timestamp timestamp,
+            @Nullable BitSet columnsProjection,
+            @Nullable IntPredicate partitionFilter
+    ) {
+        boolean includeLower = (flags & GREATER_OR_EQUAL) != 0;
+        boolean includeUpper = (flags & LESS_OR_EQUAL) != 0;
+
+        NavigableSet<BinaryRow> index = this.index;
+
+        // Swap bounds and flip index for backwards scan.
+        if ((flags & BACKWARDS) != 0) {
+            index = index.descendingSet();
+
+            boolean tempBoolean = includeLower;
+            includeLower = includeUpper;
+            includeUpper = tempBoolean;
+
+            IndexRowPrefix tempBound = lowerBound;
+            lowerBound = upperBound;
+            upperBound = tempBound;
+        }
+
+        ToIntFunction<BinaryRow> lowerCmp = lowerBound == null ? row -> -1 : new PrefixComparator(descriptor, lowerBound)::compare;
+        ToIntFunction<BinaryRow> upperCmp = upperBound == null ? row -> -1 : new PrefixComparator(descriptor, upperBound)::compare;
+
+        boolean includeLower0 = includeLower;
+        boolean includeUpper0 = includeUpper;
+
+        Iterator<IndexRowEx> iterator = index.stream()
+                .dropWhile(binaryRow -> {
+                    int cmp = lowerCmp.applyAsInt(binaryRow);
+
+                    return includeLower0 && cmp < 0 || !includeLower0 && cmp <= 0;
+                })
+                .takeWhile(binaryRow -> {
+                    int cmp = upperCmp.applyAsInt(binaryRow);
+
+                    return includeUpper0 && cmp >= 0 || !includeUpper0 && cmp > 0;
+                })
+                .filter(binaryRow -> {
+                    int partition = binaryRow.hash() % partitions;
+
+                    // This code has been copy-pasted.

Review comment:
       ```suggestion
   ```




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



[GitHub] [ignite-3] AMashenkov commented on a change in pull request #739: IGNITE-16697 MV storage methods and reference implementation

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #739:
URL: https://github.com/apache/ignite-3/pull/739#discussion_r833009297



##########
File path: modules/storage-api/src/main/java/org/apache/ignite/internal/storage/index/SortedIndexStorage.java
##########
@@ -58,12 +64,63 @@
      */
     void remove(IndexRow row);
 
+    /** Exclude lower bound. */
+    byte GREATER = 0;
+
+    /** Include lower bound. */
+    byte GREATER_OR_EQUAL = 1;
+
+    /** Exclude upper bound. */
+    byte LESS = 0;
+
+    /** Include upper bound. */
+    byte LESS_OR_EQUAL = 1 << 1;
+
+    byte FORWARD = 0;
+
+    byte BACKWARDS = 1 << 2;
+
     /**
      * Returns a range of index values between the lower bound (inclusive) and the upper bound (inclusive).
      */
     // TODO: add options https://issues.apache.org/jira/browse/IGNITE-16059
     Cursor<IndexRow> range(IndexRowPrefix lowerBound, IndexRowPrefix upperBound);
 
+    /**
+     * Returns a range of index values between the lower bound and the upper bound, consistent with the passed timestamp.
+     *
+     * @param lowerBound Lower bound. Exclusivity is controlled by a {@link #GREATER_OR_EQUAL} or {@link #GREATER} flag.
+     *      {@code null} means unbounded.
+     * @param upperBound Upper bound. Exclusivity is controlled by a {@link #LESS} or {@link #LESS_OR_EQUAL} flag.
+     *      {@code null} means unbounded.
+     * @param flags Control flags. {@link #GREATER} | {@link #LESS} | {@link #FORWARD} by default. Other available values
+     *      are {@link #GREATER_OR_EQUAL}, {@link #LESS_OR_EQUAL} and {@link #BACKWARDS}.
+     * @param timestamp Timestamp value for consistent multiversioned index scan.
+     * @param columnsProjection Bit set with column indexes to return. {@code null} means returning all available columns.
+     * @param partitionFilter Partition filter predicate. {@code null} means returning data from all partitions.
+     * @return Cursor with fetched index rows.
+     * @throws IllegalArgumentException If backwards flag is passed and backwards iteration is not supported by the storage.
+     */
+    default Cursor<IndexRowEx> scan(
+            @Nullable IndexRowPrefix lowerBound,
+            @Nullable IndexRowPrefix upperBound,
+            @MagicConstant(flagsFromClass = SortedIndexStorage.class) byte flags,
+            Timestamp timestamp,
+            @Nullable BitSet columnsProjection,
+            @Nullable IntPredicate partitionFilter
+    ) {
+        throw new UnsupportedOperationException("scan");
+    }
+
+    /**
+     * The sole purpose of this class is to avoid massive refactoring while changing the original IndexRow.
+     */
+    interface IndexRowEx {

Review comment:
       Is it temporary class or you are going to fix it within this PR?
   Would you mind to add a TODO with a ticket number to rework this?




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



[GitHub] [ignite-3] AMashenkov commented on a change in pull request #739: IGNITE-16697 MV storage methods and reference implementation

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #739:
URL: https://github.com/apache/ignite-3/pull/739#discussion_r833002638



##########
File path: modules/storage-api/src/main/java/org/apache/ignite/internal/storage/index/SortedIndexStorage.java
##########
@@ -58,12 +64,63 @@
      */
     void remove(IndexRow row);
 
+    /** Exclude lower bound. */
+    byte GREATER = 0;
+
+    /** Include lower bound. */
+    byte GREATER_OR_EQUAL = 1;
+
+    /** Exclude upper bound. */
+    byte LESS = 0;
+
+    /** Include upper bound. */
+    byte LESS_OR_EQUAL = 1 << 1;
+
+    byte FORWARD = 0;
+
+    byte BACKWARDS = 1 << 2;
+
     /**
      * Returns a range of index values between the lower bound (inclusive) and the upper bound (inclusive).
      */
     // TODO: add options https://issues.apache.org/jira/browse/IGNITE-16059
     Cursor<IndexRow> range(IndexRowPrefix lowerBound, IndexRowPrefix upperBound);
 
+    /**
+     * Returns a range of index values between the lower bound and the upper bound, consistent with the passed timestamp.
+     *
+     * @param lowerBound Lower bound. Exclusivity is controlled by a {@link #GREATER_OR_EQUAL} or {@link #GREATER} flag.
+     *      {@code null} means unbounded.
+     * @param upperBound Upper bound. Exclusivity is controlled by a {@link #LESS} or {@link #LESS_OR_EQUAL} flag.
+     *      {@code null} means unbounded.
+     * @param flags Control flags. {@link #GREATER} | {@link #LESS} | {@link #FORWARD} by default. Other available values
+     *      are {@link #GREATER_OR_EQUAL}, {@link #LESS_OR_EQUAL} and {@link #BACKWARDS}.
+     * @param timestamp Timestamp value for consistent multiversioned index scan.
+     * @param columnsProjection Bit set with column indexes to return. {@code null} means returning all available columns.
+     * @param partitionFilter Partition filter predicate. {@code null} means returning data from all partitions.
+     * @return Cursor with fetched index rows.
+     * @throws IllegalArgumentException If backwards flag is passed and backwards iteration is not supported by the storage.

Review comment:
       Should we add smth like UnsupportedBackwardIndexScanException exception for this, 
   and\or `isBackwardScanSupported()` method?




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



[GitHub] [ignite-3] AMashenkov commented on a change in pull request #739: IGNITE-16697 MV storage methods and reference implementation

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #739:
URL: https://github.com/apache/ignite-3/pull/739#discussion_r833002061



##########
File path: modules/storage-api/src/main/java/org/apache/ignite/internal/storage/index/SortedIndexStorage.java
##########
@@ -58,12 +64,63 @@
      */
     void remove(IndexRow row);
 
+    /** Exclude lower bound. */
+    byte GREATER = 0;
+
+    /** Include lower bound. */
+    byte GREATER_OR_EQUAL = 1;
+
+    /** Exclude upper bound. */
+    byte LESS = 0;
+
+    /** Include upper bound. */
+    byte LESS_OR_EQUAL = 1 << 1;
+
+    byte FORWARD = 0;
+
+    byte BACKWARDS = 1 << 2;

Review comment:
       Does you mean that every index storage MUST support Backward iteration?
   If it is not mandatory requirement, then let's add smth like UnsupportedBackwardIndexScanException to method contract into "@throws" section.




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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #739: IGNITE-16697 MV storage methods and reference implementation

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #739:
URL: https://github.com/apache/ignite-3/pull/739#discussion_r840407615



##########
File path: modules/storage-api/src/test/java/org/apache/ignite/internal/storage/AbstractMvPartitionStorageTest.java
##########
@@ -0,0 +1,206 @@
+/*
+ * 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;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.UUID;
+import java.util.function.Predicate;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.internal.util.Cursor;
+import org.junit.jupiter.api.Test;
+
+/**
+ * Base test for MV partition storages.
+ */
+public abstract class AbstractMvPartitionStorageTest extends BaseMvStoragesTest {
+    /**
+     * Creates a storage instance for testing.
+     */
+    protected abstract MvPartitionStorage partitionStorage();
+
+    /**
+     * Tests that reads and scan from empty storage return empty results.
+     */
+    @Test
+    public void testEmpty() throws Exception {
+        MvPartitionStorage pk = partitionStorage();
+
+        BinaryRow binaryKey = binaryKey(new TestKey(10, "foo"));
+
+        // Read.
+        assertNull(pk.read(binaryKey, null));
+        assertNull(pk.read(binaryKey, Timestamp.nextVersion()));
+
+        // Scan.
+        assertEquals(List.of(), convert(pk.scan(row -> true, null)));
+        assertEquals(List.of(), convert(pk.scan(row -> true, Timestamp.nextVersion())));
+    }
+
+    /**
+     * Tests basic invariants of {@link MvPartitionStorage#addWrite(BinaryRow, UUID)}.
+     */
+    @Test
+    public void testAddWrite() throws Exception {
+        MvPartitionStorage pk = partitionStorage();
+
+        TestKey key = new TestKey(10, "foo");
+        TestValue value = new TestValue(20, "bar");
+
+        BinaryRow binaryRow = binaryRow(key, value);
+
+        pk.addWrite(binaryRow, UUID.randomUUID());
+
+        // Attempt to write from another transaction.
+        assertThrows(TxIdMismatchException.class, () -> pk.addWrite(binaryRow, UUID.randomUUID()));
+
+        // Read without timestamp returns uncommited row.
+        assertEquals(value, value(pk.read(binaryKey(key), null)));
+
+        // Read with timestamp returns null.
+        assertNull(pk.read(binaryKey(key), Timestamp.nextVersion()));
+    }
+
+    /**
+     * Tests basic invariants of {@link MvPartitionStorage#abortWrite(BinaryRow)}.
+     */
+    @Test
+    public void testAbortWrite() throws Exception {
+        MvPartitionStorage pk = partitionStorage();
+
+        TestKey key = new TestKey(10, "foo");
+        TestValue value = new TestValue(20, "bar");
+
+        pk.addWrite(binaryRow(key, value), UUID.randomUUID());
+
+        pk.abortWrite(binaryKey(key));
+
+        // Aborted row can't be read.
+        assertNull(pk.read(binaryKey(key), null));
+    }
+
+    /**
+     * Tests basic invariants of {@link MvPartitionStorage#commitWrite(BinaryRow, Timestamp)}.
+     */
+    @Test
+    public void testCommitWrite() throws Exception {
+        MvPartitionStorage pk = partitionStorage();
+
+        TestKey key = new TestKey(10, "foo");
+        TestValue value = new TestValue(20, "bar");
+
+        BinaryRow binaryRow = binaryRow(key, value);
+
+        pk.addWrite(binaryRow, UUID.randomUUID());
+
+        Timestamp tsBefore = Timestamp.nextVersion();
+
+        Timestamp tsExact = Timestamp.nextVersion();
+        pk.commitWrite(binaryRow, tsExact);
+
+        Timestamp tsAfter = Timestamp.nextVersion();
+
+        // Row is invisible at the time before writing.
+        assertNull(pk.read(binaryRow, tsBefore));
+
+        // Row is valid at the time during and after writing.
+        assertEquals(value, value(pk.read(binaryRow, null)));
+        assertEquals(value, value(pk.read(binaryRow, tsExact)));
+        assertEquals(value, value(pk.read(binaryRow, tsAfter)));
+
+        TestValue newValue = new TestValue(30, "duh");
+
+        pk.addWrite(binaryRow(key, newValue), UUID.randomUUID());
+
+        // Same checks, but now there are two different versions.
+        assertNull(pk.read(binaryRow, tsBefore));
+
+        assertEquals(newValue, value(pk.read(binaryRow, null)));
+
+        assertEquals(value, value(pk.read(binaryRow, tsExact)));
+        assertEquals(value, value(pk.read(binaryRow, tsAfter)));
+        assertEquals(value, value(pk.read(binaryRow, Timestamp.nextVersion())));
+
+        pk.commitWrite(binaryKey(key), Timestamp.nextVersion());
+
+        assertEquals(newValue, value(pk.read(binaryRow, null)));

Review comment:
       Why not




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