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


##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/AbstractPageMemoryMvPartitionStorage.java:
##########
@@ -1068,4 +1059,48 @@ public CompletableFuture<Void> destroyIndex(int indexId) 
{
      * @see MvPartitionStorage#estimatedSize
      */
     public abstract void decrementEstimatedSize();
+
+    private List<GcEntry> peekSingleGcEntryBusy(HybridTimestamp lowWatermark) {
+        GcRowVersion head = renewableState.gcQueue().getFirst();
+
+        // Garbage collection queue is empty.
+        if (head == null) {
+            return List.of();
+        }
+
+        HybridTimestamp rowTimestamp = head.getTimestamp();
+
+        // There are no versions in the garbage collection queue before 
watermark.
+        if (rowTimestamp.compareTo(lowWatermark) > 0) {
+            return List.of();
+        }
+
+        return List.of(head);
+    }
+
+    private List<GcEntry> peekGcEntriesBusy(HybridTimestamp lowWatermark, int 
count) {
+        var res = new ArrayList<GcEntry>(count);
+
+        try (Cursor<GcRowVersion> cursor = renewableState.gcQueue().find(null, 
null)) {
+            while (res.size() < count && cursor.hasNext()) {
+                GcRowVersion next = cursor.next();
+
+                if (next.getTimestamp().compareTo(lowWatermark) > 0) {
+                    break;
+                }
+
+                res.add(next);

Review Comment:
   The "preloading" step is missing, please also read the data here before 
returning it to the caller. This is the whole point of the issue - to 
heuristically reduce the IO inside of "runConsistently"



##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/gc/GcUpdateHandler.java:
##########
@@ -130,46 +133,32 @@ private VacuumResult internalVacuumBatch(HybridTimestamp 
lowWatermark, IntHolder
         });
     }
 
-    private VacuumResult internalVacuum(HybridTimestamp lowWatermark, Locker 
locker, boolean useTryLock) {
-        while (true) {
-            // Check if the storage engine needs resources before continuing.
-            if (locker.shouldRelease()) {
-                return VacuumResult.SHOULD_RELEASE;
-            }
-
-            GcEntry gcEntry = storage.peek(lowWatermark);
-
-            if (gcEntry == null) {
-                return VacuumResult.NO_GARBAGE_LEFT;
-            }
-
-            RowId rowId = gcEntry.getRowId();
+    private VacuumResult internalVacuum(GcEntry gcEntry, Locker locker, 
boolean useTryLock) {
+        RowId rowId = gcEntry.getRowId();
 
-            if (useTryLock) {
-                if (!locker.tryLock(rowId)) {
-                    return VacuumResult.FAILED_ACQUIRE_LOCK;
-                }
-            } else {
-                locker.lock(rowId);
+        if (useTryLock) {
+            if (!locker.tryLock(rowId)) {
+                return VacuumResult.FAILED_ACQUIRE_LOCK;
             }
+        } else {
+            locker.lock(rowId);
+        }
 
-            BinaryRow binaryRow = storage.vacuum(gcEntry);
-
-            if (binaryRow == null) {
-                // Removed by another thread, let's try to take another.
-                continue;
-            }
+        BinaryRow binaryRow = storage.vacuum(gcEntry);
 
-            try (Cursor<ReadResult> cursor = storage.scanVersions(rowId)) {
-                // TODO: IGNITE-21005 We need to choose only those indexes 
that are not available for transactions
-                indexUpdateHandler.tryRemoveFromIndexes(binaryRow, rowId, 
cursor, null);
-            }
+        if (binaryRow == null) {
+            return VacuumResult.REMOVED_BY_ANOTHER_THREAD;

Review Comment:
   I wonder why we introduced the "removed by another thread" case, does it 
ever happen in tests? How is it possible?



##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/AbstractPageMemoryMvPartitionStorage.java:
##########
@@ -962,27 +962,18 @@ public T map(VersionChain treeRow) {
     }
 
     @Override
-    public @Nullable GcEntry peek(HybridTimestamp lowWatermark) {
-        assert THREAD_LOCAL_LOCKER.get() != null;
-
-        // Assertion above guarantees that we're in "runConsistently" closure.
-        throwExceptionIfStorageNotInRunnableState();
-
-        GcRowVersion head = renewableState.gcQueue().getFirst();
-
-        // Garbage collection queue is empty.
-        if (head == null) {
-            return null;
-        }
-
-        HybridTimestamp rowTimestamp = head.getTimestamp();
+    public List<GcEntry> peek(HybridTimestamp lowWatermark, int count) {
+        return busy(() -> {
+            throwExceptionIfStorageNotInRunnableState();
 
-        // There are no versions in the garbage collection queue before 
watermark.
-        if (rowTimestamp.compareTo(lowWatermark) > 0) {
-            return null;
-        }
+            if (count <= 0) {
+                return List.of();
+            } else if (count == 1) {
+                return peekSingleGcEntryBusy(lowWatermark);
+            }
 
-        return head;
+            return peekGcEntriesBusy(lowWatermark, count);

Review Comment:
   Why is it necessary to have a dedicated method for the case of a single 
entry? Could you please clarify that with 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: [email protected]

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

Reply via email to