tkalkirill commented on code in PR #4437:
URL: https://github.com/apache/ignite-3/pull/4437#discussion_r1773193997


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/checkpoint/CheckpointPages.java:
##########
@@ -17,75 +17,174 @@
 
 package org.apache.ignite.internal.pagememory.persistence.checkpoint;
 
+import static 
org.apache.ignite.internal.pagememory.persistence.checkpoint.CheckpointState.PAGES_SORTED;
 import static org.apache.ignite.internal.util.IgniteUtils.getUninterruptibly;
 
 import java.util.Set;
 import java.util.concurrent.CancellationException;
-import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ExecutionException;
-import org.apache.ignite.internal.lang.IgniteInternalCheckedException;
 import org.apache.ignite.internal.pagememory.FullPageId;
+import org.apache.ignite.internal.pagememory.persistence.GroupPartitionId;
+import org.apache.ignite.internal.pagememory.persistence.PersistentPageMemory;
+import org.apache.ignite.internal.pagememory.persistence.store.FilePageStore;
+import 
org.apache.ignite.internal.pagememory.persistence.store.FilePageStoreManager;
+import org.apache.ignite.internal.storage.StorageException;
+import org.jetbrains.annotations.Nullable;
 
 /**
- * View of pages which should be stored during current checkpoint.
+ * Class contains dirty pages of the segment that will need to be written at a 
checkpoint or page replacement. It also contains helper
+ * methods before writing pages.
+ *
+ * <p>For correct parallel operation of the checkpoint writer and page 
replacement, external synchronization must be used, for example,
+ * segment read and write locks.</p>
  */
 public class CheckpointPages {
-    private final Set<FullPageId> segmentPages;
+    private final Set<FullPageId> pageIds;
 
-    private final CompletableFuture<?> allowToReplace;
+    private final CheckpointProgressImpl checkpointProgress;
 
     /**
      * Constructor.
      *
-     * @param pages Pages which would be stored to disk in current checkpoint, 
does not copy the set.
-     * @param replaceFuture The sign which allows replacing pages from a 
checkpoint by page replacer.
+     * @param pageIds Dirty page IDs in the segment that should be written at 
a checkpoint or page replacement.
+     * @param checkpointProgress Progress of the current checkpoint at which 
the object was created.
      */
-    public CheckpointPages(Set<FullPageId> pages, CompletableFuture<?> 
replaceFuture) {
-        segmentPages = pages;
-        allowToReplace = replaceFuture;
+    public CheckpointPages(Set<FullPageId> pageIds, CheckpointProgress 
checkpointProgress) {
+        this.pageIds = pageIds;
+        this.checkpointProgress = (CheckpointProgressImpl) checkpointProgress;
     }
 
     /**
-     * Returns {@code true} If fullPageId is allowable to store to disk.
+     * Checks if the page is available for replacement, without removing the 
page.
+     *
+     * <p>Page is available for replacement if the following conditions are 
met:</p>
+     * <ul>
+     *     <li>If the dirty page sorting phase is complete, otherwise we wait 
for it. This is necessary so that we can safely create
+     *     partition delta files in which the dirty page order must be 
preserved.</li>
+     *     <li>If the checkpoint dirty page writer has not started writing the 
page or has already written it.</li>
+     *     <li>If the delta file fsync phase is not ready to start or is not 
in progress. This is necessary so that the data remains
+     *     consistent after the fsync phase is complete. If the phase has not 
yet begun, we will block it until we complete the
+     *     replacement.</li>
+     * </ul>
      *
-     * @param fullPageId Page id for checking.
-     * @throws IgniteInternalCheckedException If the waiting sign which allows 
replacing pages from a checkpoint by page replacer fails.
+     * <p>It is expected that if the method returns {@code true}, it will not 
be invoked again for the same page ID, then
+     * {@link #finishReplace} will be invoked later.</p>
+     *
+     * @param pageId Page ID of the replacement candidate.
+     * @return {@code True} if the page is available for replacement, {@code 
false} if not.
+     * @throws StorageException If any error occurred while waiting for the 
dirty page sorting phase to complete at a checkpoint.
+     * @see #finishReplace(FullPageId, Throwable)
      */
-    public boolean allowToSave(FullPageId fullPageId) throws 
IgniteInternalCheckedException {
+    public boolean allowToReplace(FullPageId pageId) throws StorageException {

Review Comment:
   I also thought that we would need to correct both the body and the method 
name.



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