You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@ignite.apache.org by "ibessonov (via GitHub)" <gi...@apache.org> on 2023/05/04 07:50:57 UTC

[GitHub] [ignite-3] ibessonov opened a new pull request, #2021: IGNITE-19394 "peek" invariants strengthened, to avoid missing data on RW scans.

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

   https://issues.apache.org/jira/browse/IGNITE-19394


-- 
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] tkalkirill commented on a diff in pull request #2021: IGNITE-19394 "peek" contract improved to avoid missing data on RW scans.

Posted by "tkalkirill (via GitHub)" <gi...@apache.org>.
tkalkirill commented on code in PR #2021:
URL: https://github.com/apache/ignite-3/pull/2021#discussion_r1185865922


##########
modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/AbstractSortedIndexStorageTest.java:
##########
@@ -215,12 +134,17 @@ protected SortedIndexStorage createIndexStorage(ColumnarIndexDefinition indexDef
         return tableStorage.getOrCreateSortedIndex(TEST_PARTITION, indexConfig.id());
     }
 
+    @Override
+    protected SortedIndexDescriptor indexDescriptor(SortedIndexStorage index) {
+        return index.indexDescriptor();
+    }
+
     /**
      * Tests that columns of all types are correctly serialized and deserialized.
      */
     @Test
     void testRowSerialization() {
-        SortedIndexStorage indexStorage = createIndexStorage(ALL_TYPES_COLUMN_DEFINITIONS);
+        SortedIndexStorage indexStorage = createIndexStorage("TEST_IDX", ALL_TYPES_COLUMN_DEFINITIONS);

Review Comment:
   ```suggestion
           SortedIndexStorage indexStorage = createIndexStorage(INDEX_NAME, ALL_TYPES_COLUMN_DEFINITIONS);
   ```



-- 
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 diff in pull request #2021: IGNITE-19394 "peek" contract improved to avoid missing data on RW scans.

Posted by "ibessonov (via GitHub)" <gi...@apache.org>.
ibessonov commented on code in PR #2021:
URL: https://github.com/apache/ignite-3/pull/2021#discussion_r1184799821


##########
modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/AbstractSortedIndexStorageTest.java:
##########
@@ -1506,6 +1505,49 @@ void testScanPeekReplaceRow() {
         assertNull(scan.peek());
     }
 
+    @Test
+    void testScanPeekRemoveNext() {
+        SortedIndexDefinition indexDefinition = SchemaBuilders.sortedIndex("TEST_IDX")
+                .addIndexColumn(ColumnType.INT32.typeSpec().name()).asc().done()
+                .build();
+
+        SortedIndexStorage indexStorage = createIndexStorage(indexDefinition);
+
+        BinaryTupleRowSerializer serializer = new BinaryTupleRowSerializer(indexStorage.indexDescriptor());
+
+        PeekCursor<IndexRow> scan = indexStorage.scan(null, null, 0);
+
+        RowId rowId = new RowId(TEST_PARTITION);
+
+        // index  =  [0]
+        // cursor = ^ with no cached row
+        put(indexStorage, serializer.serializeRow(new Object[]{0}, rowId));
+
+        // index  =  [0] [1]
+        // cursor = ^ with no cached row
+        assertEquals(SimpleRow.of(0, rowId), SimpleRow.of(scan.peek(), firstColumn(serializer)));
+
+        // index  =
+        // cursor = ^ with no cached row (but it remembers the last peek call)
+        remove(indexStorage, serializer.serializeRow(new Object[]{0}, rowId));
+
+        // "hasNext" and "next" must return the result of last "peek" operation. This is crucial for RW scans.
+        assertTrue(scan.hasNext());
+        assertEquals(SimpleRow.of(0, rowId), SimpleRow.of(scan.next(), firstColumn(serializer)));
+
+        // index  =
+        // cursor = ^ with no cached row
+        assertNull(scan.peek());

Review Comment:
   Current test is already too long. I test specific contract in this smaller 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] tkalkirill commented on a diff in pull request #2021: IGNITE-19394 "peek" contract improved to avoid missing data on RW scans.

Posted by "tkalkirill (via GitHub)" <gi...@apache.org>.
tkalkirill commented on code in PR #2021:
URL: https://github.com/apache/ignite-3/pull/2021#discussion_r1184804188


##########
modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/impl/TestSortedIndexStorage.java:
##########
@@ -83,10 +79,19 @@ Iterator<RowId> getRowIdIteratorForGetByBinaryTuple(BinaryTuple key) {
         BinaryTuplePrefix higherBound = BinaryTuplePrefix.fromBinaryTuple(key);
 
         //noinspection resource
-        return scan(lowerBound, higherBound, GREATER_OR_EQUAL | LESS_OR_EQUAL)
-                .stream()
-                .map(IndexRow::rowId)
-                .iterator();
+        PeekCursor<IndexRow> peekCursor = scan(lowerBound, higherBound, GREATER_OR_EQUAL | LESS_OR_EQUAL);
+
+        return new Iterator<>() {

Review Comment:
   Why an inner class and not a method or a separate class.
   You can use for example `org.apache.ignite.internal.util.CollectionUtils#viewReadOnly(java.util.Collection<? extends T1>, java.util.function.Function<? super T1,? extends T2>)`.



-- 
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 diff in pull request #2021: IGNITE-19394 "peek" contract improved to avoid missing data on RW scans.

Posted by "ibessonov (via GitHub)" <gi...@apache.org>.
ibessonov commented on code in PR #2021:
URL: https://github.com/apache/ignite-3/pull/2021#discussion_r1184785584


##########
modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/impl/TestSortedIndexStorage.java:
##########
@@ -175,112 +192,54 @@ public void close() {
         public boolean hasNext() {
             checkStorageClosedOrInProcessOfRebalance();
 
-            advanceIfNeeded();
+            if (hasNext != null) {
+                return hasNext;
+            }
+
+            currentRow = peekedRow == NO_PEEKED_ROW ? peek() : peekedRow;
+            peekedRow = NO_PEEKED_ROW;
 
+            hasNext = currentRow != null;
             return hasNext;
         }
 
         @Override
         public IndexRow next() {
-            checkStorageClosedOrInProcessOfRebalance();
-
-            advanceIfNeeded();
-
-            boolean hasNext = this.hasNext;
-
-            if (!hasNext) {
+            if (!hasNext()) {
                 throw new NoSuchElementException();
             }
 
             this.hasNext = null;
 
-            return new IndexRowImpl(new BinaryTuple(descriptor.binaryTupleSchema(), currentEntry.getKey()), rowId);
+            return currentRow;
         }
 
         @Override
         public @Nullable IndexRow peek() {
             checkStorageClosedOrInProcessOfRebalance();
 
             if (hasNext != null) {
-                if (hasNext) {
-                    return new IndexRowImpl(new BinaryTuple(descriptor.binaryTupleSchema(), currentEntry.getKey()), rowId);
-                }
-
-                return null;
+                return currentRow;
             }
 
-            Entry<ByteBuffer, NavigableMap<RowId, Object>> indexMapEntry0 = currentEntry == null ? indexMap.firstEntry() : currentEntry;
-
-            RowId nextRowId = null;
-
-            if (rowId == null) {
-                if (indexMapEntry0 != null) {
-                    nextRowId = getRowId(indexMapEntry0.getValue().firstEntry());
-                }
-            } else {
-                Entry<RowId, Object> nextRowIdEntry = indexMapEntry0.getValue().higherEntry(rowId);
-
-                if (nextRowIdEntry != null) {
-                    nextRowId = nextRowIdEntry.getKey();
-                } else {
-                    indexMapEntry0 = indexMap.higherEntry(indexMapEntry0.getKey());
-
-                    if (indexMapEntry0 != null) {
-                        nextRowId = getRowId(indexMapEntry0.getValue().firstEntry());
-                    }
-                }
-            }
-
-            return nextRowId == null
-                    ? null : new IndexRowImpl(new BinaryTuple(descriptor.binaryTupleSchema(), indexMapEntry0.getKey()), nextRowId);
-        }
-
-        private void advanceIfNeeded() {
-            if (hasNext != null) {
-                return;
-            }
-
-            if (currentEntry == null) {
-                currentEntry = indexMap.firstEntry();
-            }
-
-            if (rowId == null) {
-                if (currentEntry != null) {
-                    rowId = getRowId(currentEntry.getValue().firstEntry());
+            if (currentRow == null) {
+                try {
+                    peekedRow = indexSet.first();
+                } catch (NoSuchElementException e) {
+                    peekedRow = null;

Review Comment:
   Why is this an anti-pattern, if I use Java API the way it's intended to be used?
   I think that using Map instead of Set is anti-pattern



##########
modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/impl/AbstractTestIndexStorage.java:
##########
@@ -30,6 +30,8 @@
  * Test-only abstract index storage class.
  */
 abstract class AbstractTestIndexStorage implements IndexStorage {
+    protected final int partitionId;

Review Comment:
   Ok



-- 
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] tkalkirill commented on a diff in pull request #2021: IGNITE-19394 "peek" contract improved to avoid missing data on RW scans.

Posted by "tkalkirill (via GitHub)" <gi...@apache.org>.
tkalkirill commented on code in PR #2021:
URL: https://github.com/apache/ignite-3/pull/2021#discussion_r1184801914


##########
modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/impl/TestSortedIndexStorage.java:
##########
@@ -72,31 +78,29 @@ public SortedIndexDescriptor indexDescriptor() {
 
     @Override
     Iterator<RowId> getRowIdIteratorForGetByBinaryTuple(BinaryTuple key) {
-        return index.getOrDefault(key.byteBuffer(), emptyNavigableMap()).keySet().iterator();
+        // These must be two different instances, because "scan" call messes up headers.
+        BinaryTuplePrefix lowerBound = BinaryTuplePrefix.fromBinaryTuple(key);
+        BinaryTuplePrefix higherBound = BinaryTuplePrefix.fromBinaryTuple(key);
+
+        //noinspection resource
+        return scan(lowerBound, higherBound, GREATER_OR_EQUAL | LESS_OR_EQUAL)
+                .stream()
+                .map(IndexRow::rowId)
+                .iterator();

Review Comment:
   That's no reason not to fix it.



-- 
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 diff in pull request #2021: IGNITE-19394 "peek" contract improved to avoid missing data on RW scans.

Posted by "ibessonov (via GitHub)" <gi...@apache.org>.
ibessonov commented on code in PR #2021:
URL: https://github.com/apache/ignite-3/pull/2021#discussion_r1184784848


##########
modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/impl/TestSortedIndexStorage.java:
##########
@@ -118,23 +122,34 @@ public PeekCursor<IndexRow> scan(
             setEqualityFlag(upperBound);
         }
 
-        NavigableMap<ByteBuffer, NavigableMap<RowId, Object>> navigableMap;
+        NavigableSet<IndexRow> navigableSet;
 
         if (lowerBound == null && upperBound == null) {
-            navigableMap = index;
+            navigableSet = index;
         } else if (lowerBound == null) {
-            navigableMap = index.headMap(upperBound.byteBuffer());
+            navigableSet = index.headSet(prefixToIndexRow(upperBound, highestRowId(partitionId)), true);
         } else if (upperBound == null) {
-            navigableMap = index.tailMap(lowerBound.byteBuffer());
+            navigableSet = index.tailSet(prefixToIndexRow(lowerBound, lowestRowId(partitionId)), true);
         } else {
             try {
-                navigableMap = index.subMap(lowerBound.byteBuffer(), upperBound.byteBuffer());
+                navigableSet = index.subSet(
+                        prefixToIndexRow(lowerBound, lowestRowId(partitionId)),
+                        true,
+                        prefixToIndexRow(upperBound, highestRowId(partitionId)),
+                        true
+                );
             } catch (IllegalArgumentException e) {
-                navigableMap = emptyNavigableMap();
+                navigableSet = emptyNavigableSet();

Review Comment:
   This was here before. Exception may happen if upper bound is below the lower bound.



-- 
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 diff in pull request #2021: IGNITE-19394 "peek" contract improved to avoid missing data on RW scans.

Posted by "ibessonov (via GitHub)" <gi...@apache.org>.
ibessonov commented on code in PR #2021:
URL: https://github.com/apache/ignite-3/pull/2021#discussion_r1184806989


##########
modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/impl/TestSortedIndexStorage.java:
##########
@@ -175,112 +192,54 @@ public void close() {
         public boolean hasNext() {
             checkStorageClosedOrInProcessOfRebalance();
 
-            advanceIfNeeded();
+            if (hasNext != null) {
+                return hasNext;
+            }
+
+            currentRow = peekedRow == NO_PEEKED_ROW ? peek() : peekedRow;
+            peekedRow = NO_PEEKED_ROW;
 
+            hasNext = currentRow != null;
             return hasNext;
         }
 
         @Override
         public IndexRow next() {
-            checkStorageClosedOrInProcessOfRebalance();
-
-            advanceIfNeeded();
-
-            boolean hasNext = this.hasNext;
-
-            if (!hasNext) {
+            if (!hasNext()) {
                 throw new NoSuchElementException();
             }
 
             this.hasNext = null;
 
-            return new IndexRowImpl(new BinaryTuple(descriptor.binaryTupleSchema(), currentEntry.getKey()), rowId);
+            return currentRow;
         }
 
         @Override
         public @Nullable IndexRow peek() {
             checkStorageClosedOrInProcessOfRebalance();
 
             if (hasNext != null) {
-                if (hasNext) {
-                    return new IndexRowImpl(new BinaryTuple(descriptor.binaryTupleSchema(), currentEntry.getKey()), rowId);
-                }
-
-                return null;
+                return currentRow;
             }
 
-            Entry<ByteBuffer, NavigableMap<RowId, Object>> indexMapEntry0 = currentEntry == null ? indexMap.firstEntry() : currentEntry;
-
-            RowId nextRowId = null;
-
-            if (rowId == null) {
-                if (indexMapEntry0 != null) {
-                    nextRowId = getRowId(indexMapEntry0.getValue().firstEntry());
-                }
-            } else {
-                Entry<RowId, Object> nextRowIdEntry = indexMapEntry0.getValue().higherEntry(rowId);
-
-                if (nextRowIdEntry != null) {
-                    nextRowId = nextRowIdEntry.getKey();
-                } else {
-                    indexMapEntry0 = indexMap.higherEntry(indexMapEntry0.getKey());
-
-                    if (indexMapEntry0 != null) {
-                        nextRowId = getRowId(indexMapEntry0.getValue().firstEntry());
-                    }
-                }
-            }
-
-            return nextRowId == null
-                    ? null : new IndexRowImpl(new BinaryTuple(descriptor.binaryTupleSchema(), indexMapEntry0.getKey()), nextRowId);
-        }
-
-        private void advanceIfNeeded() {
-            if (hasNext != null) {
-                return;
-            }
-
-            if (currentEntry == null) {
-                currentEntry = indexMap.firstEntry();
-            }
-
-            if (rowId == null) {
-                if (currentEntry != null) {
-                    rowId = getRowId(currentEntry.getValue().firstEntry());
+            if (currentRow == null) {
+                try {
+                    peekedRow = indexSet.first();
+                } catch (NoSuchElementException e) {
+                    peekedRow = null;

Review Comment:
   I wouldn't use the performance argument in such case, it's too preliminary.
   > it can return null instead of an exception, this is better ...
   
   Ok, why is it better? Is this really a good reason to use the structure that doesn't fit my goals?



-- 
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 diff in pull request #2021: IGNITE-19394 "peek" contract improved to avoid missing data on RW scans.

Posted by "ibessonov (via GitHub)" <gi...@apache.org>.
ibessonov commented on code in PR #2021:
URL: https://github.com/apache/ignite-3/pull/2021#discussion_r1184788936


##########
modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/AbstractSortedIndexStorageTest.java:
##########
@@ -1506,6 +1505,49 @@ void testScanPeekReplaceRow() {
         assertNull(scan.peek());
     }
 
+    @Test
+    void testScanPeekRemoveNext() {
+        SortedIndexDefinition indexDefinition = SchemaBuilders.sortedIndex("TEST_IDX")
+                .addIndexColumn(ColumnType.INT32.typeSpec().name()).asc().done()
+                .build();
+
+        SortedIndexStorage indexStorage = createIndexStorage(indexDefinition);
+
+        BinaryTupleRowSerializer serializer = new BinaryTupleRowSerializer(indexStorage.indexDescriptor());
+
+        PeekCursor<IndexRow> scan = indexStorage.scan(null, null, 0);
+
+        RowId rowId = new RowId(TEST_PARTITION);
+
+        // index  =  [0]
+        // cursor = ^ with no cached row
+        put(indexStorage, serializer.serializeRow(new Object[]{0}, rowId));
+
+        // index  =  [0] [1]
+        // cursor = ^ with no cached row
+        assertEquals(SimpleRow.of(0, rowId), SimpleRow.of(scan.peek(), firstColumn(serializer)));
+
+        // index  =
+        // cursor = ^ with no cached row (but it remembers the last peek call)
+        remove(indexStorage, serializer.serializeRow(new Object[]{0}, rowId));
+
+        // "hasNext" and "next" must return the result of last "peek" operation. This is crucial for RW scans.
+        assertTrue(scan.hasNext());
+        assertEquals(SimpleRow.of(0, rowId), SimpleRow.of(scan.next(), firstColumn(serializer)));
+
+        // index  =
+        // cursor = ^ with no cached row
+        assertNull(scan.peek());

Review Comment:
   It is covered in `testScanPeekForFinishedCursor`. I don't want to duplicate test scenarios for no reason



-- 
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] tkalkirill commented on a diff in pull request #2021: IGNITE-19394 "peek" contract improved to avoid missing data on RW scans.

Posted by "tkalkirill (via GitHub)" <gi...@apache.org>.
tkalkirill commented on code in PR #2021:
URL: https://github.com/apache/ignite-3/pull/2021#discussion_r1185126021


##########
modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/AbstractSortedIndexStorageTest.java:
##########
@@ -238,40 +162,11 @@ void testRowSerialization() {
 
     @Test
     void testEmpty() {
-        SortedIndexStorage index = createIndexStorage(shuffledRandomDefinitions());
+        SortedIndexStorage index = createIndexStorage("TEST_IDX", shuffledRandomDefinitions());

Review Comment:
   ```suggestion
           SortedIndexStorage index = createIndexStorage(INDEX_NAME", shuffledRandomDefinitions());
   ```



##########
modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/AbstractSortedIndexStorageTest.java:
##########
@@ -1539,7 +1375,7 @@ private List<ColumnDefinition> shuffledDefinitions(Predicate<ColumnDefinition> f
      * be removed.
      */
     private void testPutGetRemove(List<ColumnDefinition> indexSchema) {
-        SortedIndexStorage indexStorage = createIndexStorage(indexSchema);
+        SortedIndexStorage indexStorage = createIndexStorage("TEST_IDX", indexSchema);

Review Comment:
   ```suggestion
           SortedIndexStorage indexStorage = createIndexStorage(INDEX_NAME, indexSchema);
   ```



-- 
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] tkalkirill commented on a diff in pull request #2021: IGNITE-19394 "peek" contract improved to avoid missing data on RW scans.

Posted by "tkalkirill (via GitHub)" <gi...@apache.org>.
tkalkirill commented on code in PR #2021:
URL: https://github.com/apache/ignite-3/pull/2021#discussion_r1184789186


##########
modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/impl/TestSortedIndexStorage.java:
##########
@@ -175,112 +192,54 @@ public void close() {
         public boolean hasNext() {
             checkStorageClosedOrInProcessOfRebalance();
 
-            advanceIfNeeded();
+            if (hasNext != null) {
+                return hasNext;
+            }
+
+            currentRow = peekedRow == NO_PEEKED_ROW ? peek() : peekedRow;
+            peekedRow = NO_PEEKED_ROW;
 
+            hasNext = currentRow != null;
             return hasNext;
         }
 
         @Override
         public IndexRow next() {
-            checkStorageClosedOrInProcessOfRebalance();
-
-            advanceIfNeeded();
-
-            boolean hasNext = this.hasNext;
-
-            if (!hasNext) {
+            if (!hasNext()) {
                 throw new NoSuchElementException();
             }
 
             this.hasNext = null;
 
-            return new IndexRowImpl(new BinaryTuple(descriptor.binaryTupleSchema(), currentEntry.getKey()), rowId);
+            return currentRow;
         }
 
         @Override
         public @Nullable IndexRow peek() {
             checkStorageClosedOrInProcessOfRebalance();
 
             if (hasNext != null) {
-                if (hasNext) {
-                    return new IndexRowImpl(new BinaryTuple(descriptor.binaryTupleSchema(), currentEntry.getKey()), rowId);
-                }
-
-                return null;
+                return currentRow;
             }
 
-            Entry<ByteBuffer, NavigableMap<RowId, Object>> indexMapEntry0 = currentEntry == null ? indexMap.firstEntry() : currentEntry;
-
-            RowId nextRowId = null;
-
-            if (rowId == null) {
-                if (indexMapEntry0 != null) {
-                    nextRowId = getRowId(indexMapEntry0.getValue().firstEntry());
-                }
-            } else {
-                Entry<RowId, Object> nextRowIdEntry = indexMapEntry0.getValue().higherEntry(rowId);
-
-                if (nextRowIdEntry != null) {
-                    nextRowId = nextRowIdEntry.getKey();
-                } else {
-                    indexMapEntry0 = indexMap.higherEntry(indexMapEntry0.getKey());
-
-                    if (indexMapEntry0 != null) {
-                        nextRowId = getRowId(indexMapEntry0.getValue().firstEntry());
-                    }
-                }
-            }
-
-            return nextRowId == null
-                    ? null : new IndexRowImpl(new BinaryTuple(descriptor.binaryTupleSchema(), indexMapEntry0.getKey()), nextRowId);
-        }
-
-        private void advanceIfNeeded() {
-            if (hasNext != null) {
-                return;
-            }
-
-            if (currentEntry == null) {
-                currentEntry = indexMap.firstEntry();
-            }
-
-            if (rowId == null) {
-                if (currentEntry != null) {
-                    rowId = getRowId(currentEntry.getValue().firstEntry());
+            if (currentRow == null) {
+                try {
+                    peekedRow = indexSet.first();
+                } catch (NoSuchElementException e) {
+                    peekedRow = null;

Review Comment:
   If you use a map, then it can return `null` instead of an exception, this is better and more performant.



-- 
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 diff in pull request #2021: IGNITE-19394 "peek" contract improved to avoid missing data on RW scans.

Posted by "sashapolo (via GitHub)" <gi...@apache.org>.
sashapolo commented on code in PR #2021:
URL: https://github.com/apache/ignite-3/pull/2021#discussion_r1184753129


##########
modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/impl/TestSortedIndexStorage.java:
##########
@@ -61,8 +54,21 @@ public class TestSortedIndexStorage extends AbstractTestIndexStorage implements
     public TestSortedIndexStorage(int partitionId, SortedIndexDescriptor descriptor) {
         super(partitionId);
 
+        BinaryTupleComparator binaryTupleComparator = new BinaryTupleComparator(descriptor);
+
         this.descriptor = descriptor;
-        this.index = new ConcurrentSkipListMap<>(new BinaryTupleComparator(descriptor));
+        this.index = new ConcurrentSkipListSet<>((indexRow0, indexRow1) -> {
+            int cmp = binaryTupleComparator.compare(

Review Comment:
   Can be written shorter:
   ```
   Comparator<IndexRow> comparator = Comparator.comparing((IndexRow row) -> row.indexColumns().byteBuffer(), binaryTupleComparator)
           .thenComparing(IndexRow::rowId);
   ```
           



##########
modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/impl/AbstractTestIndexStorage.java:
##########
@@ -30,6 +30,8 @@
  * Test-only abstract index storage class.
  */
 abstract class AbstractTestIndexStorage implements IndexStorage {
+    protected final int partitionId;

Review Comment:
   This field is only used in `TestSortedIndexStorage` maybe it should be moved there



-- 
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] tkalkirill commented on a diff in pull request #2021: IGNITE-19394 "peek" contract improved to avoid missing data on RW scans.

Posted by "tkalkirill (via GitHub)" <gi...@apache.org>.
tkalkirill commented on code in PR #2021:
URL: https://github.com/apache/ignite-3/pull/2021#discussion_r1184741354


##########
modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/impl/TestSortedIndexStorage.java:
##########
@@ -61,8 +54,21 @@ public class TestSortedIndexStorage extends AbstractTestIndexStorage implements
     public TestSortedIndexStorage(int partitionId, SortedIndexDescriptor descriptor) {
         super(partitionId);
 
+        BinaryTupleComparator binaryTupleComparator = new BinaryTupleComparator(descriptor);
+
         this.descriptor = descriptor;
-        this.index = new ConcurrentSkipListMap<>(new BinaryTupleComparator(descriptor));
+        this.index = new ConcurrentSkipListSet<>((indexRow0, indexRow1) -> {

Review Comment:
   A separate class for `Camparator` would be more useful I guess.



##########
modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/AbstractSortedIndexStorageTest.java:
##########
@@ -1506,6 +1505,49 @@ void testScanPeekReplaceRow() {
         assertNull(scan.peek());
     }
 
+    @Test
+    void testScanPeekRemoveNext() {
+        SortedIndexDefinition indexDefinition = SchemaBuilders.sortedIndex("TEST_IDX")
+                .addIndexColumn(ColumnType.INT32.typeSpec().name()).asc().done()
+                .build();
+
+        SortedIndexStorage indexStorage = createIndexStorage(indexDefinition);
+
+        BinaryTupleRowSerializer serializer = new BinaryTupleRowSerializer(indexStorage.indexDescriptor());
+
+        PeekCursor<IndexRow> scan = indexStorage.scan(null, null, 0);
+
+        RowId rowId = new RowId(TEST_PARTITION);
+
+        // index  =  [0]
+        // cursor = ^ with no cached row
+        put(indexStorage, serializer.serializeRow(new Object[]{0}, rowId));
+
+        // index  =  [0] [1]

Review Comment:
   I don't think `[1]` is here.



##########
modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/impl/TestSortedIndexStorage.java:
##########
@@ -118,23 +122,34 @@ public PeekCursor<IndexRow> scan(
             setEqualityFlag(upperBound);
         }
 
-        NavigableMap<ByteBuffer, NavigableMap<RowId, Object>> navigableMap;
+        NavigableSet<IndexRow> navigableSet;
 
         if (lowerBound == null && upperBound == null) {
-            navigableMap = index;
+            navigableSet = index;
         } else if (lowerBound == null) {
-            navigableMap = index.headMap(upperBound.byteBuffer());
+            navigableSet = index.headSet(prefixToIndexRow(upperBound, highestRowId(partitionId)), true);
         } else if (upperBound == null) {
-            navigableMap = index.tailMap(lowerBound.byteBuffer());
+            navigableSet = index.tailSet(prefixToIndexRow(lowerBound, lowestRowId(partitionId)), true);
         } else {
             try {
-                navigableMap = index.subMap(lowerBound.byteBuffer(), upperBound.byteBuffer());
+                navigableSet = index.subSet(
+                        prefixToIndexRow(lowerBound, lowestRowId(partitionId)),
+                        true,
+                        prefixToIndexRow(upperBound, highestRowId(partitionId)),
+                        true
+                );
             } catch (IllegalArgumentException e) {
-                navigableMap = emptyNavigableMap();
+                navigableSet = emptyNavigableSet();

Review Comment:
   Why is exception handling happening, what can go wrong here?



##########
modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/impl/TestSortedIndexStorage.java:
##########
@@ -175,112 +192,54 @@ public void close() {
         public boolean hasNext() {
             checkStorageClosedOrInProcessOfRebalance();
 
-            advanceIfNeeded();
+            if (hasNext != null) {
+                return hasNext;
+            }
+
+            currentRow = peekedRow == NO_PEEKED_ROW ? peek() : peekedRow;
+            peekedRow = NO_PEEKED_ROW;
 
+            hasNext = currentRow != null;
             return hasNext;
         }
 
         @Override
         public IndexRow next() {
-            checkStorageClosedOrInProcessOfRebalance();
-
-            advanceIfNeeded();
-
-            boolean hasNext = this.hasNext;
-
-            if (!hasNext) {
+            if (!hasNext()) {
                 throw new NoSuchElementException();
             }
 
             this.hasNext = null;
 
-            return new IndexRowImpl(new BinaryTuple(descriptor.binaryTupleSchema(), currentEntry.getKey()), rowId);
+            return currentRow;
         }
 
         @Override
         public @Nullable IndexRow peek() {
             checkStorageClosedOrInProcessOfRebalance();
 
             if (hasNext != null) {
-                if (hasNext) {
-                    return new IndexRowImpl(new BinaryTuple(descriptor.binaryTupleSchema(), currentEntry.getKey()), rowId);
-                }
-
-                return null;
+                return currentRow;
             }
 
-            Entry<ByteBuffer, NavigableMap<RowId, Object>> indexMapEntry0 = currentEntry == null ? indexMap.firstEntry() : currentEntry;
-
-            RowId nextRowId = null;
-
-            if (rowId == null) {
-                if (indexMapEntry0 != null) {
-                    nextRowId = getRowId(indexMapEntry0.getValue().firstEntry());
-                }
-            } else {
-                Entry<RowId, Object> nextRowIdEntry = indexMapEntry0.getValue().higherEntry(rowId);
-
-                if (nextRowIdEntry != null) {
-                    nextRowId = nextRowIdEntry.getKey();
-                } else {
-                    indexMapEntry0 = indexMap.higherEntry(indexMapEntry0.getKey());
-
-                    if (indexMapEntry0 != null) {
-                        nextRowId = getRowId(indexMapEntry0.getValue().firstEntry());
-                    }
-                }
-            }
-
-            return nextRowId == null
-                    ? null : new IndexRowImpl(new BinaryTuple(descriptor.binaryTupleSchema(), indexMapEntry0.getKey()), nextRowId);
-        }
-
-        private void advanceIfNeeded() {
-            if (hasNext != null) {
-                return;
-            }
-
-            if (currentEntry == null) {
-                currentEntry = indexMap.firstEntry();
-            }
-
-            if (rowId == null) {
-                if (currentEntry != null) {
-                    rowId = getRowId(currentEntry.getValue().firstEntry());
+            if (currentRow == null) {
+                try {
+                    peekedRow = indexSet.first();
+                } catch (NoSuchElementException e) {
+                    peekedRow = null;

Review Comment:
   Exception logic is an anti-pattern, we can avoid them if we use a `NavigableMap`.



##########
modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/AbstractSortedIndexStorageTest.java:
##########
@@ -1506,6 +1505,49 @@ void testScanPeekReplaceRow() {
         assertNull(scan.peek());
     }
 
+    @Test
+    void testScanPeekRemoveNext() {
+        SortedIndexDefinition indexDefinition = SchemaBuilders.sortedIndex("TEST_IDX")
+                .addIndexColumn(ColumnType.INT32.typeSpec().name()).asc().done()
+                .build();
+
+        SortedIndexStorage indexStorage = createIndexStorage(indexDefinition);
+
+        BinaryTupleRowSerializer serializer = new BinaryTupleRowSerializer(indexStorage.indexDescriptor());
+
+        PeekCursor<IndexRow> scan = indexStorage.scan(null, null, 0);
+
+        RowId rowId = new RowId(TEST_PARTITION);
+
+        // index  =  [0]
+        // cursor = ^ with no cached row
+        put(indexStorage, serializer.serializeRow(new Object[]{0}, rowId));
+
+        // index  =  [0] [1]
+        // cursor = ^ with no cached row
+        assertEquals(SimpleRow.of(0, rowId), SimpleRow.of(scan.peek(), firstColumn(serializer)));
+
+        // index  =
+        // cursor = ^ with no cached row (but it remembers the last peek call)
+        remove(indexStorage, serializer.serializeRow(new Object[]{0}, rowId));
+
+        // "hasNext" and "next" must return the result of last "peek" operation. This is crucial for RW scans.
+        assertTrue(scan.hasNext());
+        assertEquals(SimpleRow.of(0, rowId), SimpleRow.of(scan.next(), firstColumn(serializer)));
+
+        // index  =
+        // cursor = ^ with no cached row
+        assertNull(scan.peek());

Review Comment:
   What happens if we insert a new row and call the `peek()`?



##########
modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/impl/TestSortedIndexStorage.java:
##########
@@ -72,31 +78,29 @@ public SortedIndexDescriptor indexDescriptor() {
 
     @Override
     Iterator<RowId> getRowIdIteratorForGetByBinaryTuple(BinaryTuple key) {
-        return index.getOrDefault(key.byteBuffer(), emptyNavigableMap()).keySet().iterator();
+        // These must be two different instances, because "scan" call messes up headers.
+        BinaryTuplePrefix lowerBound = BinaryTuplePrefix.fromBinaryTuple(key);
+        BinaryTuplePrefix higherBound = BinaryTuplePrefix.fromBinaryTuple(key);
+
+        //noinspection resource
+        return scan(lowerBound, higherBound, GREATER_OR_EQUAL | LESS_OR_EQUAL)
+                .stream()
+                .map(IndexRow::rowId)
+                .iterator();

Review Comment:
   As far as I remember, there may be a problem due to row caching inside the stream api when creating an iterator.



-- 
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 diff in pull request #2021: IGNITE-19394 "peek" contract improved to avoid missing data on RW scans.

Posted by "ibessonov (via GitHub)" <gi...@apache.org>.
ibessonov commented on code in PR #2021:
URL: https://github.com/apache/ignite-3/pull/2021#discussion_r1184782560


##########
modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/impl/TestSortedIndexStorage.java:
##########
@@ -72,31 +78,29 @@ public SortedIndexDescriptor indexDescriptor() {
 
     @Override
     Iterator<RowId> getRowIdIteratorForGetByBinaryTuple(BinaryTuple key) {
-        return index.getOrDefault(key.byteBuffer(), emptyNavigableMap()).keySet().iterator();
+        // These must be two different instances, because "scan" call messes up headers.
+        BinaryTuplePrefix lowerBound = BinaryTuplePrefix.fromBinaryTuple(key);
+        BinaryTuplePrefix higherBound = BinaryTuplePrefix.fromBinaryTuple(key);
+
+        //noinspection resource
+        return scan(lowerBound, higherBound, GREATER_OR_EQUAL | LESS_OR_EQUAL)
+                .stream()
+                .map(IndexRow::rowId)
+                .iterator();

Review Comment:
   This is definitely not tested. You're right, we need to fix it and cover it with 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] ibessonov commented on a diff in pull request #2021: IGNITE-19394 "peek" contract improved to avoid missing data on RW scans.

Posted by "ibessonov (via GitHub)" <gi...@apache.org>.
ibessonov commented on code in PR #2021:
URL: https://github.com/apache/ignite-3/pull/2021#discussion_r1184806989


##########
modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/impl/TestSortedIndexStorage.java:
##########
@@ -175,112 +192,54 @@ public void close() {
         public boolean hasNext() {
             checkStorageClosedOrInProcessOfRebalance();
 
-            advanceIfNeeded();
+            if (hasNext != null) {
+                return hasNext;
+            }
+
+            currentRow = peekedRow == NO_PEEKED_ROW ? peek() : peekedRow;
+            peekedRow = NO_PEEKED_ROW;
 
+            hasNext = currentRow != null;
             return hasNext;
         }
 
         @Override
         public IndexRow next() {
-            checkStorageClosedOrInProcessOfRebalance();
-
-            advanceIfNeeded();
-
-            boolean hasNext = this.hasNext;
-
-            if (!hasNext) {
+            if (!hasNext()) {
                 throw new NoSuchElementException();
             }
 
             this.hasNext = null;
 
-            return new IndexRowImpl(new BinaryTuple(descriptor.binaryTupleSchema(), currentEntry.getKey()), rowId);
+            return currentRow;
         }
 
         @Override
         public @Nullable IndexRow peek() {
             checkStorageClosedOrInProcessOfRebalance();
 
             if (hasNext != null) {
-                if (hasNext) {
-                    return new IndexRowImpl(new BinaryTuple(descriptor.binaryTupleSchema(), currentEntry.getKey()), rowId);
-                }
-
-                return null;
+                return currentRow;
             }
 
-            Entry<ByteBuffer, NavigableMap<RowId, Object>> indexMapEntry0 = currentEntry == null ? indexMap.firstEntry() : currentEntry;
-
-            RowId nextRowId = null;
-
-            if (rowId == null) {
-                if (indexMapEntry0 != null) {
-                    nextRowId = getRowId(indexMapEntry0.getValue().firstEntry());
-                }
-            } else {
-                Entry<RowId, Object> nextRowIdEntry = indexMapEntry0.getValue().higherEntry(rowId);
-
-                if (nextRowIdEntry != null) {
-                    nextRowId = nextRowIdEntry.getKey();
-                } else {
-                    indexMapEntry0 = indexMap.higherEntry(indexMapEntry0.getKey());
-
-                    if (indexMapEntry0 != null) {
-                        nextRowId = getRowId(indexMapEntry0.getValue().firstEntry());
-                    }
-                }
-            }
-
-            return nextRowId == null
-                    ? null : new IndexRowImpl(new BinaryTuple(descriptor.binaryTupleSchema(), indexMapEntry0.getKey()), nextRowId);
-        }
-
-        private void advanceIfNeeded() {
-            if (hasNext != null) {
-                return;
-            }
-
-            if (currentEntry == null) {
-                currentEntry = indexMap.firstEntry();
-            }
-
-            if (rowId == null) {
-                if (currentEntry != null) {
-                    rowId = getRowId(currentEntry.getValue().firstEntry());
+            if (currentRow == null) {
+                try {
+                    peekedRow = indexSet.first();
+                } catch (NoSuchElementException e) {
+                    peekedRow = null;

Review Comment:
   I wouldn't use the performance argument in such case, it's too preliminary.
   > it can return null instead of an exception
   
   Ok, why is it better? Is this really a good reason to use the structure that doesn't fit my goals?



-- 
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] tkalkirill commented on a diff in pull request #2021: IGNITE-19394 "peek" contract improved to avoid missing data on RW scans.

Posted by "tkalkirill (via GitHub)" <gi...@apache.org>.
tkalkirill commented on code in PR #2021:
URL: https://github.com/apache/ignite-3/pull/2021#discussion_r1184816510


##########
modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/impl/TestSortedIndexStorage.java:
##########
@@ -83,10 +79,19 @@ Iterator<RowId> getRowIdIteratorForGetByBinaryTuple(BinaryTuple key) {
         BinaryTuplePrefix higherBound = BinaryTuplePrefix.fromBinaryTuple(key);
 
         //noinspection resource
-        return scan(lowerBound, higherBound, GREATER_OR_EQUAL | LESS_OR_EQUAL)
-                .stream()
-                .map(IndexRow::rowId)
-                .iterator();
+        PeekCursor<IndexRow> peekCursor = scan(lowerBound, higherBound, GREATER_OR_EQUAL | LESS_OR_EQUAL);
+
+        return new Iterator<>() {

Review Comment:
   Exactly, nothing will stop us from doing a similar method but for iterators.
   
   If this approach is used in `AbstractTestIndexStorage#get`, this does not mean that it is good, I think you should not foul the code with an anonymous class, but use a separate method / class.



-- 
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] tkalkirill commented on a diff in pull request #2021: IGNITE-19394 "peek" contract improved to avoid missing data on RW scans.

Posted by "tkalkirill (via GitHub)" <gi...@apache.org>.
tkalkirill commented on code in PR #2021:
URL: https://github.com/apache/ignite-3/pull/2021#discussion_r1184812297


##########
modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/impl/TestSortedIndexStorage.java:
##########
@@ -72,31 +78,29 @@ public SortedIndexDescriptor indexDescriptor() {
 
     @Override
     Iterator<RowId> getRowIdIteratorForGetByBinaryTuple(BinaryTuple key) {
-        return index.getOrDefault(key.byteBuffer(), emptyNavigableMap()).keySet().iterator();
+        // These must be two different instances, because "scan" call messes up headers.
+        BinaryTuplePrefix lowerBound = BinaryTuplePrefix.fromBinaryTuple(key);
+        BinaryTuplePrefix higherBound = BinaryTuplePrefix.fromBinaryTuple(key);
+
+        //noinspection resource
+        return scan(lowerBound, higherBound, GREATER_OR_EQUAL | LESS_OR_EQUAL)
+                .stream()
+                .map(IndexRow::rowId)
+                .iterator();

Review Comment:
   I think since you fixed it here, then you need to test it 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] tkalkirill commented on a diff in pull request #2021: IGNITE-19394 "peek" contract improved to avoid missing data on RW scans.

Posted by "tkalkirill (via GitHub)" <gi...@apache.org>.
tkalkirill commented on code in PR #2021:
URL: https://github.com/apache/ignite-3/pull/2021#discussion_r1184810811


##########
modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/AbstractSortedIndexStorageTest.java:
##########
@@ -1506,6 +1505,49 @@ void testScanPeekReplaceRow() {
         assertNull(scan.peek());
     }
 
+    @Test
+    void testScanPeekRemoveNext() {
+        SortedIndexDefinition indexDefinition = SchemaBuilders.sortedIndex("TEST_IDX")
+                .addIndexColumn(ColumnType.INT32.typeSpec().name()).asc().done()
+                .build();
+
+        SortedIndexStorage indexStorage = createIndexStorage(indexDefinition);
+
+        BinaryTupleRowSerializer serializer = new BinaryTupleRowSerializer(indexStorage.indexDescriptor());
+
+        PeekCursor<IndexRow> scan = indexStorage.scan(null, null, 0);
+
+        RowId rowId = new RowId(TEST_PARTITION);
+
+        // index  =  [0]
+        // cursor = ^ with no cached row
+        put(indexStorage, serializer.serializeRow(new Object[]{0}, rowId));
+
+        // index  =  [0] [1]
+        // cursor = ^ with no cached row
+        assertEquals(SimpleRow.of(0, rowId), SimpleRow.of(scan.peek(), firstColumn(serializer)));
+
+        // index  =
+        // cursor = ^ with no cached row (but it remembers the last peek call)
+        remove(indexStorage, serializer.serializeRow(new Object[]{0}, rowId));
+
+        // "hasNext" and "next" must return the result of last "peek" operation. This is crucial for RW scans.
+        assertTrue(scan.hasNext());
+        assertEquals(SimpleRow.of(0, rowId), SimpleRow.of(scan.next(), firstColumn(serializer)));
+
+        // index  =
+        // cursor = ^ with no cached row
+        assertNull(scan.peek());

Review Comment:
   I looked at `testScanPeekForFinishedCursor`, it checks my question.



-- 
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] tkalkirill commented on a diff in pull request #2021: IGNITE-19394 "peek" contract improved to avoid missing data on RW scans.

Posted by "tkalkirill (via GitHub)" <gi...@apache.org>.
tkalkirill commented on code in PR #2021:
URL: https://github.com/apache/ignite-3/pull/2021#discussion_r1184785303


##########
modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/AbstractSortedIndexStorageTest.java:
##########
@@ -1506,6 +1505,49 @@ void testScanPeekReplaceRow() {
         assertNull(scan.peek());
     }
 
+    @Test
+    void testScanPeekRemoveNext() {
+        SortedIndexDefinition indexDefinition = SchemaBuilders.sortedIndex("TEST_IDX")
+                .addIndexColumn(ColumnType.INT32.typeSpec().name()).asc().done()
+                .build();
+
+        SortedIndexStorage indexStorage = createIndexStorage(indexDefinition);
+
+        BinaryTupleRowSerializer serializer = new BinaryTupleRowSerializer(indexStorage.indexDescriptor());
+
+        PeekCursor<IndexRow> scan = indexStorage.scan(null, null, 0);
+
+        RowId rowId = new RowId(TEST_PARTITION);
+
+        // index  =  [0]
+        // cursor = ^ with no cached row
+        put(indexStorage, serializer.serializeRow(new Object[]{0}, rowId));
+
+        // index  =  [0] [1]
+        // cursor = ^ with no cached row
+        assertEquals(SimpleRow.of(0, rowId), SimpleRow.of(scan.peek(), firstColumn(serializer)));
+
+        // index  =
+        // cursor = ^ with no cached row (but it remembers the last peek call)
+        remove(indexStorage, serializer.serializeRow(new Object[]{0}, rowId));
+
+        // "hasNext" and "next" must return the result of last "peek" operation. This is crucial for RW scans.
+        assertTrue(scan.hasNext());
+        assertEquals(SimpleRow.of(0, rowId), SimpleRow.of(scan.next(), firstColumn(serializer)));
+
+        // index  =
+        // cursor = ^ with no cached row
+        assertNull(scan.peek());

Review Comment:
   I suggest adding this check so as not to assume.



-- 
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 diff in pull request #2021: IGNITE-19394 "peek" contract improved to avoid missing data on RW scans.

Posted by "ibessonov (via GitHub)" <gi...@apache.org>.
ibessonov commented on code in PR #2021:
URL: https://github.com/apache/ignite-3/pull/2021#discussion_r1184804387


##########
modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/impl/TestSortedIndexStorage.java:
##########
@@ -118,23 +122,34 @@ public PeekCursor<IndexRow> scan(
             setEqualityFlag(upperBound);
         }
 
-        NavigableMap<ByteBuffer, NavigableMap<RowId, Object>> navigableMap;
+        NavigableSet<IndexRow> navigableSet;
 
         if (lowerBound == null && upperBound == null) {
-            navigableMap = index;
+            navigableSet = index;
         } else if (lowerBound == null) {
-            navigableMap = index.headMap(upperBound.byteBuffer());
+            navigableSet = index.headSet(prefixToIndexRow(upperBound, highestRowId(partitionId)), true);
         } else if (upperBound == null) {
-            navigableMap = index.tailMap(lowerBound.byteBuffer());
+            navigableSet = index.tailSet(prefixToIndexRow(lowerBound, lowestRowId(partitionId)), true);
         } else {
             try {
-                navigableMap = index.subMap(lowerBound.byteBuffer(), upperBound.byteBuffer());
+                navigableSet = index.subSet(
+                        prefixToIndexRow(lowerBound, lowestRowId(partitionId)),
+                        true,
+                        prefixToIndexRow(upperBound, highestRowId(partitionId)),
+                        true
+                );
             } catch (IllegalArgumentException e) {
-                navigableMap = emptyNavigableMap();
+                navigableSet = emptyNavigableSet();

Review Comment:
   `subSet` documentation is explicit about it, but ok, I'll add a comment.



-- 
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 diff in pull request #2021: IGNITE-19394 "peek" contract improved to avoid missing data on RW scans.

Posted by "ibessonov (via GitHub)" <gi...@apache.org>.
ibessonov commented on code in PR #2021:
URL: https://github.com/apache/ignite-3/pull/2021#discussion_r1184811160


##########
modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/impl/TestSortedIndexStorage.java:
##########
@@ -72,31 +78,29 @@ public SortedIndexDescriptor indexDescriptor() {
 
     @Override
     Iterator<RowId> getRowIdIteratorForGetByBinaryTuple(BinaryTuple key) {
-        return index.getOrDefault(key.byteBuffer(), emptyNavigableMap()).keySet().iterator();
+        // These must be two different instances, because "scan" call messes up headers.
+        BinaryTuplePrefix lowerBound = BinaryTuplePrefix.fromBinaryTuple(key);
+        BinaryTuplePrefix higherBound = BinaryTuplePrefix.fromBinaryTuple(key);
+
+        //noinspection resource
+        return scan(lowerBound, higherBound, GREATER_OR_EQUAL | LESS_OR_EQUAL)
+                .stream()
+                .map(IndexRow::rowId)
+                .iterator();

Review Comment:
   I fixed it actually. Should I add tests here or in a separate issue?



-- 
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 diff in pull request #2021: IGNITE-19394 "peek" contract improved to avoid missing data on RW scans.

Posted by "ibessonov (via GitHub)" <gi...@apache.org>.
ibessonov commented on code in PR #2021:
URL: https://github.com/apache/ignite-3/pull/2021#discussion_r1185144101


##########
modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/impl/TestSortedIndexStorage.java:
##########
@@ -72,31 +78,29 @@ public SortedIndexDescriptor indexDescriptor() {
 
     @Override
     Iterator<RowId> getRowIdIteratorForGetByBinaryTuple(BinaryTuple key) {
-        return index.getOrDefault(key.byteBuffer(), emptyNavigableMap()).keySet().iterator();
+        // These must be two different instances, because "scan" call messes up headers.
+        BinaryTuplePrefix lowerBound = BinaryTuplePrefix.fromBinaryTuple(key);
+        BinaryTuplePrefix higherBound = BinaryTuplePrefix.fromBinaryTuple(key);
+
+        //noinspection resource
+        return scan(lowerBound, higherBound, GREATER_OR_EQUAL | LESS_OR_EQUAL)
+                .stream()
+                .map(IndexRow::rowId)
+                .iterator();

Review Comment:
   That's too much work. I added a test and created issue to fix all storages: https://issues.apache.org/jira/browse/IGNITE-19422



-- 
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 diff in pull request #2021: IGNITE-19394 "peek" contract improved to avoid missing data on RW scans.

Posted by "ibessonov (via GitHub)" <gi...@apache.org>.
ibessonov commented on code in PR #2021:
URL: https://github.com/apache/ignite-3/pull/2021#discussion_r1184838881


##########
modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/impl/TestSortedIndexStorage.java:
##########
@@ -83,10 +79,19 @@ Iterator<RowId> getRowIdIteratorForGetByBinaryTuple(BinaryTuple key) {
         BinaryTuplePrefix higherBound = BinaryTuplePrefix.fromBinaryTuple(key);
 
         //noinspection resource
-        return scan(lowerBound, higherBound, GREATER_OR_EQUAL | LESS_OR_EQUAL)
-                .stream()
-                .map(IndexRow::rowId)
-                .iterator();
+        PeekCursor<IndexRow> peekCursor = scan(lowerBound, higherBound, GREATER_OR_EQUAL | LESS_OR_EQUAL);
+
+        return new Iterator<>() {

Review Comment:
   Ok, I'll fix it then, not a big deal



-- 
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 diff in pull request #2021: IGNITE-19394 "peek" contract improved to avoid missing data on RW scans.

Posted by "ibessonov (via GitHub)" <gi...@apache.org>.
ibessonov commented on code in PR #2021:
URL: https://github.com/apache/ignite-3/pull/2021#discussion_r1184813457


##########
modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/impl/TestSortedIndexStorage.java:
##########
@@ -72,31 +78,29 @@ public SortedIndexDescriptor indexDescriptor() {
 
     @Override
     Iterator<RowId> getRowIdIteratorForGetByBinaryTuple(BinaryTuple key) {
-        return index.getOrDefault(key.byteBuffer(), emptyNavigableMap()).keySet().iterator();
+        // These must be two different instances, because "scan" call messes up headers.
+        BinaryTuplePrefix lowerBound = BinaryTuplePrefix.fromBinaryTuple(key);
+        BinaryTuplePrefix higherBound = BinaryTuplePrefix.fromBinaryTuple(key);
+
+        //noinspection resource
+        return scan(lowerBound, higherBound, GREATER_OR_EQUAL | LESS_OR_EQUAL)
+                .stream()
+                .map(IndexRow::rowId)
+                .iterator();

Review Comment:
   Great



-- 
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 merged pull request #2021: IGNITE-19394 "peek" contract improved to avoid missing data on RW scans.

Posted by "ibessonov (via GitHub)" <gi...@apache.org>.
ibessonov merged PR #2021:
URL: https://github.com/apache/ignite-3/pull/2021


-- 
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] tkalkirill commented on a diff in pull request #2021: IGNITE-19394 "peek" contract improved to avoid missing data on RW scans.

Posted by "tkalkirill (via GitHub)" <gi...@apache.org>.
tkalkirill commented on code in PR #2021:
URL: https://github.com/apache/ignite-3/pull/2021#discussion_r1184787821


##########
modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/impl/TestSortedIndexStorage.java:
##########
@@ -118,23 +122,34 @@ public PeekCursor<IndexRow> scan(
             setEqualityFlag(upperBound);
         }
 
-        NavigableMap<ByteBuffer, NavigableMap<RowId, Object>> navigableMap;
+        NavigableSet<IndexRow> navigableSet;
 
         if (lowerBound == null && upperBound == null) {
-            navigableMap = index;
+            navigableSet = index;
         } else if (lowerBound == null) {
-            navigableMap = index.headMap(upperBound.byteBuffer());
+            navigableSet = index.headSet(prefixToIndexRow(upperBound, highestRowId(partitionId)), true);
         } else if (upperBound == null) {
-            navigableMap = index.tailMap(lowerBound.byteBuffer());
+            navigableSet = index.tailSet(prefixToIndexRow(lowerBound, lowestRowId(partitionId)), true);
         } else {
             try {
-                navigableMap = index.subMap(lowerBound.byteBuffer(), upperBound.byteBuffer());
+                navigableSet = index.subSet(
+                        prefixToIndexRow(lowerBound, lowestRowId(partitionId)),
+                        true,
+                        prefixToIndexRow(upperBound, highestRowId(partitionId)),
+                        true
+                );
             } catch (IllegalArgumentException e) {
-                navigableMap = emptyNavigableMap();
+                navigableSet = emptyNavigableSet();

Review Comment:
   Please leave a comment.



-- 
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 diff in pull request #2021: IGNITE-19394 "peek" contract improved to avoid missing data on RW scans.

Posted by "ibessonov (via GitHub)" <gi...@apache.org>.
ibessonov commented on code in PR #2021:
URL: https://github.com/apache/ignite-3/pull/2021#discussion_r1184809717


##########
modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/impl/TestSortedIndexStorage.java:
##########
@@ -83,10 +79,19 @@ Iterator<RowId> getRowIdIteratorForGetByBinaryTuple(BinaryTuple key) {
         BinaryTuplePrefix higherBound = BinaryTuplePrefix.fromBinaryTuple(key);
 
         //noinspection resource
-        return scan(lowerBound, higherBound, GREATER_OR_EQUAL | LESS_OR_EQUAL)
-                .stream()
-                .map(IndexRow::rowId)
-                .iterator();
+        PeekCursor<IndexRow> peekCursor = scan(lowerBound, higherBound, GREATER_OR_EQUAL | LESS_OR_EQUAL);
+
+        return new Iterator<>() {

Review Comment:
   Wait a minute, it uses a collection, not iterator.
   I used anonymous class because it's small. It's the same approach that's used in `org.apache.ignite.internal.storage.index.impl.AbstractTestIndexStorage#get`, for example, do you have issues with that 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 diff in pull request #2021: IGNITE-19394 "peek" contract improved to avoid missing data on RW scans.

Posted by "ibessonov (via GitHub)" <gi...@apache.org>.
ibessonov commented on code in PR #2021:
URL: https://github.com/apache/ignite-3/pull/2021#discussion_r1184808166


##########
modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/impl/TestSortedIndexStorage.java:
##########
@@ -83,10 +79,19 @@ Iterator<RowId> getRowIdIteratorForGetByBinaryTuple(BinaryTuple key) {
         BinaryTuplePrefix higherBound = BinaryTuplePrefix.fromBinaryTuple(key);
 
         //noinspection resource
-        return scan(lowerBound, higherBound, GREATER_OR_EQUAL | LESS_OR_EQUAL)
-                .stream()
-                .map(IndexRow::rowId)
-                .iterator();
+        PeekCursor<IndexRow> peekCursor = scan(lowerBound, higherBound, GREATER_OR_EQUAL | LESS_OR_EQUAL);
+
+        return new Iterator<>() {

Review Comment:
   Ok, I'll use "viewReadOnly", I forgot about its existence.



-- 
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] tkalkirill commented on a diff in pull request #2021: IGNITE-19394 "peek" contract improved to avoid missing data on RW scans.

Posted by "tkalkirill (via GitHub)" <gi...@apache.org>.
tkalkirill commented on code in PR #2021:
URL: https://github.com/apache/ignite-3/pull/2021#discussion_r1184791475


##########
modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/AbstractSortedIndexStorageTest.java:
##########
@@ -1506,6 +1505,49 @@ void testScanPeekReplaceRow() {
         assertNull(scan.peek());
     }
 
+    @Test
+    void testScanPeekRemoveNext() {
+        SortedIndexDefinition indexDefinition = SchemaBuilders.sortedIndex("TEST_IDX")
+                .addIndexColumn(ColumnType.INT32.typeSpec().name()).asc().done()
+                .build();
+
+        SortedIndexStorage indexStorage = createIndexStorage(indexDefinition);
+
+        BinaryTupleRowSerializer serializer = new BinaryTupleRowSerializer(indexStorage.indexDescriptor());
+
+        PeekCursor<IndexRow> scan = indexStorage.scan(null, null, 0);
+
+        RowId rowId = new RowId(TEST_PARTITION);
+
+        // index  =  [0]
+        // cursor = ^ with no cached row
+        put(indexStorage, serializer.serializeRow(new Object[]{0}, rowId));
+
+        // index  =  [0] [1]
+        // cursor = ^ with no cached row
+        assertEquals(SimpleRow.of(0, rowId), SimpleRow.of(scan.peek(), firstColumn(serializer)));
+
+        // index  =
+        // cursor = ^ with no cached row (but it remembers the last peek call)
+        remove(indexStorage, serializer.serializeRow(new Object[]{0}, rowId));
+
+        // "hasNext" and "next" must return the result of last "peek" operation. This is crucial for RW scans.
+        assertTrue(scan.hasNext());
+        assertEquals(SimpleRow.of(0, rowId), SimpleRow.of(scan.next(), firstColumn(serializer)));
+
+        // index  =
+        // cursor = ^ with no cached row
+        assertNull(scan.peek());

Review Comment:
   Then why a new test, can you improve the current ones?



-- 
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 diff in pull request #2021: IGNITE-19394 "peek" contract improved to avoid missing data on RW scans.

Posted by "ibessonov (via GitHub)" <gi...@apache.org>.
ibessonov commented on code in PR #2021:
URL: https://github.com/apache/ignite-3/pull/2021#discussion_r1184793864


##########
modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/impl/TestSortedIndexStorage.java:
##########
@@ -72,31 +78,29 @@ public SortedIndexDescriptor indexDescriptor() {
 
     @Override
     Iterator<RowId> getRowIdIteratorForGetByBinaryTuple(BinaryTuple key) {
-        return index.getOrDefault(key.byteBuffer(), emptyNavigableMap()).keySet().iterator();
+        // These must be two different instances, because "scan" call messes up headers.
+        BinaryTuplePrefix lowerBound = BinaryTuplePrefix.fromBinaryTuple(key);
+        BinaryTuplePrefix higherBound = BinaryTuplePrefix.fromBinaryTuple(key);
+
+        //noinspection resource
+        return scan(lowerBound, higherBound, GREATER_OR_EQUAL | LESS_OR_EQUAL)
+                .stream()
+                .map(IndexRow::rowId)
+                .iterator();

Review Comment:
   By the way, this property is already violated in hash 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 diff in pull request #2021: IGNITE-19394 "peek" contract improved to avoid missing data on RW scans.

Posted by "ibessonov (via GitHub)" <gi...@apache.org>.
ibessonov commented on code in PR #2021:
URL: https://github.com/apache/ignite-3/pull/2021#discussion_r1184777083


##########
modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/AbstractSortedIndexStorageTest.java:
##########
@@ -1506,6 +1505,49 @@ void testScanPeekReplaceRow() {
         assertNull(scan.peek());
     }
 
+    @Test
+    void testScanPeekRemoveNext() {
+        SortedIndexDefinition indexDefinition = SchemaBuilders.sortedIndex("TEST_IDX")
+                .addIndexColumn(ColumnType.INT32.typeSpec().name()).asc().done()
+                .build();
+
+        SortedIndexStorage indexStorage = createIndexStorage(indexDefinition);
+
+        BinaryTupleRowSerializer serializer = new BinaryTupleRowSerializer(indexStorage.indexDescriptor());
+
+        PeekCursor<IndexRow> scan = indexStorage.scan(null, null, 0);
+
+        RowId rowId = new RowId(TEST_PARTITION);
+
+        // index  =  [0]
+        // cursor = ^ with no cached row
+        put(indexStorage, serializer.serializeRow(new Object[]{0}, rowId));
+
+        // index  =  [0] [1]

Review Comment:
   That's right, I forgot to remove it, thank you!



##########
modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/impl/TestSortedIndexStorage.java:
##########
@@ -61,8 +54,21 @@ public class TestSortedIndexStorage extends AbstractTestIndexStorage implements
     public TestSortedIndexStorage(int partitionId, SortedIndexDescriptor descriptor) {
         super(partitionId);
 
+        BinaryTupleComparator binaryTupleComparator = new BinaryTupleComparator(descriptor);
+
         this.descriptor = descriptor;
-        this.index = new ConcurrentSkipListMap<>(new BinaryTupleComparator(descriptor));
+        this.index = new ConcurrentSkipListSet<>((indexRow0, indexRow1) -> {

Review Comment:
   I'll make the code shorter instead



-- 
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 diff in pull request #2021: IGNITE-19394 "peek" contract improved to avoid missing data on RW scans.

Posted by "ibessonov (via GitHub)" <gi...@apache.org>.
ibessonov commented on code in PR #2021:
URL: https://github.com/apache/ignite-3/pull/2021#discussion_r1184777712


##########
modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/AbstractSortedIndexStorageTest.java:
##########
@@ -1506,6 +1505,49 @@ void testScanPeekReplaceRow() {
         assertNull(scan.peek());
     }
 
+    @Test
+    void testScanPeekRemoveNext() {
+        SortedIndexDefinition indexDefinition = SchemaBuilders.sortedIndex("TEST_IDX")
+                .addIndexColumn(ColumnType.INT32.typeSpec().name()).asc().done()
+                .build();
+
+        SortedIndexStorage indexStorage = createIndexStorage(indexDefinition);
+
+        BinaryTupleRowSerializer serializer = new BinaryTupleRowSerializer(indexStorage.indexDescriptor());
+
+        PeekCursor<IndexRow> scan = indexStorage.scan(null, null, 0);
+
+        RowId rowId = new RowId(TEST_PARTITION);
+
+        // index  =  [0]
+        // cursor = ^ with no cached row
+        put(indexStorage, serializer.serializeRow(new Object[]{0}, rowId));
+
+        // index  =  [0] [1]
+        // cursor = ^ with no cached row
+        assertEquals(SimpleRow.of(0, rowId), SimpleRow.of(scan.peek(), firstColumn(serializer)));
+
+        // index  =
+        // cursor = ^ with no cached row (but it remembers the last peek call)
+        remove(indexStorage, serializer.serializeRow(new Object[]{0}, rowId));
+
+        // "hasNext" and "next" must return the result of last "peek" operation. This is crucial for RW scans.
+        assertTrue(scan.hasNext());
+        assertEquals(SimpleRow.of(0, rowId), SimpleRow.of(scan.next(), firstColumn(serializer)));
+
+        // index  =
+        // cursor = ^ with no cached row
+        assertNull(scan.peek());

Review Comment:
   I think it's already tested in your tests. New row will be returned



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