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

[GitHub] [ignite-3] sashapolo commented on a diff in pull request #1424: IGNITE-18243 Implement a peek method for the sorted index cursor

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: notifications-unsubscribe@ignite.apache.org

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