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/07/19 08:52:25 UTC

[GitHub] [ignite-3] tkalkirill opened a new pull request, #947: IGNITE-17372 Implement DeltaFilePageStore

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

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


-- 
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 #947: IGNITE-17372 Implement DeltaFilePageStore

Posted by GitBox <gi...@apache.org>.
tkalkirill commented on code in PR #947:
URL: https://github.com/apache/ignite-3/pull/947#discussion_r928624158


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStoreFactory.java:
##########
@@ -52,45 +54,67 @@ public FilePageStoreFactory(FileIoFactory fileIoFactory, int pageSize) {
     /**
      * Creates instance of {@link FilePageStore}.
      *
-     * <p>If the file exists, an attempt will be made to read its {@link FilePageStoreHeader header} and create the {@link FilePageStore}.
+     * <p>Page stores are created based on their headers, for a file page stores with no header, the latest version is generated for delta
+     * file page store files, headers must be present.
      *
-     * @param filePath File page store path.
-     * @param headerBuffer Buffer for reading {@link FilePageStoreHeader header} from {@code filePath}.
-     * @return File page store.
+     * @param headerBuffer Buffer for reading headers.
+     * @param filePageStorePath File page store path.
+     * @param deltaFilePaths Paths to existing delta files page stores of the file page storage.
      * @throws IgniteInternalCheckedException if failed
      */
-    public FilePageStore createPageStore(Path filePath, ByteBuffer headerBuffer) throws IgniteInternalCheckedException {
+    public FilePageStore createPageStore(
+            ByteBuffer headerBuffer,
+            Path filePageStorePath,
+            Path... deltaFilePaths
+    ) throws IgniteInternalCheckedException {
         assert headerBuffer.remaining() == pageSize : headerBuffer.remaining();
 
-        if (!Files.exists(filePath)) {
-            return createPageStore(filePath, new FilePageStoreHeader(latestVersion, pageSize));
+        if (!Files.exists(filePageStorePath)) {
+            assert deltaFilePaths.length == 0 : Arrays.toString(deltaFilePaths);
+
+            return createFilePageStore(filePageStorePath, new FilePageStoreHeader(latestFilePageStoreVersion, pageSize));
         }
 
-        try (FileIo fileIo = fileIoFactory.create(filePath)) {
-            FilePageStoreHeader header = readHeader(fileIo, headerBuffer);
+        try (FileIo fileIo = fileIoFactory.create(filePageStorePath)) {
+            FilePageStoreHeader header = FilePageStoreHeader.readHeader(fileIo, headerBuffer);
 
             if (header == null) {
-                header = new FilePageStoreHeader(latestVersion, pageSize);
+                header = new FilePageStoreHeader(latestFilePageStoreVersion, pageSize);
+            }
+
+            if (deltaFilePaths.length == 0) {
+                return createFilePageStore(filePageStorePath, header);
+            }
+
+            DeltaFilePageStoreIo[] deltaFileIos = new DeltaFilePageStoreIo[deltaFilePaths.length];
+
+            for (int i = 0; i < deltaFilePaths.length; i++) {
+                Path deltaFilePath = deltaFilePaths[i];
+
+                assert Files.exists(deltaFilePath) : deltaFilePath;
+
+                try (FileIo deltaFileIo = fileIoFactory.create(deltaFilePath)) {
+                    DeltaFilePageStoreIoHeader deltaFileHeader = DeltaFilePageStoreIoHeader.readHeader(deltaFileIo, headerBuffer.rewind());
+
+                    assert deltaFileHeader != null : deltaFileHeader;
+
+                    deltaFileIos[i] = createDeltaFilePageStoreIo(deltaFilePath, deltaFileHeader);
+                }
             }
 
-            return createPageStore(filePath, header);
+            return createFilePageStore(filePageStorePath, header, deltaFileIos);
         } catch (IOException e) {
-            throw new IgniteInternalCheckedException("Error while creating file page store [file=" + filePath + "]", e);
+            throw new IgniteInternalCheckedException("Error while creating file page store [file=" + filePageStorePath + "]", e);

Review Comment:
   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] tkalkirill commented on a diff in pull request #947: IGNITE-17372 Implement DeltaFilePageStore

Posted by GitBox <gi...@apache.org>.
tkalkirill commented on code in PR #947:
URL: https://github.com/apache/ignite-3/pull/947#discussion_r928588166


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/AbstractFilePageStoreIo.java:
##########
@@ -0,0 +1,611 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.pagememory.persistence.store;
+
+import static java.nio.ByteOrder.nativeOrder;
+import static java.nio.file.StandardOpenOption.CREATE;
+import static java.nio.file.StandardOpenOption.READ;
+import static java.nio.file.StandardOpenOption.WRITE;
+import static org.apache.ignite.internal.util.IgniteUtils.atomicMoveFile;
+import static org.apache.ignite.internal.util.IgniteUtils.hexInt;
+import static org.apache.ignite.internal.util.IgniteUtils.hexLong;
+import static org.apache.ignite.internal.util.IgniteUtils.toHexString;
+import static org.apache.ignite.lang.IgniteSystemProperties.getBoolean;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.ClosedByInterruptException;
+import java.nio.channels.ClosedChannelException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.apache.ignite.internal.fileio.FileIo;
+import org.apache.ignite.internal.fileio.FileIoFactory;
+import org.apache.ignite.internal.pagememory.io.PageIo;
+import org.apache.ignite.internal.pagememory.persistence.FastCrc;
+import org.apache.ignite.internal.pagememory.persistence.IgniteInternalDataIntegrityViolationException;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Abstract class for performing IO operations on file page storage.
+ */
+public abstract class AbstractFilePageStoreIo implements Closeable {
+    private final FileIoFactory ioFactory;
+
+    private final ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
+
+    /** Skip CRC calculation flag. */
+    // TODO: IGNITE-17011 Move to config
+    private final boolean skipCrc = getBoolean("IGNITE_PDS_SKIP_CRC");
+
+    private volatile Path filePath;

Review Comment:
   Discussed in person, I will not fix.



-- 
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 #947: IGNITE-17372 Implement DeltaFilePageStore

Posted by GitBox <gi...@apache.org>.
tkalkirill commented on code in PR #947:
URL: https://github.com/apache/ignite-3/pull/947#discussion_r928721971


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/DeltaFilePageStoreIoFactory.java:
##########
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.pagememory.persistence.store;
+
+/**
+ * Factory for creating {@link DeltaFilePageStoreIo}.
+ */
+@FunctionalInterface
+public interface DeltaFilePageStoreIoFactory {

Review Comment:
   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] tkalkirill commented on a diff in pull request #947: IGNITE-17372 Implement DeltaFilePageStore

Posted by GitBox <gi...@apache.org>.
tkalkirill commented on code in PR #947:
URL: https://github.com/apache/ignite-3/pull/947#discussion_r928609841


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/DeltaFilePageStoreIo.java:
##########
@@ -0,0 +1,113 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.pagememory.persistence.store;
+
+import static java.nio.ByteOrder.nativeOrder;
+import static java.util.Arrays.binarySearch;
+import static org.apache.ignite.internal.pagememory.persistence.store.DeltaFilePageStoreIoHeader.checkFileIndex;
+import static org.apache.ignite.internal.pagememory.persistence.store.DeltaFilePageStoreIoHeader.checkFilePageIndexes;
+import static org.apache.ignite.internal.pagememory.persistence.store.DeltaFilePageStoreIoHeader.readHeader;
+import static org.apache.ignite.internal.pagememory.persistence.store.PageStoreUtils.checkFilePageSize;
+import static org.apache.ignite.internal.pagememory.persistence.store.PageStoreUtils.checkFileVersion;
+import static org.apache.ignite.internal.pagememory.util.PageIdUtils.pageIndex;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.file.Path;
+import org.apache.ignite.internal.fileio.FileIo;
+import org.apache.ignite.internal.fileio.FileIoFactory;
+
+/**
+ * Implementation of the class for working with the delta file page storage IO.
+ */
+public class DeltaFilePageStoreIo extends AbstractFilePageStoreIo {
+    private final DeltaFilePageStoreIoHeader header;
+
+    /**
+     * Constructor.
+     *
+     * @param ioFactory {@link FileIo} factory.
+     * @param filePath File page store path.
+     * @param header Delta file page store header.
+     */
+    public DeltaFilePageStoreIo(
+            FileIoFactory ioFactory,
+            Path filePath,
+            DeltaFilePageStoreIoHeader header
+    ) {
+        super(ioFactory, filePath);
+
+        this.header = header;
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public int pageSize() {
+        return header.pageSize();
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public int headerSize() {
+        return header.headerSize();
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public ByteBuffer headerBuffer() {
+        return header.toByteBuffer();
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public void checkHeader(FileIo fileIo) throws IOException {
+        DeltaFilePageStoreIoHeader header = readHeader(fileIo, ByteBuffer.allocate(pageSize()).order(nativeOrder()));
+
+        if (header == null) {
+            throw new IOException("Missing file header");
+        }
+
+        checkFileVersion(this.header.version(), header.version());
+        checkFileIndex(this.header.index(), header.index());
+        checkFilePageSize(this.header.pageSize(), header.pageSize());
+        checkFilePageIndexes(this.header.pageIndexes(), header.pageIndexes());
+    }
+
+    /**
+     * Returns page offset within the store file, {@code -1} if not found page in delta file.

Review Comment:
   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] tkalkirill commented on a diff in pull request #947: IGNITE-17372 Implement DeltaFilePageStore

Posted by GitBox <gi...@apache.org>.
tkalkirill commented on code in PR #947:
URL: https://github.com/apache/ignite-3/pull/947#discussion_r928670844


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/PartitionMetaManager.java:
##########
@@ -95,10 +95,8 @@ public PartitionMeta readOrCreateMeta(
         long partitionMetaPageId = partitionMetaPageId(groupPartitionId.getPartitionId());
 
         try {
-            if (filePageStore.size() > filePageStore.headerSize()) {
+            if (filePageStore.readWithoutPageIdCheck(partitionMetaPageId, buffer, false)) {

Review Comment:
   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] tkalkirill commented on a diff in pull request #947: IGNITE-17372 Implement DeltaFilePageStore

Posted by GitBox <gi...@apache.org>.
tkalkirill commented on code in PR #947:
URL: https://github.com/apache/ignite-3/pull/947#discussion_r928603824


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/AbstractFilePageStoreIo.java:
##########
@@ -0,0 +1,611 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.pagememory.persistence.store;
+
+import static java.nio.ByteOrder.nativeOrder;
+import static java.nio.file.StandardOpenOption.CREATE;
+import static java.nio.file.StandardOpenOption.READ;
+import static java.nio.file.StandardOpenOption.WRITE;
+import static org.apache.ignite.internal.util.IgniteUtils.atomicMoveFile;
+import static org.apache.ignite.internal.util.IgniteUtils.hexInt;
+import static org.apache.ignite.internal.util.IgniteUtils.hexLong;
+import static org.apache.ignite.internal.util.IgniteUtils.toHexString;
+import static org.apache.ignite.lang.IgniteSystemProperties.getBoolean;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.ClosedByInterruptException;
+import java.nio.channels.ClosedChannelException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.apache.ignite.internal.fileio.FileIo;
+import org.apache.ignite.internal.fileio.FileIoFactory;
+import org.apache.ignite.internal.pagememory.io.PageIo;
+import org.apache.ignite.internal.pagememory.persistence.FastCrc;
+import org.apache.ignite.internal.pagememory.persistence.IgniteInternalDataIntegrityViolationException;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Abstract class for performing IO operations on file page storage.
+ */
+public abstract class AbstractFilePageStoreIo implements Closeable {
+    private final FileIoFactory ioFactory;
+
+    private final ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
+
+    /** Skip CRC calculation flag. */
+    // TODO: IGNITE-17011 Move to config
+    private final boolean skipCrc = getBoolean("IGNITE_PDS_SKIP_CRC");
+
+    private volatile Path filePath;
+
+    private volatile @Nullable FileIo fileIo;
+
+    /** Initialized file page store IO. */
+    private volatile boolean initialized;
+
+    /** Caches the existence state of file. After it is initialized, it will be not {@code null} during lifecycle. */
+    private volatile @Nullable Boolean fileExists;
+
+    /**
+     * Constructor.
+     *
+     * @param ioFactory {@link FileIo} factory.
+     * @param filePath File page store path.
+     */
+    AbstractFilePageStoreIo(FileIoFactory ioFactory, Path filePath) {
+        this.ioFactory = ioFactory;
+        this.filePath = filePath;
+    }
+
+    /**
+     * Returns the page size in bytes.
+     */
+    public abstract int pageSize();
+
+    /**
+     * Returns the size of the header in bytes.
+     */
+    public abstract int headerSize();
+
+    /**
+     * Returns a buffer with a file page storage header.
+     */
+    public abstract ByteBuffer headerBuffer();
+
+    /**
+     * Checks the file page storage header.
+     *
+     * @param fileIo File page store IO to read the header.
+     * @throws IOException If there is an error reading the header or the header did not pass the check.
+     */
+    public abstract void checkHeader(FileIo fileIo) throws IOException;
+
+    /**
+     * Returns page offset within the store file.
+     *
+     * @param pageId Page ID.
+     */
+    public abstract long pageOffset(long pageId);
+
+    /**
+     * Stops the file page store IO.
+     *
+     * @param clean {@code True} to clean file page store.
+     * @throws IgniteInternalCheckedException If failed.
+     */
+    public void stop(boolean clean) throws IgniteInternalCheckedException {
+        try {
+            stop0(clean);
+        } catch (IOException e) {
+            throw new IgniteInternalCheckedException(
+                    "Failed to stop serving file [file=" + filePath + ", delete=" + clean + "]",
+                    e
+            );
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public void close() throws IOException {
+        stop0(false);
+    }
+
+    /**
+     * Reads a page.
+     *
+     * @param pageId Page ID.
+     * @param pageOff Page offset in the file.
+     * @param pageBuf Page buffer to read into.
+     * @param keepCrc By default, reading zeroes CRC which was on page store, but you can keep it in {@code pageBuf} if set {@code
+     * keepCrc}.
+     * @return {@code True} if the page was read successfully.
+     * @throws IgniteInternalCheckedException If reading failed (IO error occurred).
+     */
+    public boolean read(long pageId, long pageOff, ByteBuffer pageBuf, boolean keepCrc) throws IgniteInternalCheckedException {
+        return read0(pageId, pageOff, pageBuf, !skipCrc, keepCrc);
+    }
+
+    /**
+     * Writes a page.
+     *
+     * @param pageId Page ID.
+     * @param pageBuf Page buffer to write from.
+     * @param calculateCrc If {@code false} crc calculation will be forcibly skipped.
+     * @throws IgniteInternalCheckedException If page writing failed (IO error occurred).
+     */
+    public void write(long pageId, ByteBuffer pageBuf, boolean calculateCrc) throws IgniteInternalCheckedException {
+        ensure();
+
+        boolean interrupted = false;
+
+        while (true) {
+            FileIo fileIo = this.fileIo;
+
+            try {
+                readWriteLock.readLock().lock();
+
+                try {
+                    assert pageBuf.position() == 0 : pageBuf.position();
+                    assert pageBuf.order() == nativeOrder() : "Page buffer order " + pageBuf.order()
+                            + " should be same with " + nativeOrder();
+                    assert PageIo.getType(pageBuf) != 0 : "Invalid state. Type is 0! pageId = " + hexLong(pageId);
+                    assert PageIo.getVersion(pageBuf) != 0 : "Invalid state. Version is 0! pageId = " + hexLong(pageId);
+
+                    if (calculateCrc && !skipCrc) {
+                        assert PageIo.getCrc(pageBuf) == 0 : hexLong(pageId);
+
+                        PageIo.setCrc(pageBuf, calcCrc32(pageBuf, pageSize()));
+                    }
+
+                    // Check whether crc was calculated somewhere above the stack if it is forcibly skipped.
+                    assert skipCrc || PageIo.getCrc(pageBuf) != 0
+                            || calcCrc32(pageBuf, pageSize()) == 0 : "CRC hasn't been calculated, crc=0";
+
+                    assert pageBuf.position() == 0 : pageBuf.position();
+
+                    long pageOff = pageOffset(pageId);
+
+                    fileIo.writeFully(pageBuf, pageOff);
+
+                    PageIo.setCrc(pageBuf, 0);
+
+                    if (interrupted) {
+                        Thread.currentThread().interrupt();
+                    }
+
+                    return;
+                } finally {
+                    readWriteLock.readLock().unlock();
+                }
+            } catch (IOException e) {
+                if (e instanceof ClosedChannelException) {
+                    try {
+                        if (e instanceof ClosedByInterruptException) {
+                            interrupted = true;
+
+                            Thread.interrupted();
+                        }
+
+                        reinit(fileIo);
+
+                        pageBuf.position(0);
+
+                        PageIo.setCrc(pageBuf, 0);
+
+                        continue;
+                    } catch (IOException e0) {
+                        e0.addSuppressed(e);
+
+                        e = e0;
+                    }
+                }
+
+                throw new IgniteInternalCheckedException(
+                        "Failed to write page [filePath=" + filePath + ", pageId=" + pageId + "]",
+                        e
+                );
+            }
+        }
+    }
+
+    /**
+     * Sync method used to ensure that the given pages are guaranteed to be written to the file page store.
+     *
+     * @throws IgniteInternalCheckedException If sync failed (IO error occurred).
+     */
+    public void sync() throws IgniteInternalCheckedException {
+        readWriteLock.writeLock().lock();
+
+        try {
+            ensure();
+
+            FileIo fileIo = this.fileIo;
+
+            if (fileIo != null) {
+                fileIo.force();
+            }
+        } catch (IOException e) {
+            throw new IgniteInternalCheckedException("Failed to fsync file [filePath=" + filePath + ']', e);
+        } finally {
+            readWriteLock.writeLock().unlock();
+        }
+    }
+
+    /**
+     * Returns {@code true} if the file page store exists.
+     */
+    public boolean exists() {
+        if (fileExists == null) {
+            readWriteLock.readLock().lock();
+
+            try {
+                if (fileExists == null) {
+                    fileExists = Files.exists(filePath) && filePath.toFile().length() >= headerSize();
+                }
+            } finally {
+                readWriteLock.readLock().unlock();
+            }
+        }
+
+        return fileExists;
+    }
+
+    /**
+     * Initializes the file page store if it hasn't already.
+     *
+     * @throws IgniteInternalCheckedException If initialization failed (IO error occurred).
+     */
+    public void ensure() throws IgniteInternalCheckedException {
+        if (!initialized) {
+            readWriteLock.writeLock().lock();
+
+            try {
+                if (!initialized) {
+                    FileIo fileIo = null;
+
+                    IgniteInternalCheckedException err = null;
+
+                    try {
+                        boolean interrupted = false;
+
+                        while (true) {
+                            try {
+                                this.fileIo = fileIo = ioFactory.create(filePath, CREATE, READ, WRITE);
+
+                                fileExists = true;
+
+                                if (fileIo.size() < headerSize()) {
+                                    fileIo.writeFully(headerBuffer().rewind(), 0);
+                                } else {
+                                    checkHeader(fileIo);
+                                }
+
+                                if (interrupted) {
+                                    Thread.currentThread().interrupt();
+                                }
+
+                                break;
+                            } catch (ClosedByInterruptException e) {
+                                interrupted = true;
+
+                                Thread.interrupted();
+                            }
+                        }
+
+                        initialized = true;
+                    } catch (IOException e) {
+                        err = new IgniteInternalCheckedException("Failed to initialize partition file: " + filePath, e);
+
+                        throw err;
+                    } finally {
+                        if (err != null && fileIo != null) {
+                            try {
+                                fileIo.close();
+                            } catch (IOException e) {
+                                err.addSuppressed(e);
+                            }
+                        }
+                    }
+                }
+            } finally {
+                readWriteLock.writeLock().unlock();
+            }
+        }
+    }
+
+    /**
+     * Returns size of the file page store in bytes.
+     *
+     * @throws IgniteInternalCheckedException If an I/O error occurs.
+     */
+    public long size() throws IgniteInternalCheckedException {
+        readWriteLock.readLock().lock();
+
+        try {
+            FileIo io = fileIo;
+
+            return io == null ? 0 : io.size();
+        } catch (IOException e) {
+            throw new IgniteInternalCheckedException(e);
+        } finally {
+            readWriteLock.readLock().unlock();
+        }
+    }
+
+    private void stop0(boolean clean) throws IOException {
+        readWriteLock.writeLock().lock();
+
+        try {
+            if (!initialized) {
+                // Ensure the file is closed even if not initialized yet.
+                if (fileIo != null) {
+                    fileIo.close();
+                }
+
+                if (clean && exists()) {
+                    Files.delete(filePath);
+                }
+
+                return;
+            }
+
+            fileIo.force();
+
+            fileIo.close();
+
+            fileIo = null;
+
+            if (clean) {
+                Files.delete(filePath);
+
+                fileExists = false;
+            }
+        } finally {
+            initialized = false;
+
+            readWriteLock.writeLock().unlock();
+        }
+    }
+
+    private static int calcCrc32(ByteBuffer pageBuf, int pageSize) {
+        try {
+            pageBuf.position(0);
+
+            return FastCrc.calcCrc(pageBuf, pageSize);
+        } finally {
+            pageBuf.position(0);
+        }
+    }
+
+    /**
+     * Reinit page store after file channel was closed by thread interruption.
+     *
+     * @param fileIo Old fileIo.
+     */
+    private void reinit(FileIo fileIo) throws IOException {
+        if (fileIo != this.fileIo) {
+            return;
+        }
+
+        readWriteLock.writeLock().lock();
+
+        try {
+            if (fileIo != this.fileIo) {
+                return;
+            }
+
+            try {
+                boolean interrupted = false;
+
+                while (true) {
+                    try {
+                        fileIo = null;
+
+                        fileIo = ioFactory.create(filePath, CREATE, READ, WRITE);
+
+                        fileExists = true;
+
+                        checkHeader(fileIo);
+
+                        this.fileIo = fileIo;
+
+                        if (interrupted) {
+                            Thread.currentThread().interrupt();
+                        }
+
+                        break;
+                    } catch (ClosedByInterruptException e) {
+                        interrupted = true;
+
+                        Thread.interrupted();
+                    }
+                }
+            } catch (IOException e) {
+                try {
+                    if (fileIo != null) {
+                        fileIo.close();
+                    }
+                } catch (IOException e0) {
+                    e.addSuppressed(e0);
+                }
+
+                throw e;
+            }
+        } finally {
+            readWriteLock.writeLock().unlock();
+        }
+    }
+
+    /**
+     * Reads a page from the page store.
+     *
+     * @param pageId Page ID.
+     * @param pageOff Page offset in the file.
+     * @param pageBuf Page buffer to read into.
+     * @param checkCrc Check CRC on page.
+     * @param keepCrc By default reading zeroes CRC which was on file, but you can keep it in pageBuf if set keepCrc.
+     * @return {@code True} if the page was read successfully.
+     * @throws IgniteInternalCheckedException If reading failed (IO error occurred).
+     */
+    private boolean read0(
+            long pageId,
+            long pageOff,
+            ByteBuffer pageBuf,
+            boolean checkCrc,
+            boolean keepCrc
+    ) throws IgniteInternalCheckedException {
+        assert pageOff >= headerSize() : "pageOff=" + pageOff + ", headerSize=" + headerSize();
+
+        ensure();
+
+        try {
+            assert pageBuf.capacity() == pageSize() : pageBuf.capacity();
+            assert pageBuf.remaining() == pageSize() : pageBuf.remaining();
+            assert pageBuf.position() == 0 : pageBuf.position();
+            assert pageBuf.order() == nativeOrder() : pageBuf.order();
+
+            int n = readWithFailover(pageBuf, pageOff);
+
+            // If page was not written yet, nothing to read.

Review Comment:
   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] tkalkirill commented on a diff in pull request #947: IGNITE-17372 Implement DeltaFilePageStore

Posted by GitBox <gi...@apache.org>.
tkalkirill commented on code in PR #947:
URL: https://github.com/apache/ignite-3/pull/947#discussion_r928594273


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/AbstractFilePageStoreIo.java:
##########
@@ -0,0 +1,611 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.pagememory.persistence.store;
+
+import static java.nio.ByteOrder.nativeOrder;
+import static java.nio.file.StandardOpenOption.CREATE;
+import static java.nio.file.StandardOpenOption.READ;
+import static java.nio.file.StandardOpenOption.WRITE;
+import static org.apache.ignite.internal.util.IgniteUtils.atomicMoveFile;
+import static org.apache.ignite.internal.util.IgniteUtils.hexInt;
+import static org.apache.ignite.internal.util.IgniteUtils.hexLong;
+import static org.apache.ignite.internal.util.IgniteUtils.toHexString;
+import static org.apache.ignite.lang.IgniteSystemProperties.getBoolean;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.ClosedByInterruptException;
+import java.nio.channels.ClosedChannelException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.apache.ignite.internal.fileio.FileIo;
+import org.apache.ignite.internal.fileio.FileIoFactory;
+import org.apache.ignite.internal.pagememory.io.PageIo;
+import org.apache.ignite.internal.pagememory.persistence.FastCrc;
+import org.apache.ignite.internal.pagememory.persistence.IgniteInternalDataIntegrityViolationException;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Abstract class for performing IO operations on file page storage.
+ */
+public abstract class AbstractFilePageStoreIo implements Closeable {
+    private final FileIoFactory ioFactory;
+
+    private final ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
+
+    /** Skip CRC calculation flag. */
+    // TODO: IGNITE-17011 Move to config
+    private final boolean skipCrc = getBoolean("IGNITE_PDS_SKIP_CRC");
+
+    private volatile Path filePath;
+
+    private volatile @Nullable FileIo fileIo;
+
+    /** Initialized file page store IO. */
+    private volatile boolean initialized;
+
+    /** Caches the existence state of file. After it is initialized, it will be not {@code null} during lifecycle. */
+    private volatile @Nullable Boolean fileExists;
+
+    /**
+     * Constructor.
+     *
+     * @param ioFactory {@link FileIo} factory.
+     * @param filePath File page store path.
+     */
+    AbstractFilePageStoreIo(FileIoFactory ioFactory, Path filePath) {
+        this.ioFactory = ioFactory;
+        this.filePath = filePath;
+    }
+
+    /**
+     * Returns the page size in bytes.
+     */
+    public abstract int pageSize();
+
+    /**
+     * Returns the size of the header in bytes.
+     */
+    public abstract int headerSize();
+
+    /**
+     * Returns a buffer with a file page storage header.
+     */
+    public abstract ByteBuffer headerBuffer();

Review Comment:
   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] tkalkirill commented on a diff in pull request #947: IGNITE-17372 Implement DeltaFilePageStore

Posted by GitBox <gi...@apache.org>.
tkalkirill commented on code in PR #947:
URL: https://github.com/apache/ignite-3/pull/947#discussion_r928511139


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/AbstractFilePageStoreIo.java:
##########
@@ -0,0 +1,611 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.pagememory.persistence.store;
+
+import static java.nio.ByteOrder.nativeOrder;
+import static java.nio.file.StandardOpenOption.CREATE;
+import static java.nio.file.StandardOpenOption.READ;
+import static java.nio.file.StandardOpenOption.WRITE;
+import static org.apache.ignite.internal.util.IgniteUtils.atomicMoveFile;
+import static org.apache.ignite.internal.util.IgniteUtils.hexInt;
+import static org.apache.ignite.internal.util.IgniteUtils.hexLong;
+import static org.apache.ignite.internal.util.IgniteUtils.toHexString;
+import static org.apache.ignite.lang.IgniteSystemProperties.getBoolean;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.ClosedByInterruptException;
+import java.nio.channels.ClosedChannelException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.apache.ignite.internal.fileio.FileIo;
+import org.apache.ignite.internal.fileio.FileIoFactory;
+import org.apache.ignite.internal.pagememory.io.PageIo;
+import org.apache.ignite.internal.pagememory.persistence.FastCrc;
+import org.apache.ignite.internal.pagememory.persistence.IgniteInternalDataIntegrityViolationException;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Abstract class for performing IO operations on file page storage.
+ */
+public abstract class AbstractFilePageStoreIo implements Closeable {
+    private final FileIoFactory ioFactory;
+
+    private final ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
+
+    /** Skip CRC calculation flag. */
+    // TODO: IGNITE-17011 Move to config
+    private final boolean skipCrc = getBoolean("IGNITE_PDS_SKIP_CRC");
+
+    private volatile Path filePath;
+
+    private volatile @Nullable FileIo fileIo;
+
+    /** Initialized file page store IO. */
+    private volatile boolean initialized;
+
+    /** Caches the existence state of file. After it is initialized, it will be not {@code null} during lifecycle. */
+    private volatile @Nullable Boolean fileExists;

Review Comment:
   In general, I can make it non-volatile and read it under lock.



-- 
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 #947: IGNITE-17372 Implement DeltaFilePageStore

Posted by GitBox <gi...@apache.org>.
tkalkirill commented on code in PR #947:
URL: https://github.com/apache/ignite-3/pull/947#discussion_r931962351


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/AbstractFilePageStoreIo.java:
##########
@@ -486,11 +482,12 @@ private boolean read0(
 
             int n = readWithFailover(pageBuf, pageOff);
 
+            // TODO: IGNITE-17397 Investigate the ability to read a empty page

Review Comment:
   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] tkalkirill commented on a diff in pull request #947: IGNITE-17372 Implement DeltaFilePageStore

Posted by GitBox <gi...@apache.org>.
tkalkirill commented on code in PR #947:
URL: https://github.com/apache/ignite-3/pull/947#discussion_r928615230


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/DeltaFilePageStoreIoHeader.java:
##########
@@ -0,0 +1,236 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.pagememory.persistence.store;
+
+import static java.nio.ByteOrder.nativeOrder;
+import static org.apache.ignite.internal.util.IgniteUtils.hexLong;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import org.apache.ignite.internal.fileio.FileIo;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * {@link DeltaFilePageStoreIo} header.
+ *
+ * <p>Total length in bytes {@link #headerSize()}.
+ *
+ * <ul>
+ *     <li>{@link #SIGNATURE signature} (8 bytes)</li>
+ *     <li>{@link #version() version} (4 bytes)</li>
+ *     <li>{@link #pageSize() pageSize} (4 bytes)</li>
+ *     <li>{@link #pageIndexes() pageIndexes}
+ *         <ul>
+ *             <li>array length (4 bytes)</li>
+ *             <li>array elements (array length * 4 bytes)</li>
+ *         </ul>
+ *     </li>
+ * </ul>
+ */
+public class DeltaFilePageStoreIoHeader {
+    /** File signature. */
+    private static final long SIGNATURE = 0xDEAFAEE072020173L;
+
+    /** Size of the common delta file page store header for all versions, in bytes. */
+    private static final int COMMON_HEADER_SIZE =
+            8/*SIGNATURE*/ + 4/*version*/ + 4/*index*/ + 4/*page size*/ + 4/*page index array length*/;
+
+    private final int version;
+
+    private final int index;
+
+    private final int pageSize;
+
+    private final int[] pageIndexes;
+
+    private final int headerSize;
+
+    /**
+     * Constructor.
+     *
+     * @param version Delta file page store version.
+     * @param index Delta file page store index.
+     * @param pageSize Page size in bytes.
+     * @param pageIndexes Page indexes.
+     */
+    public DeltaFilePageStoreIoHeader(int version, int index, int pageSize, int[] pageIndexes) {
+        assert pageSize >= COMMON_HEADER_SIZE : pageSize;
+        assert index >= 0 : index;
+
+        this.version = version;
+        this.index = index;
+        this.pageSize = pageSize;
+        this.pageIndexes = pageIndexes;
+
+        int size = COMMON_HEADER_SIZE + (4 * pageIndexes.length);
+
+        if (size % pageSize != 0) {
+            size = ((size / pageSize) + 1) * pageSize;
+        }
+
+        headerSize = size;
+    }
+
+    /**
+     * Returns the version of the delta file page store.
+     */
+    public int version() {
+        return version;
+    }
+
+    /**
+     * Returns the index of the delta file page store.
+     */
+    public int index() {
+        return index;
+    }
+
+    /**
+     * Returns the page size in bytes.
+     */
+    public int pageSize() {
+        return pageSize;
+    }
+
+    /**
+     * Returns the size (aligned to {@link #pageSize()}) of the header in bytes.
+     */
+    public int headerSize() {
+        return headerSize;
+    }
+
+    /**
+     * Returns page indexes.
+     */
+    public int[] pageIndexes() {
+        return pageIndexes;
+    }
+
+    /**
+     * Converts the delta file page store header (aligned to {@link #pageSize()}) to a {@link ByteBuffer} for writing to a file.
+     */
+    public ByteBuffer toByteBuffer() {
+        ByteBuffer buffer = ByteBuffer.allocate(headerSize).order(nativeOrder())
+                .putLong(SIGNATURE)
+                .putInt(version)
+                .putInt(index)
+                .putInt(pageSize)
+                .putInt(pageIndexes.length);
+
+        if (pageIndexes.length > 0) {
+            buffer.asIntBuffer().put(pageIndexes);
+        }
+
+        return buffer;
+    }
+
+    /**
+     * Reads the header of the delta file page store.
+     *
+     * @param fileIo Delta file page store fileIo.
+     * @param headerBuffer Buffer for reading {@link DeltaFilePageStoreIoHeader header} from {@code fileIo}.
+     * @throws IOException If there are errors when reading the delta file page store header.
+     */
+    public static @Nullable DeltaFilePageStoreIoHeader readHeader(FileIo fileIo, ByteBuffer headerBuffer) throws IOException {
+        assert headerBuffer.remaining() >= COMMON_HEADER_SIZE : headerBuffer.remaining();
+        assert headerBuffer.order() == nativeOrder() : headerBuffer.order();
+
+        if (fileIo.size() < COMMON_HEADER_SIZE) {
+            return null;
+        }
+
+        fileIo.readFully(headerBuffer, 0);
+
+        long signature = headerBuffer.rewind().getLong();
+
+        if (SIGNATURE != signature) {
+            throw new IOException(String.format(
+                    "Invalid file signature [expected=%s, actual=%s]",
+                    hexLong(SIGNATURE),
+                    hexLong(signature))
+            );
+        }
+
+        int version = headerBuffer.getInt();
+        int index = headerBuffer.getInt();
+        int pageSize = headerBuffer.getInt();
+        int arrayLen = headerBuffer.getInt();
+
+        if (arrayLen == 0) {
+            return new DeltaFilePageStoreIoHeader(version, index, pageSize, new int[0]);
+        }
+
+        int[] pageIndexes = new int[arrayLen];
+
+        int i = 0;
+        int filePosition = headerBuffer.capacity();
+
+        while (i < arrayLen) {

Review Comment:
   Added



-- 
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 #947: IGNITE-17372 Implement DeltaFilePageStore

Posted by GitBox <gi...@apache.org>.
tkalkirill commented on code in PR #947:
URL: https://github.com/apache/ignite-3/pull/947#discussion_r928617474


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStore.java:
##########
@@ -17,122 +17,108 @@
 
 package org.apache.ignite.internal.pagememory.persistence.store;
 
-import static java.nio.ByteOrder.nativeOrder;
-import static java.nio.file.StandardOpenOption.CREATE;
-import static java.nio.file.StandardOpenOption.READ;
-import static java.nio.file.StandardOpenOption.WRITE;
-import static org.apache.ignite.internal.util.IgniteUtils.hexInt;
-import static org.apache.ignite.internal.util.IgniteUtils.hexLong;
-import static org.apache.ignite.internal.util.IgniteUtils.toHexString;
-import static org.apache.ignite.lang.IgniteSystemProperties.getBoolean;
+import static org.apache.ignite.internal.pagememory.util.PageIdUtils.pageIndex;
 
 import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.lang.invoke.VarHandle;
 import java.nio.ByteBuffer;
-import java.nio.channels.ClosedByInterruptException;
-import java.nio.channels.ClosedChannelException;
-import java.nio.file.Files;
 import java.nio.file.Path;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.locks.ReadWriteLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.function.Supplier;
 import org.apache.ignite.internal.fileio.FileIo;
-import org.apache.ignite.internal.fileio.FileIoFactory;
-import org.apache.ignite.internal.pagememory.io.PageIo;
-import org.apache.ignite.internal.pagememory.persistence.FastCrc;
-import org.apache.ignite.internal.pagememory.persistence.IgniteInternalDataIntegrityViolationException;
-import org.apache.ignite.internal.pagememory.util.PageIdUtils;
 import org.apache.ignite.lang.IgniteInternalCheckedException;
 import org.jetbrains.annotations.Nullable;
 
 /**
  * FilePageStore is a {@link PageStore} implementation that uses regular files to store pages.
  *
- * <p>Actual read and write operations are performed with {@link FileIo} abstract interface, list of its implementations is a good source
- * of information about functionality in Ignite Native Persistence.
+ * <p>It consists of the main file page store and delta file page stores, when reading the page at the beginning, the page is searched in
+ * the delta files and only then in the main file.
  *
  * <p>On a physical level each instance of {@code FilePageStore} corresponds to a partition file assigned to the local node.
  *
- * <p>Consists of:
- * <ul>
- *     <li>Header - {@link FilePageStoreHeader}. </li>
- *     <li>Body - data pages are multiples of {@link FilePageStoreHeader#pageSize() pageSize}.</li>
- * </ul>
+ * <p>Actual read and write operations are performed with {@link FileIo} abstract interface, list of its implementations is a good source
+ * of information about functionality in Ignite Native Persistence.
+ *
+ * <p>To create a delta file first invoke {@link #getOrCreateNewDeltaFile(Supplier)} then fill it and then invoke {@link
+ * #completeNewDeltaFile()}.
  */
 public class FilePageStore implements PageStore {
-    /** File version. */
-    public static final int VERSION_1 = 1;
+    private static final VarHandle PAGE_COUNT;
 
-    /** Skip CRC calculation flag. */
-    // TODO: IGNITE-17011 Move to config
-    private final boolean skipCrc = getBoolean("IGNITE_PDS_SKIP_CRC");
+    private static final VarHandle NEW_DELTA_FILE_PAGE_STORE_IO_FUTURE;
 
-    /** File page store version. */
-    private final int version;
+    static {
+        try {
+            PAGE_COUNT = MethodHandles.lookup().findVarHandle(FilePageStore.class, "pageCount", int.class);
 
-    /** Page size in bytes. */
-    private final int pageSize;
+            NEW_DELTA_FILE_PAGE_STORE_IO_FUTURE = MethodHandles.lookup().findVarHandle(
+                    FilePageStore.class,
+                    "newDeltaFilePageStoreIoFuture",
+                    CompletableFuture.class
+            );
+        } catch (ReflectiveOperationException e) {
+            throw new ExceptionInInitializerError(e);
+        }
+    }
 
-    /** Header size in bytes. Should be aligned to the {@link #pageSize}. */
-    private final int headerSize;
+    /** File page store version. */
+    public static final int VERSION_1 = 1;
 
-    /** File page store path. */
-    private final Path filePath;
+    /** Delta file page store IO version. */
+    public static final int DELTA_FILE_VERSION_1 = 1;
 
-    /** {@link FileIo} factory. */
-    private final FileIoFactory ioFactory;
+    /** File page store IO. */
+    private final FilePageStoreIo filePageStoreIo;
 
     /** Page count. */
-    private final AtomicInteger pageCount = new AtomicInteger();
+    private volatile int pageCount;
 
-    private final ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
+    /** New page allocation listener. */
+    private volatile @Nullable PageAllocationListener pageAllocationListener;
 
-    /** Caches the existence state of storage file. After it is initialized, it will be not {@code null} during lifecycle. */
-    private volatile Boolean fileExists;
+    /** Delta file page store IOs. */
+    private final List<DeltaFilePageStoreIo> deltaFilePageStoreIos;
 
-    /** {@link FileIo} for read/write operations with file. */
-    private volatile FileIo fileIo;
+    /** Future with a new delta file page store. */
+    private volatile @Nullable CompletableFuture<DeltaFilePageStoreIo> newDeltaFilePageStoreIoFuture;
 
-    /** Initialized file page store. */
-    private volatile boolean initialized;
+    /** {@link DeltaFilePageStoreIo} factory. */
+    private volatile @Nullable DeltaFilePageStoreIoFactory deltaFilePageStoreIoFactory;
 
-    /** New page allocation listener. */
-    private volatile @Nullable PageAllocationListener pageAllocationListener;
+    /** Callback on completion of delta file page store creation. */
+    private volatile @Nullable CompleteCreationDeltaFilePageStoreIoCallback completeCreationDeltaFilePageStoreIoCallback;

Review Comment:
   We personally decided that I would delete 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] tkalkirill commented on a diff in pull request #947: IGNITE-17372 Implement DeltaFilePageStore

Posted by GitBox <gi...@apache.org>.
tkalkirill commented on code in PR #947:
URL: https://github.com/apache/ignite-3/pull/947#discussion_r928604473


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/AbstractFilePageStoreIo.java:
##########
@@ -0,0 +1,611 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.pagememory.persistence.store;
+
+import static java.nio.ByteOrder.nativeOrder;
+import static java.nio.file.StandardOpenOption.CREATE;
+import static java.nio.file.StandardOpenOption.READ;
+import static java.nio.file.StandardOpenOption.WRITE;
+import static org.apache.ignite.internal.util.IgniteUtils.atomicMoveFile;
+import static org.apache.ignite.internal.util.IgniteUtils.hexInt;
+import static org.apache.ignite.internal.util.IgniteUtils.hexLong;
+import static org.apache.ignite.internal.util.IgniteUtils.toHexString;
+import static org.apache.ignite.lang.IgniteSystemProperties.getBoolean;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.ClosedByInterruptException;
+import java.nio.channels.ClosedChannelException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.apache.ignite.internal.fileio.FileIo;
+import org.apache.ignite.internal.fileio.FileIoFactory;
+import org.apache.ignite.internal.pagememory.io.PageIo;
+import org.apache.ignite.internal.pagememory.persistence.FastCrc;
+import org.apache.ignite.internal.pagememory.persistence.IgniteInternalDataIntegrityViolationException;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Abstract class for performing IO operations on file page storage.
+ */
+public abstract class AbstractFilePageStoreIo implements Closeable {
+    private final FileIoFactory ioFactory;
+
+    private final ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
+
+    /** Skip CRC calculation flag. */
+    // TODO: IGNITE-17011 Move to config
+    private final boolean skipCrc = getBoolean("IGNITE_PDS_SKIP_CRC");
+
+    private volatile Path filePath;
+
+    private volatile @Nullable FileIo fileIo;
+
+    /** Initialized file page store IO. */
+    private volatile boolean initialized;
+
+    /** Caches the existence state of file. After it is initialized, it will be not {@code null} during lifecycle. */
+    private volatile @Nullable Boolean fileExists;
+
+    /**
+     * Constructor.
+     *
+     * @param ioFactory {@link FileIo} factory.
+     * @param filePath File page store path.
+     */
+    AbstractFilePageStoreIo(FileIoFactory ioFactory, Path filePath) {
+        this.ioFactory = ioFactory;
+        this.filePath = filePath;
+    }
+
+    /**
+     * Returns the page size in bytes.
+     */
+    public abstract int pageSize();
+
+    /**
+     * Returns the size of the header in bytes.
+     */
+    public abstract int headerSize();
+
+    /**
+     * Returns a buffer with a file page storage header.
+     */
+    public abstract ByteBuffer headerBuffer();
+
+    /**
+     * Checks the file page storage header.
+     *
+     * @param fileIo File page store IO to read the header.
+     * @throws IOException If there is an error reading the header or the header did not pass the check.
+     */
+    public abstract void checkHeader(FileIo fileIo) throws IOException;
+
+    /**
+     * Returns page offset within the store file.
+     *
+     * @param pageId Page ID.
+     */
+    public abstract long pageOffset(long pageId);
+
+    /**
+     * Stops the file page store IO.
+     *
+     * @param clean {@code True} to clean file page store.
+     * @throws IgniteInternalCheckedException If failed.
+     */
+    public void stop(boolean clean) throws IgniteInternalCheckedException {
+        try {
+            stop0(clean);
+        } catch (IOException e) {
+            throw new IgniteInternalCheckedException(
+                    "Failed to stop serving file [file=" + filePath + ", delete=" + clean + "]",
+                    e
+            );
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public void close() throws IOException {
+        stop0(false);
+    }
+
+    /**
+     * Reads a page.
+     *
+     * @param pageId Page ID.
+     * @param pageOff Page offset in the file.
+     * @param pageBuf Page buffer to read into.
+     * @param keepCrc By default, reading zeroes CRC which was on page store, but you can keep it in {@code pageBuf} if set {@code
+     * keepCrc}.
+     * @return {@code True} if the page was read successfully.
+     * @throws IgniteInternalCheckedException If reading failed (IO error occurred).
+     */
+    public boolean read(long pageId, long pageOff, ByteBuffer pageBuf, boolean keepCrc) throws IgniteInternalCheckedException {
+        return read0(pageId, pageOff, pageBuf, !skipCrc, keepCrc);
+    }
+
+    /**
+     * Writes a page.
+     *
+     * @param pageId Page ID.
+     * @param pageBuf Page buffer to write from.
+     * @param calculateCrc If {@code false} crc calculation will be forcibly skipped.
+     * @throws IgniteInternalCheckedException If page writing failed (IO error occurred).
+     */
+    public void write(long pageId, ByteBuffer pageBuf, boolean calculateCrc) throws IgniteInternalCheckedException {
+        ensure();
+
+        boolean interrupted = false;
+
+        while (true) {
+            FileIo fileIo = this.fileIo;
+
+            try {
+                readWriteLock.readLock().lock();
+
+                try {
+                    assert pageBuf.position() == 0 : pageBuf.position();
+                    assert pageBuf.order() == nativeOrder() : "Page buffer order " + pageBuf.order()
+                            + " should be same with " + nativeOrder();
+                    assert PageIo.getType(pageBuf) != 0 : "Invalid state. Type is 0! pageId = " + hexLong(pageId);
+                    assert PageIo.getVersion(pageBuf) != 0 : "Invalid state. Version is 0! pageId = " + hexLong(pageId);
+
+                    if (calculateCrc && !skipCrc) {
+                        assert PageIo.getCrc(pageBuf) == 0 : hexLong(pageId);
+
+                        PageIo.setCrc(pageBuf, calcCrc32(pageBuf, pageSize()));
+                    }
+
+                    // Check whether crc was calculated somewhere above the stack if it is forcibly skipped.
+                    assert skipCrc || PageIo.getCrc(pageBuf) != 0
+                            || calcCrc32(pageBuf, pageSize()) == 0 : "CRC hasn't been calculated, crc=0";
+
+                    assert pageBuf.position() == 0 : pageBuf.position();
+
+                    long pageOff = pageOffset(pageId);
+
+                    fileIo.writeFully(pageBuf, pageOff);
+
+                    PageIo.setCrc(pageBuf, 0);
+
+                    if (interrupted) {
+                        Thread.currentThread().interrupt();
+                    }
+
+                    return;
+                } finally {
+                    readWriteLock.readLock().unlock();
+                }
+            } catch (IOException e) {
+                if (e instanceof ClosedChannelException) {
+                    try {
+                        if (e instanceof ClosedByInterruptException) {
+                            interrupted = true;
+
+                            Thread.interrupted();
+                        }
+
+                        reinit(fileIo);
+
+                        pageBuf.position(0);
+
+                        PageIo.setCrc(pageBuf, 0);
+
+                        continue;
+                    } catch (IOException e0) {
+                        e0.addSuppressed(e);
+
+                        e = e0;
+                    }
+                }
+
+                throw new IgniteInternalCheckedException(
+                        "Failed to write page [filePath=" + filePath + ", pageId=" + pageId + "]",
+                        e
+                );
+            }
+        }
+    }
+
+    /**
+     * Sync method used to ensure that the given pages are guaranteed to be written to the file page store.
+     *
+     * @throws IgniteInternalCheckedException If sync failed (IO error occurred).
+     */
+    public void sync() throws IgniteInternalCheckedException {
+        readWriteLock.writeLock().lock();
+
+        try {
+            ensure();
+
+            FileIo fileIo = this.fileIo;
+
+            if (fileIo != null) {
+                fileIo.force();
+            }
+        } catch (IOException e) {
+            throw new IgniteInternalCheckedException("Failed to fsync file [filePath=" + filePath + ']', e);
+        } finally {
+            readWriteLock.writeLock().unlock();
+        }
+    }
+
+    /**
+     * Returns {@code true} if the file page store exists.
+     */
+    public boolean exists() {
+        if (fileExists == null) {
+            readWriteLock.readLock().lock();
+
+            try {
+                if (fileExists == null) {
+                    fileExists = Files.exists(filePath) && filePath.toFile().length() >= headerSize();
+                }
+            } finally {
+                readWriteLock.readLock().unlock();
+            }
+        }
+
+        return fileExists;
+    }
+
+    /**
+     * Initializes the file page store if it hasn't already.
+     *
+     * @throws IgniteInternalCheckedException If initialization failed (IO error occurred).
+     */
+    public void ensure() throws IgniteInternalCheckedException {
+        if (!initialized) {
+            readWriteLock.writeLock().lock();
+
+            try {
+                if (!initialized) {
+                    FileIo fileIo = null;
+
+                    IgniteInternalCheckedException err = null;
+
+                    try {
+                        boolean interrupted = false;
+
+                        while (true) {
+                            try {
+                                this.fileIo = fileIo = ioFactory.create(filePath, CREATE, READ, WRITE);
+
+                                fileExists = true;
+
+                                if (fileIo.size() < headerSize()) {
+                                    fileIo.writeFully(headerBuffer().rewind(), 0);
+                                } else {
+                                    checkHeader(fileIo);
+                                }
+
+                                if (interrupted) {
+                                    Thread.currentThread().interrupt();
+                                }
+
+                                break;
+                            } catch (ClosedByInterruptException e) {
+                                interrupted = true;
+
+                                Thread.interrupted();
+                            }
+                        }
+
+                        initialized = true;
+                    } catch (IOException e) {
+                        err = new IgniteInternalCheckedException("Failed to initialize partition file: " + filePath, e);
+
+                        throw err;
+                    } finally {
+                        if (err != null && fileIo != null) {
+                            try {
+                                fileIo.close();
+                            } catch (IOException e) {
+                                err.addSuppressed(e);
+                            }
+                        }
+                    }
+                }
+            } finally {
+                readWriteLock.writeLock().unlock();
+            }
+        }
+    }
+
+    /**
+     * Returns size of the file page store in bytes.
+     *
+     * @throws IgniteInternalCheckedException If an I/O error occurs.
+     */
+    public long size() throws IgniteInternalCheckedException {
+        readWriteLock.readLock().lock();
+
+        try {
+            FileIo io = fileIo;
+
+            return io == null ? 0 : io.size();
+        } catch (IOException e) {
+            throw new IgniteInternalCheckedException(e);
+        } finally {
+            readWriteLock.readLock().unlock();
+        }
+    }
+
+    private void stop0(boolean clean) throws IOException {
+        readWriteLock.writeLock().lock();
+
+        try {
+            if (!initialized) {
+                // Ensure the file is closed even if not initialized yet.
+                if (fileIo != null) {
+                    fileIo.close();
+                }
+
+                if (clean && exists()) {
+                    Files.delete(filePath);
+                }
+
+                return;
+            }
+
+            fileIo.force();
+
+            fileIo.close();
+
+            fileIo = null;
+
+            if (clean) {
+                Files.delete(filePath);
+
+                fileExists = false;
+            }
+        } finally {
+            initialized = false;
+
+            readWriteLock.writeLock().unlock();
+        }
+    }
+
+    private static int calcCrc32(ByteBuffer pageBuf, int pageSize) {
+        try {
+            pageBuf.position(0);
+
+            return FastCrc.calcCrc(pageBuf, pageSize);
+        } finally {
+            pageBuf.position(0);
+        }
+    }
+
+    /**
+     * Reinit page store after file channel was closed by thread interruption.
+     *
+     * @param fileIo Old fileIo.
+     */
+    private void reinit(FileIo fileIo) throws IOException {
+        if (fileIo != this.fileIo) {
+            return;
+        }
+
+        readWriteLock.writeLock().lock();
+
+        try {
+            if (fileIo != this.fileIo) {
+                return;
+            }
+
+            try {
+                boolean interrupted = false;
+
+                while (true) {
+                    try {
+                        fileIo = null;
+
+                        fileIo = ioFactory.create(filePath, CREATE, READ, WRITE);
+
+                        fileExists = true;
+
+                        checkHeader(fileIo);
+
+                        this.fileIo = fileIo;
+
+                        if (interrupted) {
+                            Thread.currentThread().interrupt();
+                        }
+
+                        break;
+                    } catch (ClosedByInterruptException e) {
+                        interrupted = true;
+
+                        Thread.interrupted();
+                    }
+                }
+            } catch (IOException e) {
+                try {
+                    if (fileIo != null) {
+                        fileIo.close();
+                    }
+                } catch (IOException e0) {
+                    e.addSuppressed(e0);
+                }
+
+                throw e;
+            }
+        } finally {
+            readWriteLock.writeLock().unlock();
+        }
+    }
+
+    /**
+     * Reads a page from the page store.
+     *
+     * @param pageId Page ID.
+     * @param pageOff Page offset in the file.
+     * @param pageBuf Page buffer to read into.
+     * @param checkCrc Check CRC on page.
+     * @param keepCrc By default reading zeroes CRC which was on file, but you can keep it in pageBuf if set keepCrc.
+     * @return {@code True} if the page was read successfully.
+     * @throws IgniteInternalCheckedException If reading failed (IO error occurred).
+     */
+    private boolean read0(
+            long pageId,
+            long pageOff,
+            ByteBuffer pageBuf,
+            boolean checkCrc,
+            boolean keepCrc
+    ) throws IgniteInternalCheckedException {
+        assert pageOff >= headerSize() : "pageOff=" + pageOff + ", headerSize=" + headerSize();
+
+        ensure();
+
+        try {
+            assert pageBuf.capacity() == pageSize() : pageBuf.capacity();
+            assert pageBuf.remaining() == pageSize() : pageBuf.remaining();
+            assert pageBuf.position() == 0 : pageBuf.position();
+            assert pageBuf.order() == nativeOrder() : pageBuf.order();
+
+            int n = readWithFailover(pageBuf, pageOff);
+
+            // If page was not written yet, nothing to read.
+            if (n < 0) {
+                pageBuf.put(new byte[pageBuf.remaining()]);
+
+                return false;
+            }
+
+            int savedCrc32 = PageIo.getCrc(pageBuf);
+
+            PageIo.setCrc(pageBuf, 0);
+
+            pageBuf.position(0);
+
+            if (checkCrc) {
+                int curCrc32 = FastCrc.calcCrc(pageBuf, pageSize());
+
+                if ((savedCrc32 ^ curCrc32) != 0) {
+                    throw new IgniteInternalDataIntegrityViolationException("Failed to read page (CRC validation failed) "
+                            + "[id=" + hexLong(pageId) + ", off=" + pageOff
+                            + ", filePath=" + filePath + ", fileSize=" + fileIo.size()
+                            + ", savedCrc=" + hexInt(savedCrc32) + ", curCrc=" + hexInt(curCrc32)
+                            + ", page=" + toHexString(pageBuf) + "]");
+                }
+            }
+
+            assert PageIo.getCrc(pageBuf) == 0;
+
+            if (keepCrc) {
+                PageIo.setCrc(pageBuf, savedCrc32);
+            }
+
+            return true;
+        } catch (IOException e) {
+            throw new IgniteInternalCheckedException("Failed to read page [file=" + filePath + ", pageId=" + pageId + "]", e);
+        }
+    }
+
+    /**
+     * Reads from page storage with failover.
+     *
+     * @param destBuf Destination buffer.
+     * @param position Position.
+     * @return Number of read bytes, or {@code -1} if the given position is greater than or equal to the file's current size.
+     */
+    private int readWithFailover(ByteBuffer destBuf, long position) throws IOException {
+        boolean interrupted = false;
+
+        int bufPos = destBuf.position();
+
+        while (true) {
+            FileIo fileIo = this.fileIo;
+
+            if (fileIo == null) {
+                throw new IOException("FileIo has stopped");
+            }
+
+            try {
+                assert destBuf.remaining() > 0;
+
+                int bytesRead = fileIo.readFully(destBuf, position);
+
+                if (interrupted) {
+                    Thread.currentThread().interrupt();
+                }
+
+                return bytesRead;
+            } catch (ClosedChannelException e) {
+                destBuf.position(bufPos);
+
+                if (e instanceof ClosedByInterruptException) {
+                    interrupted = true;
+
+                    Thread.interrupted();
+                }
+
+                reinit(fileIo);
+            }
+        }
+    }
+
+    /**
+     * Returns file page store path.
+     */
+    public Path filePath() {
+        return filePath;
+    }
+
+    /**
+     * Renames the current file page store path to a new one.
+     *
+     * @param newFilePath New file page store path.
+     * @throws IOException If failed.
+     */
+    public void renameFilePath(Path newFilePath) throws IOException {
+        initialized = false;
+
+        readWriteLock.writeLock().lock();
+
+        try {
+            Path filePath = this.filePath;
+
+            if (!filePath.equals(newFilePath)) {
+                FileIo fileIo = this.fileIo;
+
+                if (fileIo != null) {

Review Comment:
   Discussed personally, I will add a check.



-- 
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 #947: IGNITE-17372 Implement DeltaFilePageStore

Posted by GitBox <gi...@apache.org>.
ibessonov commented on code in PR #947:
URL: https://github.com/apache/ignite-3/pull/947#discussion_r927712343


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/AbstractFilePageStoreIo.java:
##########
@@ -0,0 +1,611 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.pagememory.persistence.store;
+
+import static java.nio.ByteOrder.nativeOrder;
+import static java.nio.file.StandardOpenOption.CREATE;
+import static java.nio.file.StandardOpenOption.READ;
+import static java.nio.file.StandardOpenOption.WRITE;
+import static org.apache.ignite.internal.util.IgniteUtils.atomicMoveFile;
+import static org.apache.ignite.internal.util.IgniteUtils.hexInt;
+import static org.apache.ignite.internal.util.IgniteUtils.hexLong;
+import static org.apache.ignite.internal.util.IgniteUtils.toHexString;
+import static org.apache.ignite.lang.IgniteSystemProperties.getBoolean;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.ClosedByInterruptException;
+import java.nio.channels.ClosedChannelException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.apache.ignite.internal.fileio.FileIo;
+import org.apache.ignite.internal.fileio.FileIoFactory;
+import org.apache.ignite.internal.pagememory.io.PageIo;
+import org.apache.ignite.internal.pagememory.persistence.FastCrc;
+import org.apache.ignite.internal.pagememory.persistence.IgniteInternalDataIntegrityViolationException;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Abstract class for performing IO operations on file page storage.
+ */
+public abstract class AbstractFilePageStoreIo implements Closeable {
+    private final FileIoFactory ioFactory;
+
+    private final ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
+
+    /** Skip CRC calculation flag. */
+    // TODO: IGNITE-17011 Move to config
+    private final boolean skipCrc = getBoolean("IGNITE_PDS_SKIP_CRC");
+
+    private volatile Path filePath;

Review Comment:
   I believe that this field can only be updated in constructor or while holding a write lock. If we only read it holding a read lock, then "volatile" can be removed, but I don't know for sure if that's the case. I'm leaving it for you to check.



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/PartitionMetaManager.java:
##########
@@ -95,10 +95,8 @@ public PartitionMeta readOrCreateMeta(
         long partitionMetaPageId = partitionMetaPageId(groupPartitionId.getPartitionId());
 
         try {
-            if (filePageStore.size() > filePageStore.headerSize()) {
+            if (filePageStore.readWithoutPageIdCheck(partitionMetaPageId, buffer, false)) {

Review Comment:
   Ok, let's start with the very first line.
   Code became less clear right now. I would expect to see something like
   `if (filePageStore.isValid()) { .. read it ...} else { ... recreate it ... }`
   I mean, this method is not only for reading a meta page, but also for a recovery in a way.
   And, I mentioned it in previous review, we should double-check the disaster recovery procedure here. Original meta page is written to "bin" file, not in "tmp" file. This means that is can technically have a broken CRC. Of course, this can't be applied to delta files, only for a regular partition file.
   Everything that I said is very confusing probably.
   
   Anyways, "read' method doesn't have to return a boolean flag, right?



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/AbstractFilePageStoreIo.java:
##########
@@ -0,0 +1,611 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.pagememory.persistence.store;
+
+import static java.nio.ByteOrder.nativeOrder;
+import static java.nio.file.StandardOpenOption.CREATE;
+import static java.nio.file.StandardOpenOption.READ;
+import static java.nio.file.StandardOpenOption.WRITE;
+import static org.apache.ignite.internal.util.IgniteUtils.atomicMoveFile;
+import static org.apache.ignite.internal.util.IgniteUtils.hexInt;
+import static org.apache.ignite.internal.util.IgniteUtils.hexLong;
+import static org.apache.ignite.internal.util.IgniteUtils.toHexString;
+import static org.apache.ignite.lang.IgniteSystemProperties.getBoolean;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.ClosedByInterruptException;
+import java.nio.channels.ClosedChannelException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.apache.ignite.internal.fileio.FileIo;
+import org.apache.ignite.internal.fileio.FileIoFactory;
+import org.apache.ignite.internal.pagememory.io.PageIo;
+import org.apache.ignite.internal.pagememory.persistence.FastCrc;
+import org.apache.ignite.internal.pagememory.persistence.IgniteInternalDataIntegrityViolationException;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Abstract class for performing IO operations on file page storage.
+ */
+public abstract class AbstractFilePageStoreIo implements Closeable {
+    private final FileIoFactory ioFactory;
+
+    private final ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
+
+    /** Skip CRC calculation flag. */
+    // TODO: IGNITE-17011 Move to config
+    private final boolean skipCrc = getBoolean("IGNITE_PDS_SKIP_CRC");
+
+    private volatile Path filePath;
+
+    private volatile @Nullable FileIo fileIo;
+
+    /** Initialized file page store IO. */
+    private volatile boolean initialized;
+
+    /** Caches the existence state of file. After it is initialized, it will be not {@code null} during lifecycle. */
+    private volatile @Nullable Boolean fileExists;
+
+    /**
+     * Constructor.
+     *
+     * @param ioFactory {@link FileIo} factory.
+     * @param filePath File page store path.
+     */
+    AbstractFilePageStoreIo(FileIoFactory ioFactory, Path filePath) {
+        this.ioFactory = ioFactory;
+        this.filePath = filePath;
+    }
+
+    /**
+     * Returns the page size in bytes.
+     */
+    public abstract int pageSize();
+
+    /**
+     * Returns the size of the header in bytes.
+     */
+    public abstract int headerSize();
+
+    /**
+     * Returns a buffer with a file page storage header.
+     */
+    public abstract ByteBuffer headerBuffer();
+
+    /**
+     * Checks the file page storage header.
+     *
+     * @param fileIo File page store IO to read the header.
+     * @throws IOException If there is an error reading the header or the header did not pass the check.
+     */
+    public abstract void checkHeader(FileIo fileIo) throws IOException;
+
+    /**
+     * Returns page offset within the store file.
+     *
+     * @param pageId Page ID.
+     */
+    public abstract long pageOffset(long pageId);

Review Comment:
   Why is it here? Can read and write methods calculate offsets internally by themselves?



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/AbstractFilePageStoreIo.java:
##########
@@ -0,0 +1,611 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.pagememory.persistence.store;
+
+import static java.nio.ByteOrder.nativeOrder;
+import static java.nio.file.StandardOpenOption.CREATE;
+import static java.nio.file.StandardOpenOption.READ;
+import static java.nio.file.StandardOpenOption.WRITE;
+import static org.apache.ignite.internal.util.IgniteUtils.atomicMoveFile;
+import static org.apache.ignite.internal.util.IgniteUtils.hexInt;
+import static org.apache.ignite.internal.util.IgniteUtils.hexLong;
+import static org.apache.ignite.internal.util.IgniteUtils.toHexString;
+import static org.apache.ignite.lang.IgniteSystemProperties.getBoolean;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.ClosedByInterruptException;
+import java.nio.channels.ClosedChannelException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.apache.ignite.internal.fileio.FileIo;
+import org.apache.ignite.internal.fileio.FileIoFactory;
+import org.apache.ignite.internal.pagememory.io.PageIo;
+import org.apache.ignite.internal.pagememory.persistence.FastCrc;
+import org.apache.ignite.internal.pagememory.persistence.IgniteInternalDataIntegrityViolationException;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Abstract class for performing IO operations on file page storage.
+ */
+public abstract class AbstractFilePageStoreIo implements Closeable {
+    private final FileIoFactory ioFactory;
+
+    private final ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
+
+    /** Skip CRC calculation flag. */
+    // TODO: IGNITE-17011 Move to config
+    private final boolean skipCrc = getBoolean("IGNITE_PDS_SKIP_CRC");
+
+    private volatile Path filePath;
+
+    private volatile @Nullable FileIo fileIo;
+
+    /** Initialized file page store IO. */
+    private volatile boolean initialized;
+
+    /** Caches the existence state of file. After it is initialized, it will be not {@code null} during lifecycle. */
+    private volatile @Nullable Boolean fileExists;
+
+    /**
+     * Constructor.
+     *
+     * @param ioFactory {@link FileIo} factory.
+     * @param filePath File page store path.
+     */
+    AbstractFilePageStoreIo(FileIoFactory ioFactory, Path filePath) {
+        this.ioFactory = ioFactory;
+        this.filePath = filePath;
+    }
+
+    /**
+     * Returns the page size in bytes.
+     */
+    public abstract int pageSize();
+
+    /**
+     * Returns the size of the header in bytes.
+     */
+    public abstract int headerSize();
+
+    /**
+     * Returns a buffer with a file page storage header.
+     */
+    public abstract ByteBuffer headerBuffer();

Review Comment:
   Is this a buffer from the file or just a `ByteBuffer` with appropriate content? It's not clear from the name and a comment.



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/AbstractFilePageStoreIo.java:
##########
@@ -0,0 +1,611 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.pagememory.persistence.store;
+
+import static java.nio.ByteOrder.nativeOrder;
+import static java.nio.file.StandardOpenOption.CREATE;
+import static java.nio.file.StandardOpenOption.READ;
+import static java.nio.file.StandardOpenOption.WRITE;
+import static org.apache.ignite.internal.util.IgniteUtils.atomicMoveFile;
+import static org.apache.ignite.internal.util.IgniteUtils.hexInt;
+import static org.apache.ignite.internal.util.IgniteUtils.hexLong;
+import static org.apache.ignite.internal.util.IgniteUtils.toHexString;
+import static org.apache.ignite.lang.IgniteSystemProperties.getBoolean;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.ClosedByInterruptException;
+import java.nio.channels.ClosedChannelException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.apache.ignite.internal.fileio.FileIo;
+import org.apache.ignite.internal.fileio.FileIoFactory;
+import org.apache.ignite.internal.pagememory.io.PageIo;
+import org.apache.ignite.internal.pagememory.persistence.FastCrc;
+import org.apache.ignite.internal.pagememory.persistence.IgniteInternalDataIntegrityViolationException;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Abstract class for performing IO operations on file page storage.
+ */
+public abstract class AbstractFilePageStoreIo implements Closeable {
+    private final FileIoFactory ioFactory;
+
+    private final ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
+
+    /** Skip CRC calculation flag. */
+    // TODO: IGNITE-17011 Move to config
+    private final boolean skipCrc = getBoolean("IGNITE_PDS_SKIP_CRC");
+
+    private volatile Path filePath;
+
+    private volatile @Nullable FileIo fileIo;
+
+    /** Initialized file page store IO. */
+    private volatile boolean initialized;
+
+    /** Caches the existence state of file. After it is initialized, it will be not {@code null} during lifecycle. */
+    private volatile @Nullable Boolean fileExists;
+
+    /**
+     * Constructor.
+     *
+     * @param ioFactory {@link FileIo} factory.
+     * @param filePath File page store path.
+     */
+    AbstractFilePageStoreIo(FileIoFactory ioFactory, Path filePath) {
+        this.ioFactory = ioFactory;
+        this.filePath = filePath;
+    }
+
+    /**
+     * Returns the page size in bytes.
+     */
+    public abstract int pageSize();
+
+    /**
+     * Returns the size of the header in bytes.
+     */
+    public abstract int headerSize();
+
+    /**
+     * Returns a buffer with a file page storage header.
+     */
+    public abstract ByteBuffer headerBuffer();
+
+    /**
+     * Checks the file page storage header.
+     *
+     * @param fileIo File page store IO to read the header.
+     * @throws IOException If there is an error reading the header or the header did not pass the check.
+     */
+    public abstract void checkHeader(FileIo fileIo) throws IOException;
+
+    /**
+     * Returns page offset within the store file.
+     *
+     * @param pageId Page ID.
+     */
+    public abstract long pageOffset(long pageId);
+
+    /**
+     * Stops the file page store IO.
+     *
+     * @param clean {@code True} to clean file page store.
+     * @throws IgniteInternalCheckedException If failed.
+     */
+    public void stop(boolean clean) throws IgniteInternalCheckedException {
+        try {
+            stop0(clean);
+        } catch (IOException e) {
+            throw new IgniteInternalCheckedException(
+                    "Failed to stop serving file [file=" + filePath + ", delete=" + clean + "]",
+                    e
+            );
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public void close() throws IOException {
+        stop0(false);
+    }
+
+    /**
+     * Reads a page.
+     *
+     * @param pageId Page ID.
+     * @param pageOff Page offset in the file.
+     * @param pageBuf Page buffer to read into.
+     * @param keepCrc By default, reading zeroes CRC which was on page store, but you can keep it in {@code pageBuf} if set {@code

Review Comment:
   "if set true" maybe?



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/AbstractFilePageStoreIo.java:
##########
@@ -0,0 +1,611 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.pagememory.persistence.store;
+
+import static java.nio.ByteOrder.nativeOrder;
+import static java.nio.file.StandardOpenOption.CREATE;
+import static java.nio.file.StandardOpenOption.READ;
+import static java.nio.file.StandardOpenOption.WRITE;
+import static org.apache.ignite.internal.util.IgniteUtils.atomicMoveFile;
+import static org.apache.ignite.internal.util.IgniteUtils.hexInt;
+import static org.apache.ignite.internal.util.IgniteUtils.hexLong;
+import static org.apache.ignite.internal.util.IgniteUtils.toHexString;
+import static org.apache.ignite.lang.IgniteSystemProperties.getBoolean;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.ClosedByInterruptException;
+import java.nio.channels.ClosedChannelException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.apache.ignite.internal.fileio.FileIo;
+import org.apache.ignite.internal.fileio.FileIoFactory;
+import org.apache.ignite.internal.pagememory.io.PageIo;
+import org.apache.ignite.internal.pagememory.persistence.FastCrc;
+import org.apache.ignite.internal.pagememory.persistence.IgniteInternalDataIntegrityViolationException;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Abstract class for performing IO operations on file page storage.
+ */
+public abstract class AbstractFilePageStoreIo implements Closeable {
+    private final FileIoFactory ioFactory;
+
+    private final ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
+
+    /** Skip CRC calculation flag. */
+    // TODO: IGNITE-17011 Move to config
+    private final boolean skipCrc = getBoolean("IGNITE_PDS_SKIP_CRC");
+
+    private volatile Path filePath;
+
+    private volatile @Nullable FileIo fileIo;
+
+    /** Initialized file page store IO. */
+    private volatile boolean initialized;
+
+    /** Caches the existence state of file. After it is initialized, it will be not {@code null} during lifecycle. */
+    private volatile @Nullable Boolean fileExists;
+
+    /**
+     * Constructor.
+     *
+     * @param ioFactory {@link FileIo} factory.
+     * @param filePath File page store path.
+     */
+    AbstractFilePageStoreIo(FileIoFactory ioFactory, Path filePath) {
+        this.ioFactory = ioFactory;
+        this.filePath = filePath;
+    }
+
+    /**
+     * Returns the page size in bytes.
+     */
+    public abstract int pageSize();
+
+    /**
+     * Returns the size of the header in bytes.
+     */
+    public abstract int headerSize();
+
+    /**
+     * Returns a buffer with a file page storage header.
+     */
+    public abstract ByteBuffer headerBuffer();
+
+    /**
+     * Checks the file page storage header.
+     *
+     * @param fileIo File page store IO to read the header.
+     * @throws IOException If there is an error reading the header or the header did not pass the check.
+     */
+    public abstract void checkHeader(FileIo fileIo) throws IOException;
+
+    /**
+     * Returns page offset within the store file.
+     *
+     * @param pageId Page ID.
+     */
+    public abstract long pageOffset(long pageId);
+
+    /**
+     * Stops the file page store IO.
+     *
+     * @param clean {@code True} to clean file page store.
+     * @throws IgniteInternalCheckedException If failed.
+     */
+    public void stop(boolean clean) throws IgniteInternalCheckedException {
+        try {
+            stop0(clean);
+        } catch (IOException e) {
+            throw new IgniteInternalCheckedException(
+                    "Failed to stop serving file [file=" + filePath + ", delete=" + clean + "]",
+                    e
+            );
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public void close() throws IOException {
+        stop0(false);
+    }
+
+    /**
+     * Reads a page.
+     *
+     * @param pageId Page ID.
+     * @param pageOff Page offset in the file.
+     * @param pageBuf Page buffer to read into.
+     * @param keepCrc By default, reading zeroes CRC which was on page store, but you can keep it in {@code pageBuf} if set {@code
+     * keepCrc}.
+     * @return {@code True} if the page was read successfully.
+     * @throws IgniteInternalCheckedException If reading failed (IO error occurred).
+     */
+    public boolean read(long pageId, long pageOff, ByteBuffer pageBuf, boolean keepCrc) throws IgniteInternalCheckedException {
+        return read0(pageId, pageOff, pageBuf, !skipCrc, keepCrc);
+    }
+
+    /**
+     * Writes a page.
+     *
+     * @param pageId Page ID.
+     * @param pageBuf Page buffer to write from.
+     * @param calculateCrc If {@code false} crc calculation will be forcibly skipped.
+     * @throws IgniteInternalCheckedException If page writing failed (IO error occurred).
+     */
+    public void write(long pageId, ByteBuffer pageBuf, boolean calculateCrc) throws IgniteInternalCheckedException {
+        ensure();
+
+        boolean interrupted = false;
+
+        while (true) {
+            FileIo fileIo = this.fileIo;
+
+            try {
+                readWriteLock.readLock().lock();
+
+                try {
+                    assert pageBuf.position() == 0 : pageBuf.position();
+                    assert pageBuf.order() == nativeOrder() : "Page buffer order " + pageBuf.order()
+                            + " should be same with " + nativeOrder();
+                    assert PageIo.getType(pageBuf) != 0 : "Invalid state. Type is 0! pageId = " + hexLong(pageId);
+                    assert PageIo.getVersion(pageBuf) != 0 : "Invalid state. Version is 0! pageId = " + hexLong(pageId);
+
+                    if (calculateCrc && !skipCrc) {
+                        assert PageIo.getCrc(pageBuf) == 0 : hexLong(pageId);
+
+                        PageIo.setCrc(pageBuf, calcCrc32(pageBuf, pageSize()));
+                    }
+
+                    // Check whether crc was calculated somewhere above the stack if it is forcibly skipped.
+                    assert skipCrc || PageIo.getCrc(pageBuf) != 0
+                            || calcCrc32(pageBuf, pageSize()) == 0 : "CRC hasn't been calculated, crc=0";
+
+                    assert pageBuf.position() == 0 : pageBuf.position();
+
+                    long pageOff = pageOffset(pageId);
+
+                    fileIo.writeFully(pageBuf, pageOff);
+
+                    PageIo.setCrc(pageBuf, 0);
+
+                    if (interrupted) {
+                        Thread.currentThread().interrupt();
+                    }
+
+                    return;
+                } finally {
+                    readWriteLock.readLock().unlock();
+                }
+            } catch (IOException e) {
+                if (e instanceof ClosedChannelException) {
+                    try {
+                        if (e instanceof ClosedByInterruptException) {
+                            interrupted = true;
+
+                            Thread.interrupted();
+                        }
+
+                        reinit(fileIo);
+
+                        pageBuf.position(0);
+
+                        PageIo.setCrc(pageBuf, 0);
+
+                        continue;
+                    } catch (IOException e0) {
+                        e0.addSuppressed(e);
+
+                        e = e0;
+                    }
+                }
+
+                throw new IgniteInternalCheckedException(
+                        "Failed to write page [filePath=" + filePath + ", pageId=" + pageId + "]",
+                        e
+                );
+            }
+        }
+    }
+
+    /**
+     * Sync method used to ensure that the given pages are guaranteed to be written to the file page store.
+     *
+     * @throws IgniteInternalCheckedException If sync failed (IO error occurred).
+     */
+    public void sync() throws IgniteInternalCheckedException {
+        readWriteLock.writeLock().lock();

Review Comment:
   Why exactly do we have a write lock here?



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/CompleteCreationDeltaFilePageStoreIoCallback.java:
##########
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.pagememory.persistence.store;
+
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+
+/**
+ * Callback on completion of delta file page store creation.
+ */
+@FunctionalInterface
+public interface CompleteCreationDeltaFilePageStoreIoCallback {
+    /**
+     * Calls when the delta file page store is finished creating.
+     *
+     * @param deltaFilePageStoreIo Delta file page store.
+     * @throws IgniteInternalCheckedException If failed.
+     */
+    void onCompletionOfCreation(DeltaFilePageStoreIo deltaFilePageStoreIo) throws IgniteInternalCheckedException;

Review Comment:
   "onCreationCompleted" maybe? We usually don't use word "of" and many other short words in naming



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/DeltaFilePageStoreIoFactory.java:
##########
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.pagememory.persistence.store;
+
+/**
+ * Factory for creating {@link DeltaFilePageStoreIo}.
+ */
+@FunctionalInterface
+public interface DeltaFilePageStoreIoFactory {

Review Comment:
   Very interesting, I wonder why we need a factory for this



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/AbstractFilePageStoreIo.java:
##########
@@ -0,0 +1,611 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.pagememory.persistence.store;
+
+import static java.nio.ByteOrder.nativeOrder;
+import static java.nio.file.StandardOpenOption.CREATE;
+import static java.nio.file.StandardOpenOption.READ;
+import static java.nio.file.StandardOpenOption.WRITE;
+import static org.apache.ignite.internal.util.IgniteUtils.atomicMoveFile;
+import static org.apache.ignite.internal.util.IgniteUtils.hexInt;
+import static org.apache.ignite.internal.util.IgniteUtils.hexLong;
+import static org.apache.ignite.internal.util.IgniteUtils.toHexString;
+import static org.apache.ignite.lang.IgniteSystemProperties.getBoolean;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.ClosedByInterruptException;
+import java.nio.channels.ClosedChannelException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.apache.ignite.internal.fileio.FileIo;
+import org.apache.ignite.internal.fileio.FileIoFactory;
+import org.apache.ignite.internal.pagememory.io.PageIo;
+import org.apache.ignite.internal.pagememory.persistence.FastCrc;
+import org.apache.ignite.internal.pagememory.persistence.IgniteInternalDataIntegrityViolationException;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Abstract class for performing IO operations on file page storage.
+ */
+public abstract class AbstractFilePageStoreIo implements Closeable {
+    private final FileIoFactory ioFactory;
+
+    private final ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
+
+    /** Skip CRC calculation flag. */
+    // TODO: IGNITE-17011 Move to config
+    private final boolean skipCrc = getBoolean("IGNITE_PDS_SKIP_CRC");
+
+    private volatile Path filePath;
+
+    private volatile @Nullable FileIo fileIo;
+
+    /** Initialized file page store IO. */
+    private volatile boolean initialized;
+
+    /** Caches the existence state of file. After it is initialized, it will be not {@code null} during lifecycle. */
+    private volatile @Nullable Boolean fileExists;
+
+    /**
+     * Constructor.
+     *
+     * @param ioFactory {@link FileIo} factory.
+     * @param filePath File page store path.
+     */
+    AbstractFilePageStoreIo(FileIoFactory ioFactory, Path filePath) {
+        this.ioFactory = ioFactory;
+        this.filePath = filePath;
+    }
+
+    /**
+     * Returns the page size in bytes.
+     */
+    public abstract int pageSize();
+
+    /**
+     * Returns the size of the header in bytes.
+     */
+    public abstract int headerSize();
+
+    /**
+     * Returns a buffer with a file page storage header.
+     */
+    public abstract ByteBuffer headerBuffer();
+
+    /**
+     * Checks the file page storage header.
+     *
+     * @param fileIo File page store IO to read the header.
+     * @throws IOException If there is an error reading the header or the header did not pass the check.
+     */
+    public abstract void checkHeader(FileIo fileIo) throws IOException;
+
+    /**
+     * Returns page offset within the store file.
+     *
+     * @param pageId Page ID.
+     */
+    public abstract long pageOffset(long pageId);
+
+    /**
+     * Stops the file page store IO.
+     *
+     * @param clean {@code True} to clean file page store.
+     * @throws IgniteInternalCheckedException If failed.
+     */
+    public void stop(boolean clean) throws IgniteInternalCheckedException {
+        try {
+            stop0(clean);
+        } catch (IOException e) {
+            throw new IgniteInternalCheckedException(
+                    "Failed to stop serving file [file=" + filePath + ", delete=" + clean + "]",
+                    e
+            );
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public void close() throws IOException {
+        stop0(false);
+    }
+
+    /**
+     * Reads a page.
+     *
+     * @param pageId Page ID.
+     * @param pageOff Page offset in the file.
+     * @param pageBuf Page buffer to read into.
+     * @param keepCrc By default, reading zeroes CRC which was on page store, but you can keep it in {@code pageBuf} if set {@code
+     * keepCrc}.
+     * @return {@code True} if the page was read successfully.
+     * @throws IgniteInternalCheckedException If reading failed (IO error occurred).
+     */
+    public boolean read(long pageId, long pageOff, ByteBuffer pageBuf, boolean keepCrc) throws IgniteInternalCheckedException {
+        return read0(pageId, pageOff, pageBuf, !skipCrc, keepCrc);
+    }
+
+    /**
+     * Writes a page.
+     *
+     * @param pageId Page ID.
+     * @param pageBuf Page buffer to write from.
+     * @param calculateCrc If {@code false} crc calculation will be forcibly skipped.
+     * @throws IgniteInternalCheckedException If page writing failed (IO error occurred).
+     */
+    public void write(long pageId, ByteBuffer pageBuf, boolean calculateCrc) throws IgniteInternalCheckedException {
+        ensure();
+
+        boolean interrupted = false;
+
+        while (true) {
+            FileIo fileIo = this.fileIo;
+
+            try {
+                readWriteLock.readLock().lock();
+
+                try {
+                    assert pageBuf.position() == 0 : pageBuf.position();
+                    assert pageBuf.order() == nativeOrder() : "Page buffer order " + pageBuf.order()
+                            + " should be same with " + nativeOrder();
+                    assert PageIo.getType(pageBuf) != 0 : "Invalid state. Type is 0! pageId = " + hexLong(pageId);
+                    assert PageIo.getVersion(pageBuf) != 0 : "Invalid state. Version is 0! pageId = " + hexLong(pageId);
+
+                    if (calculateCrc && !skipCrc) {
+                        assert PageIo.getCrc(pageBuf) == 0 : hexLong(pageId);
+
+                        PageIo.setCrc(pageBuf, calcCrc32(pageBuf, pageSize()));
+                    }
+
+                    // Check whether crc was calculated somewhere above the stack if it is forcibly skipped.
+                    assert skipCrc || PageIo.getCrc(pageBuf) != 0
+                            || calcCrc32(pageBuf, pageSize()) == 0 : "CRC hasn't been calculated, crc=0";
+
+                    assert pageBuf.position() == 0 : pageBuf.position();
+
+                    long pageOff = pageOffset(pageId);
+
+                    fileIo.writeFully(pageBuf, pageOff);
+
+                    PageIo.setCrc(pageBuf, 0);
+
+                    if (interrupted) {
+                        Thread.currentThread().interrupt();
+                    }
+
+                    return;
+                } finally {
+                    readWriteLock.readLock().unlock();
+                }
+            } catch (IOException e) {
+                if (e instanceof ClosedChannelException) {
+                    try {
+                        if (e instanceof ClosedByInterruptException) {
+                            interrupted = true;
+
+                            Thread.interrupted();
+                        }
+
+                        reinit(fileIo);
+
+                        pageBuf.position(0);
+
+                        PageIo.setCrc(pageBuf, 0);
+
+                        continue;
+                    } catch (IOException e0) {
+                        e0.addSuppressed(e);
+
+                        e = e0;
+                    }
+                }
+
+                throw new IgniteInternalCheckedException(
+                        "Failed to write page [filePath=" + filePath + ", pageId=" + pageId + "]",
+                        e
+                );
+            }
+        }
+    }
+
+    /**
+     * Sync method used to ensure that the given pages are guaranteed to be written to the file page store.
+     *
+     * @throws IgniteInternalCheckedException If sync failed (IO error occurred).
+     */
+    public void sync() throws IgniteInternalCheckedException {
+        readWriteLock.writeLock().lock();
+
+        try {
+            ensure();
+
+            FileIo fileIo = this.fileIo;
+
+            if (fileIo != null) {
+                fileIo.force();
+            }
+        } catch (IOException e) {
+            throw new IgniteInternalCheckedException("Failed to fsync file [filePath=" + filePath + ']', e);
+        } finally {
+            readWriteLock.writeLock().unlock();
+        }
+    }
+
+    /**
+     * Returns {@code true} if the file page store exists.
+     */
+    public boolean exists() {
+        if (fileExists == null) {
+            readWriteLock.readLock().lock();
+
+            try {
+                if (fileExists == null) {
+                    fileExists = Files.exists(filePath) && filePath.toFile().length() >= headerSize();
+                }
+            } finally {
+                readWriteLock.readLock().unlock();
+            }
+        }
+
+        return fileExists;
+    }
+
+    /**
+     * Initializes the file page store if it hasn't already.
+     *
+     * @throws IgniteInternalCheckedException If initialization failed (IO error occurred).
+     */
+    public void ensure() throws IgniteInternalCheckedException {
+        if (!initialized) {
+            readWriteLock.writeLock().lock();
+
+            try {
+                if (!initialized) {
+                    FileIo fileIo = null;
+
+                    IgniteInternalCheckedException err = null;
+
+                    try {
+                        boolean interrupted = false;
+
+                        while (true) {
+                            try {
+                                this.fileIo = fileIo = ioFactory.create(filePath, CREATE, READ, WRITE);
+
+                                fileExists = true;
+
+                                if (fileIo.size() < headerSize()) {
+                                    fileIo.writeFully(headerBuffer().rewind(), 0);
+                                } else {
+                                    checkHeader(fileIo);
+                                }
+
+                                if (interrupted) {
+                                    Thread.currentThread().interrupt();
+                                }
+
+                                break;
+                            } catch (ClosedByInterruptException e) {
+                                interrupted = true;
+
+                                Thread.interrupted();
+                            }
+                        }
+
+                        initialized = true;
+                    } catch (IOException e) {
+                        err = new IgniteInternalCheckedException("Failed to initialize partition file: " + filePath, e);
+
+                        throw err;
+                    } finally {
+                        if (err != null && fileIo != null) {
+                            try {
+                                fileIo.close();
+                            } catch (IOException e) {
+                                err.addSuppressed(e);
+                            }
+                        }
+                    }
+                }
+            } finally {
+                readWriteLock.writeLock().unlock();
+            }
+        }
+    }
+
+    /**
+     * Returns size of the file page store in bytes.
+     *
+     * @throws IgniteInternalCheckedException If an I/O error occurs.
+     */
+    public long size() throws IgniteInternalCheckedException {
+        readWriteLock.readLock().lock();
+
+        try {
+            FileIo io = fileIo;
+
+            return io == null ? 0 : io.size();
+        } catch (IOException e) {
+            throw new IgniteInternalCheckedException(e);
+        } finally {
+            readWriteLock.readLock().unlock();
+        }
+    }
+
+    private void stop0(boolean clean) throws IOException {
+        readWriteLock.writeLock().lock();
+
+        try {
+            if (!initialized) {
+                // Ensure the file is closed even if not initialized yet.
+                if (fileIo != null) {
+                    fileIo.close();
+                }
+
+                if (clean && exists()) {
+                    Files.delete(filePath);
+                }
+
+                return;
+            }
+
+            fileIo.force();
+
+            fileIo.close();
+
+            fileIo = null;
+
+            if (clean) {
+                Files.delete(filePath);
+
+                fileExists = false;
+            }
+        } finally {
+            initialized = false;
+
+            readWriteLock.writeLock().unlock();
+        }
+    }
+
+    private static int calcCrc32(ByteBuffer pageBuf, int pageSize) {
+        try {
+            pageBuf.position(0);
+
+            return FastCrc.calcCrc(pageBuf, pageSize);
+        } finally {
+            pageBuf.position(0);
+        }
+    }
+
+    /**
+     * Reinit page store after file channel was closed by thread interruption.
+     *
+     * @param fileIo Old fileIo.
+     */
+    private void reinit(FileIo fileIo) throws IOException {
+        if (fileIo != this.fileIo) {
+            return;
+        }
+
+        readWriteLock.writeLock().lock();
+
+        try {
+            if (fileIo != this.fileIo) {
+                return;
+            }
+
+            try {
+                boolean interrupted = false;
+
+                while (true) {
+                    try {
+                        fileIo = null;
+
+                        fileIo = ioFactory.create(filePath, CREATE, READ, WRITE);
+
+                        fileExists = true;
+
+                        checkHeader(fileIo);
+
+                        this.fileIo = fileIo;
+
+                        if (interrupted) {
+                            Thread.currentThread().interrupt();
+                        }
+
+                        break;
+                    } catch (ClosedByInterruptException e) {
+                        interrupted = true;
+
+                        Thread.interrupted();
+                    }
+                }
+            } catch (IOException e) {
+                try {
+                    if (fileIo != null) {
+                        fileIo.close();
+                    }
+                } catch (IOException e0) {
+                    e.addSuppressed(e0);
+                }
+
+                throw e;
+            }
+        } finally {
+            readWriteLock.writeLock().unlock();
+        }
+    }
+
+    /**
+     * Reads a page from the page store.
+     *
+     * @param pageId Page ID.
+     * @param pageOff Page offset in the file.
+     * @param pageBuf Page buffer to read into.
+     * @param checkCrc Check CRC on page.
+     * @param keepCrc By default reading zeroes CRC which was on file, but you can keep it in pageBuf if set keepCrc.
+     * @return {@code True} if the page was read successfully.
+     * @throws IgniteInternalCheckedException If reading failed (IO error occurred).
+     */
+    private boolean read0(
+            long pageId,
+            long pageOff,
+            ByteBuffer pageBuf,
+            boolean checkCrc,
+            boolean keepCrc
+    ) throws IgniteInternalCheckedException {
+        assert pageOff >= headerSize() : "pageOff=" + pageOff + ", headerSize=" + headerSize();
+
+        ensure();
+
+        try {
+            assert pageBuf.capacity() == pageSize() : pageBuf.capacity();
+            assert pageBuf.remaining() == pageSize() : pageBuf.remaining();
+            assert pageBuf.position() == 0 : pageBuf.position();
+            assert pageBuf.order() == nativeOrder() : pageBuf.order();
+
+            int n = readWithFailover(pageBuf, pageOff);
+
+            // If page was not written yet, nothing to read.
+            if (n < 0) {
+                pageBuf.put(new byte[pageBuf.remaining()]);
+
+                return false;
+            }
+
+            int savedCrc32 = PageIo.getCrc(pageBuf);
+
+            PageIo.setCrc(pageBuf, 0);
+
+            pageBuf.position(0);
+
+            if (checkCrc) {
+                int curCrc32 = FastCrc.calcCrc(pageBuf, pageSize());
+
+                if ((savedCrc32 ^ curCrc32) != 0) {
+                    throw new IgniteInternalDataIntegrityViolationException("Failed to read page (CRC validation failed) "
+                            + "[id=" + hexLong(pageId) + ", off=" + pageOff
+                            + ", filePath=" + filePath + ", fileSize=" + fileIo.size()
+                            + ", savedCrc=" + hexInt(savedCrc32) + ", curCrc=" + hexInt(curCrc32)
+                            + ", page=" + toHexString(pageBuf) + "]");
+                }
+            }
+
+            assert PageIo.getCrc(pageBuf) == 0;
+
+            if (keepCrc) {
+                PageIo.setCrc(pageBuf, savedCrc32);
+            }
+
+            return true;
+        } catch (IOException e) {
+            throw new IgniteInternalCheckedException("Failed to read page [file=" + filePath + ", pageId=" + pageId + "]", e);
+        }
+    }
+
+    /**
+     * Reads from page storage with failover.
+     *
+     * @param destBuf Destination buffer.
+     * @param position Position.
+     * @return Number of read bytes, or {@code -1} if the given position is greater than or equal to the file's current size.
+     */
+    private int readWithFailover(ByteBuffer destBuf, long position) throws IOException {
+        boolean interrupted = false;
+
+        int bufPos = destBuf.position();
+
+        while (true) {
+            FileIo fileIo = this.fileIo;
+
+            if (fileIo == null) {
+                throw new IOException("FileIo has stopped");
+            }
+
+            try {
+                assert destBuf.remaining() > 0;
+
+                int bytesRead = fileIo.readFully(destBuf, position);
+
+                if (interrupted) {
+                    Thread.currentThread().interrupt();
+                }
+
+                return bytesRead;
+            } catch (ClosedChannelException e) {
+                destBuf.position(bufPos);
+
+                if (e instanceof ClosedByInterruptException) {
+                    interrupted = true;
+
+                    Thread.interrupted();
+                }
+
+                reinit(fileIo);
+            }
+        }
+    }
+
+    /**
+     * Returns file page store path.
+     */
+    public Path filePath() {
+        return filePath;
+    }
+
+    /**
+     * Renames the current file page store path to a new one.
+     *
+     * @param newFilePath New file page store path.
+     * @throws IOException If failed.
+     */
+    public void renameFilePath(Path newFilePath) throws IOException {
+        initialized = false;
+
+        readWriteLock.writeLock().lock();
+
+        try {
+            Path filePath = this.filePath;
+
+            if (!filePath.equals(newFilePath)) {
+                FileIo fileIo = this.fileIo;
+
+                if (fileIo != null) {

Review Comment:
   Shouldn't we, like, assert that `fileIo` is not null here in particular?



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/AbstractFilePageStoreIo.java:
##########
@@ -0,0 +1,611 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.pagememory.persistence.store;
+
+import static java.nio.ByteOrder.nativeOrder;
+import static java.nio.file.StandardOpenOption.CREATE;
+import static java.nio.file.StandardOpenOption.READ;
+import static java.nio.file.StandardOpenOption.WRITE;
+import static org.apache.ignite.internal.util.IgniteUtils.atomicMoveFile;
+import static org.apache.ignite.internal.util.IgniteUtils.hexInt;
+import static org.apache.ignite.internal.util.IgniteUtils.hexLong;
+import static org.apache.ignite.internal.util.IgniteUtils.toHexString;
+import static org.apache.ignite.lang.IgniteSystemProperties.getBoolean;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.ClosedByInterruptException;
+import java.nio.channels.ClosedChannelException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.apache.ignite.internal.fileio.FileIo;
+import org.apache.ignite.internal.fileio.FileIoFactory;
+import org.apache.ignite.internal.pagememory.io.PageIo;
+import org.apache.ignite.internal.pagememory.persistence.FastCrc;
+import org.apache.ignite.internal.pagememory.persistence.IgniteInternalDataIntegrityViolationException;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Abstract class for performing IO operations on file page storage.
+ */
+public abstract class AbstractFilePageStoreIo implements Closeable {
+    private final FileIoFactory ioFactory;
+
+    private final ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
+
+    /** Skip CRC calculation flag. */
+    // TODO: IGNITE-17011 Move to config
+    private final boolean skipCrc = getBoolean("IGNITE_PDS_SKIP_CRC");
+
+    private volatile Path filePath;
+
+    private volatile @Nullable FileIo fileIo;
+
+    /** Initialized file page store IO. */
+    private volatile boolean initialized;
+
+    /** Caches the existence state of file. After it is initialized, it will be not {@code null} during lifecycle. */
+    private volatile @Nullable Boolean fileExists;
+
+    /**
+     * Constructor.
+     *
+     * @param ioFactory {@link FileIo} factory.
+     * @param filePath File page store path.
+     */
+    AbstractFilePageStoreIo(FileIoFactory ioFactory, Path filePath) {
+        this.ioFactory = ioFactory;
+        this.filePath = filePath;
+    }
+
+    /**
+     * Returns the page size in bytes.
+     */
+    public abstract int pageSize();
+
+    /**
+     * Returns the size of the header in bytes.
+     */
+    public abstract int headerSize();
+
+    /**
+     * Returns a buffer with a file page storage header.
+     */
+    public abstract ByteBuffer headerBuffer();
+
+    /**
+     * Checks the file page storage header.
+     *
+     * @param fileIo File page store IO to read the header.
+     * @throws IOException If there is an error reading the header or the header did not pass the check.
+     */
+    public abstract void checkHeader(FileIo fileIo) throws IOException;
+
+    /**
+     * Returns page offset within the store file.
+     *
+     * @param pageId Page ID.
+     */
+    public abstract long pageOffset(long pageId);
+
+    /**
+     * Stops the file page store IO.
+     *
+     * @param clean {@code True} to clean file page store.
+     * @throws IgniteInternalCheckedException If failed.
+     */
+    public void stop(boolean clean) throws IgniteInternalCheckedException {
+        try {
+            stop0(clean);
+        } catch (IOException e) {
+            throw new IgniteInternalCheckedException(
+                    "Failed to stop serving file [file=" + filePath + ", delete=" + clean + "]",
+                    e
+            );
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public void close() throws IOException {
+        stop0(false);
+    }
+
+    /**
+     * Reads a page.
+     *
+     * @param pageId Page ID.
+     * @param pageOff Page offset in the file.
+     * @param pageBuf Page buffer to read into.
+     * @param keepCrc By default, reading zeroes CRC which was on page store, but you can keep it in {@code pageBuf} if set {@code
+     * keepCrc}.
+     * @return {@code True} if the page was read successfully.
+     * @throws IgniteInternalCheckedException If reading failed (IO error occurred).
+     */
+    public boolean read(long pageId, long pageOff, ByteBuffer pageBuf, boolean keepCrc) throws IgniteInternalCheckedException {
+        return read0(pageId, pageOff, pageBuf, !skipCrc, keepCrc);
+    }
+
+    /**
+     * Writes a page.
+     *
+     * @param pageId Page ID.
+     * @param pageBuf Page buffer to write from.
+     * @param calculateCrc If {@code false} crc calculation will be forcibly skipped.
+     * @throws IgniteInternalCheckedException If page writing failed (IO error occurred).
+     */
+    public void write(long pageId, ByteBuffer pageBuf, boolean calculateCrc) throws IgniteInternalCheckedException {
+        ensure();
+
+        boolean interrupted = false;
+
+        while (true) {
+            FileIo fileIo = this.fileIo;
+
+            try {
+                readWriteLock.readLock().lock();
+
+                try {
+                    assert pageBuf.position() == 0 : pageBuf.position();
+                    assert pageBuf.order() == nativeOrder() : "Page buffer order " + pageBuf.order()
+                            + " should be same with " + nativeOrder();
+                    assert PageIo.getType(pageBuf) != 0 : "Invalid state. Type is 0! pageId = " + hexLong(pageId);
+                    assert PageIo.getVersion(pageBuf) != 0 : "Invalid state. Version is 0! pageId = " + hexLong(pageId);
+
+                    if (calculateCrc && !skipCrc) {
+                        assert PageIo.getCrc(pageBuf) == 0 : hexLong(pageId);
+
+                        PageIo.setCrc(pageBuf, calcCrc32(pageBuf, pageSize()));
+                    }
+
+                    // Check whether crc was calculated somewhere above the stack if it is forcibly skipped.
+                    assert skipCrc || PageIo.getCrc(pageBuf) != 0
+                            || calcCrc32(pageBuf, pageSize()) == 0 : "CRC hasn't been calculated, crc=0";
+
+                    assert pageBuf.position() == 0 : pageBuf.position();
+
+                    long pageOff = pageOffset(pageId);
+
+                    fileIo.writeFully(pageBuf, pageOff);
+
+                    PageIo.setCrc(pageBuf, 0);
+
+                    if (interrupted) {
+                        Thread.currentThread().interrupt();
+                    }
+
+                    return;
+                } finally {
+                    readWriteLock.readLock().unlock();
+                }
+            } catch (IOException e) {
+                if (e instanceof ClosedChannelException) {
+                    try {
+                        if (e instanceof ClosedByInterruptException) {
+                            interrupted = true;
+
+                            Thread.interrupted();
+                        }
+
+                        reinit(fileIo);
+
+                        pageBuf.position(0);
+
+                        PageIo.setCrc(pageBuf, 0);
+
+                        continue;
+                    } catch (IOException e0) {
+                        e0.addSuppressed(e);
+
+                        e = e0;
+                    }
+                }
+
+                throw new IgniteInternalCheckedException(
+                        "Failed to write page [filePath=" + filePath + ", pageId=" + pageId + "]",
+                        e
+                );
+            }
+        }
+    }
+
+    /**
+     * Sync method used to ensure that the given pages are guaranteed to be written to the file page store.
+     *
+     * @throws IgniteInternalCheckedException If sync failed (IO error occurred).
+     */
+    public void sync() throws IgniteInternalCheckedException {
+        readWriteLock.writeLock().lock();
+
+        try {
+            ensure();
+
+            FileIo fileIo = this.fileIo;
+
+            if (fileIo != null) {
+                fileIo.force();
+            }
+        } catch (IOException e) {
+            throw new IgniteInternalCheckedException("Failed to fsync file [filePath=" + filePath + ']', e);
+        } finally {
+            readWriteLock.writeLock().unlock();
+        }
+    }
+
+    /**
+     * Returns {@code true} if the file page store exists.
+     */
+    public boolean exists() {
+        if (fileExists == null) {
+            readWriteLock.readLock().lock();
+
+            try {
+                if (fileExists == null) {
+                    fileExists = Files.exists(filePath) && filePath.toFile().length() >= headerSize();
+                }
+            } finally {
+                readWriteLock.readLock().unlock();
+            }
+        }
+
+        return fileExists;
+    }
+
+    /**
+     * Initializes the file page store if it hasn't already.
+     *
+     * @throws IgniteInternalCheckedException If initialization failed (IO error occurred).
+     */
+    public void ensure() throws IgniteInternalCheckedException {
+        if (!initialized) {
+            readWriteLock.writeLock().lock();
+
+            try {
+                if (!initialized) {
+                    FileIo fileIo = null;
+
+                    IgniteInternalCheckedException err = null;
+
+                    try {
+                        boolean interrupted = false;
+
+                        while (true) {
+                            try {
+                                this.fileIo = fileIo = ioFactory.create(filePath, CREATE, READ, WRITE);
+
+                                fileExists = true;
+
+                                if (fileIo.size() < headerSize()) {
+                                    fileIo.writeFully(headerBuffer().rewind(), 0);
+                                } else {
+                                    checkHeader(fileIo);
+                                }
+
+                                if (interrupted) {
+                                    Thread.currentThread().interrupt();
+                                }
+
+                                break;
+                            } catch (ClosedByInterruptException e) {
+                                interrupted = true;
+
+                                Thread.interrupted();
+                            }
+                        }
+
+                        initialized = true;
+                    } catch (IOException e) {
+                        err = new IgniteInternalCheckedException("Failed to initialize partition file: " + filePath, e);
+
+                        throw err;
+                    } finally {
+                        if (err != null && fileIo != null) {
+                            try {
+                                fileIo.close();
+                            } catch (IOException e) {
+                                err.addSuppressed(e);
+                            }
+                        }
+                    }
+                }
+            } finally {
+                readWriteLock.writeLock().unlock();
+            }
+        }
+    }
+
+    /**
+     * Returns size of the file page store in bytes.
+     *
+     * @throws IgniteInternalCheckedException If an I/O error occurs.
+     */
+    public long size() throws IgniteInternalCheckedException {
+        readWriteLock.readLock().lock();
+
+        try {
+            FileIo io = fileIo;
+
+            return io == null ? 0 : io.size();
+        } catch (IOException e) {
+            throw new IgniteInternalCheckedException(e);
+        } finally {
+            readWriteLock.readLock().unlock();
+        }
+    }
+
+    private void stop0(boolean clean) throws IOException {
+        readWriteLock.writeLock().lock();
+
+        try {
+            if (!initialized) {
+                // Ensure the file is closed even if not initialized yet.
+                if (fileIo != null) {
+                    fileIo.close();
+                }
+
+                if (clean && exists()) {
+                    Files.delete(filePath);
+                }
+
+                return;
+            }
+
+            fileIo.force();
+
+            fileIo.close();
+
+            fileIo = null;
+
+            if (clean) {
+                Files.delete(filePath);
+
+                fileExists = false;
+            }
+        } finally {
+            initialized = false;
+
+            readWriteLock.writeLock().unlock();
+        }
+    }
+
+    private static int calcCrc32(ByteBuffer pageBuf, int pageSize) {
+        try {
+            pageBuf.position(0);
+
+            return FastCrc.calcCrc(pageBuf, pageSize);
+        } finally {
+            pageBuf.position(0);
+        }
+    }
+
+    /**
+     * Reinit page store after file channel was closed by thread interruption.
+     *
+     * @param fileIo Old fileIo.
+     */
+    private void reinit(FileIo fileIo) throws IOException {
+        if (fileIo != this.fileIo) {
+            return;
+        }
+
+        readWriteLock.writeLock().lock();
+
+        try {
+            if (fileIo != this.fileIo) {
+                return;
+            }
+
+            try {
+                boolean interrupted = false;
+
+                while (true) {
+                    try {
+                        fileIo = null;
+
+                        fileIo = ioFactory.create(filePath, CREATE, READ, WRITE);
+
+                        fileExists = true;
+
+                        checkHeader(fileIo);
+
+                        this.fileIo = fileIo;
+
+                        if (interrupted) {
+                            Thread.currentThread().interrupt();
+                        }
+
+                        break;
+                    } catch (ClosedByInterruptException e) {
+                        interrupted = true;
+
+                        Thread.interrupted();
+                    }
+                }
+            } catch (IOException e) {
+                try {
+                    if (fileIo != null) {
+                        fileIo.close();
+                    }
+                } catch (IOException e0) {
+                    e.addSuppressed(e0);
+                }
+
+                throw e;
+            }
+        } finally {
+            readWriteLock.writeLock().unlock();
+        }
+    }
+
+    /**
+     * Reads a page from the page store.
+     *
+     * @param pageId Page ID.
+     * @param pageOff Page offset in the file.
+     * @param pageBuf Page buffer to read into.
+     * @param checkCrc Check CRC on page.
+     * @param keepCrc By default reading zeroes CRC which was on file, but you can keep it in pageBuf if set keepCrc.
+     * @return {@code True} if the page was read successfully.
+     * @throws IgniteInternalCheckedException If reading failed (IO error occurred).
+     */
+    private boolean read0(
+            long pageId,
+            long pageOff,
+            ByteBuffer pageBuf,
+            boolean checkCrc,
+            boolean keepCrc
+    ) throws IgniteInternalCheckedException {
+        assert pageOff >= headerSize() : "pageOff=" + pageOff + ", headerSize=" + headerSize();
+
+        ensure();
+
+        try {
+            assert pageBuf.capacity() == pageSize() : pageBuf.capacity();
+            assert pageBuf.remaining() == pageSize() : pageBuf.remaining();
+            assert pageBuf.position() == 0 : pageBuf.position();
+            assert pageBuf.order() == nativeOrder() : pageBuf.order();
+
+            int n = readWithFailover(pageBuf, pageOff);
+
+            // If page was not written yet, nothing to read.

Review Comment:
   I don't see a TODO here. I doubt that this code is really needed. We should optimize "acquirePage" so that it doesn't try to read newly allocated page from the storage, it's stupid.



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/DeltaFilePageStoreIoHeader.java:
##########
@@ -0,0 +1,236 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.pagememory.persistence.store;
+
+import static java.nio.ByteOrder.nativeOrder;
+import static org.apache.ignite.internal.util.IgniteUtils.hexLong;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import org.apache.ignite.internal.fileio.FileIo;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * {@link DeltaFilePageStoreIo} header.
+ *
+ * <p>Total length in bytes {@link #headerSize()}.
+ *
+ * <ul>
+ *     <li>{@link #SIGNATURE signature} (8 bytes)</li>
+ *     <li>{@link #version() version} (4 bytes)</li>
+ *     <li>{@link #pageSize() pageSize} (4 bytes)</li>
+ *     <li>{@link #pageIndexes() pageIndexes}
+ *         <ul>
+ *             <li>array length (4 bytes)</li>
+ *             <li>array elements (array length * 4 bytes)</li>
+ *         </ul>
+ *     </li>
+ * </ul>
+ */
+public class DeltaFilePageStoreIoHeader {
+    /** File signature. */
+    private static final long SIGNATURE = 0xDEAFAEE072020173L;
+
+    /** Size of the common delta file page store header for all versions, in bytes. */
+    private static final int COMMON_HEADER_SIZE =
+            8/*SIGNATURE*/ + 4/*version*/ + 4/*index*/ + 4/*page size*/ + 4/*page index array length*/;
+
+    private final int version;
+
+    private final int index;
+
+    private final int pageSize;
+
+    private final int[] pageIndexes;
+
+    private final int headerSize;
+
+    /**
+     * Constructor.
+     *
+     * @param version Delta file page store version.
+     * @param index Delta file page store index.
+     * @param pageSize Page size in bytes.
+     * @param pageIndexes Page indexes.
+     */
+    public DeltaFilePageStoreIoHeader(int version, int index, int pageSize, int[] pageIndexes) {
+        assert pageSize >= COMMON_HEADER_SIZE : pageSize;
+        assert index >= 0 : index;
+
+        this.version = version;
+        this.index = index;
+        this.pageSize = pageSize;
+        this.pageIndexes = pageIndexes;
+
+        int size = COMMON_HEADER_SIZE + (4 * pageIndexes.length);
+
+        if (size % pageSize != 0) {
+            size = ((size / pageSize) + 1) * pageSize;
+        }
+
+        headerSize = size;
+    }
+
+    /**
+     * Returns the version of the delta file page store.
+     */
+    public int version() {
+        return version;
+    }
+
+    /**
+     * Returns the index of the delta file page store.
+     */
+    public int index() {
+        return index;
+    }
+
+    /**
+     * Returns the page size in bytes.
+     */
+    public int pageSize() {
+        return pageSize;
+    }
+
+    /**
+     * Returns the size (aligned to {@link #pageSize()}) of the header in bytes.
+     */
+    public int headerSize() {
+        return headerSize;
+    }
+
+    /**
+     * Returns page indexes.
+     */
+    public int[] pageIndexes() {
+        return pageIndexes;
+    }
+
+    /**
+     * Converts the delta file page store header (aligned to {@link #pageSize()}) to a {@link ByteBuffer} for writing to a file.
+     */
+    public ByteBuffer toByteBuffer() {
+        ByteBuffer buffer = ByteBuffer.allocate(headerSize).order(nativeOrder())
+                .putLong(SIGNATURE)
+                .putInt(version)
+                .putInt(index)
+                .putInt(pageSize)
+                .putInt(pageIndexes.length);
+
+        if (pageIndexes.length > 0) {
+            buffer.asIntBuffer().put(pageIndexes);
+        }
+
+        return buffer;
+    }
+
+    /**
+     * Reads the header of the delta file page store.
+     *
+     * @param fileIo Delta file page store fileIo.
+     * @param headerBuffer Buffer for reading {@link DeltaFilePageStoreIoHeader header} from {@code fileIo}.
+     * @throws IOException If there are errors when reading the delta file page store header.
+     */
+    public static @Nullable DeltaFilePageStoreIoHeader readHeader(FileIo fileIo, ByteBuffer headerBuffer) throws IOException {
+        assert headerBuffer.remaining() >= COMMON_HEADER_SIZE : headerBuffer.remaining();
+        assert headerBuffer.order() == nativeOrder() : headerBuffer.order();
+
+        if (fileIo.size() < COMMON_HEADER_SIZE) {
+            return null;
+        }
+
+        fileIo.readFully(headerBuffer, 0);
+
+        long signature = headerBuffer.rewind().getLong();
+
+        if (SIGNATURE != signature) {
+            throw new IOException(String.format(
+                    "Invalid file signature [expected=%s, actual=%s]",
+                    hexLong(SIGNATURE),
+                    hexLong(signature))
+            );
+        }
+
+        int version = headerBuffer.getInt();
+        int index = headerBuffer.getInt();
+        int pageSize = headerBuffer.getInt();
+        int arrayLen = headerBuffer.getInt();
+
+        if (arrayLen == 0) {
+            return new DeltaFilePageStoreIoHeader(version, index, pageSize, new int[0]);
+        }
+
+        int[] pageIndexes = new int[arrayLen];
+
+        int i = 0;
+        int filePosition = headerBuffer.capacity();
+
+        while (i < arrayLen) {

Review Comment:
   There must be an assertion here somewhere that headerBuffer.remaining() is divisible by 4. Otherwise the code might be incorrect I think



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/DeltaFilePageStoreIo.java:
##########
@@ -0,0 +1,113 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.pagememory.persistence.store;
+
+import static java.nio.ByteOrder.nativeOrder;
+import static java.util.Arrays.binarySearch;
+import static org.apache.ignite.internal.pagememory.persistence.store.DeltaFilePageStoreIoHeader.checkFileIndex;
+import static org.apache.ignite.internal.pagememory.persistence.store.DeltaFilePageStoreIoHeader.checkFilePageIndexes;
+import static org.apache.ignite.internal.pagememory.persistence.store.DeltaFilePageStoreIoHeader.readHeader;
+import static org.apache.ignite.internal.pagememory.persistence.store.PageStoreUtils.checkFilePageSize;
+import static org.apache.ignite.internal.pagememory.persistence.store.PageStoreUtils.checkFileVersion;
+import static org.apache.ignite.internal.pagememory.util.PageIdUtils.pageIndex;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.file.Path;
+import org.apache.ignite.internal.fileio.FileIo;
+import org.apache.ignite.internal.fileio.FileIoFactory;
+
+/**
+ * Implementation of the class for working with the delta file page storage IO.
+ */
+public class DeltaFilePageStoreIo extends AbstractFilePageStoreIo {
+    private final DeltaFilePageStoreIoHeader header;
+
+    /**
+     * Constructor.
+     *
+     * @param ioFactory {@link FileIo} factory.
+     * @param filePath File page store path.
+     * @param header Delta file page store header.
+     */
+    public DeltaFilePageStoreIo(
+            FileIoFactory ioFactory,
+            Path filePath,
+            DeltaFilePageStoreIoHeader header
+    ) {
+        super(ioFactory, filePath);
+
+        this.header = header;
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public int pageSize() {
+        return header.pageSize();
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public int headerSize() {
+        return header.headerSize();
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public ByteBuffer headerBuffer() {
+        return header.toByteBuffer();
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public void checkHeader(FileIo fileIo) throws IOException {
+        DeltaFilePageStoreIoHeader header = readHeader(fileIo, ByteBuffer.allocate(pageSize()).order(nativeOrder()));
+
+        if (header == null) {
+            throw new IOException("Missing file header");
+        }
+
+        checkFileVersion(this.header.version(), header.version());
+        checkFileIndex(this.header.index(), header.index());
+        checkFilePageSize(this.header.pageSize(), header.pageSize());
+        checkFilePageIndexes(this.header.pageIndexes(), header.pageIndexes());
+    }
+
+    /**
+     * Returns page offset within the store file, {@code -1} if not found page in delta file.

Review Comment:
   "if page not found" probably. I'm not surek that my version is grammatically correct, but current comment certainly isn't



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStore.java:
##########
@@ -17,122 +17,108 @@
 
 package org.apache.ignite.internal.pagememory.persistence.store;
 
-import static java.nio.ByteOrder.nativeOrder;
-import static java.nio.file.StandardOpenOption.CREATE;
-import static java.nio.file.StandardOpenOption.READ;
-import static java.nio.file.StandardOpenOption.WRITE;
-import static org.apache.ignite.internal.util.IgniteUtils.hexInt;
-import static org.apache.ignite.internal.util.IgniteUtils.hexLong;
-import static org.apache.ignite.internal.util.IgniteUtils.toHexString;
-import static org.apache.ignite.lang.IgniteSystemProperties.getBoolean;
+import static org.apache.ignite.internal.pagememory.util.PageIdUtils.pageIndex;
 
 import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.lang.invoke.VarHandle;
 import java.nio.ByteBuffer;
-import java.nio.channels.ClosedByInterruptException;
-import java.nio.channels.ClosedChannelException;
-import java.nio.file.Files;
 import java.nio.file.Path;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.locks.ReadWriteLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.function.Supplier;
 import org.apache.ignite.internal.fileio.FileIo;
-import org.apache.ignite.internal.fileio.FileIoFactory;
-import org.apache.ignite.internal.pagememory.io.PageIo;
-import org.apache.ignite.internal.pagememory.persistence.FastCrc;
-import org.apache.ignite.internal.pagememory.persistence.IgniteInternalDataIntegrityViolationException;
-import org.apache.ignite.internal.pagememory.util.PageIdUtils;
 import org.apache.ignite.lang.IgniteInternalCheckedException;
 import org.jetbrains.annotations.Nullable;
 
 /**
  * FilePageStore is a {@link PageStore} implementation that uses regular files to store pages.
  *
- * <p>Actual read and write operations are performed with {@link FileIo} abstract interface, list of its implementations is a good source
- * of information about functionality in Ignite Native Persistence.
+ * <p>It consists of the main file page store and delta file page stores, when reading the page at the beginning, the page is searched in
+ * the delta files and only then in the main file.
  *
  * <p>On a physical level each instance of {@code FilePageStore} corresponds to a partition file assigned to the local node.
  *
- * <p>Consists of:
- * <ul>
- *     <li>Header - {@link FilePageStoreHeader}. </li>
- *     <li>Body - data pages are multiples of {@link FilePageStoreHeader#pageSize() pageSize}.</li>
- * </ul>
+ * <p>Actual read and write operations are performed with {@link FileIo} abstract interface, list of its implementations is a good source

Review Comment:
   I think that there should be a link to another file IO class, the one that you just introduced



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStore.java:
##########
@@ -141,7 +127,7 @@ public void stop(boolean clean) throws IgniteInternalCheckedException {
     public int allocatePage() throws IgniteInternalCheckedException {
         ensure();
 
-        int pageIdx = pageCount.getAndIncrement();
+        int pageIdx = (Integer) PAGE_COUNT.getAndAdd(this, 1);

Review Comment:
   Why do you cast it to `Integer` and not `int`?



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStore.java:
##########
@@ -166,503 +152,196 @@ public int pages() {
     public void pages(int pageCount) {
         assert pageCount >= 0 : pageCount;
 
-        this.pageCount.set(pageCount);
+        this.pageCount = pageCount;
     }
 
     /**
-     * Reads a page, unlike {@link #read(long, ByteBuffer, boolean)}, checks the page offset in the file not logically (pageOffset <= {@link
-     * #pages()} * {@link #pageSize}) but physically (pageOffset <= {@link #size()}), which can affect performance when used in production
-     * code.
+     * Reads a page, unlike {@link #read(long, ByteBuffer, boolean)}, does not check the {@code pageId} so that its {@code pageIdx} is not
+     * greater than the {@link #pages() number of allocated pages}.
      *
      * @param pageId Page ID.
      * @param pageBuf Page buffer to read into.
      * @param keepCrc By default, reading zeroes CRC which was on page store, but you can keep it in {@code pageBuf} if set {@code
      * keepCrc}.
+     * @return {@code True} if the page was read successfully.
      * @throws IgniteInternalCheckedException If reading failed (IO error occurred).
      */
-    public void readByPhysicalOffset(long pageId, ByteBuffer pageBuf, boolean keepCrc) throws IgniteInternalCheckedException {
-        read0(pageId, pageBuf, !skipCrc, keepCrc, false);
+    public boolean readWithoutPageIdCheck(long pageId, ByteBuffer pageBuf, boolean keepCrc) throws IgniteInternalCheckedException {
+        for (DeltaFilePageStoreIo deltaFilePageStoreIo : deltaFilePageStoreIos) {
+            long pageOff = deltaFilePageStoreIo.pageOffset(pageId);
+
+            if (pageOff >= 0) {
+                return deltaFilePageStoreIo.read(pageId, pageOff, pageBuf, keepCrc);
+            }
+        }
+
+        return filePageStoreIo.read(pageId, filePageStoreIo.pageOffset(pageId), pageBuf, keepCrc);
     }
 
     /** {@inheritDoc} */
     @Override
     public void read(long pageId, ByteBuffer pageBuf, boolean keepCrc) throws IgniteInternalCheckedException {
-        read0(pageId, pageBuf, !skipCrc, keepCrc, true);
-    }
-
-    /**
-     * Reads a page from the page store.
-     *
-     * @param pageId Page ID.
-     * @param pageBuf Page buffer to read into.
-     * @param checkCrc Check CRC on page.
-     * @param keepCrc By default reading zeroes CRC which was on file, but you can keep it in pageBuf if set keepCrc.
-     * @param checkPageOffsetLogically Check page offset by {@link #allocatedBytes} or {@link #size}.
-     * @throws IgniteInternalCheckedException If reading failed (IO error occurred).
-     */
-    private void read0(
-            long pageId,
-            ByteBuffer pageBuf,
-            boolean checkCrc,
-            boolean keepCrc,
-            boolean checkPageOffsetLogically
-    ) throws IgniteInternalCheckedException {
-        ensure();
-
-        try {
-            assert pageBuf.capacity() == pageSize : pageBuf.capacity();
-            assert pageBuf.remaining() == pageSize : pageBuf.remaining();
-            assert pageBuf.position() == 0 : pageBuf.position();
-            assert pageBuf.order() == nativeOrder() : pageBuf.order();
-
-            long pageOff = pageOffset(pageId);
-
-            if (checkPageOffsetLogically) {
-                assert pageOff <= allocatedBytes() : "calculatedOffset=" + pageOff
-                        + ", allocated=" + allocatedBytes() + ", headerSize=" + headerSize + ", filePath=" + filePath;
-            } else {
-                assert pageOff <= size() : "calculatedOffset=" + pageOff
-                        + ", size=" + size() + ", headerSize=" + headerSize + ", filePath=" + filePath;
-            }
-
-            int n = readWithFailover(pageBuf, pageOff);
+        assert pageIndex(pageId) <= pageCount : "pageIdx=" + pageIndex(pageId) + ", pageCount=" + pageCount;
 
-            // If page was not written yet, nothing to read.
-            if (n < 0) {
-                pageBuf.put(new byte[pageBuf.remaining()]);
-            }
+        for (DeltaFilePageStoreIo deltaFilePageStoreIo : deltaFilePageStoreIos) {
+            long pageOff = deltaFilePageStoreIo.pageOffset(pageId);
 
-            int savedCrc32 = PageIo.getCrc(pageBuf);
-
-            PageIo.setCrc(pageBuf, 0);
-
-            pageBuf.position(0);
-
-            if (checkCrc) {
-                int curCrc32 = FastCrc.calcCrc(pageBuf, pageSize);
-
-                if ((savedCrc32 ^ curCrc32) != 0) {
-                    throw new IgniteInternalDataIntegrityViolationException("Failed to read page (CRC validation failed) "
-                            + "[id=" + hexLong(pageId) + ", off=" + pageOff
-                            + ", filePath=" + filePath + ", fileSize=" + fileIo.size()
-                            + ", savedCrc=" + hexInt(savedCrc32) + ", curCrc=" + hexInt(curCrc32)
-                            + ", page=" + toHexString(pageBuf) + "]");
-                }
-            }
+            if (pageOff >= 0) {
+                deltaFilePageStoreIo.read(pageId, pageOff, pageBuf, keepCrc);
 
-            assert PageIo.getCrc(pageBuf) == 0;
-
-            if (keepCrc) {
-                PageIo.setCrc(pageBuf, savedCrc32);
+                return;
             }
-        } catch (IOException e) {
-            throw new IgniteInternalCheckedException("Failed to read page [file=" + filePath + ", pageId=" + pageId + "]", e);
         }
+
+        filePageStoreIo.read(pageId, filePageStoreIo.pageOffset(pageId), pageBuf, keepCrc);
     }
 
     /** {@inheritDoc} */
     @Override
     public void write(long pageId, ByteBuffer pageBuf, boolean calculateCrc) throws IgniteInternalCheckedException {
-        ensure();
-
-        boolean interrupted = false;
-
-        while (true) {
-            FileIo fileIo = this.fileIo;
-
-            try {
-                readWriteLock.readLock().lock();
-
-                try {
-                    assert pageBuf.position() == 0 : pageBuf.position();
-                    assert pageBuf.order() == nativeOrder() : "Page buffer order " + pageBuf.order()
-                            + " should be same with " + nativeOrder();
-                    assert PageIo.getType(pageBuf) != 0 : "Invalid state. Type is 0! pageId = " + hexLong(pageId);
-                    assert PageIo.getVersion(pageBuf) != 0 : "Invalid state. Version is 0! pageId = " + hexLong(pageId);
-
-                    if (calculateCrc && !skipCrc) {
-                        assert PageIo.getCrc(pageBuf) == 0 : hexLong(pageId);
-
-                        PageIo.setCrc(pageBuf, calcCrc32(pageBuf, pageSize));
-                    }
-
-                    // Check whether crc was calculated somewhere above the stack if it is forcibly skipped.
-                    assert skipCrc || PageIo.getCrc(pageBuf) != 0
-                            || calcCrc32(pageBuf, pageSize) == 0 : "CRC hasn't been calculated, crc=0";
-
-                    assert pageBuf.position() == 0 : pageBuf.position();
-
-                    long pageOff = pageOffset(pageId);
-
-                    assert pageOff <= allocatedBytes() : "calculatedOffset=" + pageOff
-                            + ", allocated=" + allocatedBytes() + ", headerSize=" + headerSize + ", filePath=" + filePath;
-
-                    fileIo.writeFully(pageBuf, pageOff);
-
-                    PageIo.setCrc(pageBuf, 0);
-
-                    if (interrupted) {
-                        Thread.currentThread().interrupt();
-                    }
-
-                    return;
-                } finally {
-                    readWriteLock.readLock().unlock();
-                }
-            } catch (IOException e) {
-                if (e instanceof ClosedChannelException) {
-                    try {
-                        if (e instanceof ClosedByInterruptException) {
-                            interrupted = true;
-
-                            Thread.interrupted();
-                        }
-
-                        reinit(fileIo);
-
-                        pageBuf.position(0);
-
-                        PageIo.setCrc(pageBuf, 0);
+        assert pageIndex(pageId) <= pageCount : "pageIdx=" + pageIndex(pageId) + ", pageCount=" + pageCount;
 
-                        continue;
-                    } catch (IOException e0) {
-                        e0.addSuppressed(e);
-
-                        e = e0;
-                    }
-                }
-
-                throw new IgniteInternalCheckedException(
-                        "Failed to write page [filePath=" + filePath + ", pageId=" + pageId + "]",
-                        e
-                );
-            }
-        }
+        filePageStoreIo.write(pageId, pageBuf, calculateCrc);
     }
 
     /** {@inheritDoc} */
     @Override
     public void sync() throws IgniteInternalCheckedException {
-        readWriteLock.writeLock().lock();
-
-        try {
-            ensure();
-
-            FileIo fileIo = this.fileIo;
-
-            if (fileIo != null) {
-                fileIo.force();
-            }
-        } catch (IOException e) {
-            throw new IgniteInternalCheckedException("Failed to fsync partition file [filePath=" + filePath + ']', e);
-        } finally {
-            readWriteLock.writeLock().unlock();
-        }
+        filePageStoreIo.sync();
     }
 
     /** {@inheritDoc} */
     @Override
     public boolean exists() {
-        if (fileExists == null) {
-            readWriteLock.writeLock().lock();
-
-            try {
-                if (fileExists == null) {
-                    fileExists = Files.exists(filePath) && filePath.toFile().length() >= headerSize;
-                }
-            } finally {
-                readWriteLock.writeLock().unlock();
-            }
-        }
-
-        return fileExists;
+        return filePageStoreIo.exists();
     }
 
     /** {@inheritDoc} */
     @Override
     public void ensure() throws IgniteInternalCheckedException {
-        if (!initialized) {
-            readWriteLock.writeLock().lock();
-
-            try {
-                if (!initialized) {
-                    FileIo fileIo = null;
-
-                    IgniteInternalCheckedException err = null;
-
-                    try {
-                        boolean interrupted = false;
-
-                        while (true) {
-                            try {
-                                this.fileIo = fileIo = ioFactory.create(filePath, CREATE, READ, WRITE);
-
-                                fileExists = true;
-
-                                if (fileIo.size() < headerSize) {
-                                    fileIo.writeFully(new FilePageStoreHeader(version, pageSize).toByteBuffer(), 0);
-                                } else {
-                                    checkHeader(fileIo);
-                                }
-
-                                if (interrupted) {
-                                    Thread.currentThread().interrupt();
-                                }
-
-                                break;
-                            } catch (ClosedByInterruptException e) {
-                                interrupted = true;
-
-                                Thread.interrupted();
-                            }
-                        }
-
-                        initialized = true;
-                    } catch (IOException e) {
-                        err = new IgniteInternalCheckedException("Failed to initialize partition file: " + filePath, e);
-
-                        throw err;
-                    } finally {
-                        if (err != null && fileIo != null) {
-                            try {
-                                fileIo.close();
-                            } catch (IOException e) {
-                                err.addSuppressed(e);
-                            }
-                        }
-                    }
-                }
-            } finally {
-                readWriteLock.writeLock().unlock();
-            }
-        }
+        filePageStoreIo.ensure();
     }
 
     /** {@inheritDoc} */
     @Override
     public void close() throws IOException {
-        stop0(false);
+        filePageStoreIo.close();
+
+        for (DeltaFilePageStoreIo deltaFilePageStoreIo : deltaFilePageStoreIos) {
+            deltaFilePageStoreIo.close();
+        }
     }
 
     /**
      * Returns size of the page store in bytes.
      *
-     * <p>May differ from {@link #pages} * {@link #pageSize} due to delayed writes or due to other implementation specific details.
+     * <p>May differ from {@link #pages} * {@link FilePageStoreIo#pageSize()} due to delayed writes or due to other implementation specific
+     * details.
      *
      * @throws IgniteInternalCheckedException If an I/O error occurs.
      */
     public long size() throws IgniteInternalCheckedException {
-        try {
-            FileIo io = fileIo;
-
-            return io == null ? 0 : io.size();
-        } catch (IOException e) {
-            throw new IgniteInternalCheckedException(e);
-        }
+        return filePageStoreIo.size();
     }
 
     /**
-     * Stops file page store.
-     *
-     * @param delete {@code True} to delete file.
-     * @throws IOException If fails.
+     * Returns file page store path.
      */
-    private void stop0(boolean delete) throws IOException {
-        readWriteLock.writeLock().lock();
-
-        try {
-            if (!initialized) {
-                // Ensure the file is closed even if not initialized yet.
-                if (fileIo != null) {
-                    fileIo.close();
-                }
-
-                if (delete && exists()) {
-                    Files.delete(filePath);
-                }
-
-                return;
-            }
-
-            fileIo.force();
-
-            fileIo.close();
-
-            fileIo = null;
-
-            if (delete) {
-                Files.delete(filePath);
-
-                fileExists = false;
-            }
-        } finally {
-            initialized = false;
+    public Path filePath() {
+        return filePageStoreIo.filePath();
+    }
 
-            readWriteLock.writeLock().unlock();
-        }
+    /**
+     * Returns file page store header size.
+     */
+    public int headerSize() {
+        return filePageStoreIo.headerSize();
     }
 
     /**
-     * Gets page offset within the store file.
+     * Sets the new page allocation listener.
      *
-     * @param pageId Page ID.
-     * @return Page offset.
+     * @param listener New page allocation listener.
      */
-    long pageOffset(long pageId) {
-        return (long) PageIdUtils.pageIndex(pageId) * pageSize + headerSize;
+    public void setPageAllocationListener(PageAllocationListener listener) {
+        pageAllocationListener = listener;
     }
 
     /**
-     * Reads from page storage with failover.
+     * Sets the delta file page store factory.
      *
-     * @param destBuf Destination buffer.
-     * @param position Position.
-     * @return Number of read bytes, or {@code -1} if the given position is greater than or equal to the file's current size.
+     * @param factory Factory.
      */
-    private int readWithFailover(ByteBuffer destBuf, long position) throws IOException {
-        boolean interrupted = false;
-
-        int bufPos = destBuf.position();
-
-        while (true) {
-            FileIo fileIo = this.fileIo;
-
-            if (fileIo == null) {
-                throw new IOException("FileIo has stopped");
-            }
-
-            try {
-                assert destBuf.remaining() > 0;
-
-                int bytesRead = fileIo.readFully(destBuf, position);
-
-                if (interrupted) {
-                    Thread.currentThread().interrupt();
-                }
-
-                return bytesRead;
-            } catch (ClosedChannelException e) {
-                destBuf.position(bufPos);
-
-                if (e instanceof ClosedByInterruptException) {
-                    interrupted = true;
-
-                    Thread.interrupted();
-                }
-
-                reinit(fileIo);
-            }
-        }
+    public void setDeltaFilePageStoreIoFactory(DeltaFilePageStoreIoFactory factory) {
+        deltaFilePageStoreIoFactory = factory;
     }
 
     /**
-     * Reinit page store after file channel was closed by thread interruption.
+     * Sets the callback on completion of delta file page store creation.
      *
-     * @param fileIo Old fileIo.
+     * @param callback Callback.
      */
-    private void reinit(FileIo fileIo) throws IOException {
-        if (!initialized) {
-            return;
-        }
-
-        if (fileIo != this.fileIo) {
-            return;
-        }
-
-        readWriteLock.writeLock().lock();
-
-        try {
-            if (fileIo != this.fileIo) {
-                return;
-            }
-
-            try {
-                boolean interrupted = false;
-
-                while (true) {
-                    try {
-                        fileIo = null;
-
-                        fileIo = ioFactory.create(filePath, CREATE, READ, WRITE);
+    public void setCompleteCreationDeltaFilePageStoreIoCallback(CompleteCreationDeltaFilePageStoreIoCallback callback) {
+        completeCreationDeltaFilePageStoreIoCallback = callback;
+    }
 
-                        fileExists = true;
+    /**
+     * Gets or creates a new delta file, a new delta file will be created when the previous one is {@link #completeNewDeltaFile()
+     * completed}.
+     *
+     * <p>Thread safe.
+     *
+     * @param pageIndexesSupplier Page indexes supplier for the new delta file page store.
+     * @return Future that will be completed when the new delta file page store is created.
+     */
+    public CompletableFuture<DeltaFilePageStoreIo> getOrCreateNewDeltaFile(Supplier<int[]> pageIndexesSupplier) {

Review Comment:
   Supplier? Why?



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStoreManager.java:
##########
@@ -49,12 +52,24 @@ public class FilePageStoreManager implements PageReadWriteManager {
     /** File suffix. */
     public static final String FILE_SUFFIX = ".bin";
 
+    /** Suffix of the temporary file. */
+    public static final String TMP_FILE_SUFFIX = ".tmp";
+
     /** Partition file prefix. */
     public static final String PART_FILE_PREFIX = "part-";
 
-    /** Partition file template. */
+    /** Partition delta file prefix. */
+    public static final String PART_DELTA_FILE_PREFIX = "part-%d-delta-";

Review Comment:
   You could write `PART_FILE_PREFIX + ...`, why duplicating these strings?



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStore.java:
##########
@@ -166,503 +152,196 @@ public int pages() {
     public void pages(int pageCount) {
         assert pageCount >= 0 : pageCount;
 
-        this.pageCount.set(pageCount);
+        this.pageCount = pageCount;
     }
 
     /**
-     * Reads a page, unlike {@link #read(long, ByteBuffer, boolean)}, checks the page offset in the file not logically (pageOffset <= {@link
-     * #pages()} * {@link #pageSize}) but physically (pageOffset <= {@link #size()}), which can affect performance when used in production
-     * code.
+     * Reads a page, unlike {@link #read(long, ByteBuffer, boolean)}, does not check the {@code pageId} so that its {@code pageIdx} is not
+     * greater than the {@link #pages() number of allocated pages}.
      *
      * @param pageId Page ID.
      * @param pageBuf Page buffer to read into.
      * @param keepCrc By default, reading zeroes CRC which was on page store, but you can keep it in {@code pageBuf} if set {@code
      * keepCrc}.
+     * @return {@code True} if the page was read successfully.
      * @throws IgniteInternalCheckedException If reading failed (IO error occurred).
      */
-    public void readByPhysicalOffset(long pageId, ByteBuffer pageBuf, boolean keepCrc) throws IgniteInternalCheckedException {
-        read0(pageId, pageBuf, !skipCrc, keepCrc, false);
+    public boolean readWithoutPageIdCheck(long pageId, ByteBuffer pageBuf, boolean keepCrc) throws IgniteInternalCheckedException {
+        for (DeltaFilePageStoreIo deltaFilePageStoreIo : deltaFilePageStoreIos) {
+            long pageOff = deltaFilePageStoreIo.pageOffset(pageId);
+
+            if (pageOff >= 0) {
+                return deltaFilePageStoreIo.read(pageId, pageOff, pageBuf, keepCrc);
+            }
+        }
+
+        return filePageStoreIo.read(pageId, filePageStoreIo.pageOffset(pageId), pageBuf, keepCrc);
     }
 
     /** {@inheritDoc} */
     @Override
     public void read(long pageId, ByteBuffer pageBuf, boolean keepCrc) throws IgniteInternalCheckedException {
-        read0(pageId, pageBuf, !skipCrc, keepCrc, true);
-    }
-
-    /**
-     * Reads a page from the page store.
-     *
-     * @param pageId Page ID.
-     * @param pageBuf Page buffer to read into.
-     * @param checkCrc Check CRC on page.
-     * @param keepCrc By default reading zeroes CRC which was on file, but you can keep it in pageBuf if set keepCrc.
-     * @param checkPageOffsetLogically Check page offset by {@link #allocatedBytes} or {@link #size}.
-     * @throws IgniteInternalCheckedException If reading failed (IO error occurred).
-     */
-    private void read0(
-            long pageId,
-            ByteBuffer pageBuf,
-            boolean checkCrc,
-            boolean keepCrc,
-            boolean checkPageOffsetLogically
-    ) throws IgniteInternalCheckedException {
-        ensure();
-
-        try {
-            assert pageBuf.capacity() == pageSize : pageBuf.capacity();
-            assert pageBuf.remaining() == pageSize : pageBuf.remaining();
-            assert pageBuf.position() == 0 : pageBuf.position();
-            assert pageBuf.order() == nativeOrder() : pageBuf.order();
-
-            long pageOff = pageOffset(pageId);
-
-            if (checkPageOffsetLogically) {
-                assert pageOff <= allocatedBytes() : "calculatedOffset=" + pageOff
-                        + ", allocated=" + allocatedBytes() + ", headerSize=" + headerSize + ", filePath=" + filePath;
-            } else {
-                assert pageOff <= size() : "calculatedOffset=" + pageOff
-                        + ", size=" + size() + ", headerSize=" + headerSize + ", filePath=" + filePath;
-            }
-
-            int n = readWithFailover(pageBuf, pageOff);
+        assert pageIndex(pageId) <= pageCount : "pageIdx=" + pageIndex(pageId) + ", pageCount=" + pageCount;
 
-            // If page was not written yet, nothing to read.
-            if (n < 0) {
-                pageBuf.put(new byte[pageBuf.remaining()]);
-            }
+        for (DeltaFilePageStoreIo deltaFilePageStoreIo : deltaFilePageStoreIos) {
+            long pageOff = deltaFilePageStoreIo.pageOffset(pageId);
 
-            int savedCrc32 = PageIo.getCrc(pageBuf);
-
-            PageIo.setCrc(pageBuf, 0);
-
-            pageBuf.position(0);
-
-            if (checkCrc) {
-                int curCrc32 = FastCrc.calcCrc(pageBuf, pageSize);
-
-                if ((savedCrc32 ^ curCrc32) != 0) {
-                    throw new IgniteInternalDataIntegrityViolationException("Failed to read page (CRC validation failed) "
-                            + "[id=" + hexLong(pageId) + ", off=" + pageOff
-                            + ", filePath=" + filePath + ", fileSize=" + fileIo.size()
-                            + ", savedCrc=" + hexInt(savedCrc32) + ", curCrc=" + hexInt(curCrc32)
-                            + ", page=" + toHexString(pageBuf) + "]");
-                }
-            }
+            if (pageOff >= 0) {
+                deltaFilePageStoreIo.read(pageId, pageOff, pageBuf, keepCrc);
 
-            assert PageIo.getCrc(pageBuf) == 0;
-
-            if (keepCrc) {
-                PageIo.setCrc(pageBuf, savedCrc32);
+                return;
             }
-        } catch (IOException e) {
-            throw new IgniteInternalCheckedException("Failed to read page [file=" + filePath + ", pageId=" + pageId + "]", e);
         }
+
+        filePageStoreIo.read(pageId, filePageStoreIo.pageOffset(pageId), pageBuf, keepCrc);
     }
 
     /** {@inheritDoc} */
     @Override
     public void write(long pageId, ByteBuffer pageBuf, boolean calculateCrc) throws IgniteInternalCheckedException {
-        ensure();
-
-        boolean interrupted = false;
-
-        while (true) {
-            FileIo fileIo = this.fileIo;
-
-            try {
-                readWriteLock.readLock().lock();
-
-                try {
-                    assert pageBuf.position() == 0 : pageBuf.position();
-                    assert pageBuf.order() == nativeOrder() : "Page buffer order " + pageBuf.order()
-                            + " should be same with " + nativeOrder();
-                    assert PageIo.getType(pageBuf) != 0 : "Invalid state. Type is 0! pageId = " + hexLong(pageId);
-                    assert PageIo.getVersion(pageBuf) != 0 : "Invalid state. Version is 0! pageId = " + hexLong(pageId);
-
-                    if (calculateCrc && !skipCrc) {
-                        assert PageIo.getCrc(pageBuf) == 0 : hexLong(pageId);
-
-                        PageIo.setCrc(pageBuf, calcCrc32(pageBuf, pageSize));
-                    }
-
-                    // Check whether crc was calculated somewhere above the stack if it is forcibly skipped.
-                    assert skipCrc || PageIo.getCrc(pageBuf) != 0
-                            || calcCrc32(pageBuf, pageSize) == 0 : "CRC hasn't been calculated, crc=0";
-
-                    assert pageBuf.position() == 0 : pageBuf.position();
-
-                    long pageOff = pageOffset(pageId);
-
-                    assert pageOff <= allocatedBytes() : "calculatedOffset=" + pageOff
-                            + ", allocated=" + allocatedBytes() + ", headerSize=" + headerSize + ", filePath=" + filePath;
-
-                    fileIo.writeFully(pageBuf, pageOff);
-
-                    PageIo.setCrc(pageBuf, 0);
-
-                    if (interrupted) {
-                        Thread.currentThread().interrupt();
-                    }
-
-                    return;
-                } finally {
-                    readWriteLock.readLock().unlock();
-                }
-            } catch (IOException e) {
-                if (e instanceof ClosedChannelException) {
-                    try {
-                        if (e instanceof ClosedByInterruptException) {
-                            interrupted = true;
-
-                            Thread.interrupted();
-                        }
-
-                        reinit(fileIo);
-
-                        pageBuf.position(0);
-
-                        PageIo.setCrc(pageBuf, 0);
+        assert pageIndex(pageId) <= pageCount : "pageIdx=" + pageIndex(pageId) + ", pageCount=" + pageCount;
 
-                        continue;
-                    } catch (IOException e0) {
-                        e0.addSuppressed(e);
-
-                        e = e0;
-                    }
-                }
-
-                throw new IgniteInternalCheckedException(
-                        "Failed to write page [filePath=" + filePath + ", pageId=" + pageId + "]",
-                        e
-                );
-            }
-        }
+        filePageStoreIo.write(pageId, pageBuf, calculateCrc);
     }
 
     /** {@inheritDoc} */
     @Override
     public void sync() throws IgniteInternalCheckedException {
-        readWriteLock.writeLock().lock();
-
-        try {
-            ensure();
-
-            FileIo fileIo = this.fileIo;
-
-            if (fileIo != null) {
-                fileIo.force();
-            }
-        } catch (IOException e) {
-            throw new IgniteInternalCheckedException("Failed to fsync partition file [filePath=" + filePath + ']', e);
-        } finally {
-            readWriteLock.writeLock().unlock();
-        }
+        filePageStoreIo.sync();
     }
 
     /** {@inheritDoc} */
     @Override
     public boolean exists() {
-        if (fileExists == null) {
-            readWriteLock.writeLock().lock();
-
-            try {
-                if (fileExists == null) {
-                    fileExists = Files.exists(filePath) && filePath.toFile().length() >= headerSize;
-                }
-            } finally {
-                readWriteLock.writeLock().unlock();
-            }
-        }
-
-        return fileExists;
+        return filePageStoreIo.exists();
     }
 
     /** {@inheritDoc} */
     @Override
     public void ensure() throws IgniteInternalCheckedException {
-        if (!initialized) {
-            readWriteLock.writeLock().lock();
-
-            try {
-                if (!initialized) {
-                    FileIo fileIo = null;
-
-                    IgniteInternalCheckedException err = null;
-
-                    try {
-                        boolean interrupted = false;
-
-                        while (true) {
-                            try {
-                                this.fileIo = fileIo = ioFactory.create(filePath, CREATE, READ, WRITE);
-
-                                fileExists = true;
-
-                                if (fileIo.size() < headerSize) {
-                                    fileIo.writeFully(new FilePageStoreHeader(version, pageSize).toByteBuffer(), 0);
-                                } else {
-                                    checkHeader(fileIo);
-                                }
-
-                                if (interrupted) {
-                                    Thread.currentThread().interrupt();
-                                }
-
-                                break;
-                            } catch (ClosedByInterruptException e) {
-                                interrupted = true;
-
-                                Thread.interrupted();
-                            }
-                        }
-
-                        initialized = true;
-                    } catch (IOException e) {
-                        err = new IgniteInternalCheckedException("Failed to initialize partition file: " + filePath, e);
-
-                        throw err;
-                    } finally {
-                        if (err != null && fileIo != null) {
-                            try {
-                                fileIo.close();
-                            } catch (IOException e) {
-                                err.addSuppressed(e);
-                            }
-                        }
-                    }
-                }
-            } finally {
-                readWriteLock.writeLock().unlock();
-            }
-        }
+        filePageStoreIo.ensure();
     }
 
     /** {@inheritDoc} */
     @Override
     public void close() throws IOException {
-        stop0(false);
+        filePageStoreIo.close();
+
+        for (DeltaFilePageStoreIo deltaFilePageStoreIo : deltaFilePageStoreIos) {
+            deltaFilePageStoreIo.close();
+        }
     }
 
     /**
      * Returns size of the page store in bytes.
      *
-     * <p>May differ from {@link #pages} * {@link #pageSize} due to delayed writes or due to other implementation specific details.
+     * <p>May differ from {@link #pages} * {@link FilePageStoreIo#pageSize()} due to delayed writes or due to other implementation specific
+     * details.
      *
      * @throws IgniteInternalCheckedException If an I/O error occurs.
      */
     public long size() throws IgniteInternalCheckedException {
-        try {
-            FileIo io = fileIo;
-
-            return io == null ? 0 : io.size();
-        } catch (IOException e) {
-            throw new IgniteInternalCheckedException(e);
-        }
+        return filePageStoreIo.size();
     }
 
     /**
-     * Stops file page store.
-     *
-     * @param delete {@code True} to delete file.
-     * @throws IOException If fails.
+     * Returns file page store path.
      */
-    private void stop0(boolean delete) throws IOException {
-        readWriteLock.writeLock().lock();
-
-        try {
-            if (!initialized) {
-                // Ensure the file is closed even if not initialized yet.
-                if (fileIo != null) {
-                    fileIo.close();
-                }
-
-                if (delete && exists()) {
-                    Files.delete(filePath);
-                }
-
-                return;
-            }
-
-            fileIo.force();
-
-            fileIo.close();
-
-            fileIo = null;
-
-            if (delete) {
-                Files.delete(filePath);
-
-                fileExists = false;
-            }
-        } finally {
-            initialized = false;
+    public Path filePath() {
+        return filePageStoreIo.filePath();
+    }
 
-            readWriteLock.writeLock().unlock();
-        }
+    /**
+     * Returns file page store header size.
+     */
+    public int headerSize() {
+        return filePageStoreIo.headerSize();
     }
 
     /**
-     * Gets page offset within the store file.
+     * Sets the new page allocation listener.
      *
-     * @param pageId Page ID.
-     * @return Page offset.
+     * @param listener New page allocation listener.
      */
-    long pageOffset(long pageId) {
-        return (long) PageIdUtils.pageIndex(pageId) * pageSize + headerSize;
+    public void setPageAllocationListener(PageAllocationListener listener) {
+        pageAllocationListener = listener;
     }
 
     /**
-     * Reads from page storage with failover.
+     * Sets the delta file page store factory.
      *
-     * @param destBuf Destination buffer.
-     * @param position Position.
-     * @return Number of read bytes, or {@code -1} if the given position is greater than or equal to the file's current size.
+     * @param factory Factory.
      */
-    private int readWithFailover(ByteBuffer destBuf, long position) throws IOException {
-        boolean interrupted = false;
-
-        int bufPos = destBuf.position();
-
-        while (true) {
-            FileIo fileIo = this.fileIo;
-
-            if (fileIo == null) {
-                throw new IOException("FileIo has stopped");
-            }
-
-            try {
-                assert destBuf.remaining() > 0;
-
-                int bytesRead = fileIo.readFully(destBuf, position);
-
-                if (interrupted) {
-                    Thread.currentThread().interrupt();
-                }
-
-                return bytesRead;
-            } catch (ClosedChannelException e) {
-                destBuf.position(bufPos);
-
-                if (e instanceof ClosedByInterruptException) {
-                    interrupted = true;
-
-                    Thread.interrupted();
-                }
-
-                reinit(fileIo);
-            }
-        }
+    public void setDeltaFilePageStoreIoFactory(DeltaFilePageStoreIoFactory factory) {
+        deltaFilePageStoreIoFactory = factory;
     }
 
     /**
-     * Reinit page store after file channel was closed by thread interruption.
+     * Sets the callback on completion of delta file page store creation.
      *
-     * @param fileIo Old fileIo.
+     * @param callback Callback.
      */
-    private void reinit(FileIo fileIo) throws IOException {
-        if (!initialized) {
-            return;
-        }
-
-        if (fileIo != this.fileIo) {
-            return;
-        }
-
-        readWriteLock.writeLock().lock();
-
-        try {
-            if (fileIo != this.fileIo) {
-                return;
-            }
-
-            try {
-                boolean interrupted = false;
-
-                while (true) {
-                    try {
-                        fileIo = null;
-
-                        fileIo = ioFactory.create(filePath, CREATE, READ, WRITE);
+    public void setCompleteCreationDeltaFilePageStoreIoCallback(CompleteCreationDeltaFilePageStoreIoCallback callback) {
+        completeCreationDeltaFilePageStoreIoCallback = callback;
+    }
 
-                        fileExists = true;
+    /**
+     * Gets or creates a new delta file, a new delta file will be created when the previous one is {@link #completeNewDeltaFile()
+     * completed}.
+     *
+     * <p>Thread safe.
+     *
+     * @param pageIndexesSupplier Page indexes supplier for the new delta file page store.
+     * @return Future that will be completed when the new delta file page store is created.
+     */
+    public CompletableFuture<DeltaFilePageStoreIo> getOrCreateNewDeltaFile(Supplier<int[]> pageIndexesSupplier) {
+        assert deltaFilePageStoreIoFactory != null;
 
-                        checkHeader(fileIo);
+        CompletableFuture<DeltaFilePageStoreIo> future = this.newDeltaFilePageStoreIoFuture;
 
-                        this.fileIo = fileIo;
+        if (future != null) {
+            return future;
+        }
 
-                        if (interrupted) {
-                            Thread.currentThread().interrupt();
-                        }
+        if (!NEW_DELTA_FILE_PAGE_STORE_IO_FUTURE.compareAndSet(this, null, future = new CompletableFuture<>())) {
+            // Another thread started creating a delta file.
+            return newDeltaFilePageStoreIoFuture;
+        }
 
-                        break;
-                    } catch (ClosedByInterruptException e) {
-                        interrupted = true;
+        int nextIndex = deltaFilePageStoreIos.isEmpty() ? 0 : deltaFilePageStoreIos.get(0).fileIndex() + 1;
 
-                        Thread.interrupted();
-                    }
-                }
-            } catch (IOException e) {
-                try {
-                    if (fileIo != null) {
-                        fileIo.close();
-                    }
-                } catch (IOException e0) {
-                    e.addSuppressed(e0);
-                }
+        DeltaFilePageStoreIo deltaFilePageStoreIo = deltaFilePageStoreIoFactory.create(nextIndex, pageIndexesSupplier.get());
 
-                throw e;
-            }
-        } finally {
-            readWriteLock.writeLock().unlock();
-        }
-    }
+        // Should add to the head, since read operations should always start from the most recent.
+        deltaFilePageStoreIos.add(0, deltaFilePageStoreIo);
 
-    private static int calcCrc32(ByteBuffer pageBuf, int pageSize) {
-        try {
-            pageBuf.position(0);
+        future.complete(deltaFilePageStoreIo);
 
-            return FastCrc.calcCrc(pageBuf, pageSize);
-        } finally {
-            pageBuf.position(0);
-        }
+        return future;
     }
 
     /**
-     * Returns file page store path.
+     * Completes the {@link #getOrCreateNewDeltaFile(Supplier) creation} of a new delta file.
+     *
+     * <p>Thread safe.
+     *
+     * @throws IgniteInternalCheckedException If failed.
      */
-    public Path filePath() {
-        return filePath;
-    }
+    public void completeNewDeltaFile() throws IgniteInternalCheckedException {

Review Comment:
   Future is already completed in `getOrCreateNewDeltaFile`, isn't it? I don't understand what's happening here. We clearly need more comments or a more clear code structure



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStoreManager.java:
##########
@@ -305,14 +340,99 @@ private List<FilePageStore> createFilePageStores(
 
         try {
             for (int i = 0; i < partitions; i++) {
-                Path partFilePath = groupWorkDir.resolve(String.format(PART_FILE_TEMPLATE, i));
+                int part = i;

Review Comment:
   Why not just rename `i`?



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStoreManager.java:
##########
@@ -305,14 +340,99 @@ private List<FilePageStore> createFilePageStores(
 
         try {
             for (int i = 0; i < partitions; i++) {
-                Path partFilePath = groupWorkDir.resolve(String.format(PART_FILE_TEMPLATE, i));
+                int part = i;
+
+                Path partFilePath = groupWorkDir.resolve(String.format(PART_FILE_TEMPLATE, part));
+
+                Path[] partDeltaFiles = findPartitionDeltaFiles(groupWorkDir, partitions);
+
+                FilePageStore filePageStore = filePageStoreFactory.createPageStore(buffer.rewind(), partFilePath, partDeltaFiles);
+
+                filePageStore.setDeltaFilePageStoreIoFactory((index, pageIndexes) -> createLatest(groupWorkDir, part, index, pageIndexes));
+
+                filePageStore.setCompleteCreationDeltaFilePageStoreIoCallback(deltaIo -> renameDeltaFile(groupWorkDir, part, deltaIo));
 
-                partitionFilePageStores.add(filePageStoreFactory.createPageStore(partFilePath, buffer.rewind()));
+                partitionFilePageStores.add(filePageStore);
             }
 
             return unmodifiableList(partitionFilePageStores);
         } finally {
             freeBuffer(buffer);
         }
     }
+
+    /**
+     * Returns paths (unsorted) to delta files for the requested partition.
+     *
+     * @param groupWorkDir Group directory.
+     * @param partition Partition number.
+     */
+    Path[] findPartitionDeltaFiles(Path groupWorkDir, int partition) throws IgniteInternalCheckedException {
+        assert partition >= 0 : partition;
+
+        try (Stream<Path> deltaFileStream = Files.find(
+                groupWorkDir,
+                1,
+                (path, basicFileAttributes) -> path.getFileName().toString().startsWith(String.format(PART_DELTA_FILE_PREFIX, partition)))

Review Comment:
   So, you're calling `String.format` in every invocation of this closure. Why?



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/AbstractFilePageStoreIo.java:
##########
@@ -0,0 +1,611 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.pagememory.persistence.store;
+
+import static java.nio.ByteOrder.nativeOrder;
+import static java.nio.file.StandardOpenOption.CREATE;
+import static java.nio.file.StandardOpenOption.READ;
+import static java.nio.file.StandardOpenOption.WRITE;
+import static org.apache.ignite.internal.util.IgniteUtils.atomicMoveFile;
+import static org.apache.ignite.internal.util.IgniteUtils.hexInt;
+import static org.apache.ignite.internal.util.IgniteUtils.hexLong;
+import static org.apache.ignite.internal.util.IgniteUtils.toHexString;
+import static org.apache.ignite.lang.IgniteSystemProperties.getBoolean;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.ClosedByInterruptException;
+import java.nio.channels.ClosedChannelException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.apache.ignite.internal.fileio.FileIo;
+import org.apache.ignite.internal.fileio.FileIoFactory;
+import org.apache.ignite.internal.pagememory.io.PageIo;
+import org.apache.ignite.internal.pagememory.persistence.FastCrc;
+import org.apache.ignite.internal.pagememory.persistence.IgniteInternalDataIntegrityViolationException;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Abstract class for performing IO operations on file page storage.
+ */
+public abstract class AbstractFilePageStoreIo implements Closeable {
+    private final FileIoFactory ioFactory;
+
+    private final ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
+
+    /** Skip CRC calculation flag. */
+    // TODO: IGNITE-17011 Move to config
+    private final boolean skipCrc = getBoolean("IGNITE_PDS_SKIP_CRC");
+
+    private volatile Path filePath;
+
+    private volatile @Nullable FileIo fileIo;
+
+    /** Initialized file page store IO. */
+    private volatile boolean initialized;
+
+    /** Caches the existence state of file. After it is initialized, it will be not {@code null} during lifecycle. */
+    private volatile @Nullable Boolean fileExists;
+
+    /**
+     * Constructor.
+     *
+     * @param ioFactory {@link FileIo} factory.
+     * @param filePath File page store path.
+     */
+    AbstractFilePageStoreIo(FileIoFactory ioFactory, Path filePath) {
+        this.ioFactory = ioFactory;
+        this.filePath = filePath;
+    }
+
+    /**
+     * Returns the page size in bytes.
+     */
+    public abstract int pageSize();

Review Comment:
   Can this be backed by a field in abstract class?



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/AbstractFilePageStoreIo.java:
##########
@@ -0,0 +1,611 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.pagememory.persistence.store;
+
+import static java.nio.ByteOrder.nativeOrder;
+import static java.nio.file.StandardOpenOption.CREATE;
+import static java.nio.file.StandardOpenOption.READ;
+import static java.nio.file.StandardOpenOption.WRITE;
+import static org.apache.ignite.internal.util.IgniteUtils.atomicMoveFile;
+import static org.apache.ignite.internal.util.IgniteUtils.hexInt;
+import static org.apache.ignite.internal.util.IgniteUtils.hexLong;
+import static org.apache.ignite.internal.util.IgniteUtils.toHexString;
+import static org.apache.ignite.lang.IgniteSystemProperties.getBoolean;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.ClosedByInterruptException;
+import java.nio.channels.ClosedChannelException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.apache.ignite.internal.fileio.FileIo;
+import org.apache.ignite.internal.fileio.FileIoFactory;
+import org.apache.ignite.internal.pagememory.io.PageIo;
+import org.apache.ignite.internal.pagememory.persistence.FastCrc;
+import org.apache.ignite.internal.pagememory.persistence.IgniteInternalDataIntegrityViolationException;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Abstract class for performing IO operations on file page storage.
+ */
+public abstract class AbstractFilePageStoreIo implements Closeable {
+    private final FileIoFactory ioFactory;
+
+    private final ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
+
+    /** Skip CRC calculation flag. */
+    // TODO: IGNITE-17011 Move to config
+    private final boolean skipCrc = getBoolean("IGNITE_PDS_SKIP_CRC");
+
+    private volatile Path filePath;
+
+    private volatile @Nullable FileIo fileIo;
+
+    /** Initialized file page store IO. */
+    private volatile boolean initialized;
+
+    /** Caches the existence state of file. After it is initialized, it will be not {@code null} during lifecycle. */
+    private volatile @Nullable Boolean fileExists;

Review Comment:
   Should this be volatile? Maybe not. Please check all fields, thank you!



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStore.java:
##########
@@ -17,122 +17,108 @@
 
 package org.apache.ignite.internal.pagememory.persistence.store;
 
-import static java.nio.ByteOrder.nativeOrder;
-import static java.nio.file.StandardOpenOption.CREATE;
-import static java.nio.file.StandardOpenOption.READ;
-import static java.nio.file.StandardOpenOption.WRITE;
-import static org.apache.ignite.internal.util.IgniteUtils.hexInt;
-import static org.apache.ignite.internal.util.IgniteUtils.hexLong;
-import static org.apache.ignite.internal.util.IgniteUtils.toHexString;
-import static org.apache.ignite.lang.IgniteSystemProperties.getBoolean;
+import static org.apache.ignite.internal.pagememory.util.PageIdUtils.pageIndex;
 
 import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.lang.invoke.VarHandle;
 import java.nio.ByteBuffer;
-import java.nio.channels.ClosedByInterruptException;
-import java.nio.channels.ClosedChannelException;
-import java.nio.file.Files;
 import java.nio.file.Path;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.locks.ReadWriteLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.function.Supplier;
 import org.apache.ignite.internal.fileio.FileIo;
-import org.apache.ignite.internal.fileio.FileIoFactory;
-import org.apache.ignite.internal.pagememory.io.PageIo;
-import org.apache.ignite.internal.pagememory.persistence.FastCrc;
-import org.apache.ignite.internal.pagememory.persistence.IgniteInternalDataIntegrityViolationException;
-import org.apache.ignite.internal.pagememory.util.PageIdUtils;
 import org.apache.ignite.lang.IgniteInternalCheckedException;
 import org.jetbrains.annotations.Nullable;
 
 /**
  * FilePageStore is a {@link PageStore} implementation that uses regular files to store pages.
  *
- * <p>Actual read and write operations are performed with {@link FileIo} abstract interface, list of its implementations is a good source
- * of information about functionality in Ignite Native Persistence.
+ * <p>It consists of the main file page store and delta file page stores, when reading the page at the beginning, the page is searched in
+ * the delta files and only then in the main file.
  *
  * <p>On a physical level each instance of {@code FilePageStore} corresponds to a partition file assigned to the local node.
  *
- * <p>Consists of:
- * <ul>
- *     <li>Header - {@link FilePageStoreHeader}. </li>
- *     <li>Body - data pages are multiples of {@link FilePageStoreHeader#pageSize() pageSize}.</li>
- * </ul>
+ * <p>Actual read and write operations are performed with {@link FileIo} abstract interface, list of its implementations is a good source
+ * of information about functionality in Ignite Native Persistence.
+ *
+ * <p>To create a delta file first invoke {@link #getOrCreateNewDeltaFile(Supplier)} then fill it and then invoke {@link
+ * #completeNewDeltaFile()}.
  */
 public class FilePageStore implements PageStore {
-    /** File version. */
-    public static final int VERSION_1 = 1;
+    private static final VarHandle PAGE_COUNT;
 
-    /** Skip CRC calculation flag. */
-    // TODO: IGNITE-17011 Move to config
-    private final boolean skipCrc = getBoolean("IGNITE_PDS_SKIP_CRC");
+    private static final VarHandle NEW_DELTA_FILE_PAGE_STORE_IO_FUTURE;
 
-    /** File page store version. */
-    private final int version;
+    static {
+        try {
+            PAGE_COUNT = MethodHandles.lookup().findVarHandle(FilePageStore.class, "pageCount", int.class);
 
-    /** Page size in bytes. */
-    private final int pageSize;
+            NEW_DELTA_FILE_PAGE_STORE_IO_FUTURE = MethodHandles.lookup().findVarHandle(
+                    FilePageStore.class,
+                    "newDeltaFilePageStoreIoFuture",
+                    CompletableFuture.class
+            );
+        } catch (ReflectiveOperationException e) {
+            throw new ExceptionInInitializerError(e);
+        }
+    }
 
-    /** Header size in bytes. Should be aligned to the {@link #pageSize}. */
-    private final int headerSize;
+    /** File page store version. */
+    public static final int VERSION_1 = 1;
 
-    /** File page store path. */
-    private final Path filePath;
+    /** Delta file page store IO version. */
+    public static final int DELTA_FILE_VERSION_1 = 1;
 
-    /** {@link FileIo} factory. */
-    private final FileIoFactory ioFactory;
+    /** File page store IO. */
+    private final FilePageStoreIo filePageStoreIo;
 
     /** Page count. */
-    private final AtomicInteger pageCount = new AtomicInteger();
+    private volatile int pageCount;
 
-    private final ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
+    /** New page allocation listener. */
+    private volatile @Nullable PageAllocationListener pageAllocationListener;
 
-    /** Caches the existence state of storage file. After it is initialized, it will be not {@code null} during lifecycle. */
-    private volatile Boolean fileExists;
+    /** Delta file page store IOs. */
+    private final List<DeltaFilePageStoreIo> deltaFilePageStoreIos;
 
-    /** {@link FileIo} for read/write operations with file. */
-    private volatile FileIo fileIo;
+    /** Future with a new delta file page store. */
+    private volatile @Nullable CompletableFuture<DeltaFilePageStoreIo> newDeltaFilePageStoreIoFuture;
 
-    /** Initialized file page store. */
-    private volatile boolean initialized;
+    /** {@link DeltaFilePageStoreIo} factory. */
+    private volatile @Nullable DeltaFilePageStoreIoFactory deltaFilePageStoreIoFactory;
 
-    /** New page allocation listener. */
-    private volatile @Nullable PageAllocationListener pageAllocationListener;
+    /** Callback on completion of delta file page store creation. */
+    private volatile @Nullable CompleteCreationDeltaFilePageStoreIoCallback completeCreationDeltaFilePageStoreIoCallback;

Review Comment:
   Can't this thing just be a `newDeltaFilePageStoreIoFuture` listener? Is there a necessity for a specific field and a specific functional interface? Looks over-engineered, but maybe I'm wrong



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStoreFactory.java:
##########
@@ -52,45 +54,67 @@ public FilePageStoreFactory(FileIoFactory fileIoFactory, int pageSize) {
     /**
      * Creates instance of {@link FilePageStore}.
      *
-     * <p>If the file exists, an attempt will be made to read its {@link FilePageStoreHeader header} and create the {@link FilePageStore}.
+     * <p>Page stores are created based on their headers, for a file page stores with no header, the latest version is generated for delta
+     * file page store files, headers must be present.
      *
-     * @param filePath File page store path.
-     * @param headerBuffer Buffer for reading {@link FilePageStoreHeader header} from {@code filePath}.
-     * @return File page store.
+     * @param headerBuffer Buffer for reading headers.
+     * @param filePageStorePath File page store path.

Review Comment:
   What does it mean? What file?



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStore.java:
##########
@@ -166,503 +152,196 @@ public int pages() {
     public void pages(int pageCount) {
         assert pageCount >= 0 : pageCount;
 
-        this.pageCount.set(pageCount);
+        this.pageCount = pageCount;
     }
 
     /**
-     * Reads a page, unlike {@link #read(long, ByteBuffer, boolean)}, checks the page offset in the file not logically (pageOffset <= {@link
-     * #pages()} * {@link #pageSize}) but physically (pageOffset <= {@link #size()}), which can affect performance when used in production
-     * code.
+     * Reads a page, unlike {@link #read(long, ByteBuffer, boolean)}, does not check the {@code pageId} so that its {@code pageIdx} is not
+     * greater than the {@link #pages() number of allocated pages}.
      *
      * @param pageId Page ID.
      * @param pageBuf Page buffer to read into.
      * @param keepCrc By default, reading zeroes CRC which was on page store, but you can keep it in {@code pageBuf} if set {@code
      * keepCrc}.
+     * @return {@code True} if the page was read successfully.
      * @throws IgniteInternalCheckedException If reading failed (IO error occurred).
      */
-    public void readByPhysicalOffset(long pageId, ByteBuffer pageBuf, boolean keepCrc) throws IgniteInternalCheckedException {
-        read0(pageId, pageBuf, !skipCrc, keepCrc, false);
+    public boolean readWithoutPageIdCheck(long pageId, ByteBuffer pageBuf, boolean keepCrc) throws IgniteInternalCheckedException {
+        for (DeltaFilePageStoreIo deltaFilePageStoreIo : deltaFilePageStoreIos) {
+            long pageOff = deltaFilePageStoreIo.pageOffset(pageId);
+
+            if (pageOff >= 0) {
+                return deltaFilePageStoreIo.read(pageId, pageOff, pageBuf, keepCrc);
+            }
+        }
+
+        return filePageStoreIo.read(pageId, filePageStoreIo.pageOffset(pageId), pageBuf, keepCrc);
     }
 
     /** {@inheritDoc} */
     @Override
     public void read(long pageId, ByteBuffer pageBuf, boolean keepCrc) throws IgniteInternalCheckedException {
-        read0(pageId, pageBuf, !skipCrc, keepCrc, true);
-    }
-
-    /**
-     * Reads a page from the page store.
-     *
-     * @param pageId Page ID.
-     * @param pageBuf Page buffer to read into.
-     * @param checkCrc Check CRC on page.
-     * @param keepCrc By default reading zeroes CRC which was on file, but you can keep it in pageBuf if set keepCrc.
-     * @param checkPageOffsetLogically Check page offset by {@link #allocatedBytes} or {@link #size}.
-     * @throws IgniteInternalCheckedException If reading failed (IO error occurred).
-     */
-    private void read0(
-            long pageId,
-            ByteBuffer pageBuf,
-            boolean checkCrc,
-            boolean keepCrc,
-            boolean checkPageOffsetLogically
-    ) throws IgniteInternalCheckedException {
-        ensure();
-
-        try {
-            assert pageBuf.capacity() == pageSize : pageBuf.capacity();
-            assert pageBuf.remaining() == pageSize : pageBuf.remaining();
-            assert pageBuf.position() == 0 : pageBuf.position();
-            assert pageBuf.order() == nativeOrder() : pageBuf.order();
-
-            long pageOff = pageOffset(pageId);
-
-            if (checkPageOffsetLogically) {
-                assert pageOff <= allocatedBytes() : "calculatedOffset=" + pageOff
-                        + ", allocated=" + allocatedBytes() + ", headerSize=" + headerSize + ", filePath=" + filePath;
-            } else {
-                assert pageOff <= size() : "calculatedOffset=" + pageOff
-                        + ", size=" + size() + ", headerSize=" + headerSize + ", filePath=" + filePath;
-            }
-
-            int n = readWithFailover(pageBuf, pageOff);
+        assert pageIndex(pageId) <= pageCount : "pageIdx=" + pageIndex(pageId) + ", pageCount=" + pageCount;
 
-            // If page was not written yet, nothing to read.
-            if (n < 0) {
-                pageBuf.put(new byte[pageBuf.remaining()]);
-            }
+        for (DeltaFilePageStoreIo deltaFilePageStoreIo : deltaFilePageStoreIos) {
+            long pageOff = deltaFilePageStoreIo.pageOffset(pageId);
 
-            int savedCrc32 = PageIo.getCrc(pageBuf);
-
-            PageIo.setCrc(pageBuf, 0);
-
-            pageBuf.position(0);
-
-            if (checkCrc) {
-                int curCrc32 = FastCrc.calcCrc(pageBuf, pageSize);
-
-                if ((savedCrc32 ^ curCrc32) != 0) {
-                    throw new IgniteInternalDataIntegrityViolationException("Failed to read page (CRC validation failed) "
-                            + "[id=" + hexLong(pageId) + ", off=" + pageOff
-                            + ", filePath=" + filePath + ", fileSize=" + fileIo.size()
-                            + ", savedCrc=" + hexInt(savedCrc32) + ", curCrc=" + hexInt(curCrc32)
-                            + ", page=" + toHexString(pageBuf) + "]");
-                }
-            }
+            if (pageOff >= 0) {
+                deltaFilePageStoreIo.read(pageId, pageOff, pageBuf, keepCrc);
 
-            assert PageIo.getCrc(pageBuf) == 0;
-
-            if (keepCrc) {
-                PageIo.setCrc(pageBuf, savedCrc32);
+                return;
             }
-        } catch (IOException e) {
-            throw new IgniteInternalCheckedException("Failed to read page [file=" + filePath + ", pageId=" + pageId + "]", e);
         }
+
+        filePageStoreIo.read(pageId, filePageStoreIo.pageOffset(pageId), pageBuf, keepCrc);
     }
 
     /** {@inheritDoc} */
     @Override
     public void write(long pageId, ByteBuffer pageBuf, boolean calculateCrc) throws IgniteInternalCheckedException {
-        ensure();
-
-        boolean interrupted = false;
-
-        while (true) {
-            FileIo fileIo = this.fileIo;
-
-            try {
-                readWriteLock.readLock().lock();
-
-                try {
-                    assert pageBuf.position() == 0 : pageBuf.position();
-                    assert pageBuf.order() == nativeOrder() : "Page buffer order " + pageBuf.order()
-                            + " should be same with " + nativeOrder();
-                    assert PageIo.getType(pageBuf) != 0 : "Invalid state. Type is 0! pageId = " + hexLong(pageId);
-                    assert PageIo.getVersion(pageBuf) != 0 : "Invalid state. Version is 0! pageId = " + hexLong(pageId);
-
-                    if (calculateCrc && !skipCrc) {
-                        assert PageIo.getCrc(pageBuf) == 0 : hexLong(pageId);
-
-                        PageIo.setCrc(pageBuf, calcCrc32(pageBuf, pageSize));
-                    }
-
-                    // Check whether crc was calculated somewhere above the stack if it is forcibly skipped.
-                    assert skipCrc || PageIo.getCrc(pageBuf) != 0
-                            || calcCrc32(pageBuf, pageSize) == 0 : "CRC hasn't been calculated, crc=0";
-
-                    assert pageBuf.position() == 0 : pageBuf.position();
-
-                    long pageOff = pageOffset(pageId);
-
-                    assert pageOff <= allocatedBytes() : "calculatedOffset=" + pageOff
-                            + ", allocated=" + allocatedBytes() + ", headerSize=" + headerSize + ", filePath=" + filePath;
-
-                    fileIo.writeFully(pageBuf, pageOff);
-
-                    PageIo.setCrc(pageBuf, 0);
-
-                    if (interrupted) {
-                        Thread.currentThread().interrupt();
-                    }
-
-                    return;
-                } finally {
-                    readWriteLock.readLock().unlock();
-                }
-            } catch (IOException e) {
-                if (e instanceof ClosedChannelException) {
-                    try {
-                        if (e instanceof ClosedByInterruptException) {
-                            interrupted = true;
-
-                            Thread.interrupted();
-                        }
-
-                        reinit(fileIo);
-
-                        pageBuf.position(0);
-
-                        PageIo.setCrc(pageBuf, 0);
+        assert pageIndex(pageId) <= pageCount : "pageIdx=" + pageIndex(pageId) + ", pageCount=" + pageCount;
 
-                        continue;
-                    } catch (IOException e0) {
-                        e0.addSuppressed(e);
-
-                        e = e0;
-                    }
-                }
-
-                throw new IgniteInternalCheckedException(
-                        "Failed to write page [filePath=" + filePath + ", pageId=" + pageId + "]",
-                        e
-                );
-            }
-        }
+        filePageStoreIo.write(pageId, pageBuf, calculateCrc);
     }
 
     /** {@inheritDoc} */
     @Override
     public void sync() throws IgniteInternalCheckedException {
-        readWriteLock.writeLock().lock();
-
-        try {
-            ensure();
-
-            FileIo fileIo = this.fileIo;
-
-            if (fileIo != null) {
-                fileIo.force();
-            }
-        } catch (IOException e) {
-            throw new IgniteInternalCheckedException("Failed to fsync partition file [filePath=" + filePath + ']', e);
-        } finally {
-            readWriteLock.writeLock().unlock();
-        }
+        filePageStoreIo.sync();
     }
 
     /** {@inheritDoc} */
     @Override
     public boolean exists() {
-        if (fileExists == null) {
-            readWriteLock.writeLock().lock();
-
-            try {
-                if (fileExists == null) {
-                    fileExists = Files.exists(filePath) && filePath.toFile().length() >= headerSize;
-                }
-            } finally {
-                readWriteLock.writeLock().unlock();
-            }
-        }
-
-        return fileExists;
+        return filePageStoreIo.exists();
     }
 
     /** {@inheritDoc} */
     @Override
     public void ensure() throws IgniteInternalCheckedException {
-        if (!initialized) {
-            readWriteLock.writeLock().lock();
-
-            try {
-                if (!initialized) {
-                    FileIo fileIo = null;
-
-                    IgniteInternalCheckedException err = null;
-
-                    try {
-                        boolean interrupted = false;
-
-                        while (true) {
-                            try {
-                                this.fileIo = fileIo = ioFactory.create(filePath, CREATE, READ, WRITE);
-
-                                fileExists = true;
-
-                                if (fileIo.size() < headerSize) {
-                                    fileIo.writeFully(new FilePageStoreHeader(version, pageSize).toByteBuffer(), 0);
-                                } else {
-                                    checkHeader(fileIo);
-                                }
-
-                                if (interrupted) {
-                                    Thread.currentThread().interrupt();
-                                }
-
-                                break;
-                            } catch (ClosedByInterruptException e) {
-                                interrupted = true;
-
-                                Thread.interrupted();
-                            }
-                        }
-
-                        initialized = true;
-                    } catch (IOException e) {
-                        err = new IgniteInternalCheckedException("Failed to initialize partition file: " + filePath, e);
-
-                        throw err;
-                    } finally {
-                        if (err != null && fileIo != null) {
-                            try {
-                                fileIo.close();
-                            } catch (IOException e) {
-                                err.addSuppressed(e);
-                            }
-                        }
-                    }
-                }
-            } finally {
-                readWriteLock.writeLock().unlock();
-            }
-        }
+        filePageStoreIo.ensure();
     }
 
     /** {@inheritDoc} */
     @Override
     public void close() throws IOException {
-        stop0(false);
+        filePageStoreIo.close();
+
+        for (DeltaFilePageStoreIo deltaFilePageStoreIo : deltaFilePageStoreIos) {
+            deltaFilePageStoreIo.close();
+        }
     }
 
     /**
      * Returns size of the page store in bytes.
      *
-     * <p>May differ from {@link #pages} * {@link #pageSize} due to delayed writes or due to other implementation specific details.
+     * <p>May differ from {@link #pages} * {@link FilePageStoreIo#pageSize()} due to delayed writes or due to other implementation specific
+     * details.
      *
      * @throws IgniteInternalCheckedException If an I/O error occurs.
      */
     public long size() throws IgniteInternalCheckedException {
-        try {
-            FileIo io = fileIo;
-
-            return io == null ? 0 : io.size();
-        } catch (IOException e) {
-            throw new IgniteInternalCheckedException(e);
-        }
+        return filePageStoreIo.size();
     }
 
     /**
-     * Stops file page store.
-     *
-     * @param delete {@code True} to delete file.
-     * @throws IOException If fails.
+     * Returns file page store path.
      */
-    private void stop0(boolean delete) throws IOException {
-        readWriteLock.writeLock().lock();
-
-        try {
-            if (!initialized) {
-                // Ensure the file is closed even if not initialized yet.
-                if (fileIo != null) {
-                    fileIo.close();
-                }
-
-                if (delete && exists()) {
-                    Files.delete(filePath);
-                }
-
-                return;
-            }
-
-            fileIo.force();
-
-            fileIo.close();
-
-            fileIo = null;
-
-            if (delete) {
-                Files.delete(filePath);
-
-                fileExists = false;
-            }
-        } finally {
-            initialized = false;
+    public Path filePath() {

Review Comment:
   Is this method still relevant?



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStoreManager.java:
##########
@@ -305,14 +340,99 @@ private List<FilePageStore> createFilePageStores(
 
         try {
             for (int i = 0; i < partitions; i++) {
-                Path partFilePath = groupWorkDir.resolve(String.format(PART_FILE_TEMPLATE, i));
+                int part = i;
+
+                Path partFilePath = groupWorkDir.resolve(String.format(PART_FILE_TEMPLATE, part));
+
+                Path[] partDeltaFiles = findPartitionDeltaFiles(groupWorkDir, partitions);
+
+                FilePageStore filePageStore = filePageStoreFactory.createPageStore(buffer.rewind(), partFilePath, partDeltaFiles);
+
+                filePageStore.setDeltaFilePageStoreIoFactory((index, pageIndexes) -> createLatest(groupWorkDir, part, index, pageIndexes));
+
+                filePageStore.setCompleteCreationDeltaFilePageStoreIoCallback(deltaIo -> renameDeltaFile(groupWorkDir, part, deltaIo));

Review Comment:
   Renaming the file here looks very strange. So, checkpoint will call a method on file store, that method will trigger the listener and it will call this code. Why does it have to be so complicated?



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStore.java:
##########
@@ -166,503 +152,196 @@ public int pages() {
     public void pages(int pageCount) {
         assert pageCount >= 0 : pageCount;
 
-        this.pageCount.set(pageCount);
+        this.pageCount = pageCount;
     }
 
     /**
-     * Reads a page, unlike {@link #read(long, ByteBuffer, boolean)}, checks the page offset in the file not logically (pageOffset <= {@link
-     * #pages()} * {@link #pageSize}) but physically (pageOffset <= {@link #size()}), which can affect performance when used in production
-     * code.
+     * Reads a page, unlike {@link #read(long, ByteBuffer, boolean)}, does not check the {@code pageId} so that its {@code pageIdx} is not
+     * greater than the {@link #pages() number of allocated pages}.
      *
      * @param pageId Page ID.
      * @param pageBuf Page buffer to read into.
      * @param keepCrc By default, reading zeroes CRC which was on page store, but you can keep it in {@code pageBuf} if set {@code
      * keepCrc}.
+     * @return {@code True} if the page was read successfully.
      * @throws IgniteInternalCheckedException If reading failed (IO error occurred).
      */
-    public void readByPhysicalOffset(long pageId, ByteBuffer pageBuf, boolean keepCrc) throws IgniteInternalCheckedException {
-        read0(pageId, pageBuf, !skipCrc, keepCrc, false);
+    public boolean readWithoutPageIdCheck(long pageId, ByteBuffer pageBuf, boolean keepCrc) throws IgniteInternalCheckedException {
+        for (DeltaFilePageStoreIo deltaFilePageStoreIo : deltaFilePageStoreIos) {
+            long pageOff = deltaFilePageStoreIo.pageOffset(pageId);
+
+            if (pageOff >= 0) {
+                return deltaFilePageStoreIo.read(pageId, pageOff, pageBuf, keepCrc);
+            }
+        }
+
+        return filePageStoreIo.read(pageId, filePageStoreIo.pageOffset(pageId), pageBuf, keepCrc);
     }
 
     /** {@inheritDoc} */
     @Override
     public void read(long pageId, ByteBuffer pageBuf, boolean keepCrc) throws IgniteInternalCheckedException {
-        read0(pageId, pageBuf, !skipCrc, keepCrc, true);
-    }
-
-    /**
-     * Reads a page from the page store.
-     *
-     * @param pageId Page ID.
-     * @param pageBuf Page buffer to read into.
-     * @param checkCrc Check CRC on page.
-     * @param keepCrc By default reading zeroes CRC which was on file, but you can keep it in pageBuf if set keepCrc.
-     * @param checkPageOffsetLogically Check page offset by {@link #allocatedBytes} or {@link #size}.
-     * @throws IgniteInternalCheckedException If reading failed (IO error occurred).
-     */
-    private void read0(
-            long pageId,
-            ByteBuffer pageBuf,
-            boolean checkCrc,
-            boolean keepCrc,
-            boolean checkPageOffsetLogically
-    ) throws IgniteInternalCheckedException {
-        ensure();
-
-        try {
-            assert pageBuf.capacity() == pageSize : pageBuf.capacity();
-            assert pageBuf.remaining() == pageSize : pageBuf.remaining();
-            assert pageBuf.position() == 0 : pageBuf.position();
-            assert pageBuf.order() == nativeOrder() : pageBuf.order();
-
-            long pageOff = pageOffset(pageId);
-
-            if (checkPageOffsetLogically) {
-                assert pageOff <= allocatedBytes() : "calculatedOffset=" + pageOff
-                        + ", allocated=" + allocatedBytes() + ", headerSize=" + headerSize + ", filePath=" + filePath;
-            } else {
-                assert pageOff <= size() : "calculatedOffset=" + pageOff
-                        + ", size=" + size() + ", headerSize=" + headerSize + ", filePath=" + filePath;
-            }
-
-            int n = readWithFailover(pageBuf, pageOff);
+        assert pageIndex(pageId) <= pageCount : "pageIdx=" + pageIndex(pageId) + ", pageCount=" + pageCount;
 
-            // If page was not written yet, nothing to read.
-            if (n < 0) {
-                pageBuf.put(new byte[pageBuf.remaining()]);
-            }
+        for (DeltaFilePageStoreIo deltaFilePageStoreIo : deltaFilePageStoreIos) {
+            long pageOff = deltaFilePageStoreIo.pageOffset(pageId);
 
-            int savedCrc32 = PageIo.getCrc(pageBuf);
-
-            PageIo.setCrc(pageBuf, 0);
-
-            pageBuf.position(0);
-
-            if (checkCrc) {
-                int curCrc32 = FastCrc.calcCrc(pageBuf, pageSize);
-
-                if ((savedCrc32 ^ curCrc32) != 0) {
-                    throw new IgniteInternalDataIntegrityViolationException("Failed to read page (CRC validation failed) "
-                            + "[id=" + hexLong(pageId) + ", off=" + pageOff
-                            + ", filePath=" + filePath + ", fileSize=" + fileIo.size()
-                            + ", savedCrc=" + hexInt(savedCrc32) + ", curCrc=" + hexInt(curCrc32)
-                            + ", page=" + toHexString(pageBuf) + "]");
-                }
-            }
+            if (pageOff >= 0) {
+                deltaFilePageStoreIo.read(pageId, pageOff, pageBuf, keepCrc);
 
-            assert PageIo.getCrc(pageBuf) == 0;
-
-            if (keepCrc) {
-                PageIo.setCrc(pageBuf, savedCrc32);
+                return;
             }
-        } catch (IOException e) {
-            throw new IgniteInternalCheckedException("Failed to read page [file=" + filePath + ", pageId=" + pageId + "]", e);
         }
+
+        filePageStoreIo.read(pageId, filePageStoreIo.pageOffset(pageId), pageBuf, keepCrc);
     }
 
     /** {@inheritDoc} */
     @Override
     public void write(long pageId, ByteBuffer pageBuf, boolean calculateCrc) throws IgniteInternalCheckedException {
-        ensure();
-
-        boolean interrupted = false;
-
-        while (true) {
-            FileIo fileIo = this.fileIo;
-
-            try {
-                readWriteLock.readLock().lock();
-
-                try {
-                    assert pageBuf.position() == 0 : pageBuf.position();
-                    assert pageBuf.order() == nativeOrder() : "Page buffer order " + pageBuf.order()
-                            + " should be same with " + nativeOrder();
-                    assert PageIo.getType(pageBuf) != 0 : "Invalid state. Type is 0! pageId = " + hexLong(pageId);
-                    assert PageIo.getVersion(pageBuf) != 0 : "Invalid state. Version is 0! pageId = " + hexLong(pageId);
-
-                    if (calculateCrc && !skipCrc) {
-                        assert PageIo.getCrc(pageBuf) == 0 : hexLong(pageId);
-
-                        PageIo.setCrc(pageBuf, calcCrc32(pageBuf, pageSize));
-                    }
-
-                    // Check whether crc was calculated somewhere above the stack if it is forcibly skipped.
-                    assert skipCrc || PageIo.getCrc(pageBuf) != 0
-                            || calcCrc32(pageBuf, pageSize) == 0 : "CRC hasn't been calculated, crc=0";
-
-                    assert pageBuf.position() == 0 : pageBuf.position();
-
-                    long pageOff = pageOffset(pageId);
-
-                    assert pageOff <= allocatedBytes() : "calculatedOffset=" + pageOff
-                            + ", allocated=" + allocatedBytes() + ", headerSize=" + headerSize + ", filePath=" + filePath;
-
-                    fileIo.writeFully(pageBuf, pageOff);
-
-                    PageIo.setCrc(pageBuf, 0);
-
-                    if (interrupted) {
-                        Thread.currentThread().interrupt();
-                    }
-
-                    return;
-                } finally {
-                    readWriteLock.readLock().unlock();
-                }
-            } catch (IOException e) {
-                if (e instanceof ClosedChannelException) {
-                    try {
-                        if (e instanceof ClosedByInterruptException) {
-                            interrupted = true;
-
-                            Thread.interrupted();
-                        }
-
-                        reinit(fileIo);
-
-                        pageBuf.position(0);
-
-                        PageIo.setCrc(pageBuf, 0);
+        assert pageIndex(pageId) <= pageCount : "pageIdx=" + pageIndex(pageId) + ", pageCount=" + pageCount;
 
-                        continue;
-                    } catch (IOException e0) {
-                        e0.addSuppressed(e);
-
-                        e = e0;
-                    }
-                }
-
-                throw new IgniteInternalCheckedException(
-                        "Failed to write page [filePath=" + filePath + ", pageId=" + pageId + "]",
-                        e
-                );
-            }
-        }
+        filePageStoreIo.write(pageId, pageBuf, calculateCrc);
     }
 
     /** {@inheritDoc} */
     @Override
     public void sync() throws IgniteInternalCheckedException {
-        readWriteLock.writeLock().lock();
-
-        try {
-            ensure();
-
-            FileIo fileIo = this.fileIo;
-
-            if (fileIo != null) {
-                fileIo.force();
-            }
-        } catch (IOException e) {
-            throw new IgniteInternalCheckedException("Failed to fsync partition file [filePath=" + filePath + ']', e);
-        } finally {
-            readWriteLock.writeLock().unlock();
-        }
+        filePageStoreIo.sync();
     }
 
     /** {@inheritDoc} */
     @Override
     public boolean exists() {
-        if (fileExists == null) {
-            readWriteLock.writeLock().lock();
-
-            try {
-                if (fileExists == null) {
-                    fileExists = Files.exists(filePath) && filePath.toFile().length() >= headerSize;
-                }
-            } finally {
-                readWriteLock.writeLock().unlock();
-            }
-        }
-
-        return fileExists;
+        return filePageStoreIo.exists();
     }
 
     /** {@inheritDoc} */
     @Override
     public void ensure() throws IgniteInternalCheckedException {
-        if (!initialized) {
-            readWriteLock.writeLock().lock();
-
-            try {
-                if (!initialized) {
-                    FileIo fileIo = null;
-
-                    IgniteInternalCheckedException err = null;
-
-                    try {
-                        boolean interrupted = false;
-
-                        while (true) {
-                            try {
-                                this.fileIo = fileIo = ioFactory.create(filePath, CREATE, READ, WRITE);
-
-                                fileExists = true;
-
-                                if (fileIo.size() < headerSize) {
-                                    fileIo.writeFully(new FilePageStoreHeader(version, pageSize).toByteBuffer(), 0);
-                                } else {
-                                    checkHeader(fileIo);
-                                }
-
-                                if (interrupted) {
-                                    Thread.currentThread().interrupt();
-                                }
-
-                                break;
-                            } catch (ClosedByInterruptException e) {
-                                interrupted = true;
-
-                                Thread.interrupted();
-                            }
-                        }
-
-                        initialized = true;
-                    } catch (IOException e) {
-                        err = new IgniteInternalCheckedException("Failed to initialize partition file: " + filePath, e);
-
-                        throw err;
-                    } finally {
-                        if (err != null && fileIo != null) {
-                            try {
-                                fileIo.close();
-                            } catch (IOException e) {
-                                err.addSuppressed(e);
-                            }
-                        }
-                    }
-                }
-            } finally {
-                readWriteLock.writeLock().unlock();
-            }
-        }
+        filePageStoreIo.ensure();
     }
 
     /** {@inheritDoc} */
     @Override
     public void close() throws IOException {
-        stop0(false);
+        filePageStoreIo.close();
+
+        for (DeltaFilePageStoreIo deltaFilePageStoreIo : deltaFilePageStoreIos) {
+            deltaFilePageStoreIo.close();
+        }
     }
 
     /**
      * Returns size of the page store in bytes.
      *
-     * <p>May differ from {@link #pages} * {@link #pageSize} due to delayed writes or due to other implementation specific details.
+     * <p>May differ from {@link #pages} * {@link FilePageStoreIo#pageSize()} due to delayed writes or due to other implementation specific
+     * details.
      *
      * @throws IgniteInternalCheckedException If an I/O error occurs.
      */
     public long size() throws IgniteInternalCheckedException {
-        try {
-            FileIo io = fileIo;
-
-            return io == null ? 0 : io.size();
-        } catch (IOException e) {
-            throw new IgniteInternalCheckedException(e);
-        }
+        return filePageStoreIo.size();
     }
 
     /**
-     * Stops file page store.
-     *
-     * @param delete {@code True} to delete file.
-     * @throws IOException If fails.
+     * Returns file page store path.
      */
-    private void stop0(boolean delete) throws IOException {
-        readWriteLock.writeLock().lock();
-
-        try {
-            if (!initialized) {
-                // Ensure the file is closed even if not initialized yet.
-                if (fileIo != null) {
-                    fileIo.close();
-                }
-
-                if (delete && exists()) {
-                    Files.delete(filePath);
-                }
-
-                return;
-            }
-
-            fileIo.force();
-
-            fileIo.close();
-
-            fileIo = null;
-
-            if (delete) {
-                Files.delete(filePath);
-
-                fileExists = false;
-            }
-        } finally {
-            initialized = false;
+    public Path filePath() {
+        return filePageStoreIo.filePath();
+    }
 
-            readWriteLock.writeLock().unlock();
-        }
+    /**
+     * Returns file page store header size.
+     */
+    public int headerSize() {
+        return filePageStoreIo.headerSize();
     }
 
     /**
-     * Gets page offset within the store file.
+     * Sets the new page allocation listener.
      *
-     * @param pageId Page ID.
-     * @return Page offset.
+     * @param listener New page allocation listener.
      */
-    long pageOffset(long pageId) {
-        return (long) PageIdUtils.pageIndex(pageId) * pageSize + headerSize;
+    public void setPageAllocationListener(PageAllocationListener listener) {
+        pageAllocationListener = listener;
     }
 
     /**
-     * Reads from page storage with failover.
+     * Sets the delta file page store factory.
      *
-     * @param destBuf Destination buffer.
-     * @param position Position.
-     * @return Number of read bytes, or {@code -1} if the given position is greater than or equal to the file's current size.
+     * @param factory Factory.
      */
-    private int readWithFailover(ByteBuffer destBuf, long position) throws IOException {
-        boolean interrupted = false;
-
-        int bufPos = destBuf.position();
-
-        while (true) {
-            FileIo fileIo = this.fileIo;
-
-            if (fileIo == null) {
-                throw new IOException("FileIo has stopped");
-            }
-
-            try {
-                assert destBuf.remaining() > 0;
-
-                int bytesRead = fileIo.readFully(destBuf, position);
-
-                if (interrupted) {
-                    Thread.currentThread().interrupt();
-                }
-
-                return bytesRead;
-            } catch (ClosedChannelException e) {
-                destBuf.position(bufPos);
-
-                if (e instanceof ClosedByInterruptException) {
-                    interrupted = true;
-
-                    Thread.interrupted();
-                }
-
-                reinit(fileIo);
-            }
-        }
+    public void setDeltaFilePageStoreIoFactory(DeltaFilePageStoreIoFactory factory) {
+        deltaFilePageStoreIoFactory = factory;
     }
 
     /**
-     * Reinit page store after file channel was closed by thread interruption.
+     * Sets the callback on completion of delta file page store creation.
      *
-     * @param fileIo Old fileIo.
+     * @param callback Callback.
      */
-    private void reinit(FileIo fileIo) throws IOException {
-        if (!initialized) {
-            return;
-        }
-
-        if (fileIo != this.fileIo) {
-            return;
-        }
-
-        readWriteLock.writeLock().lock();
-
-        try {
-            if (fileIo != this.fileIo) {
-                return;
-            }
-
-            try {
-                boolean interrupted = false;
-
-                while (true) {
-                    try {
-                        fileIo = null;
-
-                        fileIo = ioFactory.create(filePath, CREATE, READ, WRITE);
+    public void setCompleteCreationDeltaFilePageStoreIoCallback(CompleteCreationDeltaFilePageStoreIoCallback callback) {
+        completeCreationDeltaFilePageStoreIoCallback = callback;
+    }
 
-                        fileExists = true;
+    /**
+     * Gets or creates a new delta file, a new delta file will be created when the previous one is {@link #completeNewDeltaFile()
+     * completed}.
+     *
+     * <p>Thread safe.
+     *
+     * @param pageIndexesSupplier Page indexes supplier for the new delta file page store.
+     * @return Future that will be completed when the new delta file page store is created.
+     */
+    public CompletableFuture<DeltaFilePageStoreIo> getOrCreateNewDeltaFile(Supplier<int[]> pageIndexesSupplier) {
+        assert deltaFilePageStoreIoFactory != null;
 
-                        checkHeader(fileIo);
+        CompletableFuture<DeltaFilePageStoreIo> future = this.newDeltaFilePageStoreIoFuture;
 
-                        this.fileIo = fileIo;
+        if (future != null) {
+            return future;
+        }
 
-                        if (interrupted) {
-                            Thread.currentThread().interrupt();
-                        }
+        if (!NEW_DELTA_FILE_PAGE_STORE_IO_FUTURE.compareAndSet(this, null, future = new CompletableFuture<>())) {
+            // Another thread started creating a delta file.
+            return newDeltaFilePageStoreIoFuture;
+        }
 
-                        break;
-                    } catch (ClosedByInterruptException e) {
-                        interrupted = true;
+        int nextIndex = deltaFilePageStoreIos.isEmpty() ? 0 : deltaFilePageStoreIos.get(0).fileIndex() + 1;
 
-                        Thread.interrupted();
-                    }
-                }
-            } catch (IOException e) {
-                try {
-                    if (fileIo != null) {
-                        fileIo.close();
-                    }
-                } catch (IOException e0) {
-                    e.addSuppressed(e0);
-                }
+        DeltaFilePageStoreIo deltaFilePageStoreIo = deltaFilePageStoreIoFactory.create(nextIndex, pageIndexesSupplier.get());
 
-                throw e;
-            }
-        } finally {
-            readWriteLock.writeLock().unlock();
-        }
-    }
+        // Should add to the head, since read operations should always start from the most recent.
+        deltaFilePageStoreIos.add(0, deltaFilePageStoreIo);
 
-    private static int calcCrc32(ByteBuffer pageBuf, int pageSize) {
-        try {
-            pageBuf.position(0);
+        future.complete(deltaFilePageStoreIo);

Review Comment:
   Future is completed here, is this correct?



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStoreFactory.java:
##########
@@ -52,45 +54,67 @@ public FilePageStoreFactory(FileIoFactory fileIoFactory, int pageSize) {
     /**
      * Creates instance of {@link FilePageStore}.
      *
-     * <p>If the file exists, an attempt will be made to read its {@link FilePageStoreHeader header} and create the {@link FilePageStore}.
+     * <p>Page stores are created based on their headers, for a file page stores with no header, the latest version is generated for delta
+     * file page store files, headers must be present.
      *
-     * @param filePath File page store path.
-     * @param headerBuffer Buffer for reading {@link FilePageStoreHeader header} from {@code filePath}.
-     * @return File page store.
+     * @param headerBuffer Buffer for reading headers.
+     * @param filePageStorePath File page store path.
+     * @param deltaFilePaths Paths to existing delta files page stores of the file page storage.
      * @throws IgniteInternalCheckedException if failed
      */
-    public FilePageStore createPageStore(Path filePath, ByteBuffer headerBuffer) throws IgniteInternalCheckedException {
+    public FilePageStore createPageStore(
+            ByteBuffer headerBuffer,
+            Path filePageStorePath,
+            Path... deltaFilePaths
+    ) throws IgniteInternalCheckedException {
         assert headerBuffer.remaining() == pageSize : headerBuffer.remaining();
 
-        if (!Files.exists(filePath)) {
-            return createPageStore(filePath, new FilePageStoreHeader(latestVersion, pageSize));
+        if (!Files.exists(filePageStorePath)) {
+            assert deltaFilePaths.length == 0 : Arrays.toString(deltaFilePaths);
+
+            return createFilePageStore(filePageStorePath, new FilePageStoreHeader(latestFilePageStoreVersion, pageSize));
         }
 
-        try (FileIo fileIo = fileIoFactory.create(filePath)) {
-            FilePageStoreHeader header = readHeader(fileIo, headerBuffer);
+        try (FileIo fileIo = fileIoFactory.create(filePageStorePath)) {
+            FilePageStoreHeader header = FilePageStoreHeader.readHeader(fileIo, headerBuffer);
 
             if (header == null) {
-                header = new FilePageStoreHeader(latestVersion, pageSize);
+                header = new FilePageStoreHeader(latestFilePageStoreVersion, pageSize);
+            }
+
+            if (deltaFilePaths.length == 0) {
+                return createFilePageStore(filePageStorePath, header);
+            }
+
+            DeltaFilePageStoreIo[] deltaFileIos = new DeltaFilePageStoreIo[deltaFilePaths.length];
+
+            for (int i = 0; i < deltaFilePaths.length; i++) {
+                Path deltaFilePath = deltaFilePaths[i];
+
+                assert Files.exists(deltaFilePath) : deltaFilePath;
+
+                try (FileIo deltaFileIo = fileIoFactory.create(deltaFilePath)) {
+                    DeltaFilePageStoreIoHeader deltaFileHeader = DeltaFilePageStoreIoHeader.readHeader(deltaFileIo, headerBuffer.rewind());
+
+                    assert deltaFileHeader != null : deltaFileHeader;
+
+                    deltaFileIos[i] = createDeltaFilePageStoreIo(deltaFilePath, deltaFileHeader);
+                }
             }
 
-            return createPageStore(filePath, header);
+            return createFilePageStore(filePageStorePath, header, deltaFileIos);
         } catch (IOException e) {
-            throw new IgniteInternalCheckedException("Error while creating file page store [file=" + filePath + "]", e);
+            throw new IgniteInternalCheckedException("Error while creating file page store [file=" + filePageStorePath + "]", e);

Review Comment:
   I don't see delta files in the message



-- 
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 #947: IGNITE-17372 Implement DeltaFilePageStore

Posted by GitBox <gi...@apache.org>.
tkalkirill commented on code in PR #947:
URL: https://github.com/apache/ignite-3/pull/947#discussion_r928727200


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStore.java:
##########
@@ -17,122 +17,108 @@
 
 package org.apache.ignite.internal.pagememory.persistence.store;
 
-import static java.nio.ByteOrder.nativeOrder;
-import static java.nio.file.StandardOpenOption.CREATE;
-import static java.nio.file.StandardOpenOption.READ;
-import static java.nio.file.StandardOpenOption.WRITE;
-import static org.apache.ignite.internal.util.IgniteUtils.hexInt;
-import static org.apache.ignite.internal.util.IgniteUtils.hexLong;
-import static org.apache.ignite.internal.util.IgniteUtils.toHexString;
-import static org.apache.ignite.lang.IgniteSystemProperties.getBoolean;
+import static org.apache.ignite.internal.pagememory.util.PageIdUtils.pageIndex;
 
 import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.lang.invoke.VarHandle;
 import java.nio.ByteBuffer;
-import java.nio.channels.ClosedByInterruptException;
-import java.nio.channels.ClosedChannelException;
-import java.nio.file.Files;
 import java.nio.file.Path;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.locks.ReadWriteLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.function.Supplier;
 import org.apache.ignite.internal.fileio.FileIo;
-import org.apache.ignite.internal.fileio.FileIoFactory;
-import org.apache.ignite.internal.pagememory.io.PageIo;
-import org.apache.ignite.internal.pagememory.persistence.FastCrc;
-import org.apache.ignite.internal.pagememory.persistence.IgniteInternalDataIntegrityViolationException;
-import org.apache.ignite.internal.pagememory.util.PageIdUtils;
 import org.apache.ignite.lang.IgniteInternalCheckedException;
 import org.jetbrains.annotations.Nullable;
 
 /**
  * FilePageStore is a {@link PageStore} implementation that uses regular files to store pages.
  *
- * <p>Actual read and write operations are performed with {@link FileIo} abstract interface, list of its implementations is a good source
- * of information about functionality in Ignite Native Persistence.
+ * <p>It consists of the main file page store and delta file page stores, when reading the page at the beginning, the page is searched in
+ * the delta files and only then in the main file.
  *
  * <p>On a physical level each instance of {@code FilePageStore} corresponds to a partition file assigned to the local node.
  *
- * <p>Consists of:
- * <ul>
- *     <li>Header - {@link FilePageStoreHeader}. </li>
- *     <li>Body - data pages are multiples of {@link FilePageStoreHeader#pageSize() pageSize}.</li>
- * </ul>
+ * <p>Actual read and write operations are performed with {@link FileIo} abstract interface, list of its implementations is a good source
+ * of information about functionality in Ignite Native Persistence.
+ *
+ * <p>To create a delta file first invoke {@link #getOrCreateNewDeltaFile(Supplier)} then fill it and then invoke {@link
+ * #completeNewDeltaFile()}.
  */
 public class FilePageStore implements PageStore {
-    /** File version. */
-    public static final int VERSION_1 = 1;
+    private static final VarHandle PAGE_COUNT;
 
-    /** Skip CRC calculation flag. */
-    // TODO: IGNITE-17011 Move to config
-    private final boolean skipCrc = getBoolean("IGNITE_PDS_SKIP_CRC");
+    private static final VarHandle NEW_DELTA_FILE_PAGE_STORE_IO_FUTURE;
 
-    /** File page store version. */
-    private final int version;
+    static {
+        try {
+            PAGE_COUNT = MethodHandles.lookup().findVarHandle(FilePageStore.class, "pageCount", int.class);
 
-    /** Page size in bytes. */
-    private final int pageSize;
+            NEW_DELTA_FILE_PAGE_STORE_IO_FUTURE = MethodHandles.lookup().findVarHandle(
+                    FilePageStore.class,
+                    "newDeltaFilePageStoreIoFuture",
+                    CompletableFuture.class
+            );
+        } catch (ReflectiveOperationException e) {
+            throw new ExceptionInInitializerError(e);
+        }
+    }
 
-    /** Header size in bytes. Should be aligned to the {@link #pageSize}. */
-    private final int headerSize;
+    /** File page store version. */
+    public static final int VERSION_1 = 1;
 
-    /** File page store path. */
-    private final Path filePath;
+    /** Delta file page store IO version. */
+    public static final int DELTA_FILE_VERSION_1 = 1;
 
-    /** {@link FileIo} factory. */
-    private final FileIoFactory ioFactory;
+    /** File page store IO. */
+    private final FilePageStoreIo filePageStoreIo;
 
     /** Page count. */
-    private final AtomicInteger pageCount = new AtomicInteger();
+    private volatile int pageCount;
 
-    private final ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
+    /** New page allocation listener. */
+    private volatile @Nullable PageAllocationListener pageAllocationListener;
 
-    /** Caches the existence state of storage file. After it is initialized, it will be not {@code null} during lifecycle. */
-    private volatile Boolean fileExists;
+    /** Delta file page store IOs. */
+    private final List<DeltaFilePageStoreIo> deltaFilePageStoreIos;
 
-    /** {@link FileIo} for read/write operations with file. */
-    private volatile FileIo fileIo;
+    /** Future with a new delta file page store. */
+    private volatile @Nullable CompletableFuture<DeltaFilePageStoreIo> newDeltaFilePageStoreIoFuture;
 
-    /** Initialized file page store. */
-    private volatile boolean initialized;
+    /** {@link DeltaFilePageStoreIo} factory. */
+    private volatile @Nullable DeltaFilePageStoreIoFactory deltaFilePageStoreIoFactory;
 
-    /** New page allocation listener. */
-    private volatile @Nullable PageAllocationListener pageAllocationListener;
+    /** Callback on completion of delta file page store creation. */
+    private volatile @Nullable CompleteCreationDeltaFilePageStoreIoCallback completeCreationDeltaFilePageStoreIoCallback;

Review Comment:
   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] tkalkirill commented on a diff in pull request #947: IGNITE-17372 Implement DeltaFilePageStore

Posted by GitBox <gi...@apache.org>.
tkalkirill commented on code in PR #947:
URL: https://github.com/apache/ignite-3/pull/947#discussion_r931961765


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/PartitionMetaManager.java:
##########
@@ -95,28 +99,34 @@ public PartitionMeta readOrCreateMeta(
         long partitionMetaPageId = partitionMetaPageId(groupPartitionId.getPartitionId());
 
         try {
-            if (filePageStore.readWithoutPageIdCheck(partitionMetaPageId, buffer, false)) {
+            if (containsPartitionMetaIo(filePageStore)) {

Review Comment:
   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] tkalkirill commented on a diff in pull request #947: IGNITE-17372 Implement DeltaFilePageStore

Posted by GitBox <gi...@apache.org>.
tkalkirill commented on code in PR #947:
URL: https://github.com/apache/ignite-3/pull/947#discussion_r928594610


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/AbstractFilePageStoreIo.java:
##########
@@ -0,0 +1,611 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.pagememory.persistence.store;
+
+import static java.nio.ByteOrder.nativeOrder;
+import static java.nio.file.StandardOpenOption.CREATE;
+import static java.nio.file.StandardOpenOption.READ;
+import static java.nio.file.StandardOpenOption.WRITE;
+import static org.apache.ignite.internal.util.IgniteUtils.atomicMoveFile;
+import static org.apache.ignite.internal.util.IgniteUtils.hexInt;
+import static org.apache.ignite.internal.util.IgniteUtils.hexLong;
+import static org.apache.ignite.internal.util.IgniteUtils.toHexString;
+import static org.apache.ignite.lang.IgniteSystemProperties.getBoolean;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.ClosedByInterruptException;
+import java.nio.channels.ClosedChannelException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.apache.ignite.internal.fileio.FileIo;
+import org.apache.ignite.internal.fileio.FileIoFactory;
+import org.apache.ignite.internal.pagememory.io.PageIo;
+import org.apache.ignite.internal.pagememory.persistence.FastCrc;
+import org.apache.ignite.internal.pagememory.persistence.IgniteInternalDataIntegrityViolationException;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Abstract class for performing IO operations on file page storage.
+ */
+public abstract class AbstractFilePageStoreIo implements Closeable {
+    private final FileIoFactory ioFactory;
+
+    private final ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
+
+    /** Skip CRC calculation flag. */
+    // TODO: IGNITE-17011 Move to config
+    private final boolean skipCrc = getBoolean("IGNITE_PDS_SKIP_CRC");
+
+    private volatile Path filePath;
+
+    private volatile @Nullable FileIo fileIo;
+
+    /** Initialized file page store IO. */
+    private volatile boolean initialized;
+
+    /** Caches the existence state of file. After it is initialized, it will be not {@code null} during lifecycle. */
+    private volatile @Nullable Boolean fileExists;
+
+    /**
+     * Constructor.
+     *
+     * @param ioFactory {@link FileIo} factory.
+     * @param filePath File page store path.
+     */
+    AbstractFilePageStoreIo(FileIoFactory ioFactory, Path filePath) {
+        this.ioFactory = ioFactory;
+        this.filePath = filePath;
+    }
+
+    /**
+     * Returns the page size in bytes.
+     */
+    public abstract int pageSize();
+
+    /**
+     * Returns the size of the header in bytes.
+     */
+    public abstract int headerSize();
+
+    /**
+     * Returns a buffer with a file page storage header.
+     */
+    public abstract ByteBuffer headerBuffer();
+
+    /**
+     * Checks the file page storage header.
+     *
+     * @param fileIo File page store IO to read the header.
+     * @throws IOException If there is an error reading the header or the header did not pass the check.
+     */
+    public abstract void checkHeader(FileIo fileIo) throws IOException;
+
+    /**
+     * Returns page offset within the store file.
+     *
+     * @param pageId Page ID.
+     */
+    public abstract long pageOffset(long pageId);

Review Comment:
   Made as private methods.



-- 
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 #947: IGNITE-17372 Implement DeltaFilePageStore

Posted by GitBox <gi...@apache.org>.
tkalkirill commented on code in PR #947:
URL: https://github.com/apache/ignite-3/pull/947#discussion_r928624742


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStoreManager.java:
##########
@@ -49,12 +52,24 @@ public class FilePageStoreManager implements PageReadWriteManager {
     /** File suffix. */
     public static final String FILE_SUFFIX = ".bin";
 
+    /** Suffix of the temporary file. */
+    public static final String TMP_FILE_SUFFIX = ".tmp";
+
     /** Partition file prefix. */
     public static final String PART_FILE_PREFIX = "part-";
 
-    /** Partition file template. */
+    /** Partition delta file prefix. */
+    public static final String PART_DELTA_FILE_PREFIX = "part-%d-delta-";

Review Comment:
   fxi 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] tkalkirill commented on a diff in pull request #947: IGNITE-17372 Implement DeltaFilePageStore

Posted by GitBox <gi...@apache.org>.
tkalkirill commented on code in PR #947:
URL: https://github.com/apache/ignite-3/pull/947#discussion_r928591560


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/AbstractFilePageStoreIo.java:
##########
@@ -0,0 +1,611 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.pagememory.persistence.store;
+
+import static java.nio.ByteOrder.nativeOrder;
+import static java.nio.file.StandardOpenOption.CREATE;
+import static java.nio.file.StandardOpenOption.READ;
+import static java.nio.file.StandardOpenOption.WRITE;
+import static org.apache.ignite.internal.util.IgniteUtils.atomicMoveFile;
+import static org.apache.ignite.internal.util.IgniteUtils.hexInt;
+import static org.apache.ignite.internal.util.IgniteUtils.hexLong;
+import static org.apache.ignite.internal.util.IgniteUtils.toHexString;
+import static org.apache.ignite.lang.IgniteSystemProperties.getBoolean;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.ClosedByInterruptException;
+import java.nio.channels.ClosedChannelException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.apache.ignite.internal.fileio.FileIo;
+import org.apache.ignite.internal.fileio.FileIoFactory;
+import org.apache.ignite.internal.pagememory.io.PageIo;
+import org.apache.ignite.internal.pagememory.persistence.FastCrc;
+import org.apache.ignite.internal.pagememory.persistence.IgniteInternalDataIntegrityViolationException;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Abstract class for performing IO operations on file page storage.
+ */
+public abstract class AbstractFilePageStoreIo implements Closeable {
+    private final FileIoFactory ioFactory;
+
+    private final ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
+
+    /** Skip CRC calculation flag. */
+    // TODO: IGNITE-17011 Move to config
+    private final boolean skipCrc = getBoolean("IGNITE_PDS_SKIP_CRC");
+
+    private volatile Path filePath;
+
+    private volatile @Nullable FileIo fileIo;
+
+    /** Initialized file page store IO. */
+    private volatile boolean initialized;
+
+    /** Caches the existence state of file. After it is initialized, it will be not {@code null} during lifecycle. */
+    private volatile @Nullable Boolean fileExists;
+
+    /**
+     * Constructor.
+     *
+     * @param ioFactory {@link FileIo} factory.
+     * @param filePath File page store path.
+     */
+    AbstractFilePageStoreIo(FileIoFactory ioFactory, Path filePath) {
+        this.ioFactory = ioFactory;
+        this.filePath = filePath;
+    }
+
+    /**
+     * Returns the page size in bytes.
+     */
+    public abstract int pageSize();

Review Comment:
   Discussed in person, I will not fix.



-- 
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 #947: IGNITE-17372 Implement DeltaFilePageStore

Posted by GitBox <gi...@apache.org>.
tkalkirill commented on code in PR #947:
URL: https://github.com/apache/ignite-3/pull/947#discussion_r928601619


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/AbstractFilePageStoreIo.java:
##########
@@ -0,0 +1,611 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.pagememory.persistence.store;
+
+import static java.nio.ByteOrder.nativeOrder;
+import static java.nio.file.StandardOpenOption.CREATE;
+import static java.nio.file.StandardOpenOption.READ;
+import static java.nio.file.StandardOpenOption.WRITE;
+import static org.apache.ignite.internal.util.IgniteUtils.atomicMoveFile;
+import static org.apache.ignite.internal.util.IgniteUtils.hexInt;
+import static org.apache.ignite.internal.util.IgniteUtils.hexLong;
+import static org.apache.ignite.internal.util.IgniteUtils.toHexString;
+import static org.apache.ignite.lang.IgniteSystemProperties.getBoolean;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.ClosedByInterruptException;
+import java.nio.channels.ClosedChannelException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.apache.ignite.internal.fileio.FileIo;
+import org.apache.ignite.internal.fileio.FileIoFactory;
+import org.apache.ignite.internal.pagememory.io.PageIo;
+import org.apache.ignite.internal.pagememory.persistence.FastCrc;
+import org.apache.ignite.internal.pagememory.persistence.IgniteInternalDataIntegrityViolationException;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Abstract class for performing IO operations on file page storage.
+ */
+public abstract class AbstractFilePageStoreIo implements Closeable {
+    private final FileIoFactory ioFactory;
+
+    private final ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
+
+    /** Skip CRC calculation flag. */
+    // TODO: IGNITE-17011 Move to config
+    private final boolean skipCrc = getBoolean("IGNITE_PDS_SKIP_CRC");
+
+    private volatile Path filePath;
+
+    private volatile @Nullable FileIo fileIo;
+
+    /** Initialized file page store IO. */
+    private volatile boolean initialized;
+
+    /** Caches the existence state of file. After it is initialized, it will be not {@code null} during lifecycle. */
+    private volatile @Nullable Boolean fileExists;
+
+    /**
+     * Constructor.
+     *
+     * @param ioFactory {@link FileIo} factory.
+     * @param filePath File page store path.
+     */
+    AbstractFilePageStoreIo(FileIoFactory ioFactory, Path filePath) {
+        this.ioFactory = ioFactory;
+        this.filePath = filePath;
+    }
+
+    /**
+     * Returns the page size in bytes.
+     */
+    public abstract int pageSize();
+
+    /**
+     * Returns the size of the header in bytes.
+     */
+    public abstract int headerSize();
+
+    /**
+     * Returns a buffer with a file page storage header.
+     */
+    public abstract ByteBuffer headerBuffer();
+
+    /**
+     * Checks the file page storage header.
+     *
+     * @param fileIo File page store IO to read the header.
+     * @throws IOException If there is an error reading the header or the header did not pass the check.
+     */
+    public abstract void checkHeader(FileIo fileIo) throws IOException;
+
+    /**
+     * Returns page offset within the store file.
+     *
+     * @param pageId Page ID.
+     */
+    public abstract long pageOffset(long pageId);
+
+    /**
+     * Stops the file page store IO.
+     *
+     * @param clean {@code True} to clean file page store.
+     * @throws IgniteInternalCheckedException If failed.
+     */
+    public void stop(boolean clean) throws IgniteInternalCheckedException {
+        try {
+            stop0(clean);
+        } catch (IOException e) {
+            throw new IgniteInternalCheckedException(
+                    "Failed to stop serving file [file=" + filePath + ", delete=" + clean + "]",
+                    e
+            );
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public void close() throws IOException {
+        stop0(false);
+    }
+
+    /**
+     * Reads a page.
+     *
+     * @param pageId Page ID.
+     * @param pageOff Page offset in the file.
+     * @param pageBuf Page buffer to read into.
+     * @param keepCrc By default, reading zeroes CRC which was on page store, but you can keep it in {@code pageBuf} if set {@code
+     * keepCrc}.
+     * @return {@code True} if the page was read successfully.
+     * @throws IgniteInternalCheckedException If reading failed (IO error occurred).
+     */
+    public boolean read(long pageId, long pageOff, ByteBuffer pageBuf, boolean keepCrc) throws IgniteInternalCheckedException {
+        return read0(pageId, pageOff, pageBuf, !skipCrc, keepCrc);
+    }
+
+    /**
+     * Writes a page.
+     *
+     * @param pageId Page ID.
+     * @param pageBuf Page buffer to write from.
+     * @param calculateCrc If {@code false} crc calculation will be forcibly skipped.
+     * @throws IgniteInternalCheckedException If page writing failed (IO error occurred).
+     */
+    public void write(long pageId, ByteBuffer pageBuf, boolean calculateCrc) throws IgniteInternalCheckedException {
+        ensure();
+
+        boolean interrupted = false;
+
+        while (true) {
+            FileIo fileIo = this.fileIo;
+
+            try {
+                readWriteLock.readLock().lock();
+
+                try {
+                    assert pageBuf.position() == 0 : pageBuf.position();
+                    assert pageBuf.order() == nativeOrder() : "Page buffer order " + pageBuf.order()
+                            + " should be same with " + nativeOrder();
+                    assert PageIo.getType(pageBuf) != 0 : "Invalid state. Type is 0! pageId = " + hexLong(pageId);
+                    assert PageIo.getVersion(pageBuf) != 0 : "Invalid state. Version is 0! pageId = " + hexLong(pageId);
+
+                    if (calculateCrc && !skipCrc) {
+                        assert PageIo.getCrc(pageBuf) == 0 : hexLong(pageId);
+
+                        PageIo.setCrc(pageBuf, calcCrc32(pageBuf, pageSize()));
+                    }
+
+                    // Check whether crc was calculated somewhere above the stack if it is forcibly skipped.
+                    assert skipCrc || PageIo.getCrc(pageBuf) != 0
+                            || calcCrc32(pageBuf, pageSize()) == 0 : "CRC hasn't been calculated, crc=0";
+
+                    assert pageBuf.position() == 0 : pageBuf.position();
+
+                    long pageOff = pageOffset(pageId);
+
+                    fileIo.writeFully(pageBuf, pageOff);
+
+                    PageIo.setCrc(pageBuf, 0);
+
+                    if (interrupted) {
+                        Thread.currentThread().interrupt();
+                    }
+
+                    return;
+                } finally {
+                    readWriteLock.readLock().unlock();
+                }
+            } catch (IOException e) {
+                if (e instanceof ClosedChannelException) {
+                    try {
+                        if (e instanceof ClosedByInterruptException) {
+                            interrupted = true;
+
+                            Thread.interrupted();
+                        }
+
+                        reinit(fileIo);
+
+                        pageBuf.position(0);
+
+                        PageIo.setCrc(pageBuf, 0);
+
+                        continue;
+                    } catch (IOException e0) {
+                        e0.addSuppressed(e);
+
+                        e = e0;
+                    }
+                }
+
+                throw new IgniteInternalCheckedException(
+                        "Failed to write page [filePath=" + filePath + ", pageId=" + pageId + "]",
+                        e
+                );
+            }
+        }
+    }
+
+    /**
+     * Sync method used to ensure that the given pages are guaranteed to be written to the file page store.
+     *
+     * @throws IgniteInternalCheckedException If sync failed (IO error occurred).
+     */
+    public void sync() throws IgniteInternalCheckedException {
+        readWriteLock.writeLock().lock();

Review Comment:
   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] tkalkirill commented on a diff in pull request #947: IGNITE-17372 Implement DeltaFilePageStore

Posted by GitBox <gi...@apache.org>.
tkalkirill commented on code in PR #947:
URL: https://github.com/apache/ignite-3/pull/947#discussion_r928727700


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/CompleteCreationDeltaFilePageStoreIoCallback.java:
##########
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.pagememory.persistence.store;
+
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+
+/**
+ * Callback on completion of delta file page store creation.
+ */
+@FunctionalInterface
+public interface CompleteCreationDeltaFilePageStoreIoCallback {
+    /**
+     * Calls when the delta file page store is finished creating.
+     *
+     * @param deltaFilePageStoreIo Delta file page store.
+     * @throws IgniteInternalCheckedException If failed.
+     */
+    void onCompletionOfCreation(DeltaFilePageStoreIo deltaFilePageStoreIo) throws IgniteInternalCheckedException;

Review Comment:
   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 merged pull request #947: IGNITE-17372 Implement DeltaFilePageStore

Posted by GitBox <gi...@apache.org>.
ibessonov merged PR #947:
URL: https://github.com/apache/ignite-3/pull/947


-- 
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 #947: IGNITE-17372 Implement DeltaFilePageStore

Posted by GitBox <gi...@apache.org>.
tkalkirill commented on code in PR #947:
URL: https://github.com/apache/ignite-3/pull/947#discussion_r928616827


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStore.java:
##########
@@ -17,122 +17,108 @@
 
 package org.apache.ignite.internal.pagememory.persistence.store;
 
-import static java.nio.ByteOrder.nativeOrder;
-import static java.nio.file.StandardOpenOption.CREATE;
-import static java.nio.file.StandardOpenOption.READ;
-import static java.nio.file.StandardOpenOption.WRITE;
-import static org.apache.ignite.internal.util.IgniteUtils.hexInt;
-import static org.apache.ignite.internal.util.IgniteUtils.hexLong;
-import static org.apache.ignite.internal.util.IgniteUtils.toHexString;
-import static org.apache.ignite.lang.IgniteSystemProperties.getBoolean;
+import static org.apache.ignite.internal.pagememory.util.PageIdUtils.pageIndex;
 
 import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.lang.invoke.VarHandle;
 import java.nio.ByteBuffer;
-import java.nio.channels.ClosedByInterruptException;
-import java.nio.channels.ClosedChannelException;
-import java.nio.file.Files;
 import java.nio.file.Path;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.locks.ReadWriteLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.function.Supplier;
 import org.apache.ignite.internal.fileio.FileIo;
-import org.apache.ignite.internal.fileio.FileIoFactory;
-import org.apache.ignite.internal.pagememory.io.PageIo;
-import org.apache.ignite.internal.pagememory.persistence.FastCrc;
-import org.apache.ignite.internal.pagememory.persistence.IgniteInternalDataIntegrityViolationException;
-import org.apache.ignite.internal.pagememory.util.PageIdUtils;
 import org.apache.ignite.lang.IgniteInternalCheckedException;
 import org.jetbrains.annotations.Nullable;
 
 /**
  * FilePageStore is a {@link PageStore} implementation that uses regular files to store pages.
  *
- * <p>Actual read and write operations are performed with {@link FileIo} abstract interface, list of its implementations is a good source
- * of information about functionality in Ignite Native Persistence.
+ * <p>It consists of the main file page store and delta file page stores, when reading the page at the beginning, the page is searched in
+ * the delta files and only then in the main file.
  *
  * <p>On a physical level each instance of {@code FilePageStore} corresponds to a partition file assigned to the local node.
  *
- * <p>Consists of:
- * <ul>
- *     <li>Header - {@link FilePageStoreHeader}. </li>
- *     <li>Body - data pages are multiples of {@link FilePageStoreHeader#pageSize() pageSize}.</li>
- * </ul>
+ * <p>Actual read and write operations are performed with {@link FileIo} abstract interface, list of its implementations is a good source

Review Comment:
   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] tkalkirill commented on a diff in pull request #947: IGNITE-17372 Implement DeltaFilePageStore

Posted by GitBox <gi...@apache.org>.
tkalkirill commented on code in PR #947:
URL: https://github.com/apache/ignite-3/pull/947#discussion_r928509862


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/AbstractFilePageStoreIo.java:
##########
@@ -0,0 +1,611 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.pagememory.persistence.store;
+
+import static java.nio.ByteOrder.nativeOrder;
+import static java.nio.file.StandardOpenOption.CREATE;
+import static java.nio.file.StandardOpenOption.READ;
+import static java.nio.file.StandardOpenOption.WRITE;
+import static org.apache.ignite.internal.util.IgniteUtils.atomicMoveFile;
+import static org.apache.ignite.internal.util.IgniteUtils.hexInt;
+import static org.apache.ignite.internal.util.IgniteUtils.hexLong;
+import static org.apache.ignite.internal.util.IgniteUtils.toHexString;
+import static org.apache.ignite.lang.IgniteSystemProperties.getBoolean;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.ClosedByInterruptException;
+import java.nio.channels.ClosedChannelException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.apache.ignite.internal.fileio.FileIo;
+import org.apache.ignite.internal.fileio.FileIoFactory;
+import org.apache.ignite.internal.pagememory.io.PageIo;
+import org.apache.ignite.internal.pagememory.persistence.FastCrc;
+import org.apache.ignite.internal.pagememory.persistence.IgniteInternalDataIntegrityViolationException;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Abstract class for performing IO operations on file page storage.
+ */
+public abstract class AbstractFilePageStoreIo implements Closeable {
+    private final FileIoFactory ioFactory;
+
+    private final ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
+
+    /** Skip CRC calculation flag. */
+    // TODO: IGNITE-17011 Move to config
+    private final boolean skipCrc = getBoolean("IGNITE_PDS_SKIP_CRC");
+
+    private volatile Path filePath;

Review Comment:
   It is also used in exceptions, I can get around this by adding the use of locks and a local variable. WDYT?



-- 
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 #947: IGNITE-17372 Implement DeltaFilePageStore

Posted by GitBox <gi...@apache.org>.
tkalkirill commented on code in PR #947:
URL: https://github.com/apache/ignite-3/pull/947#discussion_r928516825


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/AbstractFilePageStoreIo.java:
##########
@@ -0,0 +1,611 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.pagememory.persistence.store;
+
+import static java.nio.ByteOrder.nativeOrder;
+import static java.nio.file.StandardOpenOption.CREATE;
+import static java.nio.file.StandardOpenOption.READ;
+import static java.nio.file.StandardOpenOption.WRITE;
+import static org.apache.ignite.internal.util.IgniteUtils.atomicMoveFile;
+import static org.apache.ignite.internal.util.IgniteUtils.hexInt;
+import static org.apache.ignite.internal.util.IgniteUtils.hexLong;
+import static org.apache.ignite.internal.util.IgniteUtils.toHexString;
+import static org.apache.ignite.lang.IgniteSystemProperties.getBoolean;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.ClosedByInterruptException;
+import java.nio.channels.ClosedChannelException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.apache.ignite.internal.fileio.FileIo;
+import org.apache.ignite.internal.fileio.FileIoFactory;
+import org.apache.ignite.internal.pagememory.io.PageIo;
+import org.apache.ignite.internal.pagememory.persistence.FastCrc;
+import org.apache.ignite.internal.pagememory.persistence.IgniteInternalDataIntegrityViolationException;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Abstract class for performing IO operations on file page storage.
+ */
+public abstract class AbstractFilePageStoreIo implements Closeable {
+    private final FileIoFactory ioFactory;
+
+    private final ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
+
+    /** Skip CRC calculation flag. */
+    // TODO: IGNITE-17011 Move to config
+    private final boolean skipCrc = getBoolean("IGNITE_PDS_SKIP_CRC");
+
+    private volatile Path filePath;
+
+    private volatile @Nullable FileIo fileIo;
+
+    /** Initialized file page store IO. */
+    private volatile boolean initialized;
+
+    /** Caches the existence state of file. After it is initialized, it will be not {@code null} during lifecycle. */
+    private volatile @Nullable Boolean fileExists;
+
+    /**
+     * Constructor.
+     *
+     * @param ioFactory {@link FileIo} factory.
+     * @param filePath File page store path.
+     */
+    AbstractFilePageStoreIo(FileIoFactory ioFactory, Path filePath) {
+        this.ioFactory = ioFactory;
+        this.filePath = filePath;
+    }
+
+    /**
+     * Returns the page size in bytes.
+     */
+    public abstract int pageSize();

Review Comment:
   In general, we store this information in the headers, I would not like to duplicate this in the class itself.



-- 
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 #947: IGNITE-17372 Implement DeltaFilePageStore

Posted by GitBox <gi...@apache.org>.
tkalkirill commented on code in PR #947:
URL: https://github.com/apache/ignite-3/pull/947#discussion_r928594610


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/AbstractFilePageStoreIo.java:
##########
@@ -0,0 +1,611 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.pagememory.persistence.store;
+
+import static java.nio.ByteOrder.nativeOrder;
+import static java.nio.file.StandardOpenOption.CREATE;
+import static java.nio.file.StandardOpenOption.READ;
+import static java.nio.file.StandardOpenOption.WRITE;
+import static org.apache.ignite.internal.util.IgniteUtils.atomicMoveFile;
+import static org.apache.ignite.internal.util.IgniteUtils.hexInt;
+import static org.apache.ignite.internal.util.IgniteUtils.hexLong;
+import static org.apache.ignite.internal.util.IgniteUtils.toHexString;
+import static org.apache.ignite.lang.IgniteSystemProperties.getBoolean;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.ClosedByInterruptException;
+import java.nio.channels.ClosedChannelException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.apache.ignite.internal.fileio.FileIo;
+import org.apache.ignite.internal.fileio.FileIoFactory;
+import org.apache.ignite.internal.pagememory.io.PageIo;
+import org.apache.ignite.internal.pagememory.persistence.FastCrc;
+import org.apache.ignite.internal.pagememory.persistence.IgniteInternalDataIntegrityViolationException;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Abstract class for performing IO operations on file page storage.
+ */
+public abstract class AbstractFilePageStoreIo implements Closeable {
+    private final FileIoFactory ioFactory;
+
+    private final ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
+
+    /** Skip CRC calculation flag. */
+    // TODO: IGNITE-17011 Move to config
+    private final boolean skipCrc = getBoolean("IGNITE_PDS_SKIP_CRC");
+
+    private volatile Path filePath;
+
+    private volatile @Nullable FileIo fileIo;
+
+    /** Initialized file page store IO. */
+    private volatile boolean initialized;
+
+    /** Caches the existence state of file. After it is initialized, it will be not {@code null} during lifecycle. */
+    private volatile @Nullable Boolean fileExists;
+
+    /**
+     * Constructor.
+     *
+     * @param ioFactory {@link FileIo} factory.
+     * @param filePath File page store path.
+     */
+    AbstractFilePageStoreIo(FileIoFactory ioFactory, Path filePath) {
+        this.ioFactory = ioFactory;
+        this.filePath = filePath;
+    }
+
+    /**
+     * Returns the page size in bytes.
+     */
+    public abstract int pageSize();
+
+    /**
+     * Returns the size of the header in bytes.
+     */
+    public abstract int headerSize();
+
+    /**
+     * Returns a buffer with a file page storage header.
+     */
+    public abstract ByteBuffer headerBuffer();
+
+    /**
+     * Checks the file page storage header.
+     *
+     * @param fileIo File page store IO to read the header.
+     * @throws IOException If there is an error reading the header or the header did not pass the check.
+     */
+    public abstract void checkHeader(FileIo fileIo) throws IOException;
+
+    /**
+     * Returns page offset within the store file.
+     *
+     * @param pageId Page ID.
+     */
+    public abstract long pageOffset(long pageId);

Review Comment:
   I tried to do it as internal, it is inconvenient to use when testing.



-- 
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 #947: IGNITE-17372 Implement DeltaFilePageStore

Posted by GitBox <gi...@apache.org>.
tkalkirill commented on code in PR #947:
URL: https://github.com/apache/ignite-3/pull/947#discussion_r928610749


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/DeltaFilePageStoreIoFactory.java:
##########
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.pagememory.persistence.store;
+
+/**
+ * Factory for creating {@link DeltaFilePageStoreIo}.
+ */
+@FunctionalInterface
+public interface DeltaFilePageStoreIoFactory {

Review Comment:
   We personally decided that I would delete it.



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/CompleteCreationDeltaFilePageStoreIoCallback.java:
##########
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.pagememory.persistence.store;
+
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+
+/**
+ * Callback on completion of delta file page store creation.
+ */
+@FunctionalInterface
+public interface CompleteCreationDeltaFilePageStoreIoCallback {
+    /**
+     * Calls when the delta file page store is finished creating.
+     *
+     * @param deltaFilePageStoreIo Delta file page store.
+     * @throws IgniteInternalCheckedException If failed.
+     */
+    void onCompletionOfCreation(DeltaFilePageStoreIo deltaFilePageStoreIo) throws IgniteInternalCheckedException;

Review Comment:
   We personally decided that I would delete 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] tkalkirill commented on a diff in pull request #947: IGNITE-17372 Implement DeltaFilePageStore

Posted by GitBox <gi...@apache.org>.
tkalkirill commented on code in PR #947:
URL: https://github.com/apache/ignite-3/pull/947#discussion_r928618638


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStore.java:
##########
@@ -141,7 +127,7 @@ public void stop(boolean clean) throws IgniteInternalCheckedException {
     public int allocatePage() throws IgniteInternalCheckedException {
         ensure();
 
-        int pageIdx = pageCount.getAndIncrement();
+        int pageIdx = (Integer) PAGE_COUNT.getAndAdd(this, 1);

Review Comment:
   Fix it



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStore.java:
##########
@@ -166,503 +152,196 @@ public int pages() {
     public void pages(int pageCount) {
         assert pageCount >= 0 : pageCount;
 
-        this.pageCount.set(pageCount);
+        this.pageCount = pageCount;
     }
 
     /**
-     * Reads a page, unlike {@link #read(long, ByteBuffer, boolean)}, checks the page offset in the file not logically (pageOffset <= {@link
-     * #pages()} * {@link #pageSize}) but physically (pageOffset <= {@link #size()}), which can affect performance when used in production
-     * code.
+     * Reads a page, unlike {@link #read(long, ByteBuffer, boolean)}, does not check the {@code pageId} so that its {@code pageIdx} is not
+     * greater than the {@link #pages() number of allocated pages}.
      *
      * @param pageId Page ID.
      * @param pageBuf Page buffer to read into.
      * @param keepCrc By default, reading zeroes CRC which was on page store, but you can keep it in {@code pageBuf} if set {@code
      * keepCrc}.
+     * @return {@code True} if the page was read successfully.
      * @throws IgniteInternalCheckedException If reading failed (IO error occurred).
      */
-    public void readByPhysicalOffset(long pageId, ByteBuffer pageBuf, boolean keepCrc) throws IgniteInternalCheckedException {
-        read0(pageId, pageBuf, !skipCrc, keepCrc, false);
+    public boolean readWithoutPageIdCheck(long pageId, ByteBuffer pageBuf, boolean keepCrc) throws IgniteInternalCheckedException {
+        for (DeltaFilePageStoreIo deltaFilePageStoreIo : deltaFilePageStoreIos) {
+            long pageOff = deltaFilePageStoreIo.pageOffset(pageId);
+
+            if (pageOff >= 0) {
+                return deltaFilePageStoreIo.read(pageId, pageOff, pageBuf, keepCrc);
+            }
+        }
+
+        return filePageStoreIo.read(pageId, filePageStoreIo.pageOffset(pageId), pageBuf, keepCrc);
     }
 
     /** {@inheritDoc} */
     @Override
     public void read(long pageId, ByteBuffer pageBuf, boolean keepCrc) throws IgniteInternalCheckedException {
-        read0(pageId, pageBuf, !skipCrc, keepCrc, true);
-    }
-
-    /**
-     * Reads a page from the page store.
-     *
-     * @param pageId Page ID.
-     * @param pageBuf Page buffer to read into.
-     * @param checkCrc Check CRC on page.
-     * @param keepCrc By default reading zeroes CRC which was on file, but you can keep it in pageBuf if set keepCrc.
-     * @param checkPageOffsetLogically Check page offset by {@link #allocatedBytes} or {@link #size}.
-     * @throws IgniteInternalCheckedException If reading failed (IO error occurred).
-     */
-    private void read0(
-            long pageId,
-            ByteBuffer pageBuf,
-            boolean checkCrc,
-            boolean keepCrc,
-            boolean checkPageOffsetLogically
-    ) throws IgniteInternalCheckedException {
-        ensure();
-
-        try {
-            assert pageBuf.capacity() == pageSize : pageBuf.capacity();
-            assert pageBuf.remaining() == pageSize : pageBuf.remaining();
-            assert pageBuf.position() == 0 : pageBuf.position();
-            assert pageBuf.order() == nativeOrder() : pageBuf.order();
-
-            long pageOff = pageOffset(pageId);
-
-            if (checkPageOffsetLogically) {
-                assert pageOff <= allocatedBytes() : "calculatedOffset=" + pageOff
-                        + ", allocated=" + allocatedBytes() + ", headerSize=" + headerSize + ", filePath=" + filePath;
-            } else {
-                assert pageOff <= size() : "calculatedOffset=" + pageOff
-                        + ", size=" + size() + ", headerSize=" + headerSize + ", filePath=" + filePath;
-            }
-
-            int n = readWithFailover(pageBuf, pageOff);
+        assert pageIndex(pageId) <= pageCount : "pageIdx=" + pageIndex(pageId) + ", pageCount=" + pageCount;
 
-            // If page was not written yet, nothing to read.
-            if (n < 0) {
-                pageBuf.put(new byte[pageBuf.remaining()]);
-            }
+        for (DeltaFilePageStoreIo deltaFilePageStoreIo : deltaFilePageStoreIos) {
+            long pageOff = deltaFilePageStoreIo.pageOffset(pageId);
 
-            int savedCrc32 = PageIo.getCrc(pageBuf);
-
-            PageIo.setCrc(pageBuf, 0);
-
-            pageBuf.position(0);
-
-            if (checkCrc) {
-                int curCrc32 = FastCrc.calcCrc(pageBuf, pageSize);
-
-                if ((savedCrc32 ^ curCrc32) != 0) {
-                    throw new IgniteInternalDataIntegrityViolationException("Failed to read page (CRC validation failed) "
-                            + "[id=" + hexLong(pageId) + ", off=" + pageOff
-                            + ", filePath=" + filePath + ", fileSize=" + fileIo.size()
-                            + ", savedCrc=" + hexInt(savedCrc32) + ", curCrc=" + hexInt(curCrc32)
-                            + ", page=" + toHexString(pageBuf) + "]");
-                }
-            }
+            if (pageOff >= 0) {
+                deltaFilePageStoreIo.read(pageId, pageOff, pageBuf, keepCrc);
 
-            assert PageIo.getCrc(pageBuf) == 0;
-
-            if (keepCrc) {
-                PageIo.setCrc(pageBuf, savedCrc32);
+                return;
             }
-        } catch (IOException e) {
-            throw new IgniteInternalCheckedException("Failed to read page [file=" + filePath + ", pageId=" + pageId + "]", e);
         }
+
+        filePageStoreIo.read(pageId, filePageStoreIo.pageOffset(pageId), pageBuf, keepCrc);
     }
 
     /** {@inheritDoc} */
     @Override
     public void write(long pageId, ByteBuffer pageBuf, boolean calculateCrc) throws IgniteInternalCheckedException {
-        ensure();
-
-        boolean interrupted = false;
-
-        while (true) {
-            FileIo fileIo = this.fileIo;
-
-            try {
-                readWriteLock.readLock().lock();
-
-                try {
-                    assert pageBuf.position() == 0 : pageBuf.position();
-                    assert pageBuf.order() == nativeOrder() : "Page buffer order " + pageBuf.order()
-                            + " should be same with " + nativeOrder();
-                    assert PageIo.getType(pageBuf) != 0 : "Invalid state. Type is 0! pageId = " + hexLong(pageId);
-                    assert PageIo.getVersion(pageBuf) != 0 : "Invalid state. Version is 0! pageId = " + hexLong(pageId);
-
-                    if (calculateCrc && !skipCrc) {
-                        assert PageIo.getCrc(pageBuf) == 0 : hexLong(pageId);
-
-                        PageIo.setCrc(pageBuf, calcCrc32(pageBuf, pageSize));
-                    }
-
-                    // Check whether crc was calculated somewhere above the stack if it is forcibly skipped.
-                    assert skipCrc || PageIo.getCrc(pageBuf) != 0
-                            || calcCrc32(pageBuf, pageSize) == 0 : "CRC hasn't been calculated, crc=0";
-
-                    assert pageBuf.position() == 0 : pageBuf.position();
-
-                    long pageOff = pageOffset(pageId);
-
-                    assert pageOff <= allocatedBytes() : "calculatedOffset=" + pageOff
-                            + ", allocated=" + allocatedBytes() + ", headerSize=" + headerSize + ", filePath=" + filePath;
-
-                    fileIo.writeFully(pageBuf, pageOff);
-
-                    PageIo.setCrc(pageBuf, 0);
-
-                    if (interrupted) {
-                        Thread.currentThread().interrupt();
-                    }
-
-                    return;
-                } finally {
-                    readWriteLock.readLock().unlock();
-                }
-            } catch (IOException e) {
-                if (e instanceof ClosedChannelException) {
-                    try {
-                        if (e instanceof ClosedByInterruptException) {
-                            interrupted = true;
-
-                            Thread.interrupted();
-                        }
-
-                        reinit(fileIo);
-
-                        pageBuf.position(0);
-
-                        PageIo.setCrc(pageBuf, 0);
+        assert pageIndex(pageId) <= pageCount : "pageIdx=" + pageIndex(pageId) + ", pageCount=" + pageCount;
 
-                        continue;
-                    } catch (IOException e0) {
-                        e0.addSuppressed(e);
-
-                        e = e0;
-                    }
-                }
-
-                throw new IgniteInternalCheckedException(
-                        "Failed to write page [filePath=" + filePath + ", pageId=" + pageId + "]",
-                        e
-                );
-            }
-        }
+        filePageStoreIo.write(pageId, pageBuf, calculateCrc);
     }
 
     /** {@inheritDoc} */
     @Override
     public void sync() throws IgniteInternalCheckedException {
-        readWriteLock.writeLock().lock();
-
-        try {
-            ensure();
-
-            FileIo fileIo = this.fileIo;
-
-            if (fileIo != null) {
-                fileIo.force();
-            }
-        } catch (IOException e) {
-            throw new IgniteInternalCheckedException("Failed to fsync partition file [filePath=" + filePath + ']', e);
-        } finally {
-            readWriteLock.writeLock().unlock();
-        }
+        filePageStoreIo.sync();
     }
 
     /** {@inheritDoc} */
     @Override
     public boolean exists() {
-        if (fileExists == null) {
-            readWriteLock.writeLock().lock();
-
-            try {
-                if (fileExists == null) {
-                    fileExists = Files.exists(filePath) && filePath.toFile().length() >= headerSize;
-                }
-            } finally {
-                readWriteLock.writeLock().unlock();
-            }
-        }
-
-        return fileExists;
+        return filePageStoreIo.exists();
     }
 
     /** {@inheritDoc} */
     @Override
     public void ensure() throws IgniteInternalCheckedException {
-        if (!initialized) {
-            readWriteLock.writeLock().lock();
-
-            try {
-                if (!initialized) {
-                    FileIo fileIo = null;
-
-                    IgniteInternalCheckedException err = null;
-
-                    try {
-                        boolean interrupted = false;
-
-                        while (true) {
-                            try {
-                                this.fileIo = fileIo = ioFactory.create(filePath, CREATE, READ, WRITE);
-
-                                fileExists = true;
-
-                                if (fileIo.size() < headerSize) {
-                                    fileIo.writeFully(new FilePageStoreHeader(version, pageSize).toByteBuffer(), 0);
-                                } else {
-                                    checkHeader(fileIo);
-                                }
-
-                                if (interrupted) {
-                                    Thread.currentThread().interrupt();
-                                }
-
-                                break;
-                            } catch (ClosedByInterruptException e) {
-                                interrupted = true;
-
-                                Thread.interrupted();
-                            }
-                        }
-
-                        initialized = true;
-                    } catch (IOException e) {
-                        err = new IgniteInternalCheckedException("Failed to initialize partition file: " + filePath, e);
-
-                        throw err;
-                    } finally {
-                        if (err != null && fileIo != null) {
-                            try {
-                                fileIo.close();
-                            } catch (IOException e) {
-                                err.addSuppressed(e);
-                            }
-                        }
-                    }
-                }
-            } finally {
-                readWriteLock.writeLock().unlock();
-            }
-        }
+        filePageStoreIo.ensure();
     }
 
     /** {@inheritDoc} */
     @Override
     public void close() throws IOException {
-        stop0(false);
+        filePageStoreIo.close();
+
+        for (DeltaFilePageStoreIo deltaFilePageStoreIo : deltaFilePageStoreIos) {
+            deltaFilePageStoreIo.close();
+        }
     }
 
     /**
      * Returns size of the page store in bytes.
      *
-     * <p>May differ from {@link #pages} * {@link #pageSize} due to delayed writes or due to other implementation specific details.
+     * <p>May differ from {@link #pages} * {@link FilePageStoreIo#pageSize()} due to delayed writes or due to other implementation specific
+     * details.
      *
      * @throws IgniteInternalCheckedException If an I/O error occurs.
      */
     public long size() throws IgniteInternalCheckedException {
-        try {
-            FileIo io = fileIo;
-
-            return io == null ? 0 : io.size();
-        } catch (IOException e) {
-            throw new IgniteInternalCheckedException(e);
-        }
+        return filePageStoreIo.size();
     }
 
     /**
-     * Stops file page store.
-     *
-     * @param delete {@code True} to delete file.
-     * @throws IOException If fails.
+     * Returns file page store path.
      */
-    private void stop0(boolean delete) throws IOException {
-        readWriteLock.writeLock().lock();
-
-        try {
-            if (!initialized) {
-                // Ensure the file is closed even if not initialized yet.
-                if (fileIo != null) {
-                    fileIo.close();
-                }
-
-                if (delete && exists()) {
-                    Files.delete(filePath);
-                }
-
-                return;
-            }
-
-            fileIo.force();
-
-            fileIo.close();
-
-            fileIo = null;
-
-            if (delete) {
-                Files.delete(filePath);
-
-                fileExists = false;
-            }
-        } finally {
-            initialized = false;
+    public Path filePath() {

Review Comment:
   Yes



-- 
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 #947: IGNITE-17372 Implement DeltaFilePageStore

Posted by GitBox <gi...@apache.org>.
tkalkirill commented on code in PR #947:
URL: https://github.com/apache/ignite-3/pull/947#discussion_r928504324


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/PartitionMetaManager.java:
##########
@@ -95,10 +95,8 @@ public PartitionMeta readOrCreateMeta(
         long partitionMetaPageId = partitionMetaPageId(groupPartitionId.getPartitionId());
 
         try {
-            if (filePageStore.size() > filePageStore.headerSize()) {
+            if (filePageStore.readWithoutPageIdCheck(partitionMetaPageId, buffer, false)) {

Review Comment:
   Perhaps ideally the code should have looked something like this:
   ```
               if (filePageStore.contains(partitionMetaPageId)) {
                   filePageStore.read(partitionMetaPageId, buffer, false);
   
                   return new PartitionMeta(checkpointId, ioRegistry.resolve(bufferAddr), bufferAddr);
               } else {
                   PartitionMetaIo io = ...;
   
                   filePageStore.write(partitionMetaPageId, buffer.rewind(), true);
   
                   filePageStore.sync();
   
                   return new PartitionMeta(checkpointId, io, bufferAddr);
               }
   ```
   But it will always go on a false branch, because **contains** will have to check it by the number of pages, which we can only find out about by reading the first page, I don’t want to write the logic for **PartitionMetaIo** inside, I think this is not correct.
   
   We can personally discuss the solution and come up with a better API, but so far it seems to be more compromise.



-- 
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 #947: IGNITE-17372 Implement DeltaFilePageStore

Posted by GitBox <gi...@apache.org>.
tkalkirill commented on code in PR #947:
URL: https://github.com/apache/ignite-3/pull/947#discussion_r928590384


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/AbstractFilePageStoreIo.java:
##########
@@ -0,0 +1,611 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.pagememory.persistence.store;
+
+import static java.nio.ByteOrder.nativeOrder;
+import static java.nio.file.StandardOpenOption.CREATE;
+import static java.nio.file.StandardOpenOption.READ;
+import static java.nio.file.StandardOpenOption.WRITE;
+import static org.apache.ignite.internal.util.IgniteUtils.atomicMoveFile;
+import static org.apache.ignite.internal.util.IgniteUtils.hexInt;
+import static org.apache.ignite.internal.util.IgniteUtils.hexLong;
+import static org.apache.ignite.internal.util.IgniteUtils.toHexString;
+import static org.apache.ignite.lang.IgniteSystemProperties.getBoolean;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.ClosedByInterruptException;
+import java.nio.channels.ClosedChannelException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.apache.ignite.internal.fileio.FileIo;
+import org.apache.ignite.internal.fileio.FileIoFactory;
+import org.apache.ignite.internal.pagememory.io.PageIo;
+import org.apache.ignite.internal.pagememory.persistence.FastCrc;
+import org.apache.ignite.internal.pagememory.persistence.IgniteInternalDataIntegrityViolationException;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Abstract class for performing IO operations on file page storage.
+ */
+public abstract class AbstractFilePageStoreIo implements Closeable {
+    private final FileIoFactory ioFactory;
+
+    private final ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
+
+    /** Skip CRC calculation flag. */
+    // TODO: IGNITE-17011 Move to config
+    private final boolean skipCrc = getBoolean("IGNITE_PDS_SKIP_CRC");
+
+    private volatile Path filePath;
+
+    private volatile @Nullable FileIo fileIo;
+
+    /** Initialized file page store IO. */
+    private volatile boolean initialized;
+
+    /** Caches the existence state of file. After it is initialized, it will be not {@code null} during lifecycle. */
+    private volatile @Nullable Boolean fileExists;

Review Comment:
   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] tkalkirill commented on a diff in pull request #947: IGNITE-17372 Implement DeltaFilePageStore

Posted by GitBox <gi...@apache.org>.
tkalkirill commented on code in PR #947:
URL: https://github.com/apache/ignite-3/pull/947#discussion_r928620902


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStore.java:
##########
@@ -166,503 +152,196 @@ public int pages() {
     public void pages(int pageCount) {
         assert pageCount >= 0 : pageCount;
 
-        this.pageCount.set(pageCount);
+        this.pageCount = pageCount;
     }
 
     /**
-     * Reads a page, unlike {@link #read(long, ByteBuffer, boolean)}, checks the page offset in the file not logically (pageOffset <= {@link
-     * #pages()} * {@link #pageSize}) but physically (pageOffset <= {@link #size()}), which can affect performance when used in production
-     * code.
+     * Reads a page, unlike {@link #read(long, ByteBuffer, boolean)}, does not check the {@code pageId} so that its {@code pageIdx} is not
+     * greater than the {@link #pages() number of allocated pages}.
      *
      * @param pageId Page ID.
      * @param pageBuf Page buffer to read into.
      * @param keepCrc By default, reading zeroes CRC which was on page store, but you can keep it in {@code pageBuf} if set {@code
      * keepCrc}.
+     * @return {@code True} if the page was read successfully.
      * @throws IgniteInternalCheckedException If reading failed (IO error occurred).
      */
-    public void readByPhysicalOffset(long pageId, ByteBuffer pageBuf, boolean keepCrc) throws IgniteInternalCheckedException {
-        read0(pageId, pageBuf, !skipCrc, keepCrc, false);
+    public boolean readWithoutPageIdCheck(long pageId, ByteBuffer pageBuf, boolean keepCrc) throws IgniteInternalCheckedException {
+        for (DeltaFilePageStoreIo deltaFilePageStoreIo : deltaFilePageStoreIos) {
+            long pageOff = deltaFilePageStoreIo.pageOffset(pageId);
+
+            if (pageOff >= 0) {
+                return deltaFilePageStoreIo.read(pageId, pageOff, pageBuf, keepCrc);
+            }
+        }
+
+        return filePageStoreIo.read(pageId, filePageStoreIo.pageOffset(pageId), pageBuf, keepCrc);
     }
 
     /** {@inheritDoc} */
     @Override
     public void read(long pageId, ByteBuffer pageBuf, boolean keepCrc) throws IgniteInternalCheckedException {
-        read0(pageId, pageBuf, !skipCrc, keepCrc, true);
-    }
-
-    /**
-     * Reads a page from the page store.
-     *
-     * @param pageId Page ID.
-     * @param pageBuf Page buffer to read into.
-     * @param checkCrc Check CRC on page.
-     * @param keepCrc By default reading zeroes CRC which was on file, but you can keep it in pageBuf if set keepCrc.
-     * @param checkPageOffsetLogically Check page offset by {@link #allocatedBytes} or {@link #size}.
-     * @throws IgniteInternalCheckedException If reading failed (IO error occurred).
-     */
-    private void read0(
-            long pageId,
-            ByteBuffer pageBuf,
-            boolean checkCrc,
-            boolean keepCrc,
-            boolean checkPageOffsetLogically
-    ) throws IgniteInternalCheckedException {
-        ensure();
-
-        try {
-            assert pageBuf.capacity() == pageSize : pageBuf.capacity();
-            assert pageBuf.remaining() == pageSize : pageBuf.remaining();
-            assert pageBuf.position() == 0 : pageBuf.position();
-            assert pageBuf.order() == nativeOrder() : pageBuf.order();
-
-            long pageOff = pageOffset(pageId);
-
-            if (checkPageOffsetLogically) {
-                assert pageOff <= allocatedBytes() : "calculatedOffset=" + pageOff
-                        + ", allocated=" + allocatedBytes() + ", headerSize=" + headerSize + ", filePath=" + filePath;
-            } else {
-                assert pageOff <= size() : "calculatedOffset=" + pageOff
-                        + ", size=" + size() + ", headerSize=" + headerSize + ", filePath=" + filePath;
-            }
-
-            int n = readWithFailover(pageBuf, pageOff);
+        assert pageIndex(pageId) <= pageCount : "pageIdx=" + pageIndex(pageId) + ", pageCount=" + pageCount;
 
-            // If page was not written yet, nothing to read.
-            if (n < 0) {
-                pageBuf.put(new byte[pageBuf.remaining()]);
-            }
+        for (DeltaFilePageStoreIo deltaFilePageStoreIo : deltaFilePageStoreIos) {
+            long pageOff = deltaFilePageStoreIo.pageOffset(pageId);
 
-            int savedCrc32 = PageIo.getCrc(pageBuf);
-
-            PageIo.setCrc(pageBuf, 0);
-
-            pageBuf.position(0);
-
-            if (checkCrc) {
-                int curCrc32 = FastCrc.calcCrc(pageBuf, pageSize);
-
-                if ((savedCrc32 ^ curCrc32) != 0) {
-                    throw new IgniteInternalDataIntegrityViolationException("Failed to read page (CRC validation failed) "
-                            + "[id=" + hexLong(pageId) + ", off=" + pageOff
-                            + ", filePath=" + filePath + ", fileSize=" + fileIo.size()
-                            + ", savedCrc=" + hexInt(savedCrc32) + ", curCrc=" + hexInt(curCrc32)
-                            + ", page=" + toHexString(pageBuf) + "]");
-                }
-            }
+            if (pageOff >= 0) {
+                deltaFilePageStoreIo.read(pageId, pageOff, pageBuf, keepCrc);
 
-            assert PageIo.getCrc(pageBuf) == 0;
-
-            if (keepCrc) {
-                PageIo.setCrc(pageBuf, savedCrc32);
+                return;
             }
-        } catch (IOException e) {
-            throw new IgniteInternalCheckedException("Failed to read page [file=" + filePath + ", pageId=" + pageId + "]", e);
         }
+
+        filePageStoreIo.read(pageId, filePageStoreIo.pageOffset(pageId), pageBuf, keepCrc);
     }
 
     /** {@inheritDoc} */
     @Override
     public void write(long pageId, ByteBuffer pageBuf, boolean calculateCrc) throws IgniteInternalCheckedException {
-        ensure();
-
-        boolean interrupted = false;
-
-        while (true) {
-            FileIo fileIo = this.fileIo;
-
-            try {
-                readWriteLock.readLock().lock();
-
-                try {
-                    assert pageBuf.position() == 0 : pageBuf.position();
-                    assert pageBuf.order() == nativeOrder() : "Page buffer order " + pageBuf.order()
-                            + " should be same with " + nativeOrder();
-                    assert PageIo.getType(pageBuf) != 0 : "Invalid state. Type is 0! pageId = " + hexLong(pageId);
-                    assert PageIo.getVersion(pageBuf) != 0 : "Invalid state. Version is 0! pageId = " + hexLong(pageId);
-
-                    if (calculateCrc && !skipCrc) {
-                        assert PageIo.getCrc(pageBuf) == 0 : hexLong(pageId);
-
-                        PageIo.setCrc(pageBuf, calcCrc32(pageBuf, pageSize));
-                    }
-
-                    // Check whether crc was calculated somewhere above the stack if it is forcibly skipped.
-                    assert skipCrc || PageIo.getCrc(pageBuf) != 0
-                            || calcCrc32(pageBuf, pageSize) == 0 : "CRC hasn't been calculated, crc=0";
-
-                    assert pageBuf.position() == 0 : pageBuf.position();
-
-                    long pageOff = pageOffset(pageId);
-
-                    assert pageOff <= allocatedBytes() : "calculatedOffset=" + pageOff
-                            + ", allocated=" + allocatedBytes() + ", headerSize=" + headerSize + ", filePath=" + filePath;
-
-                    fileIo.writeFully(pageBuf, pageOff);
-
-                    PageIo.setCrc(pageBuf, 0);
-
-                    if (interrupted) {
-                        Thread.currentThread().interrupt();
-                    }
-
-                    return;
-                } finally {
-                    readWriteLock.readLock().unlock();
-                }
-            } catch (IOException e) {
-                if (e instanceof ClosedChannelException) {
-                    try {
-                        if (e instanceof ClosedByInterruptException) {
-                            interrupted = true;
-
-                            Thread.interrupted();
-                        }
-
-                        reinit(fileIo);
-
-                        pageBuf.position(0);
-
-                        PageIo.setCrc(pageBuf, 0);
+        assert pageIndex(pageId) <= pageCount : "pageIdx=" + pageIndex(pageId) + ", pageCount=" + pageCount;
 
-                        continue;
-                    } catch (IOException e0) {
-                        e0.addSuppressed(e);
-
-                        e = e0;
-                    }
-                }
-
-                throw new IgniteInternalCheckedException(
-                        "Failed to write page [filePath=" + filePath + ", pageId=" + pageId + "]",
-                        e
-                );
-            }
-        }
+        filePageStoreIo.write(pageId, pageBuf, calculateCrc);
     }
 
     /** {@inheritDoc} */
     @Override
     public void sync() throws IgniteInternalCheckedException {
-        readWriteLock.writeLock().lock();
-
-        try {
-            ensure();
-
-            FileIo fileIo = this.fileIo;
-
-            if (fileIo != null) {
-                fileIo.force();
-            }
-        } catch (IOException e) {
-            throw new IgniteInternalCheckedException("Failed to fsync partition file [filePath=" + filePath + ']', e);
-        } finally {
-            readWriteLock.writeLock().unlock();
-        }
+        filePageStoreIo.sync();
     }
 
     /** {@inheritDoc} */
     @Override
     public boolean exists() {
-        if (fileExists == null) {
-            readWriteLock.writeLock().lock();
-
-            try {
-                if (fileExists == null) {
-                    fileExists = Files.exists(filePath) && filePath.toFile().length() >= headerSize;
-                }
-            } finally {
-                readWriteLock.writeLock().unlock();
-            }
-        }
-
-        return fileExists;
+        return filePageStoreIo.exists();
     }
 
     /** {@inheritDoc} */
     @Override
     public void ensure() throws IgniteInternalCheckedException {
-        if (!initialized) {
-            readWriteLock.writeLock().lock();
-
-            try {
-                if (!initialized) {
-                    FileIo fileIo = null;
-
-                    IgniteInternalCheckedException err = null;
-
-                    try {
-                        boolean interrupted = false;
-
-                        while (true) {
-                            try {
-                                this.fileIo = fileIo = ioFactory.create(filePath, CREATE, READ, WRITE);
-
-                                fileExists = true;
-
-                                if (fileIo.size() < headerSize) {
-                                    fileIo.writeFully(new FilePageStoreHeader(version, pageSize).toByteBuffer(), 0);
-                                } else {
-                                    checkHeader(fileIo);
-                                }
-
-                                if (interrupted) {
-                                    Thread.currentThread().interrupt();
-                                }
-
-                                break;
-                            } catch (ClosedByInterruptException e) {
-                                interrupted = true;
-
-                                Thread.interrupted();
-                            }
-                        }
-
-                        initialized = true;
-                    } catch (IOException e) {
-                        err = new IgniteInternalCheckedException("Failed to initialize partition file: " + filePath, e);
-
-                        throw err;
-                    } finally {
-                        if (err != null && fileIo != null) {
-                            try {
-                                fileIo.close();
-                            } catch (IOException e) {
-                                err.addSuppressed(e);
-                            }
-                        }
-                    }
-                }
-            } finally {
-                readWriteLock.writeLock().unlock();
-            }
-        }
+        filePageStoreIo.ensure();
     }
 
     /** {@inheritDoc} */
     @Override
     public void close() throws IOException {
-        stop0(false);
+        filePageStoreIo.close();
+
+        for (DeltaFilePageStoreIo deltaFilePageStoreIo : deltaFilePageStoreIos) {
+            deltaFilePageStoreIo.close();
+        }
     }
 
     /**
      * Returns size of the page store in bytes.
      *
-     * <p>May differ from {@link #pages} * {@link #pageSize} due to delayed writes or due to other implementation specific details.
+     * <p>May differ from {@link #pages} * {@link FilePageStoreIo#pageSize()} due to delayed writes or due to other implementation specific
+     * details.
      *
      * @throws IgniteInternalCheckedException If an I/O error occurs.
      */
     public long size() throws IgniteInternalCheckedException {
-        try {
-            FileIo io = fileIo;
-
-            return io == null ? 0 : io.size();
-        } catch (IOException e) {
-            throw new IgniteInternalCheckedException(e);
-        }
+        return filePageStoreIo.size();
     }
 
     /**
-     * Stops file page store.
-     *
-     * @param delete {@code True} to delete file.
-     * @throws IOException If fails.
+     * Returns file page store path.
      */
-    private void stop0(boolean delete) throws IOException {
-        readWriteLock.writeLock().lock();
-
-        try {
-            if (!initialized) {
-                // Ensure the file is closed even if not initialized yet.
-                if (fileIo != null) {
-                    fileIo.close();
-                }
-
-                if (delete && exists()) {
-                    Files.delete(filePath);
-                }
-
-                return;
-            }
-
-            fileIo.force();
-
-            fileIo.close();
-
-            fileIo = null;
-
-            if (delete) {
-                Files.delete(filePath);
-
-                fileExists = false;
-            }
-        } finally {
-            initialized = false;
+    public Path filePath() {
+        return filePageStoreIo.filePath();
+    }
 
-            readWriteLock.writeLock().unlock();
-        }
+    /**
+     * Returns file page store header size.
+     */
+    public int headerSize() {
+        return filePageStoreIo.headerSize();
     }
 
     /**
-     * Gets page offset within the store file.
+     * Sets the new page allocation listener.
      *
-     * @param pageId Page ID.
-     * @return Page offset.
+     * @param listener New page allocation listener.
      */
-    long pageOffset(long pageId) {
-        return (long) PageIdUtils.pageIndex(pageId) * pageSize + headerSize;
+    public void setPageAllocationListener(PageAllocationListener listener) {
+        pageAllocationListener = listener;
     }
 
     /**
-     * Reads from page storage with failover.
+     * Sets the delta file page store factory.
      *
-     * @param destBuf Destination buffer.
-     * @param position Position.
-     * @return Number of read bytes, or {@code -1} if the given position is greater than or equal to the file's current size.
+     * @param factory Factory.
      */
-    private int readWithFailover(ByteBuffer destBuf, long position) throws IOException {
-        boolean interrupted = false;
-
-        int bufPos = destBuf.position();
-
-        while (true) {
-            FileIo fileIo = this.fileIo;
-
-            if (fileIo == null) {
-                throw new IOException("FileIo has stopped");
-            }
-
-            try {
-                assert destBuf.remaining() > 0;
-
-                int bytesRead = fileIo.readFully(destBuf, position);
-
-                if (interrupted) {
-                    Thread.currentThread().interrupt();
-                }
-
-                return bytesRead;
-            } catch (ClosedChannelException e) {
-                destBuf.position(bufPos);
-
-                if (e instanceof ClosedByInterruptException) {
-                    interrupted = true;
-
-                    Thread.interrupted();
-                }
-
-                reinit(fileIo);
-            }
-        }
+    public void setDeltaFilePageStoreIoFactory(DeltaFilePageStoreIoFactory factory) {
+        deltaFilePageStoreIoFactory = factory;
     }
 
     /**
-     * Reinit page store after file channel was closed by thread interruption.
+     * Sets the callback on completion of delta file page store creation.
      *
-     * @param fileIo Old fileIo.
+     * @param callback Callback.
      */
-    private void reinit(FileIo fileIo) throws IOException {
-        if (!initialized) {
-            return;
-        }
-
-        if (fileIo != this.fileIo) {
-            return;
-        }
-
-        readWriteLock.writeLock().lock();
-
-        try {
-            if (fileIo != this.fileIo) {
-                return;
-            }
-
-            try {
-                boolean interrupted = false;
-
-                while (true) {
-                    try {
-                        fileIo = null;
-
-                        fileIo = ioFactory.create(filePath, CREATE, READ, WRITE);
+    public void setCompleteCreationDeltaFilePageStoreIoCallback(CompleteCreationDeltaFilePageStoreIoCallback callback) {
+        completeCreationDeltaFilePageStoreIoCallback = callback;
+    }
 
-                        fileExists = true;
+    /**
+     * Gets or creates a new delta file, a new delta file will be created when the previous one is {@link #completeNewDeltaFile()
+     * completed}.
+     *
+     * <p>Thread safe.
+     *
+     * @param pageIndexesSupplier Page indexes supplier for the new delta file page store.
+     * @return Future that will be completed when the new delta file page store is created.
+     */
+    public CompletableFuture<DeltaFilePageStoreIo> getOrCreateNewDeltaFile(Supplier<int[]> pageIndexesSupplier) {
+        assert deltaFilePageStoreIoFactory != null;
 
-                        checkHeader(fileIo);
+        CompletableFuture<DeltaFilePageStoreIo> future = this.newDeltaFilePageStoreIoFuture;
 
-                        this.fileIo = fileIo;
+        if (future != null) {
+            return future;
+        }
 
-                        if (interrupted) {
-                            Thread.currentThread().interrupt();
-                        }
+        if (!NEW_DELTA_FILE_PAGE_STORE_IO_FUTURE.compareAndSet(this, null, future = new CompletableFuture<>())) {
+            // Another thread started creating a delta file.
+            return newDeltaFilePageStoreIoFuture;
+        }
 
-                        break;
-                    } catch (ClosedByInterruptException e) {
-                        interrupted = true;
+        int nextIndex = deltaFilePageStoreIos.isEmpty() ? 0 : deltaFilePageStoreIos.get(0).fileIndex() + 1;
 
-                        Thread.interrupted();
-                    }
-                }
-            } catch (IOException e) {
-                try {
-                    if (fileIo != null) {
-                        fileIo.close();
-                    }
-                } catch (IOException e0) {
-                    e.addSuppressed(e0);
-                }
+        DeltaFilePageStoreIo deltaFilePageStoreIo = deltaFilePageStoreIoFactory.create(nextIndex, pageIndexesSupplier.get());
 
-                throw e;
-            }
-        } finally {
-            readWriteLock.writeLock().unlock();
-        }
-    }
+        // Should add to the head, since read operations should always start from the most recent.
+        deltaFilePageStoreIos.add(0, deltaFilePageStoreIo);
 
-    private static int calcCrc32(ByteBuffer pageBuf, int pageSize) {
-        try {
-            pageBuf.position(0);
+        future.complete(deltaFilePageStoreIo);

Review Comment:
   Yes



-- 
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 #947: IGNITE-17372 Implement DeltaFilePageStore

Posted by GitBox <gi...@apache.org>.
tkalkirill commented on code in PR #947:
URL: https://github.com/apache/ignite-3/pull/947#discussion_r928626846


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStoreManager.java:
##########
@@ -305,14 +340,99 @@ private List<FilePageStore> createFilePageStores(
 
         try {
             for (int i = 0; i < partitions; i++) {
-                Path partFilePath = groupWorkDir.resolve(String.format(PART_FILE_TEMPLATE, i));
+                int part = i;
+
+                Path partFilePath = groupWorkDir.resolve(String.format(PART_FILE_TEMPLATE, part));
+
+                Path[] partDeltaFiles = findPartitionDeltaFiles(groupWorkDir, partitions);
+
+                FilePageStore filePageStore = filePageStoreFactory.createPageStore(buffer.rewind(), partFilePath, partDeltaFiles);
+
+                filePageStore.setDeltaFilePageStoreIoFactory((index, pageIndexes) -> createLatest(groupWorkDir, part, index, pageIndexes));
+
+                filePageStore.setCompleteCreationDeltaFilePageStoreIoCallback(deltaIo -> renameDeltaFile(groupWorkDir, part, deltaIo));
 
-                partitionFilePageStores.add(filePageStoreFactory.createPageStore(partFilePath, buffer.rewind()));
+                partitionFilePageStores.add(filePageStore);
             }
 
             return unmodifiableList(partitionFilePageStores);
         } finally {
             freeBuffer(buffer);
         }
     }
+
+    /**
+     * Returns paths (unsorted) to delta files for the requested partition.
+     *
+     * @param groupWorkDir Group directory.
+     * @param partition Partition number.
+     */
+    Path[] findPartitionDeltaFiles(Path groupWorkDir, int partition) throws IgniteInternalCheckedException {
+        assert partition >= 0 : partition;
+
+        try (Stream<Path> deltaFileStream = Files.find(
+                groupWorkDir,
+                1,
+                (path, basicFileAttributes) -> path.getFileName().toString().startsWith(String.format(PART_DELTA_FILE_PREFIX, partition)))

Review Comment:
   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] tkalkirill commented on a diff in pull request #947: IGNITE-17372 Implement DeltaFilePageStore

Posted by GitBox <gi...@apache.org>.
tkalkirill commented on code in PR #947:
URL: https://github.com/apache/ignite-3/pull/947#discussion_r928625557


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStoreManager.java:
##########
@@ -305,14 +340,99 @@ private List<FilePageStore> createFilePageStores(
 
         try {
             for (int i = 0; i < partitions; i++) {
-                Path partFilePath = groupWorkDir.resolve(String.format(PART_FILE_TEMPLATE, i));
+                int part = i;
+
+                Path partFilePath = groupWorkDir.resolve(String.format(PART_FILE_TEMPLATE, part));
+
+                Path[] partDeltaFiles = findPartitionDeltaFiles(groupWorkDir, partitions);
+
+                FilePageStore filePageStore = filePageStoreFactory.createPageStore(buffer.rewind(), partFilePath, partDeltaFiles);
+
+                filePageStore.setDeltaFilePageStoreIoFactory((index, pageIndexes) -> createLatest(groupWorkDir, part, index, pageIndexes));
+
+                filePageStore.setCompleteCreationDeltaFilePageStoreIoCallback(deltaIo -> renameDeltaFile(groupWorkDir, part, deltaIo));

Review Comment:
   We personally decided that I would delete 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] tkalkirill commented on a diff in pull request #947: IGNITE-17372 Implement DeltaFilePageStore

Posted by GitBox <gi...@apache.org>.
tkalkirill commented on code in PR #947:
URL: https://github.com/apache/ignite-3/pull/947#discussion_r928631658


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/PartitionMetaManager.java:
##########
@@ -95,10 +95,8 @@ public PartitionMeta readOrCreateMeta(
         long partitionMetaPageId = partitionMetaPageId(groupPartitionId.getPartitionId());
 
         try {
-            if (filePageStore.size() > filePageStore.headerSize()) {
+            if (filePageStore.readWithoutPageIdCheck(partitionMetaPageId, buffer, false)) {

Review Comment:
   We discussed it personally and figured out how we can 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 #947: IGNITE-17372 Implement DeltaFilePageStore

Posted by GitBox <gi...@apache.org>.
ibessonov commented on code in PR #947:
URL: https://github.com/apache/ignite-3/pull/947#discussion_r930794012


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/PartitionMetaManager.java:
##########
@@ -95,28 +99,34 @@ public PartitionMeta readOrCreateMeta(
         long partitionMetaPageId = partitionMetaPageId(groupPartitionId.getPartitionId());
 
         try {
-            if (filePageStore.readWithoutPageIdCheck(partitionMetaPageId, buffer, false)) {
+            if (containsPartitionMetaIo(filePageStore)) {

Review Comment:
   Please use the word "page" instead of "IO".



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/AbstractFilePageStoreIo.java:
##########
@@ -486,11 +482,12 @@ private boolean read0(
 
             int n = readWithFailover(pageBuf, pageOff);
 
+            // TODO: IGNITE-17397 Investigate the ability to read a empty page

Review Comment:
   ```suggestion
               // TODO: IGNITE-17397 Investigate the ability to read an empty page
   ```



-- 
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 #947: IGNITE-17372 Implement DeltaFilePageStore

Posted by GitBox <gi...@apache.org>.
tkalkirill commented on code in PR #947:
URL: https://github.com/apache/ignite-3/pull/947#discussion_r928504324


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/PartitionMetaManager.java:
##########
@@ -95,10 +95,8 @@ public PartitionMeta readOrCreateMeta(
         long partitionMetaPageId = partitionMetaPageId(groupPartitionId.getPartitionId());
 
         try {
-            if (filePageStore.size() > filePageStore.headerSize()) {
+            if (filePageStore.readWithoutPageIdCheck(partitionMetaPageId, buffer, false)) {

Review Comment:
   Perhaps ideally the code should have looked something like this:
   `            
               if (filePageStore.contains(partitionMetaPageId)) {
                   filePageStore.read(partitionMetaPageId, buffer, false);
   
                   return new PartitionMeta(checkpointId, ioRegistry.resolve(bufferAddr), bufferAddr);
               } else {
                   PartitionMetaIo io = ...;
   
                   filePageStore.write(partitionMetaPageId, buffer.rewind(), true);
   
                   filePageStore.sync();
   
                   return new PartitionMeta(checkpointId, io, bufferAddr);
               }
   `
   But it will always go on a false branch, because **contains** will have to check it by the number of pages, which we can only find out about by reading the first page, I don’t want to write the logic for **PartitionMetaIo** inside, I think this is not correct.
   
   We can personally discuss the solution and come up with a better API, but so far it seems to be more compromise.



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/PartitionMetaManager.java:
##########
@@ -95,10 +95,8 @@ public PartitionMeta readOrCreateMeta(
         long partitionMetaPageId = partitionMetaPageId(groupPartitionId.getPartitionId());
 
         try {
-            if (filePageStore.size() > filePageStore.headerSize()) {
+            if (filePageStore.readWithoutPageIdCheck(partitionMetaPageId, buffer, false)) {

Review Comment:
   Perhaps ideally the code should have looked something like this:
   `    
               if (filePageStore.contains(partitionMetaPageId)) {
                   filePageStore.read(partitionMetaPageId, buffer, false);
   
                   return new PartitionMeta(checkpointId, ioRegistry.resolve(bufferAddr), bufferAddr);
               } else {
                   PartitionMetaIo io = ...;
   
                   filePageStore.write(partitionMetaPageId, buffer.rewind(), true);
   
                   filePageStore.sync();
   
                   return new PartitionMeta(checkpointId, io, bufferAddr);
               }
   `
   But it will always go on a false branch, because **contains** will have to check it by the number of pages, which we can only find out about by reading the first page, I don’t want to write the logic for **PartitionMetaIo** inside, I think this is not correct.
   
   We can personally discuss the solution and come up with a better API, but so far it seems to be more compromise.



-- 
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 #947: IGNITE-17372 Implement DeltaFilePageStore

Posted by GitBox <gi...@apache.org>.
tkalkirill commented on code in PR #947:
URL: https://github.com/apache/ignite-3/pull/947#discussion_r928594610


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/AbstractFilePageStoreIo.java:
##########
@@ -0,0 +1,611 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.pagememory.persistence.store;
+
+import static java.nio.ByteOrder.nativeOrder;
+import static java.nio.file.StandardOpenOption.CREATE;
+import static java.nio.file.StandardOpenOption.READ;
+import static java.nio.file.StandardOpenOption.WRITE;
+import static org.apache.ignite.internal.util.IgniteUtils.atomicMoveFile;
+import static org.apache.ignite.internal.util.IgniteUtils.hexInt;
+import static org.apache.ignite.internal.util.IgniteUtils.hexLong;
+import static org.apache.ignite.internal.util.IgniteUtils.toHexString;
+import static org.apache.ignite.lang.IgniteSystemProperties.getBoolean;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.ClosedByInterruptException;
+import java.nio.channels.ClosedChannelException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.apache.ignite.internal.fileio.FileIo;
+import org.apache.ignite.internal.fileio.FileIoFactory;
+import org.apache.ignite.internal.pagememory.io.PageIo;
+import org.apache.ignite.internal.pagememory.persistence.FastCrc;
+import org.apache.ignite.internal.pagememory.persistence.IgniteInternalDataIntegrityViolationException;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Abstract class for performing IO operations on file page storage.
+ */
+public abstract class AbstractFilePageStoreIo implements Closeable {
+    private final FileIoFactory ioFactory;
+
+    private final ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
+
+    /** Skip CRC calculation flag. */
+    // TODO: IGNITE-17011 Move to config
+    private final boolean skipCrc = getBoolean("IGNITE_PDS_SKIP_CRC");
+
+    private volatile Path filePath;
+
+    private volatile @Nullable FileIo fileIo;
+
+    /** Initialized file page store IO. */
+    private volatile boolean initialized;
+
+    /** Caches the existence state of file. After it is initialized, it will be not {@code null} during lifecycle. */
+    private volatile @Nullable Boolean fileExists;
+
+    /**
+     * Constructor.
+     *
+     * @param ioFactory {@link FileIo} factory.
+     * @param filePath File page store path.
+     */
+    AbstractFilePageStoreIo(FileIoFactory ioFactory, Path filePath) {
+        this.ioFactory = ioFactory;
+        this.filePath = filePath;
+    }
+
+    /**
+     * Returns the page size in bytes.
+     */
+    public abstract int pageSize();
+
+    /**
+     * Returns the size of the header in bytes.
+     */
+    public abstract int headerSize();
+
+    /**
+     * Returns a buffer with a file page storage header.
+     */
+    public abstract ByteBuffer headerBuffer();
+
+    /**
+     * Checks the file page storage header.
+     *
+     * @param fileIo File page store IO to read the header.
+     * @throws IOException If there is an error reading the header or the header did not pass the check.
+     */
+    public abstract void checkHeader(FileIo fileIo) throws IOException;
+
+    /**
+     * Returns page offset within the store file.
+     *
+     * @param pageId Page ID.
+     */
+    public abstract long pageOffset(long pageId);

Review Comment:
   Made as internal methods.



-- 
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 #947: IGNITE-17372 Implement DeltaFilePageStore

Posted by GitBox <gi...@apache.org>.
tkalkirill commented on code in PR #947:
URL: https://github.com/apache/ignite-3/pull/947#discussion_r928610749


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/DeltaFilePageStoreIoFactory.java:
##########
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.pagememory.persistence.store;
+
+/**
+ * Factory for creating {@link DeltaFilePageStoreIo}.
+ */
+@FunctionalInterface
+public interface DeltaFilePageStoreIoFactory {

Review Comment:
   I personally decided that I would delete it.



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/CompleteCreationDeltaFilePageStoreIoCallback.java:
##########
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.pagememory.persistence.store;
+
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+
+/**
+ * Callback on completion of delta file page store creation.
+ */
+@FunctionalInterface
+public interface CompleteCreationDeltaFilePageStoreIoCallback {
+    /**
+     * Calls when the delta file page store is finished creating.
+     *
+     * @param deltaFilePageStoreIo Delta file page store.
+     * @throws IgniteInternalCheckedException If failed.
+     */
+    void onCompletionOfCreation(DeltaFilePageStoreIo deltaFilePageStoreIo) throws IgniteInternalCheckedException;

Review Comment:
   I personally decided that I would delete 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] tkalkirill commented on a diff in pull request #947: IGNITE-17372 Implement DeltaFilePageStore

Posted by GitBox <gi...@apache.org>.
tkalkirill commented on code in PR #947:
URL: https://github.com/apache/ignite-3/pull/947#discussion_r928600390


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/AbstractFilePageStoreIo.java:
##########
@@ -0,0 +1,611 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.pagememory.persistence.store;
+
+import static java.nio.ByteOrder.nativeOrder;
+import static java.nio.file.StandardOpenOption.CREATE;
+import static java.nio.file.StandardOpenOption.READ;
+import static java.nio.file.StandardOpenOption.WRITE;
+import static org.apache.ignite.internal.util.IgniteUtils.atomicMoveFile;
+import static org.apache.ignite.internal.util.IgniteUtils.hexInt;
+import static org.apache.ignite.internal.util.IgniteUtils.hexLong;
+import static org.apache.ignite.internal.util.IgniteUtils.toHexString;
+import static org.apache.ignite.lang.IgniteSystemProperties.getBoolean;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.ClosedByInterruptException;
+import java.nio.channels.ClosedChannelException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.apache.ignite.internal.fileio.FileIo;
+import org.apache.ignite.internal.fileio.FileIoFactory;
+import org.apache.ignite.internal.pagememory.io.PageIo;
+import org.apache.ignite.internal.pagememory.persistence.FastCrc;
+import org.apache.ignite.internal.pagememory.persistence.IgniteInternalDataIntegrityViolationException;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Abstract class for performing IO operations on file page storage.
+ */
+public abstract class AbstractFilePageStoreIo implements Closeable {
+    private final FileIoFactory ioFactory;
+
+    private final ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
+
+    /** Skip CRC calculation flag. */
+    // TODO: IGNITE-17011 Move to config
+    private final boolean skipCrc = getBoolean("IGNITE_PDS_SKIP_CRC");
+
+    private volatile Path filePath;
+
+    private volatile @Nullable FileIo fileIo;
+
+    /** Initialized file page store IO. */
+    private volatile boolean initialized;
+
+    /** Caches the existence state of file. After it is initialized, it will be not {@code null} during lifecycle. */
+    private volatile @Nullable Boolean fileExists;
+
+    /**
+     * Constructor.
+     *
+     * @param ioFactory {@link FileIo} factory.
+     * @param filePath File page store path.
+     */
+    AbstractFilePageStoreIo(FileIoFactory ioFactory, Path filePath) {
+        this.ioFactory = ioFactory;
+        this.filePath = filePath;
+    }
+
+    /**
+     * Returns the page size in bytes.
+     */
+    public abstract int pageSize();
+
+    /**
+     * Returns the size of the header in bytes.
+     */
+    public abstract int headerSize();
+
+    /**
+     * Returns a buffer with a file page storage header.
+     */
+    public abstract ByteBuffer headerBuffer();
+
+    /**
+     * Checks the file page storage header.
+     *
+     * @param fileIo File page store IO to read the header.
+     * @throws IOException If there is an error reading the header or the header did not pass the check.
+     */
+    public abstract void checkHeader(FileIo fileIo) throws IOException;
+
+    /**
+     * Returns page offset within the store file.
+     *
+     * @param pageId Page ID.
+     */
+    public abstract long pageOffset(long pageId);
+
+    /**
+     * Stops the file page store IO.
+     *
+     * @param clean {@code True} to clean file page store.
+     * @throws IgniteInternalCheckedException If failed.
+     */
+    public void stop(boolean clean) throws IgniteInternalCheckedException {
+        try {
+            stop0(clean);
+        } catch (IOException e) {
+            throw new IgniteInternalCheckedException(
+                    "Failed to stop serving file [file=" + filePath + ", delete=" + clean + "]",
+                    e
+            );
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public void close() throws IOException {
+        stop0(false);
+    }
+
+    /**
+     * Reads a page.
+     *
+     * @param pageId Page ID.
+     * @param pageOff Page offset in the file.
+     * @param pageBuf Page buffer to read into.
+     * @param keepCrc By default, reading zeroes CRC which was on page store, but you can keep it in {@code pageBuf} if set {@code

Review Comment:
   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] tkalkirill commented on a diff in pull request #947: IGNITE-17372 Implement DeltaFilePageStore

Posted by GitBox <gi...@apache.org>.
tkalkirill commented on code in PR #947:
URL: https://github.com/apache/ignite-3/pull/947#discussion_r928620682


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStore.java:
##########
@@ -166,503 +152,196 @@ public int pages() {
     public void pages(int pageCount) {
         assert pageCount >= 0 : pageCount;
 
-        this.pageCount.set(pageCount);
+        this.pageCount = pageCount;
     }
 
     /**
-     * Reads a page, unlike {@link #read(long, ByteBuffer, boolean)}, checks the page offset in the file not logically (pageOffset <= {@link
-     * #pages()} * {@link #pageSize}) but physically (pageOffset <= {@link #size()}), which can affect performance when used in production
-     * code.
+     * Reads a page, unlike {@link #read(long, ByteBuffer, boolean)}, does not check the {@code pageId} so that its {@code pageIdx} is not
+     * greater than the {@link #pages() number of allocated pages}.
      *
      * @param pageId Page ID.
      * @param pageBuf Page buffer to read into.
      * @param keepCrc By default, reading zeroes CRC which was on page store, but you can keep it in {@code pageBuf} if set {@code
      * keepCrc}.
+     * @return {@code True} if the page was read successfully.
      * @throws IgniteInternalCheckedException If reading failed (IO error occurred).
      */
-    public void readByPhysicalOffset(long pageId, ByteBuffer pageBuf, boolean keepCrc) throws IgniteInternalCheckedException {
-        read0(pageId, pageBuf, !skipCrc, keepCrc, false);
+    public boolean readWithoutPageIdCheck(long pageId, ByteBuffer pageBuf, boolean keepCrc) throws IgniteInternalCheckedException {
+        for (DeltaFilePageStoreIo deltaFilePageStoreIo : deltaFilePageStoreIos) {
+            long pageOff = deltaFilePageStoreIo.pageOffset(pageId);
+
+            if (pageOff >= 0) {
+                return deltaFilePageStoreIo.read(pageId, pageOff, pageBuf, keepCrc);
+            }
+        }
+
+        return filePageStoreIo.read(pageId, filePageStoreIo.pageOffset(pageId), pageBuf, keepCrc);
     }
 
     /** {@inheritDoc} */
     @Override
     public void read(long pageId, ByteBuffer pageBuf, boolean keepCrc) throws IgniteInternalCheckedException {
-        read0(pageId, pageBuf, !skipCrc, keepCrc, true);
-    }
-
-    /**
-     * Reads a page from the page store.
-     *
-     * @param pageId Page ID.
-     * @param pageBuf Page buffer to read into.
-     * @param checkCrc Check CRC on page.
-     * @param keepCrc By default reading zeroes CRC which was on file, but you can keep it in pageBuf if set keepCrc.
-     * @param checkPageOffsetLogically Check page offset by {@link #allocatedBytes} or {@link #size}.
-     * @throws IgniteInternalCheckedException If reading failed (IO error occurred).
-     */
-    private void read0(
-            long pageId,
-            ByteBuffer pageBuf,
-            boolean checkCrc,
-            boolean keepCrc,
-            boolean checkPageOffsetLogically
-    ) throws IgniteInternalCheckedException {
-        ensure();
-
-        try {
-            assert pageBuf.capacity() == pageSize : pageBuf.capacity();
-            assert pageBuf.remaining() == pageSize : pageBuf.remaining();
-            assert pageBuf.position() == 0 : pageBuf.position();
-            assert pageBuf.order() == nativeOrder() : pageBuf.order();
-
-            long pageOff = pageOffset(pageId);
-
-            if (checkPageOffsetLogically) {
-                assert pageOff <= allocatedBytes() : "calculatedOffset=" + pageOff
-                        + ", allocated=" + allocatedBytes() + ", headerSize=" + headerSize + ", filePath=" + filePath;
-            } else {
-                assert pageOff <= size() : "calculatedOffset=" + pageOff
-                        + ", size=" + size() + ", headerSize=" + headerSize + ", filePath=" + filePath;
-            }
-
-            int n = readWithFailover(pageBuf, pageOff);
+        assert pageIndex(pageId) <= pageCount : "pageIdx=" + pageIndex(pageId) + ", pageCount=" + pageCount;
 
-            // If page was not written yet, nothing to read.
-            if (n < 0) {
-                pageBuf.put(new byte[pageBuf.remaining()]);
-            }
+        for (DeltaFilePageStoreIo deltaFilePageStoreIo : deltaFilePageStoreIos) {
+            long pageOff = deltaFilePageStoreIo.pageOffset(pageId);
 
-            int savedCrc32 = PageIo.getCrc(pageBuf);
-
-            PageIo.setCrc(pageBuf, 0);
-
-            pageBuf.position(0);
-
-            if (checkCrc) {
-                int curCrc32 = FastCrc.calcCrc(pageBuf, pageSize);
-
-                if ((savedCrc32 ^ curCrc32) != 0) {
-                    throw new IgniteInternalDataIntegrityViolationException("Failed to read page (CRC validation failed) "
-                            + "[id=" + hexLong(pageId) + ", off=" + pageOff
-                            + ", filePath=" + filePath + ", fileSize=" + fileIo.size()
-                            + ", savedCrc=" + hexInt(savedCrc32) + ", curCrc=" + hexInt(curCrc32)
-                            + ", page=" + toHexString(pageBuf) + "]");
-                }
-            }
+            if (pageOff >= 0) {
+                deltaFilePageStoreIo.read(pageId, pageOff, pageBuf, keepCrc);
 
-            assert PageIo.getCrc(pageBuf) == 0;
-
-            if (keepCrc) {
-                PageIo.setCrc(pageBuf, savedCrc32);
+                return;
             }
-        } catch (IOException e) {
-            throw new IgniteInternalCheckedException("Failed to read page [file=" + filePath + ", pageId=" + pageId + "]", e);
         }
+
+        filePageStoreIo.read(pageId, filePageStoreIo.pageOffset(pageId), pageBuf, keepCrc);
     }
 
     /** {@inheritDoc} */
     @Override
     public void write(long pageId, ByteBuffer pageBuf, boolean calculateCrc) throws IgniteInternalCheckedException {
-        ensure();
-
-        boolean interrupted = false;
-
-        while (true) {
-            FileIo fileIo = this.fileIo;
-
-            try {
-                readWriteLock.readLock().lock();
-
-                try {
-                    assert pageBuf.position() == 0 : pageBuf.position();
-                    assert pageBuf.order() == nativeOrder() : "Page buffer order " + pageBuf.order()
-                            + " should be same with " + nativeOrder();
-                    assert PageIo.getType(pageBuf) != 0 : "Invalid state. Type is 0! pageId = " + hexLong(pageId);
-                    assert PageIo.getVersion(pageBuf) != 0 : "Invalid state. Version is 0! pageId = " + hexLong(pageId);
-
-                    if (calculateCrc && !skipCrc) {
-                        assert PageIo.getCrc(pageBuf) == 0 : hexLong(pageId);
-
-                        PageIo.setCrc(pageBuf, calcCrc32(pageBuf, pageSize));
-                    }
-
-                    // Check whether crc was calculated somewhere above the stack if it is forcibly skipped.
-                    assert skipCrc || PageIo.getCrc(pageBuf) != 0
-                            || calcCrc32(pageBuf, pageSize) == 0 : "CRC hasn't been calculated, crc=0";
-
-                    assert pageBuf.position() == 0 : pageBuf.position();
-
-                    long pageOff = pageOffset(pageId);
-
-                    assert pageOff <= allocatedBytes() : "calculatedOffset=" + pageOff
-                            + ", allocated=" + allocatedBytes() + ", headerSize=" + headerSize + ", filePath=" + filePath;
-
-                    fileIo.writeFully(pageBuf, pageOff);
-
-                    PageIo.setCrc(pageBuf, 0);
-
-                    if (interrupted) {
-                        Thread.currentThread().interrupt();
-                    }
-
-                    return;
-                } finally {
-                    readWriteLock.readLock().unlock();
-                }
-            } catch (IOException e) {
-                if (e instanceof ClosedChannelException) {
-                    try {
-                        if (e instanceof ClosedByInterruptException) {
-                            interrupted = true;
-
-                            Thread.interrupted();
-                        }
-
-                        reinit(fileIo);
-
-                        pageBuf.position(0);
-
-                        PageIo.setCrc(pageBuf, 0);
+        assert pageIndex(pageId) <= pageCount : "pageIdx=" + pageIndex(pageId) + ", pageCount=" + pageCount;
 
-                        continue;
-                    } catch (IOException e0) {
-                        e0.addSuppressed(e);
-
-                        e = e0;
-                    }
-                }
-
-                throw new IgniteInternalCheckedException(
-                        "Failed to write page [filePath=" + filePath + ", pageId=" + pageId + "]",
-                        e
-                );
-            }
-        }
+        filePageStoreIo.write(pageId, pageBuf, calculateCrc);
     }
 
     /** {@inheritDoc} */
     @Override
     public void sync() throws IgniteInternalCheckedException {
-        readWriteLock.writeLock().lock();
-
-        try {
-            ensure();
-
-            FileIo fileIo = this.fileIo;
-
-            if (fileIo != null) {
-                fileIo.force();
-            }
-        } catch (IOException e) {
-            throw new IgniteInternalCheckedException("Failed to fsync partition file [filePath=" + filePath + ']', e);
-        } finally {
-            readWriteLock.writeLock().unlock();
-        }
+        filePageStoreIo.sync();
     }
 
     /** {@inheritDoc} */
     @Override
     public boolean exists() {
-        if (fileExists == null) {
-            readWriteLock.writeLock().lock();
-
-            try {
-                if (fileExists == null) {
-                    fileExists = Files.exists(filePath) && filePath.toFile().length() >= headerSize;
-                }
-            } finally {
-                readWriteLock.writeLock().unlock();
-            }
-        }
-
-        return fileExists;
+        return filePageStoreIo.exists();
     }
 
     /** {@inheritDoc} */
     @Override
     public void ensure() throws IgniteInternalCheckedException {
-        if (!initialized) {
-            readWriteLock.writeLock().lock();
-
-            try {
-                if (!initialized) {
-                    FileIo fileIo = null;
-
-                    IgniteInternalCheckedException err = null;
-
-                    try {
-                        boolean interrupted = false;
-
-                        while (true) {
-                            try {
-                                this.fileIo = fileIo = ioFactory.create(filePath, CREATE, READ, WRITE);
-
-                                fileExists = true;
-
-                                if (fileIo.size() < headerSize) {
-                                    fileIo.writeFully(new FilePageStoreHeader(version, pageSize).toByteBuffer(), 0);
-                                } else {
-                                    checkHeader(fileIo);
-                                }
-
-                                if (interrupted) {
-                                    Thread.currentThread().interrupt();
-                                }
-
-                                break;
-                            } catch (ClosedByInterruptException e) {
-                                interrupted = true;
-
-                                Thread.interrupted();
-                            }
-                        }
-
-                        initialized = true;
-                    } catch (IOException e) {
-                        err = new IgniteInternalCheckedException("Failed to initialize partition file: " + filePath, e);
-
-                        throw err;
-                    } finally {
-                        if (err != null && fileIo != null) {
-                            try {
-                                fileIo.close();
-                            } catch (IOException e) {
-                                err.addSuppressed(e);
-                            }
-                        }
-                    }
-                }
-            } finally {
-                readWriteLock.writeLock().unlock();
-            }
-        }
+        filePageStoreIo.ensure();
     }
 
     /** {@inheritDoc} */
     @Override
     public void close() throws IOException {
-        stop0(false);
+        filePageStoreIo.close();
+
+        for (DeltaFilePageStoreIo deltaFilePageStoreIo : deltaFilePageStoreIos) {
+            deltaFilePageStoreIo.close();
+        }
     }
 
     /**
      * Returns size of the page store in bytes.
      *
-     * <p>May differ from {@link #pages} * {@link #pageSize} due to delayed writes or due to other implementation specific details.
+     * <p>May differ from {@link #pages} * {@link FilePageStoreIo#pageSize()} due to delayed writes or due to other implementation specific
+     * details.
      *
      * @throws IgniteInternalCheckedException If an I/O error occurs.
      */
     public long size() throws IgniteInternalCheckedException {
-        try {
-            FileIo io = fileIo;
-
-            return io == null ? 0 : io.size();
-        } catch (IOException e) {
-            throw new IgniteInternalCheckedException(e);
-        }
+        return filePageStoreIo.size();
     }
 
     /**
-     * Stops file page store.
-     *
-     * @param delete {@code True} to delete file.
-     * @throws IOException If fails.
+     * Returns file page store path.
      */
-    private void stop0(boolean delete) throws IOException {
-        readWriteLock.writeLock().lock();
-
-        try {
-            if (!initialized) {
-                // Ensure the file is closed even if not initialized yet.
-                if (fileIo != null) {
-                    fileIo.close();
-                }
-
-                if (delete && exists()) {
-                    Files.delete(filePath);
-                }
-
-                return;
-            }
-
-            fileIo.force();
-
-            fileIo.close();
-
-            fileIo = null;
-
-            if (delete) {
-                Files.delete(filePath);
-
-                fileExists = false;
-            }
-        } finally {
-            initialized = false;
+    public Path filePath() {
+        return filePageStoreIo.filePath();
+    }
 
-            readWriteLock.writeLock().unlock();
-        }
+    /**
+     * Returns file page store header size.
+     */
+    public int headerSize() {
+        return filePageStoreIo.headerSize();
     }
 
     /**
-     * Gets page offset within the store file.
+     * Sets the new page allocation listener.
      *
-     * @param pageId Page ID.
-     * @return Page offset.
+     * @param listener New page allocation listener.
      */
-    long pageOffset(long pageId) {
-        return (long) PageIdUtils.pageIndex(pageId) * pageSize + headerSize;
+    public void setPageAllocationListener(PageAllocationListener listener) {
+        pageAllocationListener = listener;
     }
 
     /**
-     * Reads from page storage with failover.
+     * Sets the delta file page store factory.
      *
-     * @param destBuf Destination buffer.
-     * @param position Position.
-     * @return Number of read bytes, or {@code -1} if the given position is greater than or equal to the file's current size.
+     * @param factory Factory.
      */
-    private int readWithFailover(ByteBuffer destBuf, long position) throws IOException {
-        boolean interrupted = false;
-
-        int bufPos = destBuf.position();
-
-        while (true) {
-            FileIo fileIo = this.fileIo;
-
-            if (fileIo == null) {
-                throw new IOException("FileIo has stopped");
-            }
-
-            try {
-                assert destBuf.remaining() > 0;
-
-                int bytesRead = fileIo.readFully(destBuf, position);
-
-                if (interrupted) {
-                    Thread.currentThread().interrupt();
-                }
-
-                return bytesRead;
-            } catch (ClosedChannelException e) {
-                destBuf.position(bufPos);
-
-                if (e instanceof ClosedByInterruptException) {
-                    interrupted = true;
-
-                    Thread.interrupted();
-                }
-
-                reinit(fileIo);
-            }
-        }
+    public void setDeltaFilePageStoreIoFactory(DeltaFilePageStoreIoFactory factory) {
+        deltaFilePageStoreIoFactory = factory;
     }
 
     /**
-     * Reinit page store after file channel was closed by thread interruption.
+     * Sets the callback on completion of delta file page store creation.
      *
-     * @param fileIo Old fileIo.
+     * @param callback Callback.
      */
-    private void reinit(FileIo fileIo) throws IOException {
-        if (!initialized) {
-            return;
-        }
-
-        if (fileIo != this.fileIo) {
-            return;
-        }
-
-        readWriteLock.writeLock().lock();
-
-        try {
-            if (fileIo != this.fileIo) {
-                return;
-            }
-
-            try {
-                boolean interrupted = false;
-
-                while (true) {
-                    try {
-                        fileIo = null;
-
-                        fileIo = ioFactory.create(filePath, CREATE, READ, WRITE);
+    public void setCompleteCreationDeltaFilePageStoreIoCallback(CompleteCreationDeltaFilePageStoreIoCallback callback) {
+        completeCreationDeltaFilePageStoreIoCallback = callback;
+    }
 
-                        fileExists = true;
+    /**
+     * Gets or creates a new delta file, a new delta file will be created when the previous one is {@link #completeNewDeltaFile()
+     * completed}.
+     *
+     * <p>Thread safe.
+     *
+     * @param pageIndexesSupplier Page indexes supplier for the new delta file page store.
+     * @return Future that will be completed when the new delta file page store is created.
+     */
+    public CompletableFuture<DeltaFilePageStoreIo> getOrCreateNewDeltaFile(Supplier<int[]> pageIndexesSupplier) {
+        assert deltaFilePageStoreIoFactory != null;
 
-                        checkHeader(fileIo);
+        CompletableFuture<DeltaFilePageStoreIo> future = this.newDeltaFilePageStoreIoFuture;
 
-                        this.fileIo = fileIo;
+        if (future != null) {
+            return future;
+        }
 
-                        if (interrupted) {
-                            Thread.currentThread().interrupt();
-                        }
+        if (!NEW_DELTA_FILE_PAGE_STORE_IO_FUTURE.compareAndSet(this, null, future = new CompletableFuture<>())) {
+            // Another thread started creating a delta file.
+            return newDeltaFilePageStoreIoFuture;
+        }
 
-                        break;
-                    } catch (ClosedByInterruptException e) {
-                        interrupted = true;
+        int nextIndex = deltaFilePageStoreIos.isEmpty() ? 0 : deltaFilePageStoreIos.get(0).fileIndex() + 1;
 
-                        Thread.interrupted();
-                    }
-                }
-            } catch (IOException e) {
-                try {
-                    if (fileIo != null) {
-                        fileIo.close();
-                    }
-                } catch (IOException e0) {
-                    e.addSuppressed(e0);
-                }
+        DeltaFilePageStoreIo deltaFilePageStoreIo = deltaFilePageStoreIoFactory.create(nextIndex, pageIndexesSupplier.get());
 
-                throw e;
-            }
-        } finally {
-            readWriteLock.writeLock().unlock();
-        }
-    }
+        // Should add to the head, since read operations should always start from the most recent.
+        deltaFilePageStoreIos.add(0, deltaFilePageStoreIo);
 
-    private static int calcCrc32(ByteBuffer pageBuf, int pageSize) {
-        try {
-            pageBuf.position(0);
+        future.complete(deltaFilePageStoreIo);
 
-            return FastCrc.calcCrc(pageBuf, pageSize);
-        } finally {
-            pageBuf.position(0);
-        }
+        return future;
     }
 
     /**
-     * Returns file page store path.
+     * Completes the {@link #getOrCreateNewDeltaFile(Supplier) creation} of a new delta file.
+     *
+     * <p>Thread safe.
+     *
+     * @throws IgniteInternalCheckedException If failed.
      */
-    public Path filePath() {
-        return filePath;
-    }
+    public void completeNewDeltaFile() throws IgniteInternalCheckedException {

Review Comment:
   I will add more comments later.



-- 
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 #947: IGNITE-17372 Implement DeltaFilePageStore

Posted by GitBox <gi...@apache.org>.
tkalkirill commented on code in PR #947:
URL: https://github.com/apache/ignite-3/pull/947#discussion_r928619704


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStore.java:
##########
@@ -166,503 +152,196 @@ public int pages() {
     public void pages(int pageCount) {
         assert pageCount >= 0 : pageCount;
 
-        this.pageCount.set(pageCount);
+        this.pageCount = pageCount;
     }
 
     /**
-     * Reads a page, unlike {@link #read(long, ByteBuffer, boolean)}, checks the page offset in the file not logically (pageOffset <= {@link
-     * #pages()} * {@link #pageSize}) but physically (pageOffset <= {@link #size()}), which can affect performance when used in production
-     * code.
+     * Reads a page, unlike {@link #read(long, ByteBuffer, boolean)}, does not check the {@code pageId} so that its {@code pageIdx} is not
+     * greater than the {@link #pages() number of allocated pages}.
      *
      * @param pageId Page ID.
      * @param pageBuf Page buffer to read into.
      * @param keepCrc By default, reading zeroes CRC which was on page store, but you can keep it in {@code pageBuf} if set {@code
      * keepCrc}.
+     * @return {@code True} if the page was read successfully.
      * @throws IgniteInternalCheckedException If reading failed (IO error occurred).
      */
-    public void readByPhysicalOffset(long pageId, ByteBuffer pageBuf, boolean keepCrc) throws IgniteInternalCheckedException {
-        read0(pageId, pageBuf, !skipCrc, keepCrc, false);
+    public boolean readWithoutPageIdCheck(long pageId, ByteBuffer pageBuf, boolean keepCrc) throws IgniteInternalCheckedException {
+        for (DeltaFilePageStoreIo deltaFilePageStoreIo : deltaFilePageStoreIos) {
+            long pageOff = deltaFilePageStoreIo.pageOffset(pageId);
+
+            if (pageOff >= 0) {
+                return deltaFilePageStoreIo.read(pageId, pageOff, pageBuf, keepCrc);
+            }
+        }
+
+        return filePageStoreIo.read(pageId, filePageStoreIo.pageOffset(pageId), pageBuf, keepCrc);
     }
 
     /** {@inheritDoc} */
     @Override
     public void read(long pageId, ByteBuffer pageBuf, boolean keepCrc) throws IgniteInternalCheckedException {
-        read0(pageId, pageBuf, !skipCrc, keepCrc, true);
-    }
-
-    /**
-     * Reads a page from the page store.
-     *
-     * @param pageId Page ID.
-     * @param pageBuf Page buffer to read into.
-     * @param checkCrc Check CRC on page.
-     * @param keepCrc By default reading zeroes CRC which was on file, but you can keep it in pageBuf if set keepCrc.
-     * @param checkPageOffsetLogically Check page offset by {@link #allocatedBytes} or {@link #size}.
-     * @throws IgniteInternalCheckedException If reading failed (IO error occurred).
-     */
-    private void read0(
-            long pageId,
-            ByteBuffer pageBuf,
-            boolean checkCrc,
-            boolean keepCrc,
-            boolean checkPageOffsetLogically
-    ) throws IgniteInternalCheckedException {
-        ensure();
-
-        try {
-            assert pageBuf.capacity() == pageSize : pageBuf.capacity();
-            assert pageBuf.remaining() == pageSize : pageBuf.remaining();
-            assert pageBuf.position() == 0 : pageBuf.position();
-            assert pageBuf.order() == nativeOrder() : pageBuf.order();
-
-            long pageOff = pageOffset(pageId);
-
-            if (checkPageOffsetLogically) {
-                assert pageOff <= allocatedBytes() : "calculatedOffset=" + pageOff
-                        + ", allocated=" + allocatedBytes() + ", headerSize=" + headerSize + ", filePath=" + filePath;
-            } else {
-                assert pageOff <= size() : "calculatedOffset=" + pageOff
-                        + ", size=" + size() + ", headerSize=" + headerSize + ", filePath=" + filePath;
-            }
-
-            int n = readWithFailover(pageBuf, pageOff);
+        assert pageIndex(pageId) <= pageCount : "pageIdx=" + pageIndex(pageId) + ", pageCount=" + pageCount;
 
-            // If page was not written yet, nothing to read.
-            if (n < 0) {
-                pageBuf.put(new byte[pageBuf.remaining()]);
-            }
+        for (DeltaFilePageStoreIo deltaFilePageStoreIo : deltaFilePageStoreIos) {
+            long pageOff = deltaFilePageStoreIo.pageOffset(pageId);
 
-            int savedCrc32 = PageIo.getCrc(pageBuf);
-
-            PageIo.setCrc(pageBuf, 0);
-
-            pageBuf.position(0);
-
-            if (checkCrc) {
-                int curCrc32 = FastCrc.calcCrc(pageBuf, pageSize);
-
-                if ((savedCrc32 ^ curCrc32) != 0) {
-                    throw new IgniteInternalDataIntegrityViolationException("Failed to read page (CRC validation failed) "
-                            + "[id=" + hexLong(pageId) + ", off=" + pageOff
-                            + ", filePath=" + filePath + ", fileSize=" + fileIo.size()
-                            + ", savedCrc=" + hexInt(savedCrc32) + ", curCrc=" + hexInt(curCrc32)
-                            + ", page=" + toHexString(pageBuf) + "]");
-                }
-            }
+            if (pageOff >= 0) {
+                deltaFilePageStoreIo.read(pageId, pageOff, pageBuf, keepCrc);
 
-            assert PageIo.getCrc(pageBuf) == 0;
-
-            if (keepCrc) {
-                PageIo.setCrc(pageBuf, savedCrc32);
+                return;
             }
-        } catch (IOException e) {
-            throw new IgniteInternalCheckedException("Failed to read page [file=" + filePath + ", pageId=" + pageId + "]", e);
         }
+
+        filePageStoreIo.read(pageId, filePageStoreIo.pageOffset(pageId), pageBuf, keepCrc);
     }
 
     /** {@inheritDoc} */
     @Override
     public void write(long pageId, ByteBuffer pageBuf, boolean calculateCrc) throws IgniteInternalCheckedException {
-        ensure();
-
-        boolean interrupted = false;
-
-        while (true) {
-            FileIo fileIo = this.fileIo;
-
-            try {
-                readWriteLock.readLock().lock();
-
-                try {
-                    assert pageBuf.position() == 0 : pageBuf.position();
-                    assert pageBuf.order() == nativeOrder() : "Page buffer order " + pageBuf.order()
-                            + " should be same with " + nativeOrder();
-                    assert PageIo.getType(pageBuf) != 0 : "Invalid state. Type is 0! pageId = " + hexLong(pageId);
-                    assert PageIo.getVersion(pageBuf) != 0 : "Invalid state. Version is 0! pageId = " + hexLong(pageId);
-
-                    if (calculateCrc && !skipCrc) {
-                        assert PageIo.getCrc(pageBuf) == 0 : hexLong(pageId);
-
-                        PageIo.setCrc(pageBuf, calcCrc32(pageBuf, pageSize));
-                    }
-
-                    // Check whether crc was calculated somewhere above the stack if it is forcibly skipped.
-                    assert skipCrc || PageIo.getCrc(pageBuf) != 0
-                            || calcCrc32(pageBuf, pageSize) == 0 : "CRC hasn't been calculated, crc=0";
-
-                    assert pageBuf.position() == 0 : pageBuf.position();
-
-                    long pageOff = pageOffset(pageId);
-
-                    assert pageOff <= allocatedBytes() : "calculatedOffset=" + pageOff
-                            + ", allocated=" + allocatedBytes() + ", headerSize=" + headerSize + ", filePath=" + filePath;
-
-                    fileIo.writeFully(pageBuf, pageOff);
-
-                    PageIo.setCrc(pageBuf, 0);
-
-                    if (interrupted) {
-                        Thread.currentThread().interrupt();
-                    }
-
-                    return;
-                } finally {
-                    readWriteLock.readLock().unlock();
-                }
-            } catch (IOException e) {
-                if (e instanceof ClosedChannelException) {
-                    try {
-                        if (e instanceof ClosedByInterruptException) {
-                            interrupted = true;
-
-                            Thread.interrupted();
-                        }
-
-                        reinit(fileIo);
-
-                        pageBuf.position(0);
-
-                        PageIo.setCrc(pageBuf, 0);
+        assert pageIndex(pageId) <= pageCount : "pageIdx=" + pageIndex(pageId) + ", pageCount=" + pageCount;
 
-                        continue;
-                    } catch (IOException e0) {
-                        e0.addSuppressed(e);
-
-                        e = e0;
-                    }
-                }
-
-                throw new IgniteInternalCheckedException(
-                        "Failed to write page [filePath=" + filePath + ", pageId=" + pageId + "]",
-                        e
-                );
-            }
-        }
+        filePageStoreIo.write(pageId, pageBuf, calculateCrc);
     }
 
     /** {@inheritDoc} */
     @Override
     public void sync() throws IgniteInternalCheckedException {
-        readWriteLock.writeLock().lock();
-
-        try {
-            ensure();
-
-            FileIo fileIo = this.fileIo;
-
-            if (fileIo != null) {
-                fileIo.force();
-            }
-        } catch (IOException e) {
-            throw new IgniteInternalCheckedException("Failed to fsync partition file [filePath=" + filePath + ']', e);
-        } finally {
-            readWriteLock.writeLock().unlock();
-        }
+        filePageStoreIo.sync();
     }
 
     /** {@inheritDoc} */
     @Override
     public boolean exists() {
-        if (fileExists == null) {
-            readWriteLock.writeLock().lock();
-
-            try {
-                if (fileExists == null) {
-                    fileExists = Files.exists(filePath) && filePath.toFile().length() >= headerSize;
-                }
-            } finally {
-                readWriteLock.writeLock().unlock();
-            }
-        }
-
-        return fileExists;
+        return filePageStoreIo.exists();
     }
 
     /** {@inheritDoc} */
     @Override
     public void ensure() throws IgniteInternalCheckedException {
-        if (!initialized) {
-            readWriteLock.writeLock().lock();
-
-            try {
-                if (!initialized) {
-                    FileIo fileIo = null;
-
-                    IgniteInternalCheckedException err = null;
-
-                    try {
-                        boolean interrupted = false;
-
-                        while (true) {
-                            try {
-                                this.fileIo = fileIo = ioFactory.create(filePath, CREATE, READ, WRITE);
-
-                                fileExists = true;
-
-                                if (fileIo.size() < headerSize) {
-                                    fileIo.writeFully(new FilePageStoreHeader(version, pageSize).toByteBuffer(), 0);
-                                } else {
-                                    checkHeader(fileIo);
-                                }
-
-                                if (interrupted) {
-                                    Thread.currentThread().interrupt();
-                                }
-
-                                break;
-                            } catch (ClosedByInterruptException e) {
-                                interrupted = true;
-
-                                Thread.interrupted();
-                            }
-                        }
-
-                        initialized = true;
-                    } catch (IOException e) {
-                        err = new IgniteInternalCheckedException("Failed to initialize partition file: " + filePath, e);
-
-                        throw err;
-                    } finally {
-                        if (err != null && fileIo != null) {
-                            try {
-                                fileIo.close();
-                            } catch (IOException e) {
-                                err.addSuppressed(e);
-                            }
-                        }
-                    }
-                }
-            } finally {
-                readWriteLock.writeLock().unlock();
-            }
-        }
+        filePageStoreIo.ensure();
     }
 
     /** {@inheritDoc} */
     @Override
     public void close() throws IOException {
-        stop0(false);
+        filePageStoreIo.close();
+
+        for (DeltaFilePageStoreIo deltaFilePageStoreIo : deltaFilePageStoreIos) {
+            deltaFilePageStoreIo.close();
+        }
     }
 
     /**
      * Returns size of the page store in bytes.
      *
-     * <p>May differ from {@link #pages} * {@link #pageSize} due to delayed writes or due to other implementation specific details.
+     * <p>May differ from {@link #pages} * {@link FilePageStoreIo#pageSize()} due to delayed writes or due to other implementation specific
+     * details.
      *
      * @throws IgniteInternalCheckedException If an I/O error occurs.
      */
     public long size() throws IgniteInternalCheckedException {
-        try {
-            FileIo io = fileIo;
-
-            return io == null ? 0 : io.size();
-        } catch (IOException e) {
-            throw new IgniteInternalCheckedException(e);
-        }
+        return filePageStoreIo.size();
     }
 
     /**
-     * Stops file page store.
-     *
-     * @param delete {@code True} to delete file.
-     * @throws IOException If fails.
+     * Returns file page store path.
      */
-    private void stop0(boolean delete) throws IOException {
-        readWriteLock.writeLock().lock();
-
-        try {
-            if (!initialized) {
-                // Ensure the file is closed even if not initialized yet.
-                if (fileIo != null) {
-                    fileIo.close();
-                }
-
-                if (delete && exists()) {
-                    Files.delete(filePath);
-                }
-
-                return;
-            }
-
-            fileIo.force();
-
-            fileIo.close();
-
-            fileIo = null;
-
-            if (delete) {
-                Files.delete(filePath);
-
-                fileExists = false;
-            }
-        } finally {
-            initialized = false;
+    public Path filePath() {
+        return filePageStoreIo.filePath();
+    }
 
-            readWriteLock.writeLock().unlock();
-        }
+    /**
+     * Returns file page store header size.
+     */
+    public int headerSize() {
+        return filePageStoreIo.headerSize();
     }
 
     /**
-     * Gets page offset within the store file.
+     * Sets the new page allocation listener.
      *
-     * @param pageId Page ID.
-     * @return Page offset.
+     * @param listener New page allocation listener.
      */
-    long pageOffset(long pageId) {
-        return (long) PageIdUtils.pageIndex(pageId) * pageSize + headerSize;
+    public void setPageAllocationListener(PageAllocationListener listener) {
+        pageAllocationListener = listener;
     }
 
     /**
-     * Reads from page storage with failover.
+     * Sets the delta file page store factory.
      *
-     * @param destBuf Destination buffer.
-     * @param position Position.
-     * @return Number of read bytes, or {@code -1} if the given position is greater than or equal to the file's current size.
+     * @param factory Factory.
      */
-    private int readWithFailover(ByteBuffer destBuf, long position) throws IOException {
-        boolean interrupted = false;
-
-        int bufPos = destBuf.position();
-
-        while (true) {
-            FileIo fileIo = this.fileIo;
-
-            if (fileIo == null) {
-                throw new IOException("FileIo has stopped");
-            }
-
-            try {
-                assert destBuf.remaining() > 0;
-
-                int bytesRead = fileIo.readFully(destBuf, position);
-
-                if (interrupted) {
-                    Thread.currentThread().interrupt();
-                }
-
-                return bytesRead;
-            } catch (ClosedChannelException e) {
-                destBuf.position(bufPos);
-
-                if (e instanceof ClosedByInterruptException) {
-                    interrupted = true;
-
-                    Thread.interrupted();
-                }
-
-                reinit(fileIo);
-            }
-        }
+    public void setDeltaFilePageStoreIoFactory(DeltaFilePageStoreIoFactory factory) {
+        deltaFilePageStoreIoFactory = factory;
     }
 
     /**
-     * Reinit page store after file channel was closed by thread interruption.
+     * Sets the callback on completion of delta file page store creation.
      *
-     * @param fileIo Old fileIo.
+     * @param callback Callback.
      */
-    private void reinit(FileIo fileIo) throws IOException {
-        if (!initialized) {
-            return;
-        }
-
-        if (fileIo != this.fileIo) {
-            return;
-        }
-
-        readWriteLock.writeLock().lock();
-
-        try {
-            if (fileIo != this.fileIo) {
-                return;
-            }
-
-            try {
-                boolean interrupted = false;
-
-                while (true) {
-                    try {
-                        fileIo = null;
-
-                        fileIo = ioFactory.create(filePath, CREATE, READ, WRITE);
+    public void setCompleteCreationDeltaFilePageStoreIoCallback(CompleteCreationDeltaFilePageStoreIoCallback callback) {
+        completeCreationDeltaFilePageStoreIoCallback = callback;
+    }
 
-                        fileExists = true;
+    /**
+     * Gets or creates a new delta file, a new delta file will be created when the previous one is {@link #completeNewDeltaFile()
+     * completed}.
+     *
+     * <p>Thread safe.
+     *
+     * @param pageIndexesSupplier Page indexes supplier for the new delta file page store.
+     * @return Future that will be completed when the new delta file page store is created.
+     */
+    public CompletableFuture<DeltaFilePageStoreIo> getOrCreateNewDeltaFile(Supplier<int[]> pageIndexesSupplier) {

Review Comment:
   Discussed personally, it is needed to lazily calculate page indexes only when creating delta files.



-- 
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 #947: IGNITE-17372 Implement DeltaFilePageStore

Posted by GitBox <gi...@apache.org>.
tkalkirill commented on code in PR #947:
URL: https://github.com/apache/ignite-3/pull/947#discussion_r928622604


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStoreFactory.java:
##########
@@ -52,45 +54,67 @@ public FilePageStoreFactory(FileIoFactory fileIoFactory, int pageSize) {
     /**
      * Creates instance of {@link FilePageStore}.
      *
-     * <p>If the file exists, an attempt will be made to read its {@link FilePageStoreHeader header} and create the {@link FilePageStore}.
+     * <p>Page stores are created based on their headers, for a file page stores with no header, the latest version is generated for delta
+     * file page store files, headers must be present.
      *
-     * @param filePath File page store path.
-     * @param headerBuffer Buffer for reading {@link FilePageStoreHeader header} from {@code filePath}.
-     * @return File page store.
+     * @param headerBuffer Buffer for reading headers.
+     * @param filePageStorePath File page store path.

Review Comment:
   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] tkalkirill commented on a diff in pull request #947: IGNITE-17372 Implement DeltaFilePageStore

Posted by GitBox <gi...@apache.org>.
tkalkirill commented on code in PR #947:
URL: https://github.com/apache/ignite-3/pull/947#discussion_r928625187


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStoreManager.java:
##########
@@ -305,14 +340,99 @@ private List<FilePageStore> createFilePageStores(
 
         try {
             for (int i = 0; i < partitions; i++) {
-                Path partFilePath = groupWorkDir.resolve(String.format(PART_FILE_TEMPLATE, i));
+                int part = i;

Review Comment:
   because it is used in lambdas



-- 
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 #947: IGNITE-17372 Implement DeltaFilePageStore

Posted by GitBox <gi...@apache.org>.
tkalkirill commented on code in PR #947:
URL: https://github.com/apache/ignite-3/pull/947#discussion_r928728118


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStoreManager.java:
##########
@@ -305,14 +340,99 @@ private List<FilePageStore> createFilePageStores(
 
         try {
             for (int i = 0; i < partitions; i++) {
-                Path partFilePath = groupWorkDir.resolve(String.format(PART_FILE_TEMPLATE, i));
+                int part = i;
+
+                Path partFilePath = groupWorkDir.resolve(String.format(PART_FILE_TEMPLATE, part));
+
+                Path[] partDeltaFiles = findPartitionDeltaFiles(groupWorkDir, partitions);
+
+                FilePageStore filePageStore = filePageStoreFactory.createPageStore(buffer.rewind(), partFilePath, partDeltaFiles);
+
+                filePageStore.setDeltaFilePageStoreIoFactory((index, pageIndexes) -> createLatest(groupWorkDir, part, index, pageIndexes));
+
+                filePageStore.setCompleteCreationDeltaFilePageStoreIoCallback(deltaIo -> renameDeltaFile(groupWorkDir, part, deltaIo));

Review Comment:
   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