sashapolo commented on code in PR #1424:
URL: https://github.com/apache/ignite-3/pull/1424#discussion_r1045867457


##########
modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/index/RocksDbSortedIndexStorage.java:
##########
@@ -202,14 +208,47 @@ public ByteBuffer next() {
 
                 this.hasNext = null;
 
-                return ByteBuffer.wrap(key).order(ORDER);
+                return mapper.apply(ByteBuffer.wrap(key).order(ORDER));
+            }
+
+            @Override
+            public @Nullable T peek() {
+                if (hasNext != null && !hasNext) {
+                    return null;
+                }
+
+                T next = null;
+
+                refreshAndPrepareRocksIterator();
+
+                if (!it.isValid()) {
+                    RocksUtils.checkIterator(it);
+                } else {
+                    next = 
mapper.apply(ByteBuffer.wrap(it.key()).order(ORDER));
+                }
+
+                return next;
             }
 
             private void advanceIfNeeded() throws StorageException {
                 if (hasNext != null) {
                     return;
                 }
 
+                refreshAndPrepareRocksIterator();

Review Comment:
   We will now refresh the iterator on every `next` call as well, is that 
intended?



##########
modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/AbstractSortedIndexStorageTest.java:
##########
@@ -1233,6 +1235,276 @@ void testScanContractRemoveNextAndAddFirstRow() {
         assertThrows(NoSuchElementException.class, scan::next);
     }
 
+
+    @Test
+    void testScanPeekForFinishedCursor() {
+        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> scan0 = indexStorage.scan(null, null, 0);
+        PeekCursor<IndexRow> scan1 = indexStorage.scan(null, null, 0);
+
+        // index   =
+        // cursor0 = ^ already finished
+        assertFalse(scan0.hasNext());
+        assertNull(scan0.peek());
+
+        // index   =
+        // cursor1 = ^ already finished
+        assertThrows(NoSuchElementException.class, scan1::next);
+        assertNull(scan1.peek());
+
+        // index   =  [0]
+        // cursor0 = ^ already finished
+        // cursor1 = ^ already finished
+        put(indexStorage, serializer.serializeRow(new Object[]{0}, new 
RowId(TEST_PARTITION)));
+
+        assertNull(scan0.peek());

Review Comment:
   Shouldn't these cursor start returning 0, after it was inserted?



##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/sorted/PageMemorySortedIndexStorage.java:
##########
@@ -337,6 +338,31 @@ public IndexRow next() {
             }
         }
 
+        @Override
+        public @Nullable IndexRow peek() {
+            if (hasNext != null && !hasNext) {
+                return null;
+            }
+
+            try {
+                SortedIndexRow nextTreeRow;
+
+                if (treeRow == null) {
+                    nextTreeRow = lower == null ? sortedIndexTree.findFirst() 
: sortedIndexTree.findNext(lower, true);

Review Comment:
   Same here



##########
modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/AbstractSortedIndexStorageTest.java:
##########
@@ -1233,6 +1235,276 @@ void testScanContractRemoveNextAndAddFirstRow() {
         assertThrows(NoSuchElementException.class, scan::next);
     }
 
+
+    @Test
+    void testScanPeekForFinishedCursor() {
+        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> scan0 = indexStorage.scan(null, null, 0);
+        PeekCursor<IndexRow> scan1 = indexStorage.scan(null, null, 0);
+
+        // index   =
+        // cursor0 = ^ already finished
+        assertFalse(scan0.hasNext());
+        assertNull(scan0.peek());
+
+        // index   =
+        // cursor1 = ^ already finished
+        assertThrows(NoSuchElementException.class, scan1::next);
+        assertNull(scan1.peek());
+
+        // index   =  [0]
+        // cursor0 = ^ already finished
+        // cursor1 = ^ already finished
+        put(indexStorage, serializer.serializeRow(new Object[]{0}, new 
RowId(TEST_PARTITION)));
+
+        assertNull(scan0.peek());
+        assertNull(scan1.peek());
+
+        // index   =  [0]
+        // cursor0 = ^ no cached row
+        // cursor1 = ^ no cached row
+        scan0 = indexStorage.scan(null, null, 0);
+        scan1 = indexStorage.scan(null, null, 0);
+
+        assertEquals(0, serializer.deserializeColumns(scan0.peek())[0]);
+        assertEquals(0, serializer.deserializeColumns(scan1.peek())[0]);
+
+        // index   = [0]
+        // cursor0 =    ^ cached [0]
+        assertTrue(scan0.hasNext());

Review Comment:
   This is very strange: why does calling `hasNext` invalidate what `peek` 
returns? I thought only `next` can do that



##########
modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/impl/TestSortedIndexStorage.java:
##########
@@ -237,6 +238,38 @@ public IndexRow next() {
             return new IndexRowImpl(new 
BinaryTuple(descriptor.binaryTupleSchema(), indexMapEntry.getKey()), rowId);
         }
 
+        @Override
+        public @Nullable IndexRow peek() {
+            if (hasNext != null && !hasNext) {

Review Comment:
   This method is very similar to `advanceIfNeeded`, can we extract the common 
code?



##########
modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/index/RocksDbSortedIndexStorage.java:
##########
@@ -202,14 +208,47 @@ public ByteBuffer next() {
 
                 this.hasNext = null;
 
-                return ByteBuffer.wrap(key).order(ORDER);
+                return mapper.apply(ByteBuffer.wrap(key).order(ORDER));
+            }
+
+            @Override
+            public @Nullable T peek() {
+                if (hasNext != null && !hasNext) {
+                    return null;
+                }
+
+                T next = null;

Review Comment:
   This variable is redundant



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to