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/11/08 17:40:45 UTC

[GitHub] [ignite-3] tkalkirill opened a new pull request, #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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

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


-- 
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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/compaction/Compactor.java:
##########
@@ -164,77 +170,90 @@ void waitDeltaFiles() {
     }
 
     /**
-     * Adds the number of delta files to compact.
-     *
-     * @param count Number of delta files.
+     * Callback on adding delta files so we can start compacting them.
      */
-    public void addDeltaFiles(int count) {
-        assert count >= 0;
-
-        if (count > 0) {
-            deltaFileCount.addAndGet(count);
+    public void onAddingDeltaFiles() {
+        synchronized (mux) {
+            addedDeltaFiles = true;
 
-            synchronized (mux) {
-                mux.notifyAll();
-            }
+            mux.notifyAll();
         }
     }
 
     /**
      * Merges delta files with partition files.
      */
     void doCompaction() {
-        // Let's collect one delta file for each partition.
-        Queue<IgniteBiTuple<FilePageStore, DeltaFilePageStoreIo>> queue = filePageStoreManager.allPageStores().stream()
-                .flatMap(List::stream)
-                .map(filePageStore -> {
+        while (true) {
+            // Let's collect one delta file for each partition.
+            ConcurrentLinkedQueue<DeltaFileToCompaction> queue = new ConcurrentLinkedQueue<>();
+
+            for (GroupPageStores<FilePageStore> groupPageStores : filePageStoreManager.allPageStores()) {
+                for (PartitionPageStore<FilePageStore> partitionPageStore : groupPageStores.getAll()) {
+                    FilePageStore filePageStore = partitionPageStore.pageStore();
+
                     DeltaFilePageStoreIo deltaFileToCompaction = filePageStore.getDeltaFileToCompaction();
 
-                    return deltaFileToCompaction == null ? null : new IgniteBiTuple<>(filePageStore, deltaFileToCompaction);
-                })
-                .filter(Objects::nonNull)
-                .collect(toCollection(ConcurrentLinkedQueue::new));
+                    if (!filePageStore.isMarkedToDestroy() && deltaFileToCompaction != null) {
+                        queue.add(new DeltaFileToCompaction(
+                                groupPageStores.groupId(),
+                                partitionPageStore.partitionId(),
+                                filePageStore,
+                                deltaFileToCompaction
+                        ));
+                    }
+                }
+            }
+
+            if (queue.isEmpty()) {
+                break;
+            }
+
+            updateHeartbeat();
 
-        assert !queue.isEmpty();
+            int threads = threadPoolExecutor == null ? 1 : threadPoolExecutor.getMaximumPoolSize();
 
-        updateHeartbeat();
+            CompletableFuture<?>[] futures = new CompletableFuture[threads];
 
-        int threads = threadPoolExecutor == null ? 1 : threadPoolExecutor.getMaximumPoolSize();
+            for (int i = 0; i < threads; i++) {
+                CompletableFuture<?> future = futures[i] = new CompletableFuture<>();
 
-        CompletableFuture<?>[] futures = new CompletableFuture[threads];
+                Runnable merger = () -> {
+                    DeltaFileToCompaction toMerge;
 
-        for (int i = 0; i < threads; i++) {
-            CompletableFuture<?> future = futures[i] = new CompletableFuture<>();
+                    try {
+                        while ((toMerge = queue.poll()) != null) {

Review Comment:
   I tried to 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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/compaction/Compactor.java:
##########
@@ -164,77 +170,90 @@ void waitDeltaFiles() {
     }
 
     /**
-     * Adds the number of delta files to compact.
-     *
-     * @param count Number of delta files.
+     * Callback on adding delta files so we can start compacting them.
      */
-    public void addDeltaFiles(int count) {
-        assert count >= 0;
-
-        if (count > 0) {
-            deltaFileCount.addAndGet(count);
+    public void onAddingDeltaFiles() {
+        synchronized (mux) {
+            addedDeltaFiles = true;
 
-            synchronized (mux) {
-                mux.notifyAll();
-            }
+            mux.notifyAll();
         }
     }
 
     /**
      * Merges delta files with partition files.
      */
     void doCompaction() {
-        // Let's collect one delta file for each partition.
-        Queue<IgniteBiTuple<FilePageStore, DeltaFilePageStoreIo>> queue = filePageStoreManager.allPageStores().stream()
-                .flatMap(List::stream)
-                .map(filePageStore -> {
+        while (true) {
+            // Let's collect one delta file for each partition.
+            ConcurrentLinkedQueue<DeltaFileToCompaction> queue = new ConcurrentLinkedQueue<>();

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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/compaction/Compactor.java:
##########
@@ -164,77 +170,90 @@ void waitDeltaFiles() {
     }
 
     /**
-     * Adds the number of delta files to compact.
-     *
-     * @param count Number of delta files.
+     * Callback on adding delta files so we can start compacting them.
      */
-    public void addDeltaFiles(int count) {
-        assert count >= 0;
-
-        if (count > 0) {
-            deltaFileCount.addAndGet(count);
+    public void onAddingDeltaFiles() {
+        synchronized (mux) {

Review Comment:
   He also guards `Compactor#addedDeltaFiles`, I think it's not worth renaming.



-- 
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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/checkpoint/CheckpointManager.java:
##########
@@ -307,11 +312,23 @@ static int[] pageIndexesForDeltaFilePageStore(CheckpointDirtyPagesView partition
     }
 
     /**
-     * Adds the number of delta files to compact.
+     * Callback on adding delta files so we can start compacting them.
+     */
+    public void onAddingDeltaFiles() {

Review Comment:
   Tried to 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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/PartitionProcessingCounter.java:
##########
@@ -0,0 +1,83 @@
+/*
+ * 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;
+
+import java.lang.invoke.MethodHandles;
+import java.lang.invoke.VarHandle;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Helper class for tracking the completion of partition processing.
+ *
+ * <p>At the start of partition processing, you need to call {@link #onStartPartitionProcessing()}, at the end
+ * {@link #onFinishPartitionProcessing()}. When all partition processing is completed, the {@link #future()} will be completed.
+ *
+ * <p>It is recommended to use external synchronization for the correct operation of the {@link #counter partition processing counter} and
+ * the {@link #future future}.
+ */
+public class PartitionProcessingCounter {
+    private static final VarHandle COUNTER;
+
+    static {
+        try {
+            COUNTER = MethodHandles.lookup().findVarHandle(PartitionProcessingCounter.class, "counter", int.class);
+        } catch (ReflectiveOperationException e) {
+            throw new ExceptionInInitializerError(e);
+        }
+    }
+
+    /** Partition processing counter must be greater than or equal to zero. */
+    @SuppressWarnings("unused")
+    private volatile int counter;
+
+    /** Future that will be completed when the {@link #counter} is zero. */
+    private final CompletableFuture<Void> future = new CompletableFuture<>();
+
+    /**
+     * Callback at the start of partition processing.

Review Comment:
   Technically, it's not a callback. It's just a method that one should call



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/checkpoint/CheckpointPagesWriterFactory.java:
##########
@@ -106,7 +104,7 @@ CheckpointPagesWriter build(
             BooleanSupplier shutdownNow
     ) {
         return new CheckpointPagesWriter(
-                log,
+                LOG,

Review Comment:
   Should the writer also have its own logger?



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/checkpoint/CheckpointManager.java:
##########
@@ -307,11 +312,23 @@ static int[] pageIndexesForDeltaFilePageStore(CheckpointDirtyPagesView partition
     }
 
     /**
-     * Adds the number of delta files to compact.
+     * Callback on adding delta files so we can start compacting them.
+     */
+    public void onAddingDeltaFiles() {

Review Comment:
   I would prefer another name, but I don't think that I have a good example. "on***" methods are good in listeners, but this call must be explicit. So I expect an explicit action in the name, like, "triggerCompaction" or something



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/checkpoint/Checkpointer.java:
##########
@@ -452,9 +456,21 @@ private void syncUpdatedPageStores(
                     return;
                 }
 
-                fsyncDeltaFilePageStoreOnCheckpointThread(entry.getKey(), entry.getValue());
+                GroupPartitionId partitionId = entry.getKey();
+
+                FilePageStore filePageStore = filePageStoreManager.getStore(partitionId.getGroupId(), partitionId.getPartitionId());
+
+                if (filePageStore == null || filePageStore.isMarkedToDestroy()) {
+                    continue;
+                }
+
+                currentCheckpointProgress.onStartPartitionProcessing(partitionId.getGroupId(), partitionId.getPartitionId());
+
+                fsyncDeltaFilePageStoreOnCheckpointThread(filePageStore, entry.getValue());
 
-                renameDeltaFileOnCheckpointThread(entry.getKey());
+                renameDeltaFileOnCheckpointThread(filePageStore, partitionId);
+
+                currentCheckpointProgress.onFinishPartitionProcessing(partitionId.getGroupId(), partitionId.getPartitionId());

Review Comment:
   Is there a chance of having a future that will never be completed if something goes wrong? I don't see a try/finally section here. I don't see it in other places, that use these methods, either. There may be bugs lurking in you code because of that



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/compaction/Compactor.java:
##########
@@ -164,77 +170,90 @@ void waitDeltaFiles() {
     }
 
     /**
-     * Adds the number of delta files to compact.
-     *
-     * @param count Number of delta files.
+     * Callback on adding delta files so we can start compacting them.
      */
-    public void addDeltaFiles(int count) {
-        assert count >= 0;
-
-        if (count > 0) {
-            deltaFileCount.addAndGet(count);
+    public void onAddingDeltaFiles() {
+        synchronized (mux) {
+            addedDeltaFiles = true;
 
-            synchronized (mux) {
-                mux.notifyAll();
-            }
+            mux.notifyAll();
         }
     }
 
     /**
      * Merges delta files with partition files.
      */
     void doCompaction() {
-        // Let's collect one delta file for each partition.
-        Queue<IgniteBiTuple<FilePageStore, DeltaFilePageStoreIo>> queue = filePageStoreManager.allPageStores().stream()
-                .flatMap(List::stream)
-                .map(filePageStore -> {
+        while (true) {

Review Comment:
   Please explain this loop. It's not obvious



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/compaction/Compactor.java:
##########
@@ -338,6 +360,10 @@ void mergeDeltaFileToMainFile(
                 return;
             }
 
+            if (filePageStore.isMarkedToDestroy()) {

Review Comment:
   Why do we double-check it?
   Is there a synchronization? What would happen if you removed this particular check and store is marked for destruction?



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStoreManager.java:
##########
@@ -139,31 +149,63 @@ public void start() throws IgniteInternalCheckedException {
             throw new IgniteInternalCheckedException("Could not create work directory for page stores: " + dbDir, e);
         }
 
-        if (log.isWarnEnabled()) {
+        if (LOG.isWarnEnabled()) {
             String tmpDir = System.getProperty("java.io.tmpdir");
 
             if (tmpDir != null && this.dbDir.startsWith(tmpDir)) {
-                log.warn("Persistence store directory is in the temp directory and may be cleaned. "
+                LOG.warn("Persistence store directory is in the temp directory and may be cleaned. "
                         + "To avoid this change location of persistence directories [currentDir={}]", this.dbDir);
             }
         }
 
+        List<Path> toDelete = new ArrayList<>();
+
         try (Stream<Path> tmpFileStream = Files.find(
                 dbDir,
                 Integer.MAX_VALUE,
-                (path, basicFileAttributes) -> path.getFileName().toString().endsWith(TMP_FILE_SUFFIX)
-        )) {
-            List<Path> tmpFiles = tmpFileStream.collect(toList());
+                (path, basicFileAttributes) -> path.getFileName().toString().endsWith(TMP_FILE_SUFFIX))
+        ) {
+            toDelete.addAll(tmpFileStream.collect(toList()));
+        } catch (IOException e) {
+            throw new IgniteInternalCheckedException("Error while searching temporary files:" + dbDir, e);
+        }
+
+        Pattern delPartitionFilePatter = Pattern.compile(DEL_PART_FILE_REGEXP);
+
+        try (Stream<Path> delFileStream = Files.find(
+                dbDir,
+                Integer.MAX_VALUE,
+                (path, basicFileAttributes) -> path.getFileName().toString().endsWith(DEL_FILE_SUFFIX))
+        ) {
+            delFileStream.forEach(delFilePath -> {
+                Matcher matcher = delPartitionFilePatter.matcher(delFilePath.getFileName().toString());
 
-            if (!tmpFiles.isEmpty()) {
-                if (log.isInfoEnabled()) {
-                    log.info("Temporary files to be deleted: {}", tmpFiles.size());
+                if (!matcher.matches()) {
+                    throw new IgniteInternalException("Unknown file: " + delFilePath);
                 }
 
-                tmpFiles.forEach(IgniteUtils::deleteIfExists);
-            }
+                Path tableWorkDir = delFilePath.getParent();
+
+                int partitionId = Integer.parseInt(matcher.group(1));
+
+                toDelete.add(tableWorkDir.resolve(String.format(PART_FILE_TEMPLATE, partitionId)));
+
+                try {
+                    toDelete.addAll(List.of(findPartitionDeltaFiles(tableWorkDir, partitionId)));
+                } catch (IgniteInternalCheckedException e) {
+                    throw new IgniteInternalException("Error when searching delta files for partition:" + delFilePath, e);
+                }
+
+                toDelete.add(delFilePath);
+            });
         } catch (IOException e) {
-            throw new IgniteInternalCheckedException("Could not create work directory for page stores: " + dbDir, e);
+            throw new IgniteInternalCheckedException("Error while searching temporary files:" + dbDir, e);
+        }
+
+        if (!toDelete.isEmpty()) {
+            LOG.info("Files to be deleted: {}", toDelete);

Review Comment:
   Please add a line with reasons to delete these files. Something generic, like "Cleaning obsolete files for table x"



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStoreManager.java:
##########
@@ -443,4 +461,50 @@ public Path tmpDeltaFilePageStorePath(int groupId, int partitionId, int index) {
     public Path deltaFilePageStorePath(int groupId, int partitionId, int index) {
         return dbDir.resolve(GROUP_DIR_PREFIX + groupId).resolve(String.format(PART_DELTA_FILE_TEMPLATE, partitionId, index));
     }
+
+    /**
+     * Callback on destruction of the partition of the corresponding group.
+     *
+     * <p>Deletes the partition pages tore and all its delta files. Before that, it creates a marker file (for example,

Review Comment:
   ```suggestion
        * <p>Deletes the partition pages store and all its delta files. Before that, it creates a marker file (for example,
   ```



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/GroupPageStoresMap.java:
##########
@@ -47,52 +50,156 @@ public GroupPageStoresMap(LongOperationAsyncExecutor longOperationAsyncExecutor)
     }
 
     /**
-     * Puts the page stores for the group.
+     * Puts the page store for the group partition.
+     *
+     * @param groupId Group ID.
+     * @param partitionId Partition ID.
+     * @param pageStore Page store.
+     * @return Previous page store.
+     */
+    public @Nullable T put(Integer groupId, Integer partitionId, T pageStore) {
+        return longOperationAsyncExecutor.afterAsyncCompletion(() -> {
+                    PartitionPageStore<T> previous = groupIdPageStores
+                            .computeIfAbsent(groupId, id -> new GroupPageStores<>(groupId))
+                            .partitionIdPageStore
+                            .put(partitionId, new PartitionPageStore<>(partitionId, pageStore));
+
+                    return previous == null ? null : previous.pageStore;
+                }
+        );
+    }
+
+    /**
+     * Removes the page store for the group partition.
      *
-     * @param grpId Group ID.
-     * @param pageStores Page stores.
-     * @return Previous page stores.
+     * @param groupId Group ID.
+     * @param partitionId Partition ID.
+     * @return Removed page store.
      */
-    public @Nullable List<T> put(Integer grpId, List<T> pageStores) {
-        return longOperationAsyncExecutor.afterAsyncCompletion(() -> groupPageStores.put(grpId, pageStores));
+    public @Nullable T remove(Integer groupId, Integer partitionId) {
+        AtomicReference<PartitionPageStore<T>> partitionPageStoreRef = new AtomicReference<>();
+
+        groupIdPageStores.compute(groupId, (id, groupPageStores) -> {
+            if (groupPageStores == null) {
+                return null;
+            }
+
+            partitionPageStoreRef.set(groupPageStores.partitionIdPageStore.remove(partitionId));
+
+            if (groupPageStores.partitionIdPageStore.isEmpty()) {
+                return null;
+            }
+
+            return groupPageStores;
+        });
+
+        PartitionPageStore<T> partitionPageStore = partitionPageStoreRef.get();
+
+        return partitionPageStore == null ? null : partitionPageStore.pageStore;
     }
 
     /**
      * Returns the page stores for the group.
      *
-     * @param grpId Group ID.
+     * @param groupId Group ID.
      */
-    public @Nullable List<T> get(Integer grpId) {
-        return groupPageStores.get(grpId);
+    public @Nullable GroupPageStores<T> get(Integer groupId) {
+        return groupIdPageStores.get(groupId);
     }
 
     /**
-     * Returns {@code true} if a page stores exists for the group.
+     * Returns {@code true} if a page store exists for the group partition.
      *
-     * @param grpId Group ID.
+     * @param groupId Group ID.
+     * @param partitionId Partition ID.
      */
-    public boolean containsPageStores(Integer grpId) {
-        return groupPageStores.containsKey(grpId);
+    public boolean contains(Integer groupId, Integer partitionId) {
+        GroupPageStores<T> groupPageStores = groupIdPageStores.get(groupId);
+
+        return groupPageStores != null && groupPageStores.partitionIdPageStore.containsKey(partitionId);
     }
 
     /**
-     * Returns all page stores of all groups.
+     * Returns a view of all page stores of all groups.
      */
-    public Collection<List<T>> allPageStores() {
-        return groupPageStores.values();
+    public Collection<GroupPageStores<T>> getAll() {
+        return unmodifiableCollection(groupIdPageStores.values());
     }
 
     /**
      * Clears all page stores of all groups.
      */
     public void clear() {
-        groupPageStores.clear();
+        groupIdPageStores.clear();
     }
 
     /**
      * Returns the number of groups for which there are page stores.
      */
     public int groupCount() {
-        return groupPageStores.size();
+        return groupIdPageStores.size();
+    }
+
+    /**
+     * Group partition page stores.
+     */
+    public static class GroupPageStores<T extends PageStore> {
+        private final int groupId;
+
+        private final ConcurrentMap<Integer, PartitionPageStore<T>> partitionIdPageStore = new ConcurrentHashMap<>();
+
+        private GroupPageStores(int groupId) {
+            this.groupId = groupId;
+        }
+
+        /**
+         * Returns the group ID.
+         */
+        public int groupId() {
+            return groupId;
+        }
+
+        /**
+         * Returns the partition's page store.
+         */
+        @Nullable
+        public PartitionPageStore<T> get(Integer partitionId) {
+            return partitionIdPageStore.get(partitionId);
+        }
+
+        /**
+         * Returns a view of the group's partition page stores.
+         */
+        public Collection<PartitionPageStore<T>> getAll() {
+            return unmodifiableCollection(partitionIdPageStore.values());
+        }
+    }
+
+    /**
+     * Partition page store.
+     */
+    public static class PartitionPageStore<T extends PageStore> {
+        private final int partitionId;
+
+        private final T pageStore;

Review Comment:
   Does PagesStore have a getter for partitionId? Maybe it should, please validate this idea



##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/PersistentPageMemoryTableStorage.java:
##########
@@ -86,41 +97,29 @@ public PersistentPageMemoryDataRegion dataRegion() {
         return dataRegion;
     }
 
-    /** {@inheritDoc} */
     @Override
     public boolean isVolatile() {
         return false;
     }
 
-    /** {@inheritDoc} */
-    @Override
-    public void start() throws StorageException {
-        super.start();
-
-        TableView tableView = tableCfg.value();
-
-        try {
-            dataRegion.filePageStoreManager().initialize(tableView.name(), tableView.tableId(), tableView.partitions());
-
-            int deltaFileCount = dataRegion.filePageStoreManager().getStores(tableView.tableId()).stream()
-                    .mapToInt(FilePageStore::deltaFileCount)
-                    .sum();
-
-            dataRegion.checkpointManager().addDeltaFileCountForCompaction(deltaFileCount);
-        } catch (IgniteInternalCheckedException e) {
-            throw new StorageException("Error initializing file page stores for table: " + tableView.name(), e);
-        }
-    }
-
-    /** {@inheritDoc} */
     @Override
     public void destroy() throws StorageException {
         close(true);
     }
 
-    /** {@inheritDoc} */
     @Override
     public PersistentPageMemoryMvPartitionStorage createMvPartitionStorage(int partitionId) {
+        CompletableFuture<Void> partitionDestroyFuture = destroyFutureByPartitionId.get(partitionId);
+
+        if (partitionDestroyFuture != null) {
+            try {
+                // Time is chosen randomly (long enough) so as not to call #join().
+                partitionDestroyFuture.get(10, TimeUnit.SECONDS);

Review Comment:
   Again, "randomly" and "long enough" - I don't like it. This particular method is trickier, we should think about what to do here and how to design things.



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/compaction/Compactor.java:
##########
@@ -164,77 +170,90 @@ void waitDeltaFiles() {
     }
 
     /**
-     * Adds the number of delta files to compact.
-     *
-     * @param count Number of delta files.
+     * Callback on adding delta files so we can start compacting them.
      */
-    public void addDeltaFiles(int count) {
-        assert count >= 0;
-
-        if (count > 0) {
-            deltaFileCount.addAndGet(count);
+    public void onAddingDeltaFiles() {
+        synchronized (mux) {
+            addedDeltaFiles = true;
 
-            synchronized (mux) {
-                mux.notifyAll();
-            }
+            mux.notifyAll();
         }
     }
 
     /**
      * Merges delta files with partition files.
      */
     void doCompaction() {

Review Comment:
   Who calls this method? Pretty small description for a method this big. I see that it waits for some futures at the end. This means that it blocks a caller thread. Is this necessary? Please describe what's going on, code itself is hard to understand.



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/checkpoint/Checkpointer.java:
##########
@@ -775,4 +799,41 @@ private void renameDeltaFileOnCheckpointThread(GroupPartitionId partitionId) thr
     void updateLastProgressAfterReleaseWriteLock() {
         afterReleaseWriteLockCheckpointProgress = currentCheckpointProgress;
     }
+
+    /**
+     * Callback on destruction of the partition of the corresponding group.
+     *
+     * <p>If the checkpoint is in progress, then wait until it finishes processing the partition that we are going to destroy, in order to
+     * prevent the situation when we want to destroy the partition file along with its delta files, and at this time the checkpoint performs
+     * I/O operations on them.
+     *
+     * @param groupId Group ID.
+     * @param partitionId Partition ID.
+     * @throws IgniteInternalCheckedException If there are errors while processing the callback.
+     */
+    void onPartitionDestruction(int groupId, int partitionId) throws IgniteInternalCheckedException {
+        CheckpointProgressImpl currentCheckpointProgress = this.currentCheckpointProgress;
+
+        if (currentCheckpointProgress == null || !currentCheckpointProgress.inProgress()) {
+            return;
+        }
+
+        CompletableFuture<Void> processedPartitionFuture = currentCheckpointProgress.getProcessedPartitionFuture(groupId, partitionId);
+
+        if (processedPartitionFuture != null) {
+            try {
+                // Time is taken arbitrarily, but long enough to allow time for the future to complete.
+                processedPartitionFuture.get(10, SECONDS);

Review Comment:
   Wait, what?
   First of all, please don't hard-code such constants.
   Second, we're constantly suffer from other people using "get" or "join" on futures, saying that it's a horrible practice. And now I see this, without proper explanation of why it needs to be here.
   So, either explain thoroughly, or even better, re-write this code to avoid explicit await.



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/PartitionProcessingCounter.java:
##########
@@ -0,0 +1,83 @@
+/*
+ * 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;
+
+import java.lang.invoke.MethodHandles;
+import java.lang.invoke.VarHandle;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Helper class for tracking the completion of partition processing.
+ *
+ * <p>At the start of partition processing, you need to call {@link #onStartPartitionProcessing()}, at the end
+ * {@link #onFinishPartitionProcessing()}. When all partition processing is completed, the {@link #future()} will be completed.
+ *
+ * <p>It is recommended to use external synchronization for the correct operation of the {@link #counter partition processing counter} and
+ * the {@link #future future}.
+ */
+public class PartitionProcessingCounter {

Review Comment:
   Should this class be package-private?



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/compaction/Compactor.java:
##########
@@ -65,13 +66,17 @@ public class Compactor extends IgniteWorker {
 
     private final @Nullable ThreadPoolExecutor threadPoolExecutor;
 
-    private final AtomicInteger deltaFileCount = new AtomicInteger();
+    /** Guarded by {@link #mux}. */
+    private boolean addedDeltaFiles;
 
     private final FilePageStoreManager filePageStoreManager;
 
     /** Thread local with buffers for the compaction threads. */
     private final ThreadLocal<ByteBuffer> threadBuf;

Review Comment:
   By the way, why not "static". Is it possible that several compactor workers operate in a single thread? I don't think so



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/checkpoint/CheckpointManager.java:
##########
@@ -307,11 +312,23 @@ static int[] pageIndexesForDeltaFilePageStore(CheckpointDirtyPagesView partition
     }
 
     /**
-     * Adds the number of delta files to compact.
+     * Callback on adding delta files so we can start compacting them.
+     */
+    public void onAddingDeltaFiles() {
+        compactor.onAddingDeltaFiles();
+    }
+
+    /**
+     * Callback on destruction of the partition of the corresponding group.
+     *
+     * <p>Prepares the checkpointer and compactor for partition destruction.
      *
-     * @param count Number of delta files.
+     * @param groupId Group ID.
+     * @param partitionId Partition ID.
+     * @throws IgniteInternalCheckedException If there are errors while processing the callback.
      */
-    public void addDeltaFileCountForCompaction(int count) {
-        compactor.addDeltaFiles(count);
+    public void onPartitionDestruction(int groupId, int partitionId) throws IgniteInternalCheckedException {

Review Comment:
   This one doesn't return any future. I guess we get them differently?



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/compaction/Compactor.java:
##########
@@ -164,77 +170,90 @@ void waitDeltaFiles() {
     }
 
     /**
-     * Adds the number of delta files to compact.
-     *
-     * @param count Number of delta files.
+     * Callback on adding delta files so we can start compacting them.
      */
-    public void addDeltaFiles(int count) {
-        assert count >= 0;
-
-        if (count > 0) {
-            deltaFileCount.addAndGet(count);
+    public void onAddingDeltaFiles() {
+        synchronized (mux) {
+            addedDeltaFiles = true;
 
-            synchronized (mux) {
-                mux.notifyAll();
-            }
+            mux.notifyAll();
         }
     }
 
     /**
      * Merges delta files with partition files.
      */
     void doCompaction() {
-        // Let's collect one delta file for each partition.
-        Queue<IgniteBiTuple<FilePageStore, DeltaFilePageStoreIo>> queue = filePageStoreManager.allPageStores().stream()
-                .flatMap(List::stream)
-                .map(filePageStore -> {
+        while (true) {
+            // Let's collect one delta file for each partition.
+            ConcurrentLinkedQueue<DeltaFileToCompaction> queue = new ConcurrentLinkedQueue<>();
+
+            for (GroupPageStores<FilePageStore> groupPageStores : filePageStoreManager.allPageStores()) {
+                for (PartitionPageStore<FilePageStore> partitionPageStore : groupPageStores.getAll()) {
+                    FilePageStore filePageStore = partitionPageStore.pageStore();
+
                     DeltaFilePageStoreIo deltaFileToCompaction = filePageStore.getDeltaFileToCompaction();
 
-                    return deltaFileToCompaction == null ? null : new IgniteBiTuple<>(filePageStore, deltaFileToCompaction);
-                })
-                .filter(Objects::nonNull)
-                .collect(toCollection(ConcurrentLinkedQueue::new));
+                    if (!filePageStore.isMarkedToDestroy() && deltaFileToCompaction != null) {
+                        queue.add(new DeltaFileToCompaction(
+                                groupPageStores.groupId(),
+                                partitionPageStore.partitionId(),
+                                filePageStore,
+                                deltaFileToCompaction
+                        ));
+                    }
+                }
+            }
+
+            if (queue.isEmpty()) {

Review Comment:
   So, with this queue you're supervising the thread-pool, right?
   You read a batch of delta files, distribute them across the pool, and then read the next batch and so on.
   If what I described is correct, you're under-utilizing resources. All threads must wait until the last thread finishes its delta file, and only then the next batch is created.
   Why have you decided to use this approach? Was this all intentional? Did you realize that threads are under-utilized in this implementation?



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/compaction/Compactor.java:
##########
@@ -146,7 +150,7 @@ protected void body() throws InterruptedException {
     void waitDeltaFiles() {

Review Comment:
   How many threads execute this method? If more than one, then I see a huge problem.
   First thread to wake up will set "addedDeltaFiles" to false. Other threads will see that and continue the loop indefinitely (or until cancellation, to be precise).
   Basically, with this change you ruined multi-threading in compactor, please be careful.



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/compaction/Compactor.java:
##########
@@ -164,77 +170,90 @@ void waitDeltaFiles() {
     }
 
     /**
-     * Adds the number of delta files to compact.
-     *
-     * @param count Number of delta files.
+     * Callback on adding delta files so we can start compacting them.
      */
-    public void addDeltaFiles(int count) {
-        assert count >= 0;
-
-        if (count > 0) {
-            deltaFileCount.addAndGet(count);
+    public void onAddingDeltaFiles() {
+        synchronized (mux) {
+            addedDeltaFiles = true;
 
-            synchronized (mux) {
-                mux.notifyAll();
-            }
+            mux.notifyAll();
         }
     }
 
     /**
      * Merges delta files with partition files.
      */
     void doCompaction() {
-        // Let's collect one delta file for each partition.
-        Queue<IgniteBiTuple<FilePageStore, DeltaFilePageStoreIo>> queue = filePageStoreManager.allPageStores().stream()
-                .flatMap(List::stream)
-                .map(filePageStore -> {
+        while (true) {
+            // Let's collect one delta file for each partition.
+            ConcurrentLinkedQueue<DeltaFileToCompaction> queue = new ConcurrentLinkedQueue<>();
+
+            for (GroupPageStores<FilePageStore> groupPageStores : filePageStoreManager.allPageStores()) {
+                for (PartitionPageStore<FilePageStore> partitionPageStore : groupPageStores.getAll()) {
+                    FilePageStore filePageStore = partitionPageStore.pageStore();
+
                     DeltaFilePageStoreIo deltaFileToCompaction = filePageStore.getDeltaFileToCompaction();
 
-                    return deltaFileToCompaction == null ? null : new IgniteBiTuple<>(filePageStore, deltaFileToCompaction);
-                })
-                .filter(Objects::nonNull)
-                .collect(toCollection(ConcurrentLinkedQueue::new));
+                    if (!filePageStore.isMarkedToDestroy() && deltaFileToCompaction != null) {
+                        queue.add(new DeltaFileToCompaction(
+                                groupPageStores.groupId(),
+                                partitionPageStore.partitionId(),
+                                filePageStore,
+                                deltaFileToCompaction
+                        ));
+                    }
+                }
+            }
+
+            if (queue.isEmpty()) {
+                break;
+            }
+
+            updateHeartbeat();
 
-        assert !queue.isEmpty();
+            int threads = threadPoolExecutor == null ? 1 : threadPoolExecutor.getMaximumPoolSize();

Review Comment:
   How can it be null? Is it possible to set it up in this way?



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/compaction/Compactor.java:
##########
@@ -357,14 +387,70 @@ void mergeDeltaFileToMainFile(
             return;
         }
 
+        if (filePageStore.isMarkedToDestroy()) {
+            return;
+        }
+
         boolean removed = filePageStore.removeDeltaFile(deltaFilePageStore);
 
         assert removed : filePageStore.filePath();
 
         deltaFilePageStore.markMergedToFilePageStore();
 
         deltaFilePageStore.stop(true);
+    }
+
+    /**
+     * Callback on destruction of the partition of the corresponding group.
+     *
+     * <p>If the partition compaction is in progress, then we will wait until it is completed so that there are no errors when we want to
+     * destroy the partition file and its delta file, and at this time its compaction occurs.
+     *
+     * @param groupId Group ID.
+     * @param partitionId Partition ID.
+     */
+    public void onPartitionDestruction(int groupId, int partitionId) throws IgniteInternalCheckedException {
+        CompletableFuture<Void> partitionProcessingFuture = processedPartitionMap.getProcessedPartitionFuture(groupId, partitionId);
+
+        if (partitionProcessingFuture != null) {
+            try {
+                // Time is taken arbitrarily, but long enough to allow time for the future to complete.
+                partitionProcessingFuture.get(10, SECONDS);

Review Comment:
   Again, I don't like this code. I very-very don't like it.
   For some clients, checkpoints may take, like, a minute. So 10 seconds isn't enough.
   And, even if it is enough, having all these "get" calls is a bad practice.



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/compaction/Compactor.java:
##########
@@ -357,14 +387,70 @@ void mergeDeltaFileToMainFile(
             return;
         }
 
+        if (filePageStore.isMarkedToDestroy()) {
+            return;
+        }
+
         boolean removed = filePageStore.removeDeltaFile(deltaFilePageStore);
 
         assert removed : filePageStore.filePath();
 
         deltaFilePageStore.markMergedToFilePageStore();
 
         deltaFilePageStore.stop(true);
+    }
+
+    /**
+     * Callback on destruction of the partition of the corresponding group.
+     *
+     * <p>If the partition compaction is in progress, then we will wait until it is completed so that there are no errors when we want to
+     * destroy the partition file and its delta file, and at this time its compaction occurs.
+     *
+     * @param groupId Group ID.
+     * @param partitionId Partition ID.
+     */
+    public void onPartitionDestruction(int groupId, int partitionId) throws IgniteInternalCheckedException {
+        CompletableFuture<Void> partitionProcessingFuture = processedPartitionMap.getProcessedPartitionFuture(groupId, partitionId);
+
+        if (partitionProcessingFuture != null) {
+            try {
+                // Time is taken arbitrarily, but long enough to allow time for the future to complete.
+                partitionProcessingFuture.get(10, SECONDS);
+            } catch (Exception e) {
+                throw new IgniteInternalCheckedException(
+                        IgniteStringFormatter.format(
+                                "Error waiting for partition processing to complete on compaction: [groupId={}, partitionId={}]",
+                                groupId,
+                                partitionId
+                        ),
+                        e
+                );
+            }
+        }
+    }
+
+    /**
+     * Delta file for compaction.
+     */
+    private static class DeltaFileToCompaction {

Review Comment:
   It's "for compaction", or "to compact", there's no other option



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStoreManager.java:
##########
@@ -320,9 +369,9 @@ void stopAllGroupFilePageStores(boolean cleanFiles) {
             try {
                 stopGroupFilePageStores(partitionPageStores, cleanFiles);
 
-                log.info("Cleanup cache stores [total={}, cleanFiles={}]", partitionPageStores.size(), cleanFiles);
+                LOG.info("Cleanup cache stores [total={}, cleanFiles={}]", partitionPageStores.size(), cleanFiles);
             } catch (Exception e) {
-                log.info("Failed to gracefully stop page store managers", e);
+                LOG.info("Failed to gracefully stop page store managers", e);

Review Comment:
   Are you sure this is "info" level event?



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/GroupPageStoresMap.java:
##########
@@ -47,52 +50,156 @@ public GroupPageStoresMap(LongOperationAsyncExecutor longOperationAsyncExecutor)
     }
 
     /**
-     * Puts the page stores for the group.
+     * Puts the page store for the group partition.
+     *
+     * @param groupId Group ID.
+     * @param partitionId Partition ID.
+     * @param pageStore Page store.
+     * @return Previous page store.
+     */
+    public @Nullable T put(Integer groupId, Integer partitionId, T pageStore) {
+        return longOperationAsyncExecutor.afterAsyncCompletion(() -> {
+                    PartitionPageStore<T> previous = groupIdPageStores
+                            .computeIfAbsent(groupId, id -> new GroupPageStores<>(groupId))
+                            .partitionIdPageStore
+                            .put(partitionId, new PartitionPageStore<>(partitionId, pageStore));
+
+                    return previous == null ? null : previous.pageStore;
+                }
+        );
+    }
+
+    /**
+     * Removes the page store for the group partition.
      *
-     * @param grpId Group ID.
-     * @param pageStores Page stores.
-     * @return Previous page stores.
+     * @param groupId Group ID.
+     * @param partitionId Partition ID.
+     * @return Removed page store.
      */
-    public @Nullable List<T> put(Integer grpId, List<T> pageStores) {
-        return longOperationAsyncExecutor.afterAsyncCompletion(() -> groupPageStores.put(grpId, pageStores));
+    public @Nullable T remove(Integer groupId, Integer partitionId) {
+        AtomicReference<PartitionPageStore<T>> partitionPageStoreRef = new AtomicReference<>();
+
+        groupIdPageStores.compute(groupId, (id, groupPageStores) -> {

Review Comment:
   Should we use a map with primitive keys?



##########
modules/page-memory/src/test/java/org/apache/ignite/internal/pagememory/persistence/store/GroupPageStoresMapTest.java:
##########
@@ -52,125 +57,265 @@ void setUp() {
         when(longOperationAsyncExecutor.afterAsyncCompletion(any(Supplier.class)))
                 .then(answer -> ((Supplier<?>) answer.getArgument(0)).get());
 
-        groupPageStores = new GroupPageStoresMap<>(longOperationAsyncExecutor);
+        groupPageStoresMap = new GroupPageStoresMap<>(longOperationAsyncExecutor);
     }
 
     @Test
     void testGroupCount() {
-        assertEquals(0, groupPageStores.groupCount());
+        assertEquals(0, groupPageStoresMap.groupCount());
 
-        groupPageStores.put(0, List.of());
+        groupPageStoresMap.put(0, 0, mock(FilePageStore.class));
 
-        assertEquals(1, groupPageStores.groupCount());
+        assertEquals(1, groupPageStoresMap.groupCount());
 
-        groupPageStores.put(0, List.of());
+        groupPageStoresMap.put(0, 0, mock(FilePageStore.class));
 
-        assertEquals(1, groupPageStores.groupCount());
+        assertEquals(1, groupPageStoresMap.groupCount());
 
-        groupPageStores.put(1, List.of());
+        groupPageStoresMap.put(1, 0, mock(FilePageStore.class));
 
-        assertEquals(2, groupPageStores.groupCount());
+        assertEquals(2, groupPageStoresMap.groupCount());
 
-        groupPageStores.clear();
+        groupPageStoresMap.clear();
 
-        assertEquals(0, groupPageStores.groupCount());
+        assertEquals(0, groupPageStoresMap.groupCount());
     }
 
     @Test
     void testPut() {
-        List<PageStore> holder0 = List.of();
+        FilePageStore filePageStore0 = mock(FilePageStore.class);
 
-        assertNull(groupPageStores.put(0, holder0));
+        assertNull(groupPageStoresMap.put(0, 0, filePageStore0));
         checkInvokeAfterAsyncCompletion(1);
 
-        List<PageStore> holder1 = List.of();
+        FilePageStore filePageStore1 = mock(FilePageStore.class);
 
-        assertSame(holder0, groupPageStores.put(0, holder1));
+        assertSame(filePageStore0, groupPageStoresMap.put(0, 0, filePageStore1));
         checkInvokeAfterAsyncCompletion(2);
 
-        assertSame(holder1, groupPageStores.get(0));
-        assertNull(groupPageStores.get(1));
+        assertNull(groupPageStoresMap.get(1));
+
+        assertThat(getAll(groupPageStoresMap.get(0)), containsInAnyOrder(new TestPartitionPageStore<>(0, filePageStore1)));
     }
 
     @Test
     void testGet() {
-        assertNull(groupPageStores.get(0));
-        assertNull(groupPageStores.get(1));
+        assertNull(groupPageStoresMap.get(0));
+        assertNull(groupPageStoresMap.get(1));
 
-        List<PageStore> pageStores0 = List.of();
+        FilePageStore filePageStore0 = mock(FilePageStore.class);
 
-        groupPageStores.put(0, pageStores0);
+        groupPageStoresMap.put(0, 0, filePageStore0);
 
-        assertSame(pageStores0, groupPageStores.get(0));
-        assertNull(groupPageStores.get(1));
+        GroupPageStores<PageStore> groupPageStores0 = groupPageStoresMap.get(0);
 
-        List<PageStore> pageStores1 = List.of();
+        assertThat(getAll(groupPageStores0), containsInAnyOrder(new TestPartitionPageStore<>(0, filePageStore0)));
+        assertNull(groupPageStoresMap.get(1));
 
-        groupPageStores.put(1, pageStores1);
+        FilePageStore filePageStore1 = mock(FilePageStore.class);
 
-        assertSame(pageStores0, groupPageStores.get(0));
-        assertSame(pageStores1, groupPageStores.get(1));
+        groupPageStoresMap.put(1, 1, filePageStore1);
 
-        groupPageStores.clear();
+        GroupPageStores<PageStore> groupPageStores1 = groupPageStoresMap.get(1);
 
-        assertNull(groupPageStores.get(0));
-        assertNull(groupPageStores.get(1));
+        assertThat(getAll(groupPageStores0), containsInAnyOrder(new TestPartitionPageStore<>(0, filePageStore0)));
+        assertThat(getAll(groupPageStores1), containsInAnyOrder(new TestPartitionPageStore<>(1, filePageStore1)));
+
+        assertEquals(TestPartitionPageStore.of(groupPageStores0.get(0)), new TestPartitionPageStore<>(0, filePageStore0));
+        assertEquals(TestPartitionPageStore.of(groupPageStores1.get(1)), new TestPartitionPageStore<>(1, filePageStore1));
+
+        assertNull(groupPageStores0.get(2));
+        assertNull(groupPageStores1.get(2));
+
+        groupPageStoresMap.clear();
+
+        assertNull(groupPageStoresMap.get(0));
+        assertNull(groupPageStoresMap.get(1));
     }
 
     @Test
     void testContainsPageStores() {
-        assertFalse(groupPageStores.containsPageStores(0));
-        assertFalse(groupPageStores.containsPageStores(1));
+        assertFalse(groupPageStoresMap.contains(0, 0));
+        assertFalse(groupPageStoresMap.contains(1, 0));
+
+        FilePageStore filePageStore0 = mock(FilePageStore.class);
 
-        groupPageStores.put(0, List.of());
+        groupPageStoresMap.put(0, 0, filePageStore0);
 
-        assertTrue(groupPageStores.containsPageStores(0));
-        assertFalse(groupPageStores.containsPageStores(1));
+        assertTrue(groupPageStoresMap.contains(0, 0));
+        assertFalse(groupPageStoresMap.contains(1, 0));
+        assertFalse(groupPageStoresMap.contains(0, 1));
 
-        List<PageStore> pageStores1 = List.of();
+        FilePageStore filePageStore1 = mock(FilePageStore.class);
 
-        groupPageStores.put(1, List.of());
+        groupPageStoresMap.put(1, 0, filePageStore1);
 
-        assertTrue(groupPageStores.containsPageStores(0));
-        assertTrue(groupPageStores.containsPageStores(1));
+        assertTrue(groupPageStoresMap.contains(0, 0));
+        assertTrue(groupPageStoresMap.contains(1, 0));
 
-        groupPageStores.clear();
+        groupPageStoresMap.clear();
 
-        assertFalse(groupPageStores.containsPageStores(0));
-        assertFalse(groupPageStores.containsPageStores(1));
+        assertFalse(groupPageStoresMap.contains(0, 0));
+        assertFalse(groupPageStoresMap.contains(1, 0));
     }
 
     @Test
-    void testAllPageStores() {
-        assertThat(groupPageStores.allPageStores(), empty());
+    void testGetAll() {
+        assertThat(groupPageStoresMap.getAll(), empty());
 
-        List<PageStore> pageStores0 = List.of();
-        List<PageStore> pageStores1 = List.of();
+        FilePageStore filePageStore0 = mock(FilePageStore.class);
+        FilePageStore filePageStore1 = mock(FilePageStore.class);
 
-        groupPageStores.put(0, pageStores0);
+        groupPageStoresMap.put(0, 0, filePageStore0);
 
-        assertThat(groupPageStores.allPageStores(), containsInAnyOrder(pageStores0));
+        assertThat(
+                getAll(groupPageStoresMap),
+                containsInAnyOrder(new TestGroupPartitionPageStore<>(0, 0, filePageStore0))
+        );
 
-        groupPageStores.put(1, pageStores1);
+        groupPageStoresMap.put(1, 1, filePageStore1);
 
-        assertThat(groupPageStores.allPageStores(), containsInAnyOrder(pageStores0, pageStores1));
+        assertThat(
+                getAll(groupPageStoresMap),
+                containsInAnyOrder(
+                        new TestGroupPartitionPageStore<>(0, 0, filePageStore0),
+                        new TestGroupPartitionPageStore<>(1, 1, filePageStore1)
+                )
+        );
 
-        groupPageStores.clear();
+        groupPageStoresMap.clear();
 
-        assertThat(groupPageStores.allPageStores(), empty());
+        assertThat(groupPageStoresMap.getAll(), empty());
     }
 
     @Test
     void testClear() {
-        assertDoesNotThrow(groupPageStores::clear);
+        assertDoesNotThrow(groupPageStoresMap::clear);
+
+        groupPageStoresMap.put(0, 0, mock(FilePageStore.class));
+
+        assertDoesNotThrow(groupPageStoresMap::clear);
+        assertEquals(0, groupPageStoresMap.groupCount());
+
+        assertNull(groupPageStoresMap.get(0));
+        assertFalse(groupPageStoresMap.contains(0, 0));
+    }
+
+    @Test
+    void testRemove() {
+        assertNull(groupPageStoresMap.remove(0, 0));
+
+        FilePageStore filePageStore = mock(FilePageStore.class);
+
+        groupPageStoresMap.put(0, 0, filePageStore);
+
+        GroupPageStores<PageStore> groupPageStores = groupPageStoresMap.get(0);
 
-        groupPageStores.put(0, List.of());
+        assertNull(groupPageStoresMap.remove(0, 1));
+        assertNull(groupPageStoresMap.remove(1, 0));
 
-        assertDoesNotThrow(groupPageStores::clear);
-        assertEquals(0, groupPageStores.groupCount());
+        assertSame(filePageStore, groupPageStoresMap.remove(0, 0));
+
+        assertNull(groupPageStoresMap.get(0));
+        assertNull(groupPageStores.get(0));
+
+        assertFalse(groupPageStoresMap.contains(0, 0));
+        assertEquals(0, groupPageStoresMap.groupCount());
     }
 
     private void checkInvokeAfterAsyncCompletion(int times) {
         verify(longOperationAsyncExecutor, times(times)).afterAsyncCompletion(any(Supplier.class));
     }
+
+    private static <T extends PageStore> Collection<TestGroupPartitionPageStore<T>> getAll(GroupPageStoresMap<T> groupPageStoresMap) {
+        return groupPageStoresMap.getAll().stream()
+                .flatMap(groupPageStores -> groupPageStores.getAll().stream()
+                        .map(partitionPageStore -> new TestGroupPartitionPageStore<>(
+                                groupPageStores.groupId(),
+                                partitionPageStore.partitionId(),
+                                partitionPageStore.pageStore()
+                        ))
+                )
+                .collect(toList());
+    }
+
+    private static <T extends PageStore> Collection<TestPartitionPageStore<T>> getAll(GroupPageStores<T> groupPageStores) {
+        return groupPageStores.getAll().stream()
+                .map(partitionPageStore -> new TestPartitionPageStore<>(partitionPageStore.partitionId(), partitionPageStore.pageStore()))
+                .collect(toList());
+    }
+
+    /**
+     * Inner class for tests.
+     */
+    private static class TestGroupPartitionPageStore<T extends PageStore> extends TestPartitionPageStore<T> {
+        @IgniteToStringInclude
+        final int groupId;
+
+        private TestGroupPartitionPageStore(int groupId, int partitionId, T filePageStore) {
+            super(partitionId, filePageStore);
+
+            this.groupId = groupId;
+        }
+
+        @Override
+        public boolean equals(Object obj) {
+            if (obj == this) {
+                return true;
+            }
+
+            if (obj instanceof TestGroupPartitionPageStore) {
+                TestGroupPartitionPageStore that = (TestGroupPartitionPageStore) obj;
+
+                return groupId == that.groupId && partitionId == that.partitionId && filePageStore == that.filePageStore;
+            }
+
+            return false;
+        }
+
+        @Override
+        public String toString() {
+            return S.toString(TestGroupPartitionPageStore.class, this, "partitionId", partitionId, "filePageStore", filePageStore);
+        }
+    }
+
+    /**
+     * Inner class for tests.
+     */
+    private static class TestPartitionPageStore<T extends PageStore> {
+        @IgniteToStringInclude
+        final int partitionId;
+
+        @IgniteToStringInclude
+        final T filePageStore;
+
+        private TestPartitionPageStore(int partitionId, T filePageStore) {
+            this.partitionId = partitionId;
+            this.filePageStore = filePageStore;
+        }
+
+        @Override
+        public boolean equals(Object obj) {
+            if (obj == this) {
+                return true;
+            }
+
+            if (obj instanceof TestPartitionPageStore) {
+                TestPartitionPageStore that = (TestPartitionPageStore) obj;
+
+                return partitionId == that.partitionId && filePageStore == that.filePageStore;

Review Comment:
   Why do you use a reference equality here?



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/GroupPageStoresMap.java:
##########
@@ -47,52 +50,156 @@ public GroupPageStoresMap(LongOperationAsyncExecutor longOperationAsyncExecutor)
     }
 
     /**
-     * Puts the page stores for the group.
+     * Puts the page store for the group partition.
+     *
+     * @param groupId Group ID.
+     * @param partitionId Partition ID.
+     * @param pageStore Page store.
+     * @return Previous page store.
+     */
+    public @Nullable T put(Integer groupId, Integer partitionId, T pageStore) {
+        return longOperationAsyncExecutor.afterAsyncCompletion(() -> {
+                    PartitionPageStore<T> previous = groupIdPageStores
+                            .computeIfAbsent(groupId, id -> new GroupPageStores<>(groupId))

Review Comment:
   I'd rather pass `id` into the constructor and not enclose the external variable



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/GroupPageStoresMap.java:
##########
@@ -47,52 +50,156 @@ public GroupPageStoresMap(LongOperationAsyncExecutor longOperationAsyncExecutor)
     }
 
     /**
-     * Puts the page stores for the group.
+     * Puts the page store for the group partition.
+     *
+     * @param groupId Group ID.
+     * @param partitionId Partition ID.
+     * @param pageStore Page store.
+     * @return Previous page store.
+     */
+    public @Nullable T put(Integer groupId, Integer partitionId, T pageStore) {
+        return longOperationAsyncExecutor.afterAsyncCompletion(() -> {
+                    PartitionPageStore<T> previous = groupIdPageStores

Review Comment:
   Isn't this padding too big?



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/compaction/Compactor.java:
##########
@@ -164,77 +170,90 @@ void waitDeltaFiles() {
     }
 
     /**
-     * Adds the number of delta files to compact.
-     *
-     * @param count Number of delta files.
+     * Callback on adding delta files so we can start compacting them.
      */
-    public void addDeltaFiles(int count) {
-        assert count >= 0;
-
-        if (count > 0) {
-            deltaFileCount.addAndGet(count);
+    public void onAddingDeltaFiles() {
+        synchronized (mux) {
+            addedDeltaFiles = true;
 
-            synchronized (mux) {
-                mux.notifyAll();
-            }
+            mux.notifyAll();
         }
     }
 
     /**
      * Merges delta files with partition files.
      */
     void doCompaction() {
-        // Let's collect one delta file for each partition.
-        Queue<IgniteBiTuple<FilePageStore, DeltaFilePageStoreIo>> queue = filePageStoreManager.allPageStores().stream()
-                .flatMap(List::stream)
-                .map(filePageStore -> {
+        while (true) {
+            // Let's collect one delta file for each partition.
+            ConcurrentLinkedQueue<DeltaFileToCompaction> queue = new ConcurrentLinkedQueue<>();
+
+            for (GroupPageStores<FilePageStore> groupPageStores : filePageStoreManager.allPageStores()) {
+                for (PartitionPageStore<FilePageStore> partitionPageStore : groupPageStores.getAll()) {
+                    FilePageStore filePageStore = partitionPageStore.pageStore();
+
                     DeltaFilePageStoreIo deltaFileToCompaction = filePageStore.getDeltaFileToCompaction();
 
-                    return deltaFileToCompaction == null ? null : new IgniteBiTuple<>(filePageStore, deltaFileToCompaction);
-                })
-                .filter(Objects::nonNull)
-                .collect(toCollection(ConcurrentLinkedQueue::new));
+                    if (!filePageStore.isMarkedToDestroy() && deltaFileToCompaction != null) {
+                        queue.add(new DeltaFileToCompaction(
+                                groupPageStores.groupId(),
+                                partitionPageStore.partitionId(),
+                                filePageStore,
+                                deltaFileToCompaction
+                        ));
+                    }
+                }
+            }
+
+            if (queue.isEmpty()) {

Review Comment:
   Maybe I'm worrying about it too much, we don't even know how fast or how frequent it'll work



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/compaction/Compactor.java:
##########
@@ -338,6 +360,10 @@ void mergeDeltaFileToMainFile(
                 return;
             }
 
+            if (filePageStore.isMarkedToDestroy()) {

Review Comment:
   I have a feeling that this code is not safe. We had similar code with cache-group stopping during the snapshot (in GG). It never worked properly.



##########
modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/AbstractMvTableStorageTest.java:
##########
@@ -419,11 +505,69 @@ private static void createTestTable(TableConfiguration tableConfig) {
         assertThat(createTableFuture, willCompleteSuccessfully());
     }
 
-    private static <T> List<T> getAll(Cursor<T> cursor) {
+    private static <T> List<T> getAll(Cursor<T> cursor) throws Exception {
         try (cursor) {
             return cursor.stream().collect(Collectors.toList());
-        } catch (Exception e) {
-            throw new RuntimeException(e);
         }
     }
+
+    private void checkStorageDestroyed(MvPartitionStorage storage) {
+        int partId = PARTITION_ID;
+
+        assertThrows(StorageClosedException.class, () -> storage.runConsistently(() -> null));
+
+        assertThrows(StorageClosedException.class, storage::flush);
+
+        assertThrows(StorageClosedException.class, storage::lastAppliedIndex);
+        assertThrows(StorageClosedException.class, storage::lastAppliedTerm);

Review Comment:
   These two don't have to throw exceptions, but ok



##########
modules/storage-api/src/main/java/org/apache/ignite/internal/storage/engine/MvTableStorage.java:
##########
@@ -59,6 +60,10 @@ public interface MvTableStorage {
     /**
      * Destroys a partition and all associated indices.
      *
+     * <p>This method will do nothing if there is no partition by ID, when trying to call methods to read or write (as well as all previous
+     * open cursors) for {@link MvPartitionStorage}, {@link HashIndexStorage} and {@link SortedIndexStorage}, {@link StorageClosedException}
+     * will be thrown.
+     *

Review Comment:
   This description is close to impossible to understand, please use re-structure your text and expand it.



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/PartitionProcessingCounterMap.java:
##########
@@ -0,0 +1,90 @@
+/*
+ * 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;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Helper class for thread-safe work with {@link PartitionProcessingCounter} for any partition of any group.
+ */
+public class PartitionProcessingCounterMap {
+    private final ConcurrentMap<GroupPartitionId, PartitionProcessingCounter> processedPartitions = new ConcurrentHashMap<>();
+
+    /**
+     * Callback at the beginning of checkpoint processing of a partition, for example, when writing dirty pages or executing a fsync.
+     *
+     * @param groupId Group ID.
+     * @param partitionId Partition ID.
+     */
+    public void onStartPartitionProcessing(int groupId, int partitionId) {
+        GroupPartitionId groupPartitionId = new GroupPartitionId(groupId, partitionId);
+
+        processedPartitions.compute(groupPartitionId, (id, partitionProcessingCounter) -> {
+            if (partitionProcessingCounter == null) {
+                PartitionProcessingCounter counter = new PartitionProcessingCounter();
+
+                counter.onStartPartitionProcessing();
+
+                return counter;
+            }
+
+            partitionProcessingCounter.onStartPartitionProcessing();
+
+            return partitionProcessingCounter;
+        });
+    }
+
+    /**
+     * Callback on completion of partition processing, for example, when writing dirty pages or executing a fsync.
+     *
+     * @param groupId Group ID.
+     * @param partitionId Partition ID.
+     */
+    public void onFinishPartitionProcessing(int groupId, int partitionId) {
+        GroupPartitionId groupPartitionId = new GroupPartitionId(groupId, partitionId);
+
+        processedPartitions.compute(groupPartitionId, (id, partitionProcessingCounter) -> {
+            assert partitionProcessingCounter != null : id;
+            assert !partitionProcessingCounter.future().isDone() : id;
+
+            partitionProcessingCounter.onFinishPartitionProcessing();
+
+            return partitionProcessingCounter.future().isDone() ? null : partitionProcessingCounter;
+        });
+    }
+
+    /**
+     * Returns the future if the partition according to the given parameters is currently being processed, for example, dirty pages are
+     * being written or fsync is being done, {@code null} if the partition is not currently being processed.
+     *
+     * <p>Future will be added on {@link #onStartPartitionProcessing(int, int)} call and completed on
+     * {@link #onFinishPartitionProcessing(int, int)} call (equal to the number of {@link #onFinishPartitionProcessing(int, int)} calls).
+     *
+     * @param groupId Group ID.
+     * @param partitionId Partition ID.
+     */
+    @Nullable
+    public CompletableFuture<Void> getProcessedPartitionFuture(int groupId, int partitionId) {

Review Comment:
   Is there a chance of having a race condition when this method returns null? Caller thinks that all partitions are processed, but then, suddenly, another thread calls "start" before you know it, and then it's all doomed...



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/GroupPageStoresMap.java:
##########
@@ -47,52 +50,156 @@ public GroupPageStoresMap(LongOperationAsyncExecutor longOperationAsyncExecutor)
     }
 
     /**
-     * Puts the page stores for the group.
+     * Puts the page store for the group partition.
+     *
+     * @param groupId Group ID.
+     * @param partitionId Partition ID.
+     * @param pageStore Page store.
+     * @return Previous page store.
+     */
+    public @Nullable T put(Integer groupId, Integer partitionId, T pageStore) {
+        return longOperationAsyncExecutor.afterAsyncCompletion(() -> {
+                    PartitionPageStore<T> previous = groupIdPageStores
+                            .computeIfAbsent(groupId, id -> new GroupPageStores<>(groupId))
+                            .partitionIdPageStore
+                            .put(partitionId, new PartitionPageStore<>(partitionId, pageStore));
+
+                    return previous == null ? null : previous.pageStore;
+                }
+        );
+    }
+
+    /**
+     * Removes the page store for the group partition.
      *
-     * @param grpId Group ID.
-     * @param pageStores Page stores.
-     * @return Previous page stores.
+     * @param groupId Group ID.
+     * @param partitionId Partition ID.
+     * @return Removed page store.
      */
-    public @Nullable List<T> put(Integer grpId, List<T> pageStores) {
-        return longOperationAsyncExecutor.afterAsyncCompletion(() -> groupPageStores.put(grpId, pageStores));
+    public @Nullable T remove(Integer groupId, Integer partitionId) {
+        AtomicReference<PartitionPageStore<T>> partitionPageStoreRef = new AtomicReference<>();
+
+        groupIdPageStores.compute(groupId, (id, groupPageStores) -> {

Review Comment:
   Why don't we use a composite key (groupId, partitionId)? Is this because we need to be able to get all group stores at the same time fast?



##########
modules/storage-api/src/main/java/org/apache/ignite/internal/storage/StorageClosedException.java:
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.storage;
+
+/**
+ *  Exception that will be thrown when the storage is closed.
+ */
+public class StorageClosedException extends StorageException {
+    /**
+     * Constructor.
+     *
+     * @param message Error message.
+     */
+    public StorageClosedException(String message) {
+        super(message);
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param message Error message.
+     * @param cause The cause.
+     */
+    public StorageClosedException(String message, Throwable cause) {
+        super(message, cause);

Review Comment:
   Where's the error code? :)
   I guess we should make it soon



-- 
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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/sorted/PageMemorySortedIndexStorage.java:
##########
@@ -57,6 +69,13 @@ public class PageMemorySortedIndexStorage implements SortedIndexStorage {
     /** Highest possible RowId according to signed long ordering. */
     private final RowId highestRowId;
 
+    /** Busy lock for synchronous closing. */
+    private final IgniteSpinBusyLock closeBusyLock = new IgniteSpinBusyLock();
+
+    /** To avoid double closure. */
+    @SuppressWarnings("unused")
+    private volatile boolean started = true;

Review Comment:
   Change 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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStoreManager.java:
##########
@@ -75,52 +93,44 @@ public class FilePageStoreManager implements PageReadWriteManager {
     /** Group directory prefix. */
     public static final String GROUP_DIR_PREFIX = "table-";
 
-    /** Logger. */
-    private final IgniteLogger log;
-
     /** Starting directory for all file page stores, for example: 'db/group-123/index.bin'. */
     private final Path dbDir;
 
     /** Page size in bytes. */
     private final int pageSize;
 
-    /**
-     * Executor to disallow running code that modifies data in {@link #groupPageStores} concurrently with cleanup of file page store.
-     */
+    /** Executor to disallow running code that modifies data in {@link #groupPageStores} concurrently with cleanup of file page store. */
     private final LongOperationAsyncExecutor cleanupAsyncExecutor;
 
-    /** Mapping: group ID -> page store list. */
+    /** Mapping: group ID -> group page stores. */
     private final GroupPageStoresMap<FilePageStore> groupPageStores;
 
-    /** Group directory initialization lock. */
-    private final IgniteStripedLock initGroupDirLock = new IgniteStripedLock(Math.max(Runtime.getRuntime().availableProcessors(), 8));
+    /** Striped lock. */
+    private final IgniteStripedLock stripedLock = new IgniteStripedLock(Math.max(Runtime.getRuntime().availableProcessors(), 8));

Review Comment:
   Sounds reasonable, tried to change.



-- 
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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java:
##########
@@ -378,21 +381,38 @@ public RocksDbMvPartitionStorage getOrCreateMvPartition(int partitionId) throws
     }
 
     @Override
-    public void destroyPartition(int partitionId) throws StorageException {
+    public CompletableFuture<Void> destroyPartition(int partitionId) {
         checkPartitionId(partitionId);
 
+        CompletableFuture<Void> destroyPartitionFuture = new CompletableFuture<>();
+
+        CompletableFuture<Void> previousDestroyPartitionFuture = partitionIdDestroyFutureMap.putIfAbsent(
+                partitionId,
+                destroyPartitionFuture
+        );
+
+        if (previousDestroyPartitionFuture != null) {
+            return previousDestroyPartitionFuture;
+        }
+
         RocksDbMvPartitionStorage mvPartition = partitions.getAndSet(partitionId, null);
 
         if (mvPartition != null) {
-            //TODO IGNITE-17626 Destroy indexes as well...
-            mvPartition.destroy();
-
             try {
+                //TODO IGNITE-17626 Destroy indexes as well...
+                mvPartition.destroy();

Review Comment:
   Added a comment.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: 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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/sorted/PageMemorySortedIndexStorage.java:
##########
@@ -120,22 +188,63 @@ public void remove(IndexRow row) {
             remove.afterCompletion();
         } catch (IgniteInternalCheckedException e) {
             throw new StorageException("Failed to remove value from index", e);
+        } finally {
+            closeBusyLock.leaveBusy();
         }
     }
 
     @Override
     public Cursor<IndexRow> scan(@Nullable BinaryTuplePrefix lowerBound, @Nullable BinaryTuplePrefix upperBound, int flags) {
-        boolean includeLower = (flags & GREATER_OR_EQUAL) != 0;
-        boolean includeUpper = (flags & LESS_OR_EQUAL) != 0;
-
-        SortedIndexRowKey lower = createBound(lowerBound, !includeLower);
-
-        SortedIndexRowKey upper = createBound(upperBound, includeUpper);
+        if (!closeBusyLock.enterBusy()) {
+            throwStorageClosedException();
+        }
 
         try {
-            return map(sortedIndexTree.find(lower, upper), this::toIndexRowImpl);
+            boolean includeLower = (flags & GREATER_OR_EQUAL) != 0;
+            boolean includeUpper = (flags & LESS_OR_EQUAL) != 0;
+
+            SortedIndexRowKey lower = createBound(lowerBound, !includeLower);
+
+            SortedIndexRowKey upper = createBound(upperBound, includeUpper);
+
+            Cursor<SortedIndexRow> cursor = sortedIndexTree.find(lower, upper);
+
+            return new Cursor<>() {
+                @Override
+                public void close() {
+                    cursor.close();
+                }
+
+                @Override
+                public boolean hasNext() {

Review Comment:
   Sounds reasonable, tried to 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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/PersistentPageMemoryTableStorage.java:
##########
@@ -418,4 +430,43 @@ public CompletableFuture<Void> finishRebalanceMvPartition(int partitionId) {
         // TODO: IGNITE-18029 Implement
         throw new UnsupportedOperationException();
     }
+
+    @Override
+    public void destroyMvPartitionStorage(AbstractPageMemoryMvPartitionStorage mvPartitionStorage) throws StorageException {
+        int partitionId = mvPartitionStorage.partitionId();
+
+        CompletableFuture<Void> previousFuture = destroyFutureByPartitionId.put(partitionId, new CompletableFuture<>());
+
+        assert previousFuture == null : "Previous destruction of the partition has not completed: " + partitionId;
+
+        // It is enough for us to close the partition storage and its indexes (do not destroy), perform some actions, and then simply

Review Comment:
   Tried to 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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/test/java/org/apache/ignite/internal/pagememory/persistence/store/GroupPageStoresMapTest.java:
##########
@@ -52,125 +57,265 @@ void setUp() {
         when(longOperationAsyncExecutor.afterAsyncCompletion(any(Supplier.class)))
                 .then(answer -> ((Supplier<?>) answer.getArgument(0)).get());
 
-        groupPageStores = new GroupPageStoresMap<>(longOperationAsyncExecutor);
+        groupPageStoresMap = new GroupPageStoresMap<>(longOperationAsyncExecutor);
     }
 
     @Test
     void testGroupCount() {
-        assertEquals(0, groupPageStores.groupCount());
+        assertEquals(0, groupPageStoresMap.groupCount());
 
-        groupPageStores.put(0, List.of());
+        groupPageStoresMap.put(0, 0, mock(FilePageStore.class));
 
-        assertEquals(1, groupPageStores.groupCount());
+        assertEquals(1, groupPageStoresMap.groupCount());
 
-        groupPageStores.put(0, List.of());
+        groupPageStoresMap.put(0, 0, mock(FilePageStore.class));
 
-        assertEquals(1, groupPageStores.groupCount());
+        assertEquals(1, groupPageStoresMap.groupCount());
 
-        groupPageStores.put(1, List.of());
+        groupPageStoresMap.put(1, 0, mock(FilePageStore.class));
 
-        assertEquals(2, groupPageStores.groupCount());
+        assertEquals(2, groupPageStoresMap.groupCount());
 
-        groupPageStores.clear();
+        groupPageStoresMap.clear();
 
-        assertEquals(0, groupPageStores.groupCount());
+        assertEquals(0, groupPageStoresMap.groupCount());
     }
 
     @Test
     void testPut() {
-        List<PageStore> holder0 = List.of();
+        FilePageStore filePageStore0 = mock(FilePageStore.class);
 
-        assertNull(groupPageStores.put(0, holder0));
+        assertNull(groupPageStoresMap.put(0, 0, filePageStore0));
         checkInvokeAfterAsyncCompletion(1);
 
-        List<PageStore> holder1 = List.of();
+        FilePageStore filePageStore1 = mock(FilePageStore.class);
 
-        assertSame(holder0, groupPageStores.put(0, holder1));
+        assertSame(filePageStore0, groupPageStoresMap.put(0, 0, filePageStore1));
         checkInvokeAfterAsyncCompletion(2);
 
-        assertSame(holder1, groupPageStores.get(0));
-        assertNull(groupPageStores.get(1));
+        assertNull(groupPageStoresMap.get(1));
+
+        assertThat(getAll(groupPageStoresMap.get(0)), containsInAnyOrder(new TestPartitionPageStore<>(0, filePageStore1)));
     }
 
     @Test
     void testGet() {
-        assertNull(groupPageStores.get(0));
-        assertNull(groupPageStores.get(1));
+        assertNull(groupPageStoresMap.get(0));
+        assertNull(groupPageStoresMap.get(1));
 
-        List<PageStore> pageStores0 = List.of();
+        FilePageStore filePageStore0 = mock(FilePageStore.class);
 
-        groupPageStores.put(0, pageStores0);
+        groupPageStoresMap.put(0, 0, filePageStore0);
 
-        assertSame(pageStores0, groupPageStores.get(0));
-        assertNull(groupPageStores.get(1));
+        GroupPageStores<PageStore> groupPageStores0 = groupPageStoresMap.get(0);
 
-        List<PageStore> pageStores1 = List.of();
+        assertThat(getAll(groupPageStores0), containsInAnyOrder(new TestPartitionPageStore<>(0, filePageStore0)));
+        assertNull(groupPageStoresMap.get(1));
 
-        groupPageStores.put(1, pageStores1);
+        FilePageStore filePageStore1 = mock(FilePageStore.class);
 
-        assertSame(pageStores0, groupPageStores.get(0));
-        assertSame(pageStores1, groupPageStores.get(1));
+        groupPageStoresMap.put(1, 1, filePageStore1);
 
-        groupPageStores.clear();
+        GroupPageStores<PageStore> groupPageStores1 = groupPageStoresMap.get(1);
 
-        assertNull(groupPageStores.get(0));
-        assertNull(groupPageStores.get(1));
+        assertThat(getAll(groupPageStores0), containsInAnyOrder(new TestPartitionPageStore<>(0, filePageStore0)));
+        assertThat(getAll(groupPageStores1), containsInAnyOrder(new TestPartitionPageStore<>(1, filePageStore1)));
+
+        assertEquals(TestPartitionPageStore.of(groupPageStores0.get(0)), new TestPartitionPageStore<>(0, filePageStore0));
+        assertEquals(TestPartitionPageStore.of(groupPageStores1.get(1)), new TestPartitionPageStore<>(1, filePageStore1));
+
+        assertNull(groupPageStores0.get(2));
+        assertNull(groupPageStores1.get(2));
+
+        groupPageStoresMap.clear();
+
+        assertNull(groupPageStoresMap.get(0));
+        assertNull(groupPageStoresMap.get(1));
     }
 
     @Test
     void testContainsPageStores() {
-        assertFalse(groupPageStores.containsPageStores(0));
-        assertFalse(groupPageStores.containsPageStores(1));
+        assertFalse(groupPageStoresMap.contains(0, 0));
+        assertFalse(groupPageStoresMap.contains(1, 0));
+
+        FilePageStore filePageStore0 = mock(FilePageStore.class);
 
-        groupPageStores.put(0, List.of());
+        groupPageStoresMap.put(0, 0, filePageStore0);
 
-        assertTrue(groupPageStores.containsPageStores(0));
-        assertFalse(groupPageStores.containsPageStores(1));
+        assertTrue(groupPageStoresMap.contains(0, 0));
+        assertFalse(groupPageStoresMap.contains(1, 0));
+        assertFalse(groupPageStoresMap.contains(0, 1));
 
-        List<PageStore> pageStores1 = List.of();
+        FilePageStore filePageStore1 = mock(FilePageStore.class);
 
-        groupPageStores.put(1, List.of());
+        groupPageStoresMap.put(1, 0, filePageStore1);
 
-        assertTrue(groupPageStores.containsPageStores(0));
-        assertTrue(groupPageStores.containsPageStores(1));
+        assertTrue(groupPageStoresMap.contains(0, 0));
+        assertTrue(groupPageStoresMap.contains(1, 0));
 
-        groupPageStores.clear();
+        groupPageStoresMap.clear();
 
-        assertFalse(groupPageStores.containsPageStores(0));
-        assertFalse(groupPageStores.containsPageStores(1));
+        assertFalse(groupPageStoresMap.contains(0, 0));
+        assertFalse(groupPageStoresMap.contains(1, 0));
     }
 
     @Test
-    void testAllPageStores() {
-        assertThat(groupPageStores.allPageStores(), empty());
+    void testGetAll() {
+        assertThat(groupPageStoresMap.getAll(), empty());
 
-        List<PageStore> pageStores0 = List.of();
-        List<PageStore> pageStores1 = List.of();
+        FilePageStore filePageStore0 = mock(FilePageStore.class);
+        FilePageStore filePageStore1 = mock(FilePageStore.class);
 
-        groupPageStores.put(0, pageStores0);
+        groupPageStoresMap.put(0, 0, filePageStore0);
 
-        assertThat(groupPageStores.allPageStores(), containsInAnyOrder(pageStores0));
+        assertThat(
+                getAll(groupPageStoresMap),
+                containsInAnyOrder(new TestGroupPartitionPageStore<>(0, 0, filePageStore0))
+        );
 
-        groupPageStores.put(1, pageStores1);
+        groupPageStoresMap.put(1, 1, filePageStore1);
 
-        assertThat(groupPageStores.allPageStores(), containsInAnyOrder(pageStores0, pageStores1));
+        assertThat(
+                getAll(groupPageStoresMap),
+                containsInAnyOrder(
+                        new TestGroupPartitionPageStore<>(0, 0, filePageStore0),
+                        new TestGroupPartitionPageStore<>(1, 1, filePageStore1)
+                )
+        );
 
-        groupPageStores.clear();
+        groupPageStoresMap.clear();
 
-        assertThat(groupPageStores.allPageStores(), empty());
+        assertThat(groupPageStoresMap.getAll(), empty());
     }
 
     @Test
     void testClear() {
-        assertDoesNotThrow(groupPageStores::clear);
+        assertDoesNotThrow(groupPageStoresMap::clear);
+
+        groupPageStoresMap.put(0, 0, mock(FilePageStore.class));
+
+        assertDoesNotThrow(groupPageStoresMap::clear);
+        assertEquals(0, groupPageStoresMap.groupCount());
+
+        assertNull(groupPageStoresMap.get(0));
+        assertFalse(groupPageStoresMap.contains(0, 0));
+    }
+
+    @Test
+    void testRemove() {
+        assertNull(groupPageStoresMap.remove(0, 0));
+
+        FilePageStore filePageStore = mock(FilePageStore.class);
+
+        groupPageStoresMap.put(0, 0, filePageStore);
+
+        GroupPageStores<PageStore> groupPageStores = groupPageStoresMap.get(0);
 
-        groupPageStores.put(0, List.of());
+        assertNull(groupPageStoresMap.remove(0, 1));
+        assertNull(groupPageStoresMap.remove(1, 0));
 
-        assertDoesNotThrow(groupPageStores::clear);
-        assertEquals(0, groupPageStores.groupCount());
+        assertSame(filePageStore, groupPageStoresMap.remove(0, 0));
+
+        assertNull(groupPageStoresMap.get(0));
+        assertNull(groupPageStores.get(0));
+
+        assertFalse(groupPageStoresMap.contains(0, 0));
+        assertEquals(0, groupPageStoresMap.groupCount());
     }
 
     private void checkInvokeAfterAsyncCompletion(int times) {
         verify(longOperationAsyncExecutor, times(times)).afterAsyncCompletion(any(Supplier.class));
     }
+
+    private static <T extends PageStore> Collection<TestGroupPartitionPageStore<T>> getAll(GroupPageStoresMap<T> groupPageStoresMap) {
+        return groupPageStoresMap.getAll().stream()
+                .flatMap(groupPageStores -> groupPageStores.getAll().stream()
+                        .map(partitionPageStore -> new TestGroupPartitionPageStore<>(
+                                groupPageStores.groupId(),
+                                partitionPageStore.partitionId(),
+                                partitionPageStore.pageStore()
+                        ))
+                )
+                .collect(toList());
+    }
+
+    private static <T extends PageStore> Collection<TestPartitionPageStore<T>> getAll(GroupPageStores<T> groupPageStores) {
+        return groupPageStores.getAll().stream()
+                .map(partitionPageStore -> new TestPartitionPageStore<>(partitionPageStore.partitionId(), partitionPageStore.pageStore()))
+                .collect(toList());
+    }
+
+    /**
+     * Inner class for tests.
+     */
+    private static class TestGroupPartitionPageStore<T extends PageStore> extends TestPartitionPageStore<T> {
+        @IgniteToStringInclude
+        final int groupId;
+
+        private TestGroupPartitionPageStore(int groupId, int partitionId, T filePageStore) {
+            super(partitionId, filePageStore);
+
+            this.groupId = groupId;
+        }
+
+        @Override
+        public boolean equals(Object obj) {
+            if (obj == this) {
+                return true;
+            }
+
+            if (obj instanceof TestGroupPartitionPageStore) {
+                TestGroupPartitionPageStore that = (TestGroupPartitionPageStore) obj;
+
+                return groupId == that.groupId && partitionId == that.partitionId && filePageStore == that.filePageStore;
+            }
+
+            return false;
+        }
+
+        @Override
+        public String toString() {
+            return S.toString(TestGroupPartitionPageStore.class, this, "partitionId", partitionId, "filePageStore", filePageStore);
+        }
+    }
+
+    /**
+     * Inner class for tests.
+     */
+    private static class TestPartitionPageStore<T extends PageStore> {
+        @IgniteToStringInclude
+        final int partitionId;
+
+        @IgniteToStringInclude
+        final T filePageStore;
+
+        private TestPartitionPageStore(int partitionId, T filePageStore) {
+            this.partitionId = partitionId;
+            this.filePageStore = filePageStore;
+        }
+
+        @Override
+        public boolean equals(Object obj) {
+            if (obj == this) {
+                return true;
+            }
+
+            if (obj instanceof TestPartitionPageStore) {
+                TestPartitionPageStore that = (TestPartitionPageStore) obj;
+
+                return partitionId == that.partitionId && filePageStore == that.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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/GroupPageStoresMap.java:
##########
@@ -47,52 +50,156 @@ public GroupPageStoresMap(LongOperationAsyncExecutor longOperationAsyncExecutor)
     }
 
     /**
-     * Puts the page stores for the group.
+     * Puts the page store for the group partition.
+     *
+     * @param groupId Group ID.
+     * @param partitionId Partition ID.
+     * @param pageStore Page store.
+     * @return Previous page store.
+     */
+    public @Nullable T put(Integer groupId, Integer partitionId, T pageStore) {
+        return longOperationAsyncExecutor.afterAsyncCompletion(() -> {
+                    PartitionPageStore<T> previous = groupIdPageStores
+                            .computeIfAbsent(groupId, id -> new GroupPageStores<>(groupId))
+                            .partitionIdPageStore
+                            .put(partitionId, new PartitionPageStore<>(partitionId, pageStore));
+
+                    return previous == null ? null : previous.pageStore;
+                }
+        );
+    }
+
+    /**
+     * Removes the page store for the group partition.
      *
-     * @param grpId Group ID.
-     * @param pageStores Page stores.
-     * @return Previous page stores.
+     * @param groupId Group ID.
+     * @param partitionId Partition ID.
+     * @return Removed page store.
      */
-    public @Nullable List<T> put(Integer grpId, List<T> pageStores) {
-        return longOperationAsyncExecutor.afterAsyncCompletion(() -> groupPageStores.put(grpId, pageStores));
+    public @Nullable T remove(Integer groupId, Integer partitionId) {
+        AtomicReference<PartitionPageStore<T>> partitionPageStoreRef = new AtomicReference<>();
+
+        groupIdPageStores.compute(groupId, (id, groupPageStores) -> {
+            if (groupPageStores == null) {
+                return null;
+            }
+
+            partitionPageStoreRef.set(groupPageStores.partitionIdPageStore.remove(partitionId));
+
+            if (groupPageStores.partitionIdPageStore.isEmpty()) {
+                return null;
+            }
+
+            return groupPageStores;
+        });
+
+        PartitionPageStore<T> partitionPageStore = partitionPageStoreRef.get();
+
+        return partitionPageStore == null ? null : partitionPageStore.pageStore;
     }
 
     /**
      * Returns the page stores for the group.
      *
-     * @param grpId Group ID.
+     * @param groupId Group ID.
      */
-    public @Nullable List<T> get(Integer grpId) {
-        return groupPageStores.get(grpId);
+    public @Nullable GroupPageStores<T> get(Integer groupId) {
+        return groupIdPageStores.get(groupId);
     }
 
     /**
-     * Returns {@code true} if a page stores exists for the group.
+     * Returns {@code true} if a page store exists for the group partition.
      *
-     * @param grpId Group ID.
+     * @param groupId Group ID.
+     * @param partitionId Partition ID.
      */
-    public boolean containsPageStores(Integer grpId) {
-        return groupPageStores.containsKey(grpId);
+    public boolean contains(Integer groupId, Integer partitionId) {
+        GroupPageStores<T> groupPageStores = groupIdPageStores.get(groupId);
+
+        return groupPageStores != null && groupPageStores.partitionIdPageStore.containsKey(partitionId);
     }
 
     /**
-     * Returns all page stores of all groups.
+     * Returns a view of all page stores of all groups.
      */
-    public Collection<List<T>> allPageStores() {
-        return groupPageStores.values();
+    public Collection<GroupPageStores<T>> getAll() {
+        return unmodifiableCollection(groupIdPageStores.values());
     }
 
     /**
      * Clears all page stores of all groups.
      */
     public void clear() {
-        groupPageStores.clear();
+        groupIdPageStores.clear();
     }
 
     /**
      * Returns the number of groups for which there are page stores.
      */
     public int groupCount() {
-        return groupPageStores.size();
+        return groupIdPageStores.size();
+    }
+
+    /**
+     * Group partition page stores.
+     */
+    public static class GroupPageStores<T extends PageStore> {
+        private final int groupId;
+
+        private final ConcurrentMap<Integer, PartitionPageStore<T>> partitionIdPageStore = new ConcurrentHashMap<>();
+
+        private GroupPageStores(int groupId) {
+            this.groupId = groupId;
+        }
+
+        /**
+         * Returns the group ID.
+         */
+        public int groupId() {
+            return groupId;
+        }
+
+        /**
+         * Returns the partition's page store.
+         */
+        @Nullable
+        public PartitionPageStore<T> get(Integer partitionId) {
+            return partitionIdPageStore.get(partitionId);
+        }
+
+        /**
+         * Returns a view of the group's partition page stores.
+         */
+        public Collection<PartitionPageStore<T>> getAll() {
+            return unmodifiableCollection(partitionIdPageStore.values());
+        }
+    }
+
+    /**
+     * Partition page store.
+     */
+    public static class PartitionPageStore<T extends PageStore> {
+        private final int partitionId;
+
+        private final T pageStore;

Review Comment:
   There is no getter, in general we can add it, but it is better in a separate ticket.



-- 
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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/checkpoint/Checkpointer.java:
##########
@@ -775,4 +799,41 @@ private void renameDeltaFileOnCheckpointThread(GroupPartitionId partitionId) thr
     void updateLastProgressAfterReleaseWriteLock() {
         afterReleaseWriteLockCheckpointProgress = currentCheckpointProgress;
     }
+
+    /**
+     * Callback on destruction of the partition of the corresponding group.
+     *
+     * <p>If the checkpoint is in progress, then wait until it finishes processing the partition that we are going to destroy, in order to
+     * prevent the situation when we want to destroy the partition file along with its delta files, and at this time the checkpoint performs
+     * I/O operations on them.
+     *
+     * @param groupId Group ID.
+     * @param partitionId Partition ID.
+     * @throws IgniteInternalCheckedException If there are errors while processing the callback.
+     */
+    void onPartitionDestruction(int groupId, int partitionId) throws IgniteInternalCheckedException {
+        CheckpointProgressImpl currentCheckpointProgress = this.currentCheckpointProgress;
+
+        if (currentCheckpointProgress == null || !currentCheckpointProgress.inProgress()) {
+            return;
+        }
+
+        CompletableFuture<Void> processedPartitionFuture = currentCheckpointProgress.getProcessedPartitionFuture(groupId, partitionId);

Review Comment:
   Tried to 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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/hash/PageMemoryHashIndexStorage.java:
##########
@@ -123,11 +188,34 @@ public void remove(IndexRow row) throws StorageException {
             remove.afterCompletion();
         } catch (IgniteInternalCheckedException e) {
             throw new StorageException("Failed to remove value from index", e);
+        } finally {
+            closeBusyLock.leaveBusy();
         }
     }
 
     @Override
     public void destroy() throws StorageException {
-        //TODO IGNITE-17626 Implement.
+        // TODO: IGNITE-17626 Remove it
+        throw new UnsupportedOperationException();
+    }
+
+    /**
+     * Closes the hash index storage.
+     */
+    public void close() {
+        if (!STARTED.compareAndSet(this, true, false)) {

Review Comment:
   Change name



##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/sorted/PageMemorySortedIndexStorage.java:
##########
@@ -120,22 +188,63 @@ public void remove(IndexRow row) {
             remove.afterCompletion();
         } catch (IgniteInternalCheckedException e) {
             throw new StorageException("Failed to remove value from index", e);
+        } finally {
+            closeBusyLock.leaveBusy();
         }
     }
 
     @Override
     public Cursor<IndexRow> scan(@Nullable BinaryTuplePrefix lowerBound, @Nullable BinaryTuplePrefix upperBound, int flags) {
-        boolean includeLower = (flags & GREATER_OR_EQUAL) != 0;
-        boolean includeUpper = (flags & LESS_OR_EQUAL) != 0;
-
-        SortedIndexRowKey lower = createBound(lowerBound, !includeLower);
-
-        SortedIndexRowKey upper = createBound(upperBound, includeUpper);
+        if (!closeBusyLock.enterBusy()) {
+            throwStorageClosedException();
+        }
 
         try {
-            return map(sortedIndexTree.find(lower, upper), this::toIndexRowImpl);
+            boolean includeLower = (flags & GREATER_OR_EQUAL) != 0;
+            boolean includeUpper = (flags & LESS_OR_EQUAL) != 0;
+
+            SortedIndexRowKey lower = createBound(lowerBound, !includeLower);
+
+            SortedIndexRowKey upper = createBound(upperBound, includeUpper);
+
+            Cursor<SortedIndexRow> cursor = sortedIndexTree.find(lower, upper);
+
+            return new Cursor<>() {

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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/AbstractPageMemoryMvPartitionStorage.java:
##########
@@ -834,23 +958,45 @@ public boolean hasNext() {
             }
 
             while (true) {
-                if (!treeCursor.hasNext()) {
-                    iterationExhausted = true;
-                    return false;
+                if (!closeBusyLock.enterBusy()) {
+                    throwStorageClosedException();
                 }
 
-                VersionChain chain = treeCursor.next();
-                ReadResult result = findLatestRowVersion(chain);
+                try {
+                    if (!treeCursor.hasNext()) {
+                        iterationExhausted = true;
+                        return false;
+                    }
 
-                if (result.isEmpty() && !result.isWriteIntent()) {
-                    continue;
-                }
+                    VersionChain chain = treeCursor.next();
+                    ReadResult result = findLatestRowVersion(chain);
 
-                nextRead = result;
-                currentChain = chain;
+                    if (result.isEmpty() && !result.isWriteIntent()) {
+                        continue;
+                    }
 
-                return true;
+                    nextRead = result;
+                    currentChain = chain;
+
+                    return true;
+                } finally {
+                    closeBusyLock.leaveBusy();
+                }
             }
         }
     }
+
+    /**
+     * Throws an exception that the storage is already closed.
+     */
+    protected void throwStorageClosedException() {
+        throw new StorageClosedException("Storage is already closed");
+    }
+
+    /**
+     * Returns the partition ID.
+     */
+    public int partitionId() {

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 commented on a diff in pull request #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/GroupPageStoresMap.java:
##########
@@ -47,52 +50,156 @@ public GroupPageStoresMap(LongOperationAsyncExecutor longOperationAsyncExecutor)
     }
 
     /**
-     * Puts the page stores for the group.
+     * Puts the page store for the group partition.
+     *
+     * @param groupId Group ID.
+     * @param partitionId Partition ID.
+     * @param pageStore Page store.
+     * @return Previous page store.
+     */
+    public @Nullable T put(Integer groupId, Integer partitionId, T pageStore) {
+        return longOperationAsyncExecutor.afterAsyncCompletion(() -> {
+                    PartitionPageStore<T> previous = groupIdPageStores
+                            .computeIfAbsent(groupId, id -> new GroupPageStores<>(groupId))

Review Comment:
   Can you clarify, what exactly wasn't clear?
   I recommended changing `id -> new GroupPageStores<>(groupId)` to `id -> new GroupPageStores<>(id)`, maybe even using a constructor reference (IDEA will suggest it I guess).



-- 
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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/raft/snapshot/incoming/IncomingSnapshotCopier.java:
##########
@@ -195,8 +195,8 @@ private CompletableFuture<?> prepareMvPartitionStorageForRebalance(Executor exec
             return completedFuture(null);
         }
 
-        return CompletableFuture.supplyAsync(() -> partitionSnapshotStorage.partition().reCreateMvPartitionStorage(), executor)
-                .thenCompose(mvPartitionStorage -> {
+        return partitionSnapshotStorage.partition().reCreateMvPartitionStorage()

Review Comment:
   Partition destruction is now an asynchronous operation that will be performed in some pool, and we need to wait for the future to complete.



-- 
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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/AbstractPageMemoryTableStorage.java:
##########
@@ -49,6 +51,8 @@ public abstract class AbstractPageMemoryTableStorage implements MvTableStorage {
 
     private volatile AtomicReferenceArray<AbstractPageMemoryMvPartitionStorage> mvPartitions;
 
+    protected final ConcurrentMap<Integer, CompletableFuture<Void>> partitionIdDestroyFuture = new ConcurrentHashMap<>();

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 commented on a diff in pull request #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/VolatilePageMemoryTableStorage.java:
##########
@@ -157,4 +155,10 @@ public CompletableFuture<Void> finishRebalanceMvPartition(int partitionId) {
         // TODO: IGNITE-18028 Implement
         throw new UnsupportedOperationException();
     }
+
+    @Override
+    public void destroyMvPartitionStorage(AbstractPageMemoryMvPartitionStorage mvPartitionStorage) throws StorageException {

Review Comment:
   Shouldn't this method be protected instead?



##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/sorted/PageMemorySortedIndexStorage.java:
##########
@@ -120,22 +188,63 @@ public void remove(IndexRow row) {
             remove.afterCompletion();
         } catch (IgniteInternalCheckedException e) {
             throw new StorageException("Failed to remove value from index", e);
+        } finally {
+            closeBusyLock.leaveBusy();
         }
     }
 
     @Override
     public Cursor<IndexRow> scan(@Nullable BinaryTuplePrefix lowerBound, @Nullable BinaryTuplePrefix upperBound, int flags) {
-        boolean includeLower = (flags & GREATER_OR_EQUAL) != 0;
-        boolean includeUpper = (flags & LESS_OR_EQUAL) != 0;
-
-        SortedIndexRowKey lower = createBound(lowerBound, !includeLower);
-
-        SortedIndexRowKey upper = createBound(upperBound, includeUpper);
+        if (!closeBusyLock.enterBusy()) {
+            throwStorageClosedException();
+        }
 
         try {
-            return map(sortedIndexTree.find(lower, upper), this::toIndexRowImpl);
+            boolean includeLower = (flags & GREATER_OR_EQUAL) != 0;
+            boolean includeUpper = (flags & LESS_OR_EQUAL) != 0;
+
+            SortedIndexRowKey lower = createBound(lowerBound, !includeLower);
+
+            SortedIndexRowKey upper = createBound(upperBound, includeUpper);
+
+            Cursor<SortedIndexRow> cursor = sortedIndexTree.find(lower, upper);
+
+            return new Cursor<>() {

Review Comment:
   Maybe we can extract a class to be reused



##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/PersistentPageMemoryTableStorage.java:
##########
@@ -418,4 +430,43 @@ public CompletableFuture<Void> finishRebalanceMvPartition(int partitionId) {
         // TODO: IGNITE-18029 Implement
         throw new UnsupportedOperationException();
     }
+
+    @Override
+    public void destroyMvPartitionStorage(AbstractPageMemoryMvPartitionStorage mvPartitionStorage) throws StorageException {
+        int partitionId = mvPartitionStorage.partitionId();
+
+        CompletableFuture<Void> previousFuture = destroyFutureByPartitionId.put(partitionId, new CompletableFuture<>());
+
+        assert previousFuture == null : "Previous destruction of the partition has not completed: " + partitionId;

Review Comment:
   What's up with the future anyway? No one uses it, what's the point?



##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/AbstractPageMemoryMvPartitionStorage.java:
##########
@@ -834,23 +958,45 @@ public boolean hasNext() {
             }
 
             while (true) {
-                if (!treeCursor.hasNext()) {
-                    iterationExhausted = true;
-                    return false;
+                if (!closeBusyLock.enterBusy()) {
+                    throwStorageClosedException();
                 }
 
-                VersionChain chain = treeCursor.next();
-                ReadResult result = findLatestRowVersion(chain);
+                try {
+                    if (!treeCursor.hasNext()) {
+                        iterationExhausted = true;
+                        return false;
+                    }
 
-                if (result.isEmpty() && !result.isWriteIntent()) {
-                    continue;
-                }
+                    VersionChain chain = treeCursor.next();
+                    ReadResult result = findLatestRowVersion(chain);
 
-                nextRead = result;
-                currentChain = chain;
+                    if (result.isEmpty() && !result.isWriteIntent()) {
+                        continue;
+                    }
 
-                return true;
+                    nextRead = result;
+                    currentChain = chain;
+
+                    return true;
+                } finally {
+                    closeBusyLock.leaveBusy();
+                }
             }
         }
     }
+
+    /**
+     * Throws an exception that the storage is already closed.
+     */
+    protected void throwStorageClosedException() {
+        throw new StorageClosedException("Storage is already closed");
+    }
+
+    /**
+     * Returns the partition ID.
+     */
+    public int partitionId() {

Review Comment:
   Please move this method a bit higher, ideally before any data access methods



##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/hash/PageMemoryHashIndexStorage.java:
##########
@@ -123,11 +188,34 @@ public void remove(IndexRow row) throws StorageException {
             remove.afterCompletion();
         } catch (IgniteInternalCheckedException e) {
             throw new StorageException("Failed to remove value from index", e);
+        } finally {
+            closeBusyLock.leaveBusy();
         }
     }
 
     @Override
     public void destroy() throws StorageException {
-        //TODO IGNITE-17626 Implement.
+        // TODO: IGNITE-17626 Remove it
+        throw new UnsupportedOperationException();
+    }
+
+    /**
+     * Closes the hash index storage.
+     */
+    public void close() {
+        if (!STARTED.compareAndSet(this, true, false)) {

Review Comment:
   Usually "not started" means "not YET started". Why don't you use same names as everyone else? Why does it have to be different?
   Uniformity in code is very important, it's always easier when you see a common pattern.



##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/PersistentPageMemoryTableStorage.java:
##########
@@ -418,4 +430,43 @@ public CompletableFuture<Void> finishRebalanceMvPartition(int partitionId) {
         // TODO: IGNITE-18029 Implement
         throw new UnsupportedOperationException();
     }
+
+    @Override
+    public void destroyMvPartitionStorage(AbstractPageMemoryMvPartitionStorage mvPartitionStorage) throws StorageException {
+        int partitionId = mvPartitionStorage.partitionId();
+
+        CompletableFuture<Void> previousFuture = destroyFutureByPartitionId.put(partitionId, new CompletableFuture<>());
+
+        assert previousFuture == null : "Previous destruction of the partition has not completed: " + partitionId;

Review Comment:
   Why not just return the old future?



-- 
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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/compaction/Compactor.java:
##########
@@ -65,13 +66,17 @@ public class Compactor extends IgniteWorker {
 
     private final @Nullable ThreadPoolExecutor threadPoolExecutor;
 
-    private final AtomicInteger deltaFileCount = new AtomicInteger();
+    /** Guarded by {@link #mux}. */
+    private boolean addedDeltaFiles;
 
     private final FilePageStoreManager filePageStoreManager;
 
     /** Thread local with buffers for the compaction threads. */
     private final ThreadLocal<ByteBuffer> threadBuf;

Review Comment:
   It will not work to use static, because we do not know the page size at the time the class is initialized.



-- 
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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/PartitionProcessingCounter.java:
##########
@@ -0,0 +1,83 @@
+/*
+ * 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;
+
+import java.lang.invoke.MethodHandles;
+import java.lang.invoke.VarHandle;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Helper class for tracking the completion of partition processing.
+ *
+ * <p>At the start of partition processing, you need to call {@link #onStartPartitionProcessing()}, at the end
+ * {@link #onFinishPartitionProcessing()}. When all partition processing is completed, the {@link #future()} will be completed.
+ *
+ * <p>It is recommended to use external synchronization for the correct operation of the {@link #counter partition processing counter} and
+ * the {@link #future future}.
+ */
+public class PartitionProcessingCounter {
+    private static final VarHandle COUNTER;
+
+    static {
+        try {
+            COUNTER = MethodHandles.lookup().findVarHandle(PartitionProcessingCounter.class, "counter", int.class);
+        } catch (ReflectiveOperationException e) {
+            throw new ExceptionInInitializerError(e);
+        }
+    }
+
+    /** Partition processing counter must be greater than or equal to zero. */
+    @SuppressWarnings("unused")
+    private volatile int counter;
+
+    /** Future that will be completed when the {@link #counter} is zero. */
+    private final CompletableFuture<Void> future = new CompletableFuture<>();
+
+    /**
+     * Callback at the start of partition processing.
+     */
+    public void onStartPartitionProcessing() {
+        assert !future.isDone();
+
+        int updatedValue = (int) COUNTER.getAndAdd(this, 1) + 1;

Review Comment:
   It is, but it's not complex code.



-- 
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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/PartitionProcessingCounter.java:
##########
@@ -0,0 +1,83 @@
+/*
+ * 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;
+
+import java.lang.invoke.MethodHandles;
+import java.lang.invoke.VarHandle;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Helper class for tracking the completion of partition processing.
+ *
+ * <p>At the start of partition processing, you need to call {@link #onStartPartitionProcessing()}, at the end
+ * {@link #onFinishPartitionProcessing()}. When all partition processing is completed, the {@link #future()} will be completed.
+ *
+ * <p>It is recommended to use external synchronization for the correct operation of the {@link #counter partition processing counter} and
+ * the {@link #future future}.
+ */
+public class PartitionProcessingCounter {
+    private static final VarHandle COUNTER;
+
+    static {
+        try {
+            COUNTER = MethodHandles.lookup().findVarHandle(PartitionProcessingCounter.class, "counter", int.class);

Review Comment:
   In this case, we will save memory and at the same time the complexity of the code will not increase.



-- 
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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/compaction/Compactor.java:
##########
@@ -164,77 +170,90 @@ void waitDeltaFiles() {
     }
 
     /**
-     * Adds the number of delta files to compact.
-     *
-     * @param count Number of delta files.
+     * Callback on adding delta files so we can start compacting them.
      */
-    public void addDeltaFiles(int count) {
-        assert count >= 0;
-
-        if (count > 0) {
-            deltaFileCount.addAndGet(count);
+    public void onAddingDeltaFiles() {
+        synchronized (mux) {
+            addedDeltaFiles = true;
 
-            synchronized (mux) {
-                mux.notifyAll();
-            }
+            mux.notifyAll();
         }
     }
 
     /**
      * Merges delta files with partition files.
      */
     void doCompaction() {
-        // Let's collect one delta file for each partition.
-        Queue<IgniteBiTuple<FilePageStore, DeltaFilePageStoreIo>> queue = filePageStoreManager.allPageStores().stream()
-                .flatMap(List::stream)
-                .map(filePageStore -> {
+        while (true) {
+            // Let's collect one delta file for each partition.
+            ConcurrentLinkedQueue<DeltaFileToCompaction> queue = new ConcurrentLinkedQueue<>();
+
+            for (GroupPageStores<FilePageStore> groupPageStores : filePageStoreManager.allPageStores()) {
+                for (PartitionPageStore<FilePageStore> partitionPageStore : groupPageStores.getAll()) {
+                    FilePageStore filePageStore = partitionPageStore.pageStore();
+
                     DeltaFilePageStoreIo deltaFileToCompaction = filePageStore.getDeltaFileToCompaction();
 
-                    return deltaFileToCompaction == null ? null : new IgniteBiTuple<>(filePageStore, deltaFileToCompaction);
-                })
-                .filter(Objects::nonNull)
-                .collect(toCollection(ConcurrentLinkedQueue::new));
+                    if (!filePageStore.isMarkedToDestroy() && deltaFileToCompaction != null) {
+                        queue.add(new DeltaFileToCompaction(
+                                groupPageStores.groupId(),
+                                partitionPageStore.partitionId(),
+                                filePageStore,
+                                deltaFileToCompaction
+                        ));
+                    }
+                }
+            }
+
+            if (queue.isEmpty()) {

Review Comment:
   It does not control, in threads we simply take delta files for compaction while they are in the pack, as soon as one pack is over, we start creating another.
   
   I do not understand the question.



-- 
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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/AbstractHashIndexStorageTest.java:
##########
@@ -216,6 +217,7 @@ public void testRemoveIdempotence() {
         assertDoesNotThrow(() -> remove(row));
     }
 
+    @Disabled("https://issues.apache.org/jira/browse/IGNITE-17626")

Review Comment:
   Fair enough, I think in this ticket it is necessary to indicate about all the storages, when we start it they will be in different tickets.



-- 
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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/GroupPageStoresMap.java:
##########
@@ -47,52 +50,156 @@ public GroupPageStoresMap(LongOperationAsyncExecutor longOperationAsyncExecutor)
     }
 
     /**
-     * Puts the page stores for the group.
+     * Puts the page store for the group partition.
+     *
+     * @param groupId Group ID.
+     * @param partitionId Partition ID.
+     * @param pageStore Page store.
+     * @return Previous page store.
+     */
+    public @Nullable T put(Integer groupId, Integer partitionId, T pageStore) {
+        return longOperationAsyncExecutor.afterAsyncCompletion(() -> {
+                    PartitionPageStore<T> previous = groupIdPageStores
+                            .computeIfAbsent(groupId, id -> new GroupPageStores<>(groupId))

Review Comment:
   I didn't understand



-- 
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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/PartitionProcessingCounterMap.java:
##########
@@ -0,0 +1,90 @@
+/*
+ * 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;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Helper class for thread-safe work with {@link PartitionProcessingCounter} for any partition of any group.
+ */
+public class PartitionProcessingCounterMap {
+    private final ConcurrentMap<GroupPartitionId, PartitionProcessingCounter> processedPartitions = new ConcurrentHashMap<>();
+
+    /**
+     * Callback at the beginning of checkpoint processing of a partition, for example, when writing dirty pages or executing a fsync.
+     *
+     * @param groupId Group ID.
+     * @param partitionId Partition ID.
+     */
+    public void onStartPartitionProcessing(int groupId, int partitionId) {
+        GroupPartitionId groupPartitionId = new GroupPartitionId(groupId, partitionId);
+
+        processedPartitions.compute(groupPartitionId, (id, partitionProcessingCounter) -> {
+            if (partitionProcessingCounter == null) {
+                PartitionProcessingCounter counter = new PartitionProcessingCounter();
+
+                counter.onStartPartitionProcessing();
+
+                return counter;
+            }
+
+            partitionProcessingCounter.onStartPartitionProcessing();
+
+            return partitionProcessingCounter;
+        });
+    }
+
+    /**
+     * Callback on completion of partition processing, for example, when writing dirty pages or executing a fsync.
+     *
+     * @param groupId Group ID.
+     * @param partitionId Partition ID.
+     */
+    public void onFinishPartitionProcessing(int groupId, int partitionId) {
+        GroupPartitionId groupPartitionId = new GroupPartitionId(groupId, partitionId);
+
+        processedPartitions.compute(groupPartitionId, (id, partitionProcessingCounter) -> {
+            assert partitionProcessingCounter != null : id;
+            assert !partitionProcessingCounter.future().isDone() : id;
+
+            partitionProcessingCounter.onFinishPartitionProcessing();
+
+            return partitionProcessingCounter.future().isDone() ? null : partitionProcessingCounter;
+        });
+    }
+
+    /**
+     * Returns the future if the partition according to the given parameters is currently being processed, for example, dirty pages are
+     * being written or fsync is being done, {@code null} if the partition is not currently being processed.
+     *
+     * <p>Future will be added on {@link #onStartPartitionProcessing(int, int)} call and completed on
+     * {@link #onFinishPartitionProcessing(int, int)} call (equal to the number of {@link #onFinishPartitionProcessing(int, int)} calls).
+     *
+     * @param groupId Group ID.
+     * @param partitionId Partition ID.
+     */
+    @Nullable
+    public CompletableFuture<Void> getProcessedPartitionFuture(int groupId, int partitionId) {

Review Comment:
   In general, you are right, but in our cases we will not suffer, because if we did not start processing the partition, then they will see that we want to destroy it, since we first notify the data region and `FilePageStore` that we want to destroy it, and only then check whether we are in the process of processing.



-- 
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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/GroupPageStoresMap.java:
##########
@@ -47,52 +50,156 @@ public GroupPageStoresMap(LongOperationAsyncExecutor longOperationAsyncExecutor)
     }
 
     /**
-     * Puts the page stores for the group.
+     * Puts the page store for the group partition.
+     *
+     * @param groupId Group ID.
+     * @param partitionId Partition ID.
+     * @param pageStore Page store.
+     * @return Previous page store.
+     */
+    public @Nullable T put(Integer groupId, Integer partitionId, T pageStore) {
+        return longOperationAsyncExecutor.afterAsyncCompletion(() -> {
+                    PartitionPageStore<T> previous = groupIdPageStores
+                            .computeIfAbsent(groupId, id -> new GroupPageStores<>(groupId))

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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/compaction/Compactor.java:
##########
@@ -401,38 +415,24 @@ void mergeDeltaFileToMainFile(
     }
 
     /**
-     * Callback on destruction of the partition of the corresponding group.
+     * Prepares the compacter to destroy a 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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/test/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStoreManagerTest.java:
##########
@@ -358,19 +347,133 @@ void testAllPageStores() throws Exception {
 
         manager.start();
 
-        manager.initialize("test0", 1, 1);
-        manager.initialize("test1", 2, 1);
+        manager.initialize("test0", 1, 0);
+        manager.initialize("test1", 2, 0);
+
+        List<Path> allPageStoreFiles = manager.allPageStores().stream()
+                .map(GroupPageStores::getAll)
+                .flatMap(Collection::stream)
+                .map(PartitionPageStore::pageStore)
+                .map(FilePageStore::filePath)
+                .collect(toList());
 
         assertThat(
-                manager.allPageStores().stream().flatMap(List::stream).map(FilePageStore::filePath).collect(toList()),
+                allPageStoreFiles,
                 containsInAnyOrder(
                         workDir.resolve("db/table-1").resolve("part-0.bin"),
                         workDir.resolve("db/table-2").resolve("part-0.bin")
                 )
         );
     }
 
+    @Test
+    void testOnPartitionDestruction() throws Exception {
+        FilePageStoreManager manager = createManager();
+
+        manager.start();
+
+        manager.initialize("test0", 0, 0);
+        manager.initialize("test1", 1, 0);
+
+        FilePageStore filePageStore0 = manager.getStore(0, 0);
+        FilePageStore filePageStore1 = manager.getStore(1, 0);
+
+        filePageStore0.ensure();

Review Comment:
   I don’t know the reasons, it came from 2.0



-- 
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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/VolatilePageMemoryTableStorage.java:
##########
@@ -157,4 +155,10 @@ public CompletableFuture<Void> finishRebalanceMvPartition(int partitionId) {
         // TODO: IGNITE-18028 Implement
         throw new UnsupportedOperationException();
     }
+
+    @Override
+    public void destroyMvPartitionStorage(AbstractPageMemoryMvPartitionStorage mvPartitionStorage) throws StorageException {

Review Comment:
   I think the package-private is enough



-- 
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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/compaction/Compactor.java:
##########
@@ -164,77 +170,90 @@ void waitDeltaFiles() {
     }
 
     /**
-     * Adds the number of delta files to compact.
-     *
-     * @param count Number of delta files.
+     * Callback on adding delta files so we can start compacting them.
      */
-    public void addDeltaFiles(int count) {
-        assert count >= 0;
-
-        if (count > 0) {
-            deltaFileCount.addAndGet(count);
+    public void onAddingDeltaFiles() {
+        synchronized (mux) {
+            addedDeltaFiles = true;
 
-            synchronized (mux) {
-                mux.notifyAll();
-            }
+            mux.notifyAll();
         }
     }
 
     /**
      * Merges delta files with partition files.
      */
     void doCompaction() {
-        // Let's collect one delta file for each partition.
-        Queue<IgniteBiTuple<FilePageStore, DeltaFilePageStoreIo>> queue = filePageStoreManager.allPageStores().stream()
-                .flatMap(List::stream)
-                .map(filePageStore -> {
+        while (true) {
+            // Let's collect one delta file for each partition.
+            ConcurrentLinkedQueue<DeltaFileToCompaction> queue = new ConcurrentLinkedQueue<>();
+
+            for (GroupPageStores<FilePageStore> groupPageStores : filePageStoreManager.allPageStores()) {
+                for (PartitionPageStore<FilePageStore> partitionPageStore : groupPageStores.getAll()) {
+                    FilePageStore filePageStore = partitionPageStore.pageStore();
+
                     DeltaFilePageStoreIo deltaFileToCompaction = filePageStore.getDeltaFileToCompaction();
 
-                    return deltaFileToCompaction == null ? null : new IgniteBiTuple<>(filePageStore, deltaFileToCompaction);
-                })
-                .filter(Objects::nonNull)
-                .collect(toCollection(ConcurrentLinkedQueue::new));
+                    if (!filePageStore.isMarkedToDestroy() && deltaFileToCompaction != null) {
+                        queue.add(new DeltaFileToCompaction(
+                                groupPageStores.groupId(),
+                                partitionPageStore.partitionId(),
+                                filePageStore,
+                                deltaFileToCompaction
+                        ));
+                    }
+                }
+            }
+
+            if (queue.isEmpty()) {

Review Comment:
   I think that this does not apply to the task of removing the partition, if necessary, we will fix it in another ticket.



-- 
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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/hash/PageMemoryHashIndexStorage.java:
##########
@@ -51,6 +64,13 @@ public class PageMemoryHashIndexStorage implements HashIndexStorage {
     /** Highest possible RowId according to signed long ordering. */
     private final RowId highestRowId;
 
+    /** Busy lock for synchronous closing. */
+    private final IgniteSpinBusyLock closeBusyLock = new IgniteSpinBusyLock();
+
+    /** To avoid double closure. */
+    @SuppressWarnings("unused")
+    private volatile boolean started = true;

Review Comment:
   Change 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] rpuch commented on a diff in pull request #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/compaction/Compactor.java:
##########
@@ -65,13 +66,17 @@ public class Compactor extends IgniteWorker {
 
     private final @Nullable ThreadPoolExecutor threadPoolExecutor;
 
-    private final AtomicInteger deltaFileCount = new AtomicInteger();
+    /** Guarded by {@link #mux}. */
+    private boolean addedDeltaFiles;
 
     private final FilePageStoreManager filePageStoreManager;
 
     /** Thread local with buffers for the compaction threads. */
     private final ThreadLocal<ByteBuffer> threadBuf;
 
+    /** Partitions currently being processed, for example, writes dirty pages to delta file. */

Review Comment:
   Do we still need 'for example' words in this case? Don't we know exactly what 'processing' means here?



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/PartitionProcessingCounter.java:
##########
@@ -0,0 +1,83 @@
+/*
+ * 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;
+
+import java.lang.invoke.MethodHandles;
+import java.lang.invoke.VarHandle;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Helper class for tracking the completion of partition processing.
+ *
+ * <p>At the start of partition processing, you need to call {@link #onStartPartitionProcessing()}, at the end
+ * {@link #onFinishPartitionProcessing()}. When all partition processing is completed, the {@link #future()} will be completed.
+ *
+ * <p>It is recommended to use external synchronization for the correct operation of the {@link #counter partition processing counter} and

Review Comment:
   'Recommended' implies that it is nice to have, but not actually mandatory. In this case, is it mandatory or not? Also, could you please elaborate in the javadoc on what kind of synchronization is needed?



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/PartitionProcessingCounterMap.java:
##########
@@ -0,0 +1,90 @@
+/*
+ * 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;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Helper class for thread-safe work with {@link PartitionProcessingCounter} for any partition of any group.
+ */
+public class PartitionProcessingCounterMap {
+    private final ConcurrentMap<GroupPartitionId, PartitionProcessingCounter> processedPartitions = new ConcurrentHashMap<>();
+
+    /**
+     * Callback at the beginning of checkpoint processing of a partition, for example, when writing dirty pages or executing a fsync.
+     *
+     * @param groupId Group ID.
+     * @param partitionId Partition ID.
+     */
+    public void onStartPartitionProcessing(int groupId, int partitionId) {
+        GroupPartitionId groupPartitionId = new GroupPartitionId(groupId, partitionId);
+
+        processedPartitions.compute(groupPartitionId, (id, partitionProcessingCounter) -> {
+            if (partitionProcessingCounter == null) {
+                PartitionProcessingCounter counter = new PartitionProcessingCounter();
+
+                counter.onStartPartitionProcessing();
+
+                return counter;
+            }
+
+            partitionProcessingCounter.onStartPartitionProcessing();
+
+            return partitionProcessingCounter;
+        });
+    }

Review Comment:
   ```suggestion
           processedPartitions.computeIfAbsent(groupPartitionId, key -> new PartitionProcessingCounter()).onStartPartitionProcessing();
       }
   ```



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/compaction/Compactor.java:
##########
@@ -164,77 +170,90 @@ void waitDeltaFiles() {
     }
 
     /**
-     * Adds the number of delta files to compact.
-     *
-     * @param count Number of delta files.
+     * Callback on adding delta files so we can start compacting them.
      */
-    public void addDeltaFiles(int count) {
-        assert count >= 0;
-
-        if (count > 0) {
-            deltaFileCount.addAndGet(count);
+    public void onAddingDeltaFiles() {
+        synchronized (mux) {
+            addedDeltaFiles = true;
 
-            synchronized (mux) {
-                mux.notifyAll();
-            }
+            mux.notifyAll();
         }
     }
 
     /**
      * Merges delta files with partition files.
      */
     void doCompaction() {
-        // Let's collect one delta file for each partition.
-        Queue<IgniteBiTuple<FilePageStore, DeltaFilePageStoreIo>> queue = filePageStoreManager.allPageStores().stream()
-                .flatMap(List::stream)
-                .map(filePageStore -> {
+        while (true) {
+            // Let's collect one delta file for each partition.
+            ConcurrentLinkedQueue<DeltaFileToCompaction> queue = new ConcurrentLinkedQueue<>();

Review Comment:
   Looks like `queue` is only used for enqueueing (via `add()`) and dequeueing (via `poll()`), so `Queue` is enough as the variable type. I suggest to change it accordingly.



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/PartitionProcessingCounterMap.java:
##########
@@ -0,0 +1,90 @@
+/*
+ * 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;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Helper class for thread-safe work with {@link PartitionProcessingCounter} for any partition of any group.
+ */
+public class PartitionProcessingCounterMap {
+    private final ConcurrentMap<GroupPartitionId, PartitionProcessingCounter> processedPartitions = new ConcurrentHashMap<>();
+
+    /**
+     * Callback at the beginning of checkpoint processing of a partition, for example, when writing dirty pages or executing a fsync.
+     *
+     * @param groupId Group ID.
+     * @param partitionId Partition ID.
+     */
+    public void onStartPartitionProcessing(int groupId, int partitionId) {
+        GroupPartitionId groupPartitionId = new GroupPartitionId(groupId, partitionId);
+
+        processedPartitions.compute(groupPartitionId, (id, partitionProcessingCounter) -> {
+            if (partitionProcessingCounter == null) {
+                PartitionProcessingCounter counter = new PartitionProcessingCounter();
+
+                counter.onStartPartitionProcessing();
+
+                return counter;
+            }
+
+            partitionProcessingCounter.onStartPartitionProcessing();
+
+            return partitionProcessingCounter;
+        });
+    }
+
+    /**
+     * Callback on completion of partition processing, for example, when writing dirty pages or executing a fsync.
+     *
+     * @param groupId Group ID.
+     * @param partitionId Partition ID.
+     */
+    public void onFinishPartitionProcessing(int groupId, int partitionId) {
+        GroupPartitionId groupPartitionId = new GroupPartitionId(groupId, partitionId);
+
+        processedPartitions.compute(groupPartitionId, (id, partitionProcessingCounter) -> {
+            assert partitionProcessingCounter != null : id;
+            assert !partitionProcessingCounter.future().isDone() : id;
+
+            partitionProcessingCounter.onFinishPartitionProcessing();
+
+            return partitionProcessingCounter.future().isDone() ? null : partitionProcessingCounter;
+        });
+    }
+
+    /**
+     * Returns the future if the partition according to the given parameters is currently being processed, for example, dirty pages are
+     * being written or fsync is being done, {@code null} if the partition is not currently being processed.
+     *
+     * <p>Future will be added on {@link #onStartPartitionProcessing(int, int)} call and completed on
+     * {@link #onFinishPartitionProcessing(int, int)} call (equal to the number of {@link #onFinishPartitionProcessing(int, int)} calls).
+     *
+     * @param groupId Group ID.
+     * @param partitionId Partition ID.
+     */
+    @Nullable
+    public CompletableFuture<Void> getProcessedPartitionFuture(int groupId, int partitionId) {
+        PartitionProcessingCounter partitionProcessingCounter = processedPartitions.get(new GroupPartitionId(groupId, partitionId));
+
+        return partitionProcessingCounter == null ? null : partitionProcessingCounter.future();

Review Comment:
   It seems weird to return a null future. According to the usages, this future just gets waited on (if not null). How about creating a static field with a completed future and return it instead of null? Then the caller would start waiting on it and just immediately unblock itself and proceed, so the effect would be the same, but avoiding a null check (hence, simplifiying the caller code). Also, less possibilities to get an NPE.



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStoreManager.java:
##########
@@ -230,78 +269,86 @@ public long allocatePage(int grpId, int partId, byte flags) throws IgniteInterna
     }
 
     /**
-     * Initializing the file page stores for a group.
+     * Initialization of the page storage for the group partition.
      *
      * @param tableName Table name.
      * @param tableId Integer table id.
-     * @param partitions Partition number, must be greater than {@code 0} and less {@link PageIdAllocator#MAX_PARTITION_ID}.
+     * @param partitionId Partition ID, must be between {@code 0} (inclusive) and {@link PageIdAllocator#MAX_PARTITION_ID} (inclusive).
      * @throws IgniteInternalCheckedException If failed.
      */
-    public void initialize(String tableName, int tableId, int partitions) throws IgniteInternalCheckedException {
-        assert partitions > 0 && partitions < MAX_PARTITION_ID : partitions;
+    public void initialize(String tableName, int tableId, int partitionId) throws IgniteInternalCheckedException {
+        assert partitionId >= 0 && partitionId <= MAX_PARTITION_ID : partitionId;
 
-        initGroupDirLock.lock(tableId);
+        stripedLock.lock(tableId + partitionId);
 
         try {
-            if (!groupPageStores.containsPageStores(tableId)) {
-                List<FilePageStore> partitionFilePageStores = createFilePageStores(tableId, partitions);
+            if (!groupPageStores.contains(tableId, partitionId)) {
+                Path tableWorkDir = ensureGroupWorkDir(tableId);

Review Comment:
   Looks like two terminologies are mixed: the old (Ingite 2?) terminology of 'group' with the new terminology where it is 'table' (because we seem to map tables to groups as 1-to-1). Should we stick to one terminology in `pagememory` everywhere?



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStoreManager.java:
##########
@@ -75,52 +93,44 @@ public class FilePageStoreManager implements PageReadWriteManager {
     /** Group directory prefix. */
     public static final String GROUP_DIR_PREFIX = "table-";
 
-    /** Logger. */
-    private final IgniteLogger log;
-
     /** Starting directory for all file page stores, for example: 'db/group-123/index.bin'. */
     private final Path dbDir;
 
     /** Page size in bytes. */
     private final int pageSize;
 
-    /**
-     * Executor to disallow running code that modifies data in {@link #groupPageStores} concurrently with cleanup of file page store.
-     */
+    /** Executor to disallow running code that modifies data in {@link #groupPageStores} concurrently with cleanup of file page store. */
     private final LongOperationAsyncExecutor cleanupAsyncExecutor;
 
-    /** Mapping: group ID -> page store list. */
+    /** Mapping: group ID -> group page stores. */
     private final GroupPageStoresMap<FilePageStore> groupPageStores;
 
-    /** Group directory initialization lock. */
-    private final IgniteStripedLock initGroupDirLock = new IgniteStripedLock(Math.max(Runtime.getRuntime().availableProcessors(), 8));
+    /** Striped lock. */
+    private final IgniteStripedLock stripedLock = new IgniteStripedLock(Math.max(Runtime.getRuntime().availableProcessors(), 8));

Review Comment:
   This lock is only used for initialization. I think it would make the code clearer if we name it accordingly, like `initPartitionLock`



##########
modules/page-memory/src/test/java/org/apache/ignite/internal/pagememory/persistence/PartitionMetaManagerTest.java:
##########
@@ -188,6 +189,23 @@ void testReadWritePartitionMeta(@WorkDirectory Path workDir) throws Exception {
         }
     }
 
+    @Test
+    void testRemoveMeta() {
+        PartitionMetaManager manager = new PartitionMetaManager(ioRegistry, PAGE_SIZE);
+
+        GroupPartitionId id = new GroupPartitionId(0, 0);
+
+        assertDoesNotThrow(() -> manager.removeMeta(id));

Review Comment:
   It looks like we don't need to explicitly add that `assertDoesNotThrow()` here: if an exception is thrown, it fails the test anyway, and this is not the last operation of the test. It is just called for its side effects which are used later.



##########
modules/page-memory/src/test/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStoreManagerTest.java:
##########
@@ -358,19 +347,133 @@ void testAllPageStores() throws Exception {
 
         manager.start();
 
-        manager.initialize("test0", 1, 1);
-        manager.initialize("test1", 2, 1);
+        manager.initialize("test0", 1, 0);
+        manager.initialize("test1", 2, 0);
+
+        List<Path> allPageStoreFiles = manager.allPageStores().stream()
+                .map(GroupPageStores::getAll)
+                .flatMap(Collection::stream)
+                .map(PartitionPageStore::pageStore)
+                .map(FilePageStore::filePath)
+                .collect(toList());
 
         assertThat(
-                manager.allPageStores().stream().flatMap(List::stream).map(FilePageStore::filePath).collect(toList()),
+                allPageStoreFiles,
                 containsInAnyOrder(
                         workDir.resolve("db/table-1").resolve("part-0.bin"),
                         workDir.resolve("db/table-2").resolve("part-0.bin")
                 )
         );
     }
 
+    @Test
+    void testOnPartitionDestruction() throws Exception {
+        FilePageStoreManager manager = createManager();
+
+        manager.start();
+
+        manager.initialize("test0", 0, 0);
+        manager.initialize("test1", 1, 0);
+
+        FilePageStore filePageStore0 = manager.getStore(0, 0);
+        FilePageStore filePageStore1 = manager.getStore(1, 0);
+
+        filePageStore0.ensure();

Review Comment:
   This should not be addressed in this PR, but I'm curious: why is the method name 'ensure()'? It looks like something is missing, I would expect it to be 'ensureSomething()', like 'ensureInitialized()'.



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStoreManager.java:
##########
@@ -443,4 +461,50 @@ public Path tmpDeltaFilePageStorePath(int groupId, int partitionId, int index) {
     public Path deltaFilePageStorePath(int groupId, int partitionId, int index) {
         return dbDir.resolve(GROUP_DIR_PREFIX + groupId).resolve(String.format(PART_DELTA_FILE_TEMPLATE, partitionId, index));
     }
+
+    /**
+     * Callback on destruction of the partition of the corresponding group.
+     *
+     * <p>Deletes the partition pages tore and all its delta files. Before that, it creates a marker file (for example,

Review Comment:
   ```suggestion
        * <p>Deletes the partition pages store and all its delta files. Before that, it creates a marker file (for example,
   ```



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/compaction/Compactor.java:
##########
@@ -164,77 +170,90 @@ void waitDeltaFiles() {
     }
 
     /**
-     * Adds the number of delta files to compact.
-     *
-     * @param count Number of delta files.
+     * Callback on adding delta files so we can start compacting them.
      */
-    public void addDeltaFiles(int count) {
-        assert count >= 0;
-
-        if (count > 0) {
-            deltaFileCount.addAndGet(count);
+    public void onAddingDeltaFiles() {
+        synchronized (mux) {
+            addedDeltaFiles = true;
 
-            synchronized (mux) {
-                mux.notifyAll();
-            }
+            mux.notifyAll();
         }
     }
 
     /**
      * Merges delta files with partition files.
      */
     void doCompaction() {
-        // Let's collect one delta file for each partition.
-        Queue<IgniteBiTuple<FilePageStore, DeltaFilePageStoreIo>> queue = filePageStoreManager.allPageStores().stream()
-                .flatMap(List::stream)
-                .map(filePageStore -> {
+        while (true) {
+            // Let's collect one delta file for each partition.
+            ConcurrentLinkedQueue<DeltaFileToCompaction> queue = new ConcurrentLinkedQueue<>();
+
+            for (GroupPageStores<FilePageStore> groupPageStores : filePageStoreManager.allPageStores()) {
+                for (PartitionPageStore<FilePageStore> partitionPageStore : groupPageStores.getAll()) {
+                    FilePageStore filePageStore = partitionPageStore.pageStore();
+
                     DeltaFilePageStoreIo deltaFileToCompaction = filePageStore.getDeltaFileToCompaction();
 
-                    return deltaFileToCompaction == null ? null : new IgniteBiTuple<>(filePageStore, deltaFileToCompaction);
-                })
-                .filter(Objects::nonNull)
-                .collect(toCollection(ConcurrentLinkedQueue::new));
+                    if (!filePageStore.isMarkedToDestroy() && deltaFileToCompaction != null) {
+                        queue.add(new DeltaFileToCompaction(
+                                groupPageStores.groupId(),
+                                partitionPageStore.partitionId(),
+                                filePageStore,
+                                deltaFileToCompaction
+                        ));
+                    }
+                }
+            }
+
+            if (queue.isEmpty()) {
+                break;
+            }
+
+            updateHeartbeat();
 
-        assert !queue.isEmpty();
+            int threads = threadPoolExecutor == null ? 1 : threadPoolExecutor.getMaximumPoolSize();
 
-        updateHeartbeat();
+            CompletableFuture<?>[] futures = new CompletableFuture[threads];
 
-        int threads = threadPoolExecutor == null ? 1 : threadPoolExecutor.getMaximumPoolSize();
+            for (int i = 0; i < threads; i++) {
+                CompletableFuture<?> future = futures[i] = new CompletableFuture<>();
 
-        CompletableFuture<?>[] futures = new CompletableFuture[threads];
+                Runnable merger = () -> {
+                    DeltaFileToCompaction toMerge;
 
-        for (int i = 0; i < threads; i++) {
-            CompletableFuture<?> future = futures[i] = new CompletableFuture<>();
+                    try {
+                        while ((toMerge = queue.poll()) != null) {
+                            GroupPartitionId partitionId = new GroupPartitionId(toMerge.groupId, toMerge.partitionId);
+
+                            processedPartitionMap.onStartPartitionProcessing(partitionId.getGroupId(), partitionId.getPartitionId());

Review Comment:
   Looks like `onStartPartitionProcessing()` and `onFinishPartitionProcessing()` are always called with group ID and partition ID taken from an instance of `GroupPartitionId`. How about just making these 2 methods (and the methods they call in `PartitionProcessingCounter`) accept `GroupPartitionId` and avoid this unpacking?



##########
modules/storage-api/src/main/java/org/apache/ignite/internal/storage/engine/MvTableStorage.java:
##########
@@ -59,6 +60,10 @@ public interface MvTableStorage {
     /**
      * Destroys a partition and all associated indices.
      *
+     * <p>This method will do nothing if there is no partition by ID, when trying to call methods to read or write (as well as all previous

Review Comment:
   I suggest to split this sentence into two sentences to make it more clear:
   
   1. First says that destruction is idempotent
   2. Second says that **after calling this method** any read/write/... will throw



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/PartitionProcessingCounter.java:
##########
@@ -0,0 +1,83 @@
+/*
+ * 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;
+
+import java.lang.invoke.MethodHandles;
+import java.lang.invoke.VarHandle;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Helper class for tracking the completion of partition processing.
+ *
+ * <p>At the start of partition processing, you need to call {@link #onStartPartitionProcessing()}, at the end
+ * {@link #onFinishPartitionProcessing()}. When all partition processing is completed, the {@link #future()} will be completed.
+ *
+ * <p>It is recommended to use external synchronization for the correct operation of the {@link #counter partition processing counter} and
+ * the {@link #future future}.
+ */
+public class PartitionProcessingCounter {
+    private static final VarHandle COUNTER;
+
+    static {
+        try {
+            COUNTER = MethodHandles.lookup().findVarHandle(PartitionProcessingCounter.class, "counter", int.class);
+        } catch (ReflectiveOperationException e) {
+            throw new ExceptionInInitializerError(e);
+        }
+    }
+
+    /** Partition processing counter must be greater than or equal to zero. */
+    @SuppressWarnings("unused")
+    private volatile int counter;
+
+    /** Future that will be completed when the {@link #counter} is zero. */
+    private final CompletableFuture<Void> future = new CompletableFuture<>();
+
+    /**
+     * Callback at the start of partition processing.
+     */
+    public void onStartPartitionProcessing() {
+        assert !future.isDone();
+
+        int updatedValue = (int) COUNTER.getAndAdd(this, 1) + 1;

Review Comment:
   Looks analogous to `counter.incrementAndGet()` (when using `AtomicInteger`)



##########
modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/AbstractMvTableStorageTest.java:
##########
@@ -419,11 +505,69 @@ private static void createTestTable(TableConfiguration tableConfig) {
         assertThat(createTableFuture, willCompleteSuccessfully());
     }
 
-    private static <T> List<T> getAll(Cursor<T> cursor) {
+    private static <T> List<T> getAll(Cursor<T> cursor) throws Exception {
         try (cursor) {
             return cursor.stream().collect(Collectors.toList());
-        } catch (Exception e) {
-            throw new RuntimeException(e);
         }
     }
+
+    private void checkStorageDestroyed(MvPartitionStorage storage) {
+        int partId = PARTITION_ID;
+
+        assertThrows(StorageClosedException.class, () -> storage.runConsistently(() -> null));
+
+        assertThrows(StorageClosedException.class, storage::flush);
+
+        assertThrows(StorageClosedException.class, storage::lastAppliedIndex);
+        assertThrows(StorageClosedException.class, storage::lastAppliedTerm);
+        assertThrows(StorageClosedException.class, storage::persistedIndex);
+
+        assertThrows(StorageClosedException.class, () -> storage.runConsistently(() -> {

Review Comment:
   Do we need this check even if `runConsistently()` with empty action throws this same exception?



##########
modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/AbstractMvTableStorageTest.java:
##########
@@ -419,11 +505,69 @@ private static void createTestTable(TableConfiguration tableConfig) {
         assertThat(createTableFuture, willCompleteSuccessfully());
     }
 
-    private static <T> List<T> getAll(Cursor<T> cursor) {
+    private static <T> List<T> getAll(Cursor<T> cursor) throws Exception {
         try (cursor) {
             return cursor.stream().collect(Collectors.toList());
-        } catch (Exception e) {
-            throw new RuntimeException(e);
         }
     }
+
+    private void checkStorageDestroyed(MvPartitionStorage storage) {
+        int partId = PARTITION_ID;
+
+        assertThrows(StorageClosedException.class, () -> storage.runConsistently(() -> null));
+
+        assertThrows(StorageClosedException.class, storage::flush);
+
+        assertThrows(StorageClosedException.class, storage::lastAppliedIndex);
+        assertThrows(StorageClosedException.class, storage::lastAppliedTerm);
+        assertThrows(StorageClosedException.class, storage::persistedIndex);

Review Comment:
   Let's add same check about `MvPartitionStorage#committedGroupConfiguration()`



##########
modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/impl/TestMvPartitionStorage.java:
##########
@@ -398,23 +421,35 @@ public ReadResult next() {
 
     @Override
     public @Nullable RowId closestRowId(RowId lowerBound) throws StorageException {
+        checkClosed();
+
         return map.ceilingKey(lowerBound);
     }
 
-    /** {@inheritDoc} */
     @Override
     public long rowsCount() {
+        checkClosed();
+
         return map.size();
     }
 
-    /** {@inheritDoc} */
     @Override
     public void close() {
-        // No-op.
+        started = false;

Review Comment:
   This looks strange: how can one make something not started when it was already started? I suggest to rename the field to `closed` and flip its semantics: initially it's `false`, here we switch it to `true`, and we check whether is was closed in `checkClosed()`.



##########
modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/impl/TestSortedIndexStorage.java:
##########
@@ -48,6 +49,8 @@ public class TestSortedIndexStorage implements SortedIndexStorage {
 
     private final SortedIndexDescriptor descriptor;
 
+    private volatile boolean started = true;

Review Comment:
   Same thing about `started` and `closed`



##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/hash/PageMemoryHashIndexStorage.java:
##########
@@ -51,6 +64,13 @@ public class PageMemoryHashIndexStorage implements HashIndexStorage {
     /** Highest possible RowId according to signed long ordering. */
     private final RowId highestRowId;
 
+    /** Busy lock for synchronous closing. */
+    private final IgniteSpinBusyLock closeBusyLock = new IgniteSpinBusyLock();
+
+    /** To avoid double closure. */
+    @SuppressWarnings("unused")
+    private volatile boolean started = true;

Review Comment:
   Same thing about `started` and `closed`



##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/sorted/PageMemorySortedIndexStorage.java:
##########
@@ -57,6 +69,13 @@ public class PageMemorySortedIndexStorage implements SortedIndexStorage {
     /** Highest possible RowId according to signed long ordering. */
     private final RowId highestRowId;
 
+    /** Busy lock for synchronous closing. */
+    private final IgniteSpinBusyLock closeBusyLock = new IgniteSpinBusyLock();
+
+    /** To avoid double closure. */
+    @SuppressWarnings("unused")
+    private volatile boolean started = true;

Review Comment:
   `started` and `closed`, as above



##########
modules/storage-page-memory/src/test/java/org/apache/ignite/internal/storage/pagememory/PersistentPageMemoryMvTableStorageTest.java:
##########
@@ -31,13 +33,13 @@
 import org.junit.jupiter.api.AfterEach;
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.Test;
 import org.junit.jupiter.api.extension.ExtendWith;
 
 /**
  * Tests for {@link PersistentPageMemoryTableStorage} class.
  */
-@ExtendWith(WorkDirectoryExtension.class)
-@ExtendWith(ConfigurationExtension.class)
+@ExtendWith({ConfigurationExtension.class, WorkDirectoryExtension.class})

Review Comment:
   Both styles seem ok, but why is this (one annotation with an array) seems better to you? It brings some syntactic noise (braces).



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/PartitionProcessingCounterMap.java:
##########
@@ -0,0 +1,90 @@
+/*
+ * 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;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Helper class for thread-safe work with {@link PartitionProcessingCounter} for any partition of any group.
+ */
+public class PartitionProcessingCounterMap {
+    private final ConcurrentMap<GroupPartitionId, PartitionProcessingCounter> processedPartitions = new ConcurrentHashMap<>();
+
+    /**
+     * Callback at the beginning of checkpoint processing of a partition, for example, when writing dirty pages or executing a fsync.
+     *
+     * @param groupId Group ID.
+     * @param partitionId Partition ID.
+     */
+    public void onStartPartitionProcessing(int groupId, int partitionId) {
+        GroupPartitionId groupPartitionId = new GroupPartitionId(groupId, partitionId);
+
+        processedPartitions.compute(groupPartitionId, (id, partitionProcessingCounter) -> {
+            if (partitionProcessingCounter == null) {
+                PartitionProcessingCounter counter = new PartitionProcessingCounter();
+
+                counter.onStartPartitionProcessing();
+
+                return counter;
+            }
+
+            partitionProcessingCounter.onStartPartitionProcessing();
+
+            return partitionProcessingCounter;
+        });
+    }

Review Comment:
   Just a suggestion. This way, the code is signiticantly shorter, but it replaces one atomic operation with 2 atomic operations (map update, then counter increment). Will this difference be critical in this case? Can processing start compete with processing finish?



##########
modules/storage-page-memory/src/test/java/org/apache/ignite/internal/storage/pagememory/PersistentPageMemoryMvTableStorageTest.java:
##########
@@ -101,4 +96,16 @@ public void testAbortRebalanceMvPartition() throws Exception {
     public void testFinishRebalanceMvPartition() throws Exception {
         super.testFinishRebalanceMvPartition();
     }
+
+    @Test
+    @Override
+    public void testDestroyPartition() throws Exception {
+        super.testDestroyPartition();
+
+        // Let's make sure that the checkpoint doesn't failed.

Review Comment:
   Probably, should be one of "hasn't failed", "didn't fail", "doesn't fail"



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/compaction/Compactor.java:
##########
@@ -164,77 +170,90 @@ void waitDeltaFiles() {
     }
 
     /**
-     * Adds the number of delta files to compact.
-     *
-     * @param count Number of delta files.
+     * Callback on adding delta files so we can start compacting them.
      */
-    public void addDeltaFiles(int count) {
-        assert count >= 0;
-
-        if (count > 0) {
-            deltaFileCount.addAndGet(count);
+    public void onAddingDeltaFiles() {
+        synchronized (mux) {

Review Comment:
   `mux` now acts more as a monitor or a conditional variable (waiting and notification) and not as a mutex (mutual exclusion). Should the variable be renamed?



##########
modules/page-memory/src/test/java/org/apache/ignite/internal/pagememory/persistence/checkpoint/CheckpointerTest.java:
##########
@@ -458,6 +448,47 @@ void testNextCheckpointInterval() throws Exception {
         );
     }
 
+    @Test
+    void testOnPartitionDestruction() throws Exception {
+        Checkpointer checkpointer = new Checkpointer(
+                "test",
+                null,
+                null,
+                mock(CheckpointWorkflow.class),
+                mock(CheckpointPagesWriterFactory.class),
+                mock(FilePageStoreManager.class),
+                mock(Compactor.class),
+                checkpointConfig
+        );
+
+        int groupId = 0;
+        int partitionId = 0;
+
+        // Everything should be fine as there is no current running checkpoint.
+        checkpointer.onPartitionDestruction(groupId, partitionId);
+
+        CheckpointProgressImpl checkpointProgress = (CheckpointProgressImpl) checkpointer.scheduledProgress();
+
+        checkpointer.startCheckpointProgress();
+
+        checkpointer.onPartitionDestruction(groupId, partitionId);
+
+        checkpointProgress.transitTo(LOCK_RELEASED);
+        assertTrue(checkpointProgress.inProgress());
+
+        // Everything should be fine so on a "working" checkpoint we don't process the partition anything.

Review Comment:
   ```suggestion
           // Everything should be fine so on a "working" checkpoint we don't process the partition anyhow.
   ```



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/PartitionProcessingCounter.java:
##########
@@ -0,0 +1,83 @@
+/*
+ * 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;
+
+import java.lang.invoke.MethodHandles;
+import java.lang.invoke.VarHandle;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Helper class for tracking the completion of partition processing.
+ *
+ * <p>At the start of partition processing, you need to call {@link #onStartPartitionProcessing()}, at the end
+ * {@link #onFinishPartitionProcessing()}. When all partition processing is completed, the {@link #future()} will be completed.
+ *
+ * <p>It is recommended to use external synchronization for the correct operation of the {@link #counter partition processing counter} and
+ * the {@link #future future}.
+ */
+public class PartitionProcessingCounter {
+    private static final VarHandle COUNTER;
+
+    static {
+        try {
+            COUNTER = MethodHandles.lookup().findVarHandle(PartitionProcessingCounter.class, "counter", int.class);
+        } catch (ReflectiveOperationException e) {
+            throw new ExceptionInInitializerError(e);
+        }
+    }
+
+    /** Partition processing counter must be greater than or equal to zero. */
+    @SuppressWarnings("unused")
+    private volatile int counter;
+
+    /** Future that will be completed when the {@link #counter} is zero. */
+    private final CompletableFuture<Void> future = new CompletableFuture<>();
+
+    /**
+     * Callback at the start of partition processing.
+     */
+    public void onStartPartitionProcessing() {
+        assert !future.isDone();
+
+        int updatedValue = (int) COUNTER.getAndAdd(this, 1) + 1;
+
+        assert updatedValue > 0 : updatedValue;
+    }
+
+    /**
+     * Callback at the finish of partition processing.
+     */
+    public void onFinishPartitionProcessing() {
+        assert !future.isDone();
+
+        int updatedValue = (int) COUNTER.getAndAdd(this, -1) - 1;

Review Comment:
   Looks analogous to `counter.decrementAndGet()` (when using `AtomicInteger`)



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/compaction/Compactor.java:
##########
@@ -164,77 +170,90 @@ void waitDeltaFiles() {
     }
 
     /**
-     * Adds the number of delta files to compact.
-     *
-     * @param count Number of delta files.
+     * Callback on adding delta files so we can start compacting them.
      */
-    public void addDeltaFiles(int count) {
-        assert count >= 0;
-
-        if (count > 0) {
-            deltaFileCount.addAndGet(count);
+    public void onAddingDeltaFiles() {
+        synchronized (mux) {
+            addedDeltaFiles = true;
 
-            synchronized (mux) {
-                mux.notifyAll();
-            }
+            mux.notifyAll();
         }
     }
 
     /**
      * Merges delta files with partition files.
      */
     void doCompaction() {
-        // Let's collect one delta file for each partition.
-        Queue<IgniteBiTuple<FilePageStore, DeltaFilePageStoreIo>> queue = filePageStoreManager.allPageStores().stream()
-                .flatMap(List::stream)
-                .map(filePageStore -> {
+        while (true) {
+            // Let's collect one delta file for each partition.
+            ConcurrentLinkedQueue<DeltaFileToCompaction> queue = new ConcurrentLinkedQueue<>();
+
+            for (GroupPageStores<FilePageStore> groupPageStores : filePageStoreManager.allPageStores()) {
+                for (PartitionPageStore<FilePageStore> partitionPageStore : groupPageStores.getAll()) {
+                    FilePageStore filePageStore = partitionPageStore.pageStore();
+
                     DeltaFilePageStoreIo deltaFileToCompaction = filePageStore.getDeltaFileToCompaction();
 
-                    return deltaFileToCompaction == null ? null : new IgniteBiTuple<>(filePageStore, deltaFileToCompaction);
-                })
-                .filter(Objects::nonNull)
-                .collect(toCollection(ConcurrentLinkedQueue::new));
+                    if (!filePageStore.isMarkedToDestroy() && deltaFileToCompaction != null) {
+                        queue.add(new DeltaFileToCompaction(
+                                groupPageStores.groupId(),
+                                partitionPageStore.partitionId(),
+                                filePageStore,
+                                deltaFileToCompaction
+                        ));
+                    }
+                }
+            }
+
+            if (queue.isEmpty()) {
+                break;
+            }
+
+            updateHeartbeat();
 
-        assert !queue.isEmpty();
+            int threads = threadPoolExecutor == null ? 1 : threadPoolExecutor.getMaximumPoolSize();
 
-        updateHeartbeat();
+            CompletableFuture<?>[] futures = new CompletableFuture[threads];
 
-        int threads = threadPoolExecutor == null ? 1 : threadPoolExecutor.getMaximumPoolSize();
+            for (int i = 0; i < threads; i++) {
+                CompletableFuture<?> future = futures[i] = new CompletableFuture<>();
 
-        CompletableFuture<?>[] futures = new CompletableFuture[threads];
+                Runnable merger = () -> {
+                    DeltaFileToCompaction toMerge;
 
-        for (int i = 0; i < threads; i++) {
-            CompletableFuture<?> future = futures[i] = new CompletableFuture<>();
+                    try {
+                        while ((toMerge = queue.poll()) != null) {

Review Comment:
   This 'assignment inside a different expression' is sneaky. It makes the code more difficult to understand, but the gain is really small (just 1 line spared?). I think this idiom is more harmful than useful.



##########
modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/AbstractMvTableStorageTest.java:
##########
@@ -382,6 +389,85 @@ public void testFinishRebalanceMvPartition() throws Exception {
         assertDoesNotThrow(() -> tableStorage.finishRebalanceMvPartition(PARTITION_ID).get(1, TimeUnit.SECONDS));
     }
 
+    @Test
+    public void testDestroyPartition() throws Exception {
+        assertThrows(
+                IllegalArgumentException.class,
+                () -> tableStorage.destroyPartition(tableStorage.configuration().partitions().value())
+        );
+
+        MvPartitionStorage mvPartitionStorage = tableStorage.getOrCreateMvPartition(PARTITION_ID);
+        HashIndexStorage hashIndexStorage = tableStorage.getOrCreateHashIndex(PARTITION_ID, hashIdx.id());
+        SortedIndexStorage sortedIndexStorage = tableStorage.getOrCreateSortedIndex(PARTITION_ID, sortedIdx.id());
+
+        RowId rowId = new RowId(PARTITION_ID);
+
+        BinaryRow binaryRow = binaryRow(new TestKey(0, "0"), new TestValue(1, "1"));
+
+        IndexRow indexRow = indexRow(binaryRow, rowId);
+
+        mvPartitionStorage.runConsistently(() -> {
+            mvPartitionStorage.addWriteCommitted(rowId, binaryRow, clock.now());
+
+            hashIndexStorage.put(indexRow);
+
+            sortedIndexStorage.put(indexRow);
+
+            return null;
+        });
+
+        Cursor<ReadResult> scanVersionsCursor = mvPartitionStorage.scanVersions(rowId);
+        PartitionTimestampCursor scanTimestampCursor = mvPartitionStorage.scan(clock.now());
+
+        Cursor<RowId> getFromHashIndexCursor = hashIndexStorage.get(indexRow.indexColumns());
+
+        Cursor<RowId> getFromSortedIndexCursor = sortedIndexStorage.get(indexRow.indexColumns());
+        Cursor<IndexRow> scanFromSortedIndexCursor = sortedIndexStorage.scan(null, null, 0);
+
+        tableStorage.destroyPartition(PARTITION_ID);
+
+        // Let's check that we won't get destroyed storages.
+        assertNull(tableStorage.getMvPartition(PARTITION_ID));
+        assertThrows(StorageException.class, () -> tableStorage.getOrCreateHashIndex(PARTITION_ID, hashIdx.id()));
+        assertThrows(StorageException.class, () -> tableStorage.getOrCreateSortedIndex(PARTITION_ID, sortedIdx.id()));
+
+        checkStorageDestroyed(mvPartitionStorage);
+        checkStorageDestroyed(hashIndexStorage);
+        checkStorageDestroyed(sortedIndexStorage);
+
+        assertThrows(StorageClosedException.class, () -> getAll(scanVersionsCursor));
+        assertThrows(StorageClosedException.class, () -> getAll(scanTimestampCursor));
+
+        assertThrows(StorageClosedException.class, () -> getAll(getFromHashIndexCursor));
+
+        assertThrows(StorageClosedException.class, () -> getAll(getFromSortedIndexCursor));
+        assertThrows(StorageClosedException.class, () -> getAll(scanFromSortedIndexCursor));
+
+        // Let's check that nothing will happen if we try to destroy a non-existing partition.
+        tableStorage.destroyPartition(PARTITION_ID);

Review Comment:
   Let's wrap this into `assertDoesNotThrow()` to give a hint to the reader that we want to make sure that nothing explodes



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/PartitionProcessingCounter.java:
##########
@@ -0,0 +1,83 @@
+/*
+ * 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;
+
+import java.lang.invoke.MethodHandles;
+import java.lang.invoke.VarHandle;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Helper class for tracking the completion of partition processing.
+ *
+ * <p>At the start of partition processing, you need to call {@link #onStartPartitionProcessing()}, at the end
+ * {@link #onFinishPartitionProcessing()}. When all partition processing is completed, the {@link #future()} will be completed.
+ *
+ * <p>It is recommended to use external synchronization for the correct operation of the {@link #counter partition processing counter} and
+ * the {@link #future future}.
+ */
+public class PartitionProcessingCounter {
+    private static final VarHandle COUNTER;
+
+    static {
+        try {
+            COUNTER = MethodHandles.lookup().findVarHandle(PartitionProcessingCounter.class, "counter", int.class);

Review Comment:
   It looks like a replacement for an `AtomicInteger`. `AtomicInteger` would make the code more concise and easier to understand, you could use `incrementAndGet()` and `decrementAndGet()` below.
   
   You probably wanted to spare object instances using this approach, but it's just one instance per partition. It seems unlikely that someone will have even a million partitions in the same JVM, so the saving is not worth code complexity.



##########
modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/AbstractMvTableStorageTest.java:
##########
@@ -198,6 +204,7 @@ public void testCreateHashIndex() {
     /**
      * Tests destroying an index.
      */
+    @Disabled("https://issues.apache.org/jira/browse/IGNITE-17626")

Review Comment:
   The mentioned Jira issue is about pagemem-based storages, but the test is disabled for all of them. Should other Jira issues be mentioned here as well?



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStoreManager.java:
##########
@@ -139,31 +149,63 @@ public void start() throws IgniteInternalCheckedException {
             throw new IgniteInternalCheckedException("Could not create work directory for page stores: " + dbDir, e);
         }
 
-        if (log.isWarnEnabled()) {
+        if (LOG.isWarnEnabled()) {
             String tmpDir = System.getProperty("java.io.tmpdir");
 
             if (tmpDir != null && this.dbDir.startsWith(tmpDir)) {
-                log.warn("Persistence store directory is in the temp directory and may be cleaned. "
+                LOG.warn("Persistence store directory is in the temp directory and may be cleaned. "
                         + "To avoid this change location of persistence directories [currentDir={}]", this.dbDir);
             }
         }
 
+        List<Path> toDelete = new ArrayList<>();
+
         try (Stream<Path> tmpFileStream = Files.find(
                 dbDir,
                 Integer.MAX_VALUE,
-                (path, basicFileAttributes) -> path.getFileName().toString().endsWith(TMP_FILE_SUFFIX)
-        )) {
-            List<Path> tmpFiles = tmpFileStream.collect(toList());
+                (path, basicFileAttributes) -> path.getFileName().toString().endsWith(TMP_FILE_SUFFIX))
+        ) {
+            toDelete.addAll(tmpFileStream.collect(toList()));
+        } catch (IOException e) {
+            throw new IgniteInternalCheckedException("Error while searching temporary files:" + dbDir, e);
+        }
+
+        Pattern delPartitionFilePatter = Pattern.compile(DEL_PART_FILE_REGEXP);

Review Comment:
   ```suggestion
           Pattern delPartitionFilePattern = Pattern.compile(DEL_PART_FILE_REGEXP);
   ```



##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/PersistentPageMemoryTableStorage.java:
##########
@@ -418,4 +430,43 @@ public CompletableFuture<Void> finishRebalanceMvPartition(int partitionId) {
         // TODO: IGNITE-18029 Implement
         throw new UnsupportedOperationException();
     }
+
+    @Override
+    public void destroyMvPartitionStorage(AbstractPageMemoryMvPartitionStorage mvPartitionStorage) throws StorageException {
+        int partitionId = mvPartitionStorage.partitionId();
+
+        CompletableFuture<Void> previousFuture = destroyFutureByPartitionId.put(partitionId, new CompletableFuture<>());
+
+        assert previousFuture == null : "Previous destruction of the partition has not completed: " + partitionId;
+
+        // It is enough for us to close the partition storage and its indexes (do not destroy), perform some actions, and then simply

Review Comment:
   What are 'some actions'? This is very mysterious :)



##########
modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/impl/TestHashIndexStorage.java:
##########
@@ -40,6 +41,8 @@ public class TestHashIndexStorage implements HashIndexStorage {
 
     private final HashIndexDescriptor descriptor;
 
+    private volatile boolean started = true;

Review Comment:
   It seems that `closed` would look more logical as making something not started after it was started is a bit strange.



##########
modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/AbstractHashIndexStorageTest.java:
##########
@@ -216,6 +217,7 @@ public void testRemoveIdempotence() {
         assertDoesNotThrow(() -> remove(row));
     }
 
+    @Disabled("https://issues.apache.org/jira/browse/IGNITE-17626")

Review Comment:
   The mentioned Jira issue is about pagemem-based indices, but the test is disabled for all of them. Should other Jira issues be mentioned here as well?



##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/sorted/PageMemorySortedIndexStorage.java:
##########
@@ -120,22 +188,63 @@ public void remove(IndexRow row) {
             remove.afterCompletion();
         } catch (IgniteInternalCheckedException e) {
             throw new StorageException("Failed to remove value from index", e);
+        } finally {
+            closeBusyLock.leaveBusy();
         }
     }
 
     @Override
     public Cursor<IndexRow> scan(@Nullable BinaryTuplePrefix lowerBound, @Nullable BinaryTuplePrefix upperBound, int flags) {
-        boolean includeLower = (flags & GREATER_OR_EQUAL) != 0;
-        boolean includeUpper = (flags & LESS_OR_EQUAL) != 0;
-
-        SortedIndexRowKey lower = createBound(lowerBound, !includeLower);
-
-        SortedIndexRowKey upper = createBound(upperBound, includeUpper);
+        if (!closeBusyLock.enterBusy()) {
+            throwStorageClosedException();
+        }
 
         try {
-            return map(sortedIndexTree.find(lower, upper), this::toIndexRowImpl);
+            boolean includeLower = (flags & GREATER_OR_EQUAL) != 0;
+            boolean includeUpper = (flags & LESS_OR_EQUAL) != 0;
+
+            SortedIndexRowKey lower = createBound(lowerBound, !includeLower);
+
+            SortedIndexRowKey upper = createBound(upperBound, includeUpper);
+
+            Cursor<SortedIndexRow> cursor = sortedIndexTree.find(lower, upper);
+
+            return new Cursor<>() {
+                @Override
+                public void close() {
+                    cursor.close();
+                }
+
+                @Override
+                public boolean hasNext() {

Review Comment:
   How about extracting a base class containing `hasNext()` and `close()` implementations like it's done in `BusyRocksIteratorAdapter`? This would allow to reduce duplication.



##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/hash/PageMemoryHashIndexStorage.java:
##########
@@ -17,22 +17,35 @@
 
 package org.apache.ignite.internal.storage.pagememory.index.hash;
 
+import java.lang.invoke.MethodHandles;
+import java.lang.invoke.VarHandle;
 import org.apache.ignite.internal.schema.BinaryTuple;
 import org.apache.ignite.internal.storage.RowId;
+import org.apache.ignite.internal.storage.StorageClosedException;
 import org.apache.ignite.internal.storage.StorageException;
 import org.apache.ignite.internal.storage.index.HashIndexDescriptor;
 import org.apache.ignite.internal.storage.index.HashIndexStorage;
 import org.apache.ignite.internal.storage.index.IndexRow;
 import org.apache.ignite.internal.storage.pagememory.index.freelist.IndexColumns;
 import org.apache.ignite.internal.storage.pagememory.index.freelist.IndexColumnsFreeList;
 import org.apache.ignite.internal.util.Cursor;
-import org.apache.ignite.internal.util.CursorUtils;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
 import org.apache.ignite.lang.IgniteInternalCheckedException;
 
 /**
- * Hash index storage implementation.
+ * Implementation of Hash index storage using Page Memory.
  */
 public class PageMemoryHashIndexStorage implements HashIndexStorage {
+    private static final VarHandle STARTED;
+
+    static {
+        try {
+            STARTED = MethodHandles.lookup().findVarHandle(PageMemoryHashIndexStorage.class, "started", boolean.class);

Review Comment:
   Let's use `AtomicBoolean` instead. It makes the code simpler and the price we pay (one object per index per partition) is tiny. The same relates to all pagemem-based index and `MvPartitionStorage` implementations.



-- 
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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStoreManager.java:
##########
@@ -139,31 +149,63 @@ public void start() throws IgniteInternalCheckedException {
             throw new IgniteInternalCheckedException("Could not create work directory for page stores: " + dbDir, e);
         }
 
-        if (log.isWarnEnabled()) {
+        if (LOG.isWarnEnabled()) {
             String tmpDir = System.getProperty("java.io.tmpdir");
 
             if (tmpDir != null && this.dbDir.startsWith(tmpDir)) {
-                log.warn("Persistence store directory is in the temp directory and may be cleaned. "
+                LOG.warn("Persistence store directory is in the temp directory and may be cleaned. "
                         + "To avoid this change location of persistence directories [currentDir={}]", this.dbDir);
             }
         }
 
+        List<Path> toDelete = new ArrayList<>();
+
         try (Stream<Path> tmpFileStream = Files.find(
                 dbDir,
                 Integer.MAX_VALUE,
-                (path, basicFileAttributes) -> path.getFileName().toString().endsWith(TMP_FILE_SUFFIX)
-        )) {
-            List<Path> tmpFiles = tmpFileStream.collect(toList());
+                (path, basicFileAttributes) -> path.getFileName().toString().endsWith(TMP_FILE_SUFFIX))
+        ) {
+            toDelete.addAll(tmpFileStream.collect(toList()));
+        } catch (IOException e) {
+            throw new IgniteInternalCheckedException("Error while searching temporary files:" + dbDir, e);
+        }
+
+        Pattern delPartitionFilePatter = Pattern.compile(DEL_PART_FILE_REGEXP);
+
+        try (Stream<Path> delFileStream = Files.find(
+                dbDir,
+                Integer.MAX_VALUE,
+                (path, basicFileAttributes) -> path.getFileName().toString().endsWith(DEL_FILE_SUFFIX))
+        ) {
+            delFileStream.forEach(delFilePath -> {
+                Matcher matcher = delPartitionFilePatter.matcher(delFilePath.getFileName().toString());
 
-            if (!tmpFiles.isEmpty()) {
-                if (log.isInfoEnabled()) {
-                    log.info("Temporary files to be deleted: {}", tmpFiles.size());
+                if (!matcher.matches()) {
+                    throw new IgniteInternalException("Unknown file: " + delFilePath);
                 }
 
-                tmpFiles.forEach(IgniteUtils::deleteIfExists);
-            }
+                Path tableWorkDir = delFilePath.getParent();
+
+                int partitionId = Integer.parseInt(matcher.group(1));
+
+                toDelete.add(tableWorkDir.resolve(String.format(PART_FILE_TEMPLATE, partitionId)));
+
+                try {
+                    toDelete.addAll(List.of(findPartitionDeltaFiles(tableWorkDir, partitionId)));
+                } catch (IgniteInternalCheckedException e) {
+                    throw new IgniteInternalException("Error when searching delta files for partition:" + delFilePath, e);
+                }
+
+                toDelete.add(delFilePath);
+            });
         } catch (IOException e) {
-            throw new IgniteInternalCheckedException("Could not create work directory for page stores: " + dbDir, e);
+            throw new IgniteInternalCheckedException("Error while searching temporary files:" + dbDir, e);
+        }
+
+        if (!toDelete.isEmpty()) {
+            LOG.info("Files to be deleted: {}", toDelete);

Review Comment:
   Maybe, I don't mind



-- 
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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStoreManager.java:
##########
@@ -443,4 +461,50 @@ public Path tmpDeltaFilePageStorePath(int groupId, int partitionId, int index) {
     public Path deltaFilePageStorePath(int groupId, int partitionId, int index) {
         return dbDir.resolve(GROUP_DIR_PREFIX + groupId).resolve(String.format(PART_DELTA_FILE_TEMPLATE, partitionId, index));
     }
+
+    /**
+     * Callback on destruction of the partition of the corresponding group.
+     *
+     * <p>Deletes the partition pages tore and all its delta files. Before that, it creates a marker file (for example,

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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/compaction/Compactor.java:
##########
@@ -338,6 +360,10 @@ void mergeDeltaFileToMainFile(
                 return;
             }
 
+            if (filePageStore.isMarkedToDestroy()) {

Review Comment:
   If at the moment we are compacting the partition, then we will wait until it is completed and only then delete the 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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStoreManager.java:
##########
@@ -230,78 +269,86 @@ public long allocatePage(int grpId, int partId, byte flags) throws IgniteInterna
     }
 
     /**
-     * Initializing the file page stores for a group.
+     * Initialization of the page storage for the group partition.
      *
      * @param tableName Table name.
      * @param tableId Integer table id.
-     * @param partitions Partition number, must be greater than {@code 0} and less {@link PageIdAllocator#MAX_PARTITION_ID}.
+     * @param partitionId Partition ID, must be between {@code 0} (inclusive) and {@link PageIdAllocator#MAX_PARTITION_ID} (inclusive).
      * @throws IgniteInternalCheckedException If failed.
      */
-    public void initialize(String tableName, int tableId, int partitions) throws IgniteInternalCheckedException {
-        assert partitions > 0 && partitions < MAX_PARTITION_ID : partitions;
+    public void initialize(String tableName, int tableId, int partitionId) throws IgniteInternalCheckedException {
+        assert partitionId >= 0 && partitionId <= MAX_PARTITION_ID : partitionId;
 
-        initGroupDirLock.lock(tableId);
+        stripedLock.lock(tableId + partitionId);
 
         try {
-            if (!groupPageStores.containsPageStores(tableId)) {
-                List<FilePageStore> partitionFilePageStores = createFilePageStores(tableId, partitions);
+            if (!groupPageStores.contains(tableId, partitionId)) {
+                Path tableWorkDir = ensureGroupWorkDir(tableId);
+
+                ByteBuffer buffer = allocateBuffer(pageSize);
+
+                try {
+                    Path partFilePath = tableWorkDir.resolve(String.format(PART_FILE_TEMPLATE, partitionId));
 
-                List<FilePageStore> old = groupPageStores.put(tableId, partitionFilePageStores);
+                    Path[] partDeltaFiles = findPartitionDeltaFiles(tableWorkDir, partitionId);
 
-                assert old == null : tableName;
+                    FilePageStore filePageStore = filePageStoreFactory.createPageStore(buffer.rewind(), partFilePath, partDeltaFiles);
+
+                    FilePageStore previous = groupPageStores.put(tableId, partitionId, filePageStore);
+
+                    assert previous == null : IgniteStringFormatter.format(
+                            "Parallel creation is not allowed: [tableName={}, tableId={}, partitionId={}]",
+                            tableName,
+                            tableId,
+                            partitionId
+                    );
+                } finally {
+                    freeBuffer(buffer);
+                }
             }
         } catch (IgniteInternalCheckedException e) {
             // TODO: IGNITE-16899 By analogy with 2.0, fail a node
 
             throw e;
         } finally {
-            initGroupDirLock.unlock(tableId);
+            stripedLock.unlock(tableId + partitionId);
         }
     }
 
     /**
-     * Returns collection of related partition file page stores for group.
+     * Returns the group partition page stores.
      *
      * @param grpId Group ID.
      */
-    public @Nullable List<FilePageStore> getStores(int grpId) {
+    public @Nullable GroupPageStores<FilePageStore> getStores(int grpId) {
         return groupPageStores.get(grpId);
     }
 
     /**
-     * Returns all page stores of all groups.
+     * Returns view for all page stores of all groups.
      */
-    public Collection<List<FilePageStore>> allPageStores() {
-        return groupPageStores.allPageStores();
+    public Collection<GroupPageStores<FilePageStore>> allPageStores() {
+        return groupPageStores.getAll();
     }
 
     /**
      * Returns partition file page store for the corresponding parameters.
      *
-     * @param grpId Group ID.
-     * @param partId Partition ID, from {@code 0} to {@link PageIdAllocator#MAX_PARTITION_ID} (inclusive).
-     * @throws IgniteInternalCheckedException If group or partition with the given ID was not created.
+     * @param groupId Group ID.
+     * @param partitionId Partition ID, from {@code 0} (inclusive) to {@link PageIdAllocator#MAX_PARTITION_ID} (inclusive).
      */
-    public FilePageStore getStore(int grpId, int partId) throws IgniteInternalCheckedException {
-        assert partId >= 0 && partId <= MAX_PARTITION_ID : partId;
+    public @Nullable FilePageStore getStore(int groupId, int partitionId) {

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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/checkpoint/Checkpointer.java:
##########
@@ -452,9 +456,21 @@ private void syncUpdatedPageStores(
                     return;
                 }
 
-                fsyncDeltaFilePageStoreOnCheckpointThread(entry.getKey(), entry.getValue());
+                GroupPartitionId partitionId = entry.getKey();
+
+                FilePageStore filePageStore = filePageStoreManager.getStore(partitionId.getGroupId(), partitionId.getPartitionId());
+
+                if (filePageStore == null || filePageStore.isMarkedToDestroy()) {
+                    continue;
+                }
+
+                currentCheckpointProgress.onStartPartitionProcessing(partitionId.getGroupId(), partitionId.getPartitionId());

Review Comment:
   Changed this code



-- 
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] rpuch commented on a diff in pull request #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStoreManager.java:
##########
@@ -230,78 +269,86 @@ public long allocatePage(int grpId, int partId, byte flags) throws IgniteInterna
     }
 
     /**
-     * Initializing the file page stores for a group.
+     * Initialization of the page storage for the group partition.
      *
      * @param tableName Table name.
      * @param tableId Integer table id.
-     * @param partitions Partition number, must be greater than {@code 0} and less {@link PageIdAllocator#MAX_PARTITION_ID}.
+     * @param partitionId Partition ID, must be between {@code 0} (inclusive) and {@link PageIdAllocator#MAX_PARTITION_ID} (inclusive).
      * @throws IgniteInternalCheckedException If failed.
      */
-    public void initialize(String tableName, int tableId, int partitions) throws IgniteInternalCheckedException {
-        assert partitions > 0 && partitions < MAX_PARTITION_ID : partitions;
+    public void initialize(String tableName, int tableId, int partitionId) throws IgniteInternalCheckedException {
+        assert partitionId >= 0 && partitionId <= MAX_PARTITION_ID : partitionId;
 
-        initGroupDirLock.lock(tableId);
+        stripedLock.lock(tableId + partitionId);
 
         try {
-            if (!groupPageStores.containsPageStores(tableId)) {
-                List<FilePageStore> partitionFilePageStores = createFilePageStores(tableId, partitions);
+            if (!groupPageStores.contains(tableId, partitionId)) {
+                Path tableWorkDir = ensureGroupWorkDir(tableId);
+
+                ByteBuffer buffer = allocateBuffer(pageSize);
+
+                try {
+                    Path partFilePath = tableWorkDir.resolve(String.format(PART_FILE_TEMPLATE, partitionId));
 
-                List<FilePageStore> old = groupPageStores.put(tableId, partitionFilePageStores);
+                    Path[] partDeltaFiles = findPartitionDeltaFiles(tableWorkDir, partitionId);
 
-                assert old == null : tableName;
+                    FilePageStore filePageStore = filePageStoreFactory.createPageStore(buffer.rewind(), partFilePath, partDeltaFiles);
+
+                    FilePageStore previous = groupPageStores.put(tableId, partitionId, filePageStore);
+
+                    assert previous == null : IgniteStringFormatter.format(
+                            "Parallel creation is not allowed: [tableName={}, tableId={}, partitionId={}]",
+                            tableName,
+                            tableId,
+                            partitionId
+                    );
+                } finally {
+                    freeBuffer(buffer);
+                }
             }
         } catch (IgniteInternalCheckedException e) {
             // TODO: IGNITE-16899 By analogy with 2.0, fail a node
 
             throw e;
         } finally {
-            initGroupDirLock.unlock(tableId);
+            stripedLock.unlock(tableId + partitionId);
         }
     }
 
     /**
-     * Returns collection of related partition file page stores for group.
+     * Returns the group partition page stores.
      *
      * @param grpId Group ID.
      */
-    public @Nullable List<FilePageStore> getStores(int grpId) {
+    public @Nullable GroupPageStores<FilePageStore> getStores(int grpId) {
         return groupPageStores.get(grpId);
     }
 
     /**
-     * Returns all page stores of all groups.
+     * Returns view for all page stores of all groups.
      */
-    public Collection<List<FilePageStore>> allPageStores() {
-        return groupPageStores.allPageStores();
+    public Collection<GroupPageStores<FilePageStore>> allPageStores() {
+        return groupPageStores.getAll();
     }
 
     /**
      * Returns partition file page store for the corresponding parameters.
      *
-     * @param grpId Group ID.
-     * @param partId Partition ID, from {@code 0} to {@link PageIdAllocator#MAX_PARTITION_ID} (inclusive).
-     * @throws IgniteInternalCheckedException If group or partition with the given ID was not created.
+     * @param groupId Group ID.
+     * @param partitionId Partition ID, from {@code 0} (inclusive) to {@link PageIdAllocator#MAX_PARTITION_ID} (inclusive).
      */
-    public FilePageStore getStore(int grpId, int partId) throws IgniteInternalCheckedException {
-        assert partId >= 0 && partId <= MAX_PARTITION_ID : partId;
+    public @Nullable FilePageStore getStore(int groupId, int partitionId) {

Review Comment:
   Please specify (in the javadoc) the conditions under which this method returns `null`. Does it happen if, and only if, the partition is destroyed?



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStoreManager.java:
##########
@@ -230,78 +269,86 @@ public long allocatePage(int grpId, int partId, byte flags) throws IgniteInterna
     }
 
     /**
-     * Initializing the file page stores for a group.
+     * Initialization of the page storage for the group partition.
      *
      * @param tableName Table name.
      * @param tableId Integer table id.
-     * @param partitions Partition number, must be greater than {@code 0} and less {@link PageIdAllocator#MAX_PARTITION_ID}.
+     * @param partitionId Partition ID, must be between {@code 0} (inclusive) and {@link PageIdAllocator#MAX_PARTITION_ID} (inclusive).
      * @throws IgniteInternalCheckedException If failed.
      */
-    public void initialize(String tableName, int tableId, int partitions) throws IgniteInternalCheckedException {
-        assert partitions > 0 && partitions < MAX_PARTITION_ID : partitions;
+    public void initialize(String tableName, int tableId, int partitionId) throws IgniteInternalCheckedException {
+        assert partitionId >= 0 && partitionId <= MAX_PARTITION_ID : partitionId;
 
-        initGroupDirLock.lock(tableId);
+        stripedLock.lock(tableId + partitionId);
 
         try {
-            if (!groupPageStores.containsPageStores(tableId)) {
-                List<FilePageStore> partitionFilePageStores = createFilePageStores(tableId, partitions);
+            if (!groupPageStores.contains(tableId, partitionId)) {
+                Path tableWorkDir = ensureGroupWorkDir(tableId);
+
+                ByteBuffer buffer = allocateBuffer(pageSize);
+
+                try {
+                    Path partFilePath = tableWorkDir.resolve(String.format(PART_FILE_TEMPLATE, partitionId));
 
-                List<FilePageStore> old = groupPageStores.put(tableId, partitionFilePageStores);
+                    Path[] partDeltaFiles = findPartitionDeltaFiles(tableWorkDir, partitionId);
 
-                assert old == null : tableName;
+                    FilePageStore filePageStore = filePageStoreFactory.createPageStore(buffer.rewind(), partFilePath, partDeltaFiles);
+
+                    FilePageStore previous = groupPageStores.put(tableId, partitionId, filePageStore);
+
+                    assert previous == null : IgniteStringFormatter.format(
+                            "Parallel creation is not allowed: [tableName={}, tableId={}, partitionId={}]",
+                            tableName,
+                            tableId,
+                            partitionId
+                    );
+                } finally {
+                    freeBuffer(buffer);
+                }
             }
         } catch (IgniteInternalCheckedException e) {
             // TODO: IGNITE-16899 By analogy with 2.0, fail a node
 
             throw e;
         } finally {
-            initGroupDirLock.unlock(tableId);
+            stripedLock.unlock(tableId + partitionId);
         }
     }
 
     /**
-     * Returns collection of related partition file page stores for group.
+     * Returns the group partition page stores.
      *
      * @param grpId Group ID.
      */
-    public @Nullable List<FilePageStore> getStores(int grpId) {
+    public @Nullable GroupPageStores<FilePageStore> getStores(int grpId) {
         return groupPageStores.get(grpId);
     }
 
     /**
-     * Returns all page stores of all groups.
+     * Returns view for all page stores of all groups.
      */
-    public Collection<List<FilePageStore>> allPageStores() {
-        return groupPageStores.allPageStores();
+    public Collection<GroupPageStores<FilePageStore>> allPageStores() {
+        return groupPageStores.getAll();
     }
 
     /**
      * Returns partition file page store for the corresponding parameters.
      *
-     * @param grpId Group ID.
-     * @param partId Partition ID, from {@code 0} to {@link PageIdAllocator#MAX_PARTITION_ID} (inclusive).
-     * @throws IgniteInternalCheckedException If group or partition with the given ID was not created.
+     * @param groupId Group ID.
+     * @param partitionId Partition ID, from {@code 0} (inclusive) to {@link PageIdAllocator#MAX_PARTITION_ID} (inclusive).
      */
-    public FilePageStore getStore(int grpId, int partId) throws IgniteInternalCheckedException {
-        assert partId >= 0 && partId <= MAX_PARTITION_ID : partId;
+    public @Nullable FilePageStore getStore(int groupId, int partitionId) {

Review Comment:
   Also, this method is called 3 times in this class (in `read()`, `write()` and `allocate()`), and the result is never checked for `null`, but methods are invoked on the result right off the bat. I suggest adding a method like `getRequiredStore()` that would call `getStore()`, throw if the result is null (with a proper explanation) and then return non-null result.



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/checkpoint/Checkpointer.java:
##########
@@ -452,9 +456,21 @@ private void syncUpdatedPageStores(
                     return;
                 }
 
-                fsyncDeltaFilePageStoreOnCheckpointThread(entry.getKey(), entry.getValue());
+                GroupPartitionId partitionId = entry.getKey();
+
+                FilePageStore filePageStore = filePageStoreManager.getStore(partitionId.getGroupId(), partitionId.getPartitionId());
+
+                if (filePageStore == null || filePageStore.isMarkedToDestroy()) {

Review Comment:
   Let's replace
   
   ```
   if (COND) continue;
   REST;
   ```
   
   with
   
   ```
   if (!COND) {
       REST
   }
   ```
   
   After this transformation, lines 459-473 will become identical to lines 498-510, so a method could be extracted to avoid code duplication.



-- 
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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/AbstractMvTableStorageTest.java:
##########
@@ -198,6 +204,7 @@ public void testCreateHashIndex() {
     /**
      * Tests destroying an index.
      */
+    @Disabled("https://issues.apache.org/jira/browse/IGNITE-17626")

Review Comment:
   Fair enough, I think in this ticket it is necessary to indicate about all the storages, when we start it they will be in different tickets.



-- 
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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/compaction/Compactor.java:
##########
@@ -146,7 +150,7 @@ protected void body() throws InterruptedException {
     void waitDeltaFiles() {

Review Comment:
   The compactor is a one-thread class, we use an additional thread only when compacting the delta files themselves, the rest of the methods are called only in one thread.



-- 
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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/compaction/Compactor.java:
##########
@@ -164,77 +170,90 @@ void waitDeltaFiles() {
     }
 
     /**
-     * Adds the number of delta files to compact.
-     *
-     * @param count Number of delta files.
+     * Callback on adding delta files so we can start compacting them.
      */
-    public void addDeltaFiles(int count) {
-        assert count >= 0;
-
-        if (count > 0) {
-            deltaFileCount.addAndGet(count);
+    public void onAddingDeltaFiles() {
+        synchronized (mux) {
+            addedDeltaFiles = true;
 
-            synchronized (mux) {
-                mux.notifyAll();
-            }
+            mux.notifyAll();
         }
     }
 
     /**
      * Merges delta files with partition files.
      */
     void doCompaction() {

Review Comment:
   Tried to 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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/storage-api/src/main/java/org/apache/ignite/internal/storage/StorageClosedException.java:
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.storage;
+
+/**
+ *  Exception that will be thrown when the storage is closed.
+ */
+public class StorageClosedException extends StorageException {
+    /**
+     * Constructor.
+     *
+     * @param message Error message.
+     */
+    public StorageClosedException(String message) {
+        super(message);
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param message Error message.
+     * @param cause The cause.
+     */
+    public StorageClosedException(String message, Throwable cause) {
+        super(message, cause);

Review Comment:
   coming soon



-- 
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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/PartitionProcessingCounter.java:
##########
@@ -0,0 +1,83 @@
+/*
+ * 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;
+
+import java.lang.invoke.MethodHandles;
+import java.lang.invoke.VarHandle;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Helper class for tracking the completion of partition processing.
+ *
+ * <p>At the start of partition processing, you need to call {@link #onStartPartitionProcessing()}, at the end
+ * {@link #onFinishPartitionProcessing()}. When all partition processing is completed, the {@link #future()} will be completed.
+ *
+ * <p>It is recommended to use external synchronization for the correct operation of the {@link #counter partition processing counter} and
+ * the {@link #future future}.
+ */
+public class PartitionProcessingCounter {
+    private static final VarHandle COUNTER;
+
+    static {
+        try {
+            COUNTER = MethodHandles.lookup().findVarHandle(PartitionProcessingCounter.class, "counter", int.class);
+        } catch (ReflectiveOperationException e) {
+            throw new ExceptionInInitializerError(e);
+        }
+    }
+
+    /** Partition processing counter must be greater than or equal to zero. */
+    @SuppressWarnings("unused")
+    private volatile int counter;
+
+    /** Future that will be completed when the {@link #counter} is zero. */
+    private final CompletableFuture<Void> future = new CompletableFuture<>();
+
+    /**
+     * Callback at the start of partition processing.

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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/checkpoint/Checkpointer.java:
##########
@@ -452,9 +456,21 @@ private void syncUpdatedPageStores(
                     return;
                 }
 
-                fsyncDeltaFilePageStoreOnCheckpointThread(entry.getKey(), entry.getValue());
+                GroupPartitionId partitionId = entry.getKey();
+
+                FilePageStore filePageStore = filePageStoreManager.getStore(partitionId.getGroupId(), partitionId.getPartitionId());
+
+                if (filePageStore == null || filePageStore.isMarkedToDestroy()) {

Review Comment:
   I would not like to change the code and also this file only has 332 lines



-- 
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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/checkpoint/Checkpointer.java:
##########
@@ -775,4 +799,41 @@ private void renameDeltaFileOnCheckpointThread(GroupPartitionId partitionId) thr
     void updateLastProgressAfterReleaseWriteLock() {
         afterReleaseWriteLockCheckpointProgress = currentCheckpointProgress;
     }
+
+    /**
+     * Callback on destruction of the partition of the corresponding group.
+     *
+     * <p>If the checkpoint is in progress, then wait until it finishes processing the partition that we are going to destroy, in order to
+     * prevent the situation when we want to destroy the partition file along with its delta files, and at this time the checkpoint performs
+     * I/O operations on them.
+     *
+     * @param groupId Group ID.
+     * @param partitionId Partition ID.
+     * @throws IgniteInternalCheckedException If there are errors while processing the callback.
+     */
+    void onPartitionDestruction(int groupId, int partitionId) throws IgniteInternalCheckedException {
+        CheckpointProgressImpl currentCheckpointProgress = this.currentCheckpointProgress;
+
+        if (currentCheckpointProgress == null || !currentCheckpointProgress.inProgress()) {
+            return;
+        }
+
+        CompletableFuture<Void> processedPartitionFuture = currentCheckpointProgress.getProcessedPartitionFuture(groupId, partitionId);
+
+        if (processedPartitionFuture != null) {
+            try {
+                // Time is taken arbitrarily, but long enough to allow time for the future to complete.
+                processedPartitionFuture.get(10, SECONDS);

Review Comment:
   Tried to fix it



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/checkpoint/CheckpointManager.java:
##########
@@ -307,11 +312,23 @@ static int[] pageIndexesForDeltaFilePageStore(CheckpointDirtyPagesView partition
     }
 
     /**
-     * Adds the number of delta files to compact.
+     * Callback on adding delta files so we can start compacting them.
+     */
+    public void onAddingDeltaFiles() {
+        compactor.onAddingDeltaFiles();
+    }
+
+    /**
+     * Callback on destruction of the partition of the corresponding group.
+     *
+     * <p>Prepares the checkpointer and compactor for partition destruction.
      *
-     * @param count Number of delta files.
+     * @param groupId Group ID.
+     * @param partitionId Partition ID.
+     * @throws IgniteInternalCheckedException If there are errors while processing the callback.
      */
-    public void addDeltaFileCountForCompaction(int count) {
-        compactor.addDeltaFiles(count);
+    public void onPartitionDestruction(int groupId, int partitionId) throws IgniteInternalCheckedException {

Review Comment:
   Tried to 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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/compaction/Compactor.java:
##########
@@ -357,14 +387,70 @@ void mergeDeltaFileToMainFile(
             return;
         }
 
+        if (filePageStore.isMarkedToDestroy()) {
+            return;
+        }
+
         boolean removed = filePageStore.removeDeltaFile(deltaFilePageStore);
 
         assert removed : filePageStore.filePath();
 
         deltaFilePageStore.markMergedToFilePageStore();
 
         deltaFilePageStore.stop(true);
+    }
+
+    /**
+     * Callback on destruction of the partition of the corresponding group.
+     *
+     * <p>If the partition compaction is in progress, then we will wait until it is completed so that there are no errors when we want to
+     * destroy the partition file and its delta file, and at this time its compaction occurs.
+     *
+     * @param groupId Group ID.
+     * @param partitionId Partition ID.
+     */
+    public void onPartitionDestruction(int groupId, int partitionId) throws IgniteInternalCheckedException {
+        CompletableFuture<Void> partitionProcessingFuture = processedPartitionMap.getProcessedPartitionFuture(groupId, partitionId);
+
+        if (partitionProcessingFuture != null) {
+            try {
+                // Time is taken arbitrarily, but long enough to allow time for the future to complete.
+                partitionProcessingFuture.get(10, SECONDS);

Review Comment:
   Tried to 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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/raft/snapshot/incoming/IncomingSnapshotCopier.java:
##########
@@ -195,8 +195,8 @@ private CompletableFuture<?> prepareMvPartitionStorageForRebalance(Executor exec
             return completedFuture(null);
         }
 
-        return CompletableFuture.supplyAsync(() -> partitionSnapshotStorage.partition().reCreateMvPartitionStorage(), executor)
-                .thenCompose(mvPartitionStorage -> {
+        return partitionSnapshotStorage.partition().reCreateMvPartitionStorage()

Review Comment:
   Partition destruction is now an asynchronous operation that will be performed in some pool/thread, and we need to wait for the future to complete.



-- 
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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/VolatilePageMemoryTableStorage.java:
##########
@@ -157,4 +155,10 @@ public CompletableFuture<Void> finishRebalanceMvPartition(int partitionId) {
         // TODO: IGNITE-18028 Implement
         throw new UnsupportedOperationException();
     }
+
+    @Override
+    public void destroyMvPartitionStorage(AbstractPageMemoryMvPartitionStorage mvPartitionStorage) throws StorageException {

Review Comment:
   Does anyone call it outside of the base class?



-- 
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] rpuch commented on a diff in pull request #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/raft/snapshot/incoming/IncomingSnapshotCopier.java:
##########
@@ -195,8 +195,8 @@ private CompletableFuture<?> prepareMvPartitionStorageForRebalance(Executor exec
             return completedFuture(null);
         }
 
-        return CompletableFuture.supplyAsync(() -> partitionSnapshotStorage.partition().reCreateMvPartitionStorage(), executor)
-                .thenCompose(mvPartitionStorage -> {
+        return partitionSnapshotStorage.partition().reCreateMvPartitionStorage()

Review Comment:
   Cool!



-- 
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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/AbstractMvTableStorageTest.java:
##########
@@ -444,7 +444,7 @@ public void testDestroyPartition() throws Exception {
         assertThrows(StorageClosedException.class, () -> getAll(scanFromSortedIndexCursor));
 
         // Let's check that nothing will happen if we try to destroy a non-existing partition.
-        tableStorage.destroyPartition(PARTITION_ID);
+        assertDoesNotThrow(() -> tableStorage.destroyPartition(PARTITION_ID));

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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/PartitionProcessingCounter.java:
##########
@@ -0,0 +1,83 @@
+/*
+ * 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;
+
+import java.lang.invoke.MethodHandles;
+import java.lang.invoke.VarHandle;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Helper class for tracking the completion of partition processing.
+ *
+ * <p>At the start of partition processing, you need to call {@link #onStartPartitionProcessing()}, at the end
+ * {@link #onFinishPartitionProcessing()}. When all partition processing is completed, the {@link #future()} will be completed.
+ *
+ * <p>It is recommended to use external synchronization for the correct operation of the {@link #counter partition processing counter} and
+ * the {@link #future future}.
+ */
+public class PartitionProcessingCounter {
+    private static final VarHandle COUNTER;
+
+    static {
+        try {
+            COUNTER = MethodHandles.lookup().findVarHandle(PartitionProcessingCounter.class, "counter", int.class);

Review Comment:
   I do not think that for us it greatly complicates the code.



-- 
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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/compaction/Compactor.java:
##########
@@ -348,6 +374,10 @@ void mergeDeltaFileToMainFile(
             return;
         }
 
+        if (filePageStore.isMarkedToDestroy()) {

Review Comment:
   I think we don't need a separate method that just calls another method, a bit redundant.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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


[GitHub] [ignite-3] rpuch commented on a diff in pull request #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/PartitionProcessingCounter.java:
##########
@@ -0,0 +1,83 @@
+/*
+ * 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;
+
+import java.lang.invoke.MethodHandles;
+import java.lang.invoke.VarHandle;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Helper class for tracking the completion of partition processing.
+ *
+ * <p>At the start of partition processing, you need to call {@link #onStartPartitionProcessing()}, at the end
+ * {@link #onFinishPartitionProcessing()}. When all partition processing is completed, the {@link #future()} will be completed.
+ *
+ * <p>It is recommended to use external synchronization for the correct operation of the {@link #counter partition processing counter} and
+ * the {@link #future future}.
+ */
+public class PartitionProcessingCounter {
+    private static final VarHandle COUNTER;
+
+    static {
+        try {
+            COUNTER = MethodHandles.lookup().findVarHandle(PartitionProcessingCounter.class, "counter", int.class);

Review Comment:
   The complexity increases in both creation (`static` block with `try` and `MethodHandles` invocation vs `new AtomicInteger()`) and usage (`incrementAndGet()` vs `getAndAdd(this, 1) + 1`). It is slightly simpler, but simpler. And even slight simplification is wortha lot (surely more than saving 10k objects per JVM).



-- 
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] rpuch commented on a diff in pull request #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/checkpoint/Checkpointer.java:
##########
@@ -452,9 +456,21 @@ private void syncUpdatedPageStores(
                     return;
                 }
 
-                fsyncDeltaFilePageStoreOnCheckpointThread(entry.getKey(), entry.getValue());
+                GroupPartitionId partitionId = entry.getKey();
+
+                FilePageStore filePageStore = filePageStoreManager.getStore(partitionId.getGroupId(), partitionId.getPartitionId());
+
+                if (filePageStore == null || filePageStore.isMarkedToDestroy()) {
+                    continue;
+                }
+
+                currentCheckpointProgress.onStartPartitionProcessing(partitionId.getGroupId(), partitionId.getPartitionId());

Review Comment:
   Looks like all actual I/O made by a `Checkpointer` has to be enclosed in `onStartPartitionProcessing()`/`onFinishPartitionProcessing()` calls. If this is true, I suggest to describe this requirement somewhere (in the class javadoc, or in a comment in the beginning of this class).



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/compaction/Compactor.java:
##########
@@ -348,6 +374,10 @@ void mergeDeltaFileToMainFile(
             return;
         }
 
+        if (filePageStore.isMarkedToDestroy()) {

Review Comment:
   These checks (whether it's cancelled, whether the store is marked for destroy) are always done together in this method. How about uniting them in a single method like `shouldNotProceedWith(FilePageStore)` and call it in all places in this method?



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/checkpoint/Checkpointer.java:
##########
@@ -775,4 +799,41 @@ private void renameDeltaFileOnCheckpointThread(GroupPartitionId partitionId) thr
     void updateLastProgressAfterReleaseWriteLock() {
         afterReleaseWriteLockCheckpointProgress = currentCheckpointProgress;
     }
+
+    /**
+     * Callback on destruction of the partition of the corresponding group.
+     *
+     * <p>If the checkpoint is in progress, then wait until it finishes processing the partition that we are going to destroy, in order to
+     * prevent the situation when we want to destroy the partition file along with its delta files, and at this time the checkpoint performs
+     * I/O operations on them.
+     *
+     * @param groupId Group ID.
+     * @param partitionId Partition ID.
+     * @throws IgniteInternalCheckedException If there are errors while processing the callback.
+     */
+    void onPartitionDestruction(int groupId, int partitionId) throws IgniteInternalCheckedException {
+        CheckpointProgressImpl currentCheckpointProgress = this.currentCheckpointProgress;
+
+        if (currentCheckpointProgress == null || !currentCheckpointProgress.inProgress()) {
+            return;
+        }
+
+        CompletableFuture<Void> processedPartitionFuture = currentCheckpointProgress.getProcessedPartitionFuture(groupId, partitionId);

Review Comment:
   For a fresh reader, it is not clear why we are not afraid of a race between calling this method and starting a checkpoint. How about adding a comment explaining why this is the case (as you explained me in a private conversation, namely about generation already having been switched, so a new checkpoint would not write any pages from the corresponding group+partition, if I understood correctly).



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/compaction/Compactor.java:
##########
@@ -164,77 +170,90 @@ void waitDeltaFiles() {
     }
 
     /**
-     * Adds the number of delta files to compact.
-     *
-     * @param count Number of delta files.
+     * Callback on adding delta files so we can start compacting them.
      */
-    public void addDeltaFiles(int count) {
-        assert count >= 0;
-
-        if (count > 0) {
-            deltaFileCount.addAndGet(count);
+    public void onAddingDeltaFiles() {
+        synchronized (mux) {
+            addedDeltaFiles = true;
 
-            synchronized (mux) {
-                mux.notifyAll();
-            }
+            mux.notifyAll();
         }
     }
 
     /**
      * Merges delta files with partition files.
      */
     void doCompaction() {
-        // Let's collect one delta file for each partition.
-        Queue<IgniteBiTuple<FilePageStore, DeltaFilePageStoreIo>> queue = filePageStoreManager.allPageStores().stream()
-                .flatMap(List::stream)
-                .map(filePageStore -> {
+        while (true) {
+            // Let's collect one delta file for each partition.
+            ConcurrentLinkedQueue<DeltaFileToCompaction> queue = new ConcurrentLinkedQueue<>();
+
+            for (GroupPageStores<FilePageStore> groupPageStores : filePageStoreManager.allPageStores()) {
+                for (PartitionPageStore<FilePageStore> partitionPageStore : groupPageStores.getAll()) {
+                    FilePageStore filePageStore = partitionPageStore.pageStore();
+
                     DeltaFilePageStoreIo deltaFileToCompaction = filePageStore.getDeltaFileToCompaction();
 
-                    return deltaFileToCompaction == null ? null : new IgniteBiTuple<>(filePageStore, deltaFileToCompaction);
-                })
-                .filter(Objects::nonNull)
-                .collect(toCollection(ConcurrentLinkedQueue::new));
+                    if (!filePageStore.isMarkedToDestroy() && deltaFileToCompaction != null) {
+                        queue.add(new DeltaFileToCompaction(
+                                groupPageStores.groupId(),
+                                partitionPageStore.partitionId(),
+                                filePageStore,
+                                deltaFileToCompaction
+                        ));
+                    }
+                }
+            }
+
+            if (queue.isEmpty()) {
+                break;
+            }
+
+            updateHeartbeat();
 
-        assert !queue.isEmpty();
+            int threads = threadPoolExecutor == null ? 1 : threadPoolExecutor.getMaximumPoolSize();
 
-        updateHeartbeat();
+            CompletableFuture<?>[] futures = new CompletableFuture[threads];
 
-        int threads = threadPoolExecutor == null ? 1 : threadPoolExecutor.getMaximumPoolSize();
+            for (int i = 0; i < threads; i++) {
+                CompletableFuture<?> future = futures[i] = new CompletableFuture<>();
 
-        CompletableFuture<?>[] futures = new CompletableFuture[threads];
+                Runnable merger = () -> {
+                    DeltaFileToCompaction toMerge;
 
-        for (int i = 0; i < threads; i++) {
-            CompletableFuture<?> future = futures[i] = new CompletableFuture<>();
+                    try {
+                        while ((toMerge = queue.poll()) != null) {
+                            GroupPartitionId partitionId = new GroupPartitionId(toMerge.groupId, toMerge.partitionId);
+
+                            processedPartitionMap.onStartPartitionProcessing(partitionId.getGroupId(), partitionId.getPartitionId());

Review Comment:
   Also, it seems that, analogously to `Checkpointer`, all the actual I/O performed by this class must be enclosed in `onStartPartitionProcessing()`/`onFinishPartitionProcessing()` invocations, this is an invariant important for the whole thing to work correctly. If this is true, it would be nice if this requirement was stated explicitly in the class javadoc/comment.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: 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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/AbstractMvTableStorageTest.java:
##########
@@ -382,6 +389,85 @@ public void testFinishRebalanceMvPartition() throws Exception {
         assertDoesNotThrow(() -> tableStorage.finishRebalanceMvPartition(PARTITION_ID).get(1, TimeUnit.SECONDS));
     }
 
+    @Test
+    public void testDestroyPartition() throws Exception {
+        assertThrows(
+                IllegalArgumentException.class,
+                () -> tableStorage.destroyPartition(tableStorage.configuration().partitions().value())
+        );
+
+        MvPartitionStorage mvPartitionStorage = tableStorage.getOrCreateMvPartition(PARTITION_ID);
+        HashIndexStorage hashIndexStorage = tableStorage.getOrCreateHashIndex(PARTITION_ID, hashIdx.id());
+        SortedIndexStorage sortedIndexStorage = tableStorage.getOrCreateSortedIndex(PARTITION_ID, sortedIdx.id());
+
+        RowId rowId = new RowId(PARTITION_ID);
+
+        BinaryRow binaryRow = binaryRow(new TestKey(0, "0"), new TestValue(1, "1"));
+
+        IndexRow indexRow = indexRow(binaryRow, rowId);
+
+        mvPartitionStorage.runConsistently(() -> {
+            mvPartitionStorage.addWriteCommitted(rowId, binaryRow, clock.now());
+
+            hashIndexStorage.put(indexRow);
+
+            sortedIndexStorage.put(indexRow);
+
+            return null;
+        });
+
+        Cursor<ReadResult> scanVersionsCursor = mvPartitionStorage.scanVersions(rowId);
+        PartitionTimestampCursor scanTimestampCursor = mvPartitionStorage.scan(clock.now());
+
+        Cursor<RowId> getFromHashIndexCursor = hashIndexStorage.get(indexRow.indexColumns());
+
+        Cursor<RowId> getFromSortedIndexCursor = sortedIndexStorage.get(indexRow.indexColumns());
+        Cursor<IndexRow> scanFromSortedIndexCursor = sortedIndexStorage.scan(null, null, 0);
+
+        tableStorage.destroyPartition(PARTITION_ID);
+
+        // Let's check that we won't get destroyed storages.
+        assertNull(tableStorage.getMvPartition(PARTITION_ID));
+        assertThrows(StorageException.class, () -> tableStorage.getOrCreateHashIndex(PARTITION_ID, hashIdx.id()));
+        assertThrows(StorageException.class, () -> tableStorage.getOrCreateSortedIndex(PARTITION_ID, sortedIdx.id()));
+
+        checkStorageDestroyed(mvPartitionStorage);
+        checkStorageDestroyed(hashIndexStorage);
+        checkStorageDestroyed(sortedIndexStorage);
+
+        assertThrows(StorageClosedException.class, () -> getAll(scanVersionsCursor));
+        assertThrows(StorageClosedException.class, () -> getAll(scanTimestampCursor));
+
+        assertThrows(StorageClosedException.class, () -> getAll(getFromHashIndexCursor));
+
+        assertThrows(StorageClosedException.class, () -> getAll(getFromSortedIndexCursor));
+        assertThrows(StorageClosedException.class, () -> getAll(scanFromSortedIndexCursor));
+
+        // Let's check that nothing will happen if we try to destroy a non-existing partition.
+        tableStorage.destroyPartition(PARTITION_ID);

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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/compaction/Compactor.java:
##########
@@ -164,77 +170,90 @@ void waitDeltaFiles() {
     }
 
     /**
-     * Adds the number of delta files to compact.
-     *
-     * @param count Number of delta files.
+     * Callback on adding delta files so we can start compacting them.
      */
-    public void addDeltaFiles(int count) {
-        assert count >= 0;
-
-        if (count > 0) {
-            deltaFileCount.addAndGet(count);
+    public void onAddingDeltaFiles() {
+        synchronized (mux) {
+            addedDeltaFiles = true;
 
-            synchronized (mux) {
-                mux.notifyAll();
-            }
+            mux.notifyAll();
         }
     }
 
     /**
      * Merges delta files with partition files.
      */
     void doCompaction() {
-        // Let's collect one delta file for each partition.
-        Queue<IgniteBiTuple<FilePageStore, DeltaFilePageStoreIo>> queue = filePageStoreManager.allPageStores().stream()
-                .flatMap(List::stream)
-                .map(filePageStore -> {
+        while (true) {
+            // Let's collect one delta file for each partition.
+            ConcurrentLinkedQueue<DeltaFileToCompaction> queue = new ConcurrentLinkedQueue<>();
+
+            for (GroupPageStores<FilePageStore> groupPageStores : filePageStoreManager.allPageStores()) {
+                for (PartitionPageStore<FilePageStore> partitionPageStore : groupPageStores.getAll()) {
+                    FilePageStore filePageStore = partitionPageStore.pageStore();
+
                     DeltaFilePageStoreIo deltaFileToCompaction = filePageStore.getDeltaFileToCompaction();
 
-                    return deltaFileToCompaction == null ? null : new IgniteBiTuple<>(filePageStore, deltaFileToCompaction);
-                })
-                .filter(Objects::nonNull)
-                .collect(toCollection(ConcurrentLinkedQueue::new));
+                    if (!filePageStore.isMarkedToDestroy() && deltaFileToCompaction != null) {
+                        queue.add(new DeltaFileToCompaction(
+                                groupPageStores.groupId(),
+                                partitionPageStore.partitionId(),
+                                filePageStore,
+                                deltaFileToCompaction
+                        ));
+                    }
+                }
+            }
+
+            if (queue.isEmpty()) {
+                break;
+            }
+
+            updateHeartbeat();
 
-        assert !queue.isEmpty();
+            int threads = threadPoolExecutor == null ? 1 : threadPoolExecutor.getMaximumPoolSize();
 
-        updateHeartbeat();
+            CompletableFuture<?>[] futures = new CompletableFuture[threads];
 
-        int threads = threadPoolExecutor == null ? 1 : threadPoolExecutor.getMaximumPoolSize();
+            for (int i = 0; i < threads; i++) {
+                CompletableFuture<?> future = futures[i] = new CompletableFuture<>();
 
-        CompletableFuture<?>[] futures = new CompletableFuture[threads];
+                Runnable merger = () -> {
+                    DeltaFileToCompaction toMerge;
 
-        for (int i = 0; i < threads; i++) {
-            CompletableFuture<?> future = futures[i] = new CompletableFuture<>();
+                    try {
+                        while ((toMerge = queue.poll()) != null) {
+                            GroupPartitionId partitionId = new GroupPartitionId(toMerge.groupId, toMerge.partitionId);
+
+                            processedPartitionMap.onStartPartitionProcessing(partitionId.getGroupId(), partitionId.getPartitionId());

Review Comment:
   For methods `onStartPartitionProcessing` and `onFinishPartitionProcessing` replaced with `GroupPartitionId`



-- 
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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/compaction/Compactor.java:
##########
@@ -65,13 +66,17 @@ public class Compactor extends IgniteWorker {
 
     private final @Nullable ThreadPoolExecutor threadPoolExecutor;
 
-    private final AtomicInteger deltaFileCount = new AtomicInteger();
+    /** Guarded by {@link #mux}. */
+    private boolean addedDeltaFiles;
 
     private final FilePageStoreManager filePageStoreManager;
 
     /** Thread local with buffers for the compaction threads. */
     private final ThreadLocal<ByteBuffer> threadBuf;
 
+    /** Partitions currently being processed, for example, writes dirty pages to delta file. */

Review Comment:
   Tried to 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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/PartitionProcessingCounter.java:
##########
@@ -0,0 +1,83 @@
+/*
+ * 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;
+
+import java.lang.invoke.MethodHandles;
+import java.lang.invoke.VarHandle;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Helper class for tracking the completion of partition processing.
+ *
+ * <p>At the start of partition processing, you need to call {@link #onStartPartitionProcessing()}, at the end
+ * {@link #onFinishPartitionProcessing()}. When all partition processing is completed, the {@link #future()} will be completed.
+ *
+ * <p>It is recommended to use external synchronization for the correct operation of the {@link #counter partition processing counter} and
+ * the {@link #future future}.
+ */
+public class PartitionProcessingCounter {

Review Comment:
   Why not, fixed 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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/GroupPageStoresMap.java:
##########
@@ -47,52 +50,156 @@ public GroupPageStoresMap(LongOperationAsyncExecutor longOperationAsyncExecutor)
     }
 
     /**
-     * Puts the page stores for the group.
+     * Puts the page store for the group partition.
+     *
+     * @param groupId Group ID.
+     * @param partitionId Partition ID.
+     * @param pageStore Page store.
+     * @return Previous page store.
+     */
+    public @Nullable T put(Integer groupId, Integer partitionId, T pageStore) {
+        return longOperationAsyncExecutor.afterAsyncCompletion(() -> {
+                    PartitionPageStore<T> previous = groupIdPageStores

Review Comment:
   Check style does not swear.



-- 
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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/compaction/Compactor.java:
##########
@@ -348,6 +374,10 @@ void mergeDeltaFileToMainFile(
             return;
         }
 
+        if (filePageStore.isMarkedToDestroy()) {

Review Comment:
   But for what?



-- 
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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/PartitionProcessingCounter.java:
##########
@@ -0,0 +1,83 @@
+/*
+ * 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;
+
+import java.lang.invoke.MethodHandles;
+import java.lang.invoke.VarHandle;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Helper class for tracking the completion of partition processing.
+ *
+ * <p>At the start of partition processing, you need to call {@link #onStartPartitionProcessing()}, at the end
+ * {@link #onFinishPartitionProcessing()}. When all partition processing is completed, the {@link #future()} will be completed.
+ *
+ * <p>It is recommended to use external synchronization for the correct operation of the {@link #counter partition processing counter} and
+ * the {@link #future future}.
+ */
+public class PartitionProcessingCounter {
+    private static final VarHandle COUNTER;
+
+    static {
+        try {
+            COUNTER = MethodHandles.lookup().findVarHandle(PartitionProcessingCounter.class, "counter", int.class);
+        } catch (ReflectiveOperationException e) {
+            throw new ExceptionInInitializerError(e);
+        }
+    }
+
+    /** Partition processing counter must be greater than or equal to zero. */
+    @SuppressWarnings("unused")
+    private volatile int counter;
+
+    /** Future that will be completed when the {@link #counter} is zero. */
+    private final CompletableFuture<Void> future = new CompletableFuture<>();
+
+    /**
+     * Callback at the start of partition processing.
+     */
+    public void onStartPartitionProcessing() {
+        assert !future.isDone();
+
+        int updatedValue = (int) COUNTER.getAndAdd(this, 1) + 1;
+
+        assert updatedValue > 0 : updatedValue;
+    }
+
+    /**
+     * Callback at the finish of partition processing.
+     */
+    public void onFinishPartitionProcessing() {
+        assert !future.isDone();
+
+        int updatedValue = (int) COUNTER.getAndAdd(this, -1) - 1;

Review Comment:
   It is, but it's not complex code.



-- 
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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/impl/TestHashIndexStorage.java:
##########
@@ -40,6 +41,8 @@ public class TestHashIndexStorage implements HashIndexStorage {
 
     private final HashIndexDescriptor descriptor;
 
+    private volatile boolean started = true;

Review Comment:
   change it



##########
modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/impl/TestSortedIndexStorage.java:
##########
@@ -48,6 +49,8 @@ public class TestSortedIndexStorage implements SortedIndexStorage {
 
     private final SortedIndexDescriptor descriptor;
 
+    private volatile boolean started = true;

Review Comment:
   Change 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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/PartitionProcessingCounterMap.java:
##########
@@ -0,0 +1,90 @@
+/*
+ * 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;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Helper class for thread-safe work with {@link PartitionProcessingCounter} for any partition of any group.
+ */
+public class PartitionProcessingCounterMap {
+    private final ConcurrentMap<GroupPartitionId, PartitionProcessingCounter> processedPartitions = new ConcurrentHashMap<>();
+
+    /**
+     * Callback at the beginning of checkpoint processing of a partition, for example, when writing dirty pages or executing a fsync.
+     *
+     * @param groupId Group ID.
+     * @param partitionId Partition ID.
+     */
+    public void onStartPartitionProcessing(int groupId, int partitionId) {
+        GroupPartitionId groupPartitionId = new GroupPartitionId(groupId, partitionId);
+
+        processedPartitions.compute(groupPartitionId, (id, partitionProcessingCounter) -> {
+            if (partitionProcessingCounter == null) {
+                PartitionProcessingCounter counter = new PartitionProcessingCounter();
+
+                counter.onStartPartitionProcessing();
+
+                return counter;
+            }
+
+            partitionProcessingCounter.onStartPartitionProcessing();
+
+            return partitionProcessingCounter;
+        });
+    }
+
+    /**
+     * Callback on completion of partition processing, for example, when writing dirty pages or executing a fsync.
+     *
+     * @param groupId Group ID.
+     * @param partitionId Partition ID.
+     */
+    public void onFinishPartitionProcessing(int groupId, int partitionId) {
+        GroupPartitionId groupPartitionId = new GroupPartitionId(groupId, partitionId);
+
+        processedPartitions.compute(groupPartitionId, (id, partitionProcessingCounter) -> {
+            assert partitionProcessingCounter != null : id;
+            assert !partitionProcessingCounter.future().isDone() : id;
+
+            partitionProcessingCounter.onFinishPartitionProcessing();
+
+            return partitionProcessingCounter.future().isDone() ? null : partitionProcessingCounter;
+        });
+    }
+
+    /**
+     * Returns the future if the partition according to the given parameters is currently being processed, for example, dirty pages are
+     * being written or fsync is being done, {@code null} if the partition is not currently being processed.
+     *
+     * <p>Future will be added on {@link #onStartPartitionProcessing(int, int)} call and completed on
+     * {@link #onFinishPartitionProcessing(int, int)} call (equal to the number of {@link #onFinishPartitionProcessing(int, int)} calls).
+     *
+     * @param groupId Group ID.
+     * @param partitionId Partition ID.
+     */
+    @Nullable
+    public CompletableFuture<Void> getProcessedPartitionFuture(int groupId, int partitionId) {
+        PartitionProcessingCounter partitionProcessingCounter = processedPartitions.get(new GroupPartitionId(groupId, partitionId));
+
+        return partitionProcessingCounter == null ? null : partitionProcessingCounter.future();

Review Comment:
   But this reflects the essence, if the partition is now somehow processed, then there will be futures, otherwise `null`



-- 
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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/PartitionProcessingCounter.java:
##########
@@ -0,0 +1,83 @@
+/*
+ * 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;
+
+import java.lang.invoke.MethodHandles;
+import java.lang.invoke.VarHandle;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Helper class for tracking the completion of partition processing.
+ *
+ * <p>At the start of partition processing, you need to call {@link #onStartPartitionProcessing()}, at the end
+ * {@link #onFinishPartitionProcessing()}. When all partition processing is completed, the {@link #future()} will be completed.
+ *
+ * <p>It is recommended to use external synchronization for the correct operation of the {@link #counter partition processing counter} and

Review Comment:
   Tried to 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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/compaction/Compactor.java:
##########
@@ -65,13 +66,17 @@ public class Compactor extends IgniteWorker {
 
     private final @Nullable ThreadPoolExecutor threadPoolExecutor;
 
-    private final AtomicInteger deltaFileCount = new AtomicInteger();
+    /** Guarded by {@link #mux}. */
+    private boolean addedDeltaFiles;
 
     private final FilePageStoreManager filePageStoreManager;
 
     /** Thread local with buffers for the compaction threads. */
     private final ThreadLocal<ByteBuffer> threadBuf;

Review Comment:
   So what? Initial value will be 0, that's fine, you can change it during the start



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/compaction/Compactor.java:
##########
@@ -65,13 +66,17 @@ public class Compactor extends IgniteWorker {
 
     private final @Nullable ThreadPoolExecutor threadPoolExecutor;
 
-    private final AtomicInteger deltaFileCount = new AtomicInteger();
+    /** Guarded by {@link #mux}. */
+    private boolean addedDeltaFiles;
 
     private final FilePageStoreManager filePageStoreManager;
 
     /** Thread local with buffers for the compaction threads. */
     private final ThreadLocal<ByteBuffer> threadBuf;

Review Comment:
   So what? Initial value will be null, that's fine, you can change it during the start



-- 
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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/AbstractMvTableStorageTest.java:
##########
@@ -419,11 +505,69 @@ private static void createTestTable(TableConfiguration tableConfig) {
         assertThat(createTableFuture, willCompleteSuccessfully());
     }
 
-    private static <T> List<T> getAll(Cursor<T> cursor) {
+    private static <T> List<T> getAll(Cursor<T> cursor) throws Exception {
         try (cursor) {
             return cursor.stream().collect(Collectors.toList());
-        } catch (Exception e) {
-            throw new RuntimeException(e);
         }
     }
+
+    private void checkStorageDestroyed(MvPartitionStorage storage) {
+        int partId = PARTITION_ID;
+
+        assertThrows(StorageClosedException.class, () -> storage.runConsistently(() -> null));
+
+        assertThrows(StorageClosedException.class, storage::flush);
+
+        assertThrows(StorageClosedException.class, storage::lastAppliedIndex);
+        assertThrows(StorageClosedException.class, storage::lastAppliedTerm);
+        assertThrows(StorageClosedException.class, storage::persistedIndex);
+
+        assertThrows(StorageClosedException.class, () -> storage.runConsistently(() -> {

Review Comment:
   I agree that it can be removed.



-- 
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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/test/java/org/apache/ignite/internal/pagememory/persistence/PartitionMetaManagerTest.java:
##########
@@ -188,6 +189,23 @@ void testReadWritePartitionMeta(@WorkDirectory Path workDir) throws Exception {
         }
     }
 
+    @Test
+    void testRemoveMeta() {
+        PartitionMetaManager manager = new PartitionMetaManager(ioRegistry, PAGE_SIZE);
+
+        GroupPartitionId id = new GroupPartitionId(0, 0);
+
+        assertDoesNotThrow(() -> manager.removeMeta(id));

Review Comment:
   Remove 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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/checkpoint/CheckpointPagesWriterFactory.java:
##########
@@ -106,7 +104,7 @@ CheckpointPagesWriter build(
             BooleanSupplier shutdownNow
     ) {
         return new CheckpointPagesWriter(
-                log,
+                LOG,

Review Comment:
   I think you're right, I'll 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] rpuch commented on a diff in pull request #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/checkpoint/Checkpointer.java:
##########
@@ -500,13 +501,13 @@ private void syncUpdatedPageStores(
                             FilePageStore filePageStore = filePageStoreManager.getStore(partId.getGroupId(), partId.getPartitionId());
 
                             if (filePageStore != null && !filePageStore.isMarkedToDestroy()) {
-                                currentCheckpointProgress.onStartPartitionProcessing(partId.getGroupId(), partId.getPartitionId());
+                                currentCheckpointProgress.onStartPartitionProcessing(partId);
 
                                 fsyncDeltaFilePageStoreOnCheckpointThread(filePageStore, entry.getValue());
 
                                 renameDeltaFileOnCheckpointThread(filePageStore, partId);
 
-                                currentCheckpointProgress.onFinishPartitionProcessing(partId.getGroupId(), partId.getPartitionId());
+                                currentCheckpointProgress.onFinishPartitionProcessing(partId);

Review Comment:
   Shouldn't this also be invoked in a `finally` block?



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/compaction/Compactor.java:
##########
@@ -42,15 +43,13 @@
 import org.apache.ignite.internal.util.IgniteUtils;
 import org.apache.ignite.internal.util.worker.IgniteWorker;
 import org.apache.ignite.internal.util.worker.IgniteWorkerListener;
-import org.apache.ignite.lang.IgniteInternalCheckedException;
 import org.apache.ignite.lang.IgniteInternalException;
-import org.apache.ignite.lang.IgniteStringFormatter;
 import org.jetbrains.annotations.Nullable;
 
 /**
  * Entity to compact delta files.
  *
- * <p>To start compacting delta files, you need to notify about the appearance of {@link #onAddingDeltaFiles() delta files ready for
+ * <p>To start compacting delta files, you need to notify about the appearance of {@link #triggerCompaction() delta files ready for

Review Comment:
   Probably this javadoc needs to be reworded as the method seems to be now about triggering the compaction, not about appearance of something



##########
modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/AbstractMvTableStorageTest.java:
##########
@@ -444,7 +444,7 @@ public void testDestroyPartition() throws Exception {
         assertThrows(StorageClosedException.class, () -> getAll(scanFromSortedIndexCursor));
 
         // Let's check that nothing will happen if we try to destroy a non-existing partition.
-        tableStorage.destroyPartition(PARTITION_ID);
+        assertDoesNotThrow(() -> tableStorage.destroyPartition(PARTITION_ID));

Review Comment:
   Let's also check that the returned future will successfully complete



##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/raft/snapshot/incoming/IncomingSnapshotCopier.java:
##########
@@ -195,8 +195,8 @@ private CompletableFuture<?> prepareMvPartitionStorageForRebalance(Executor exec
             return completedFuture(null);
         }
 
-        return CompletableFuture.supplyAsync(() -> partitionSnapshotStorage.partition().reCreateMvPartitionStorage(), executor)
-                .thenCompose(mvPartitionStorage -> {
+        return partitionSnapshotStorage.partition().reCreateMvPartitionStorage()

Review Comment:
   Looks like MV partition will be destroyed on the current thread. Before this change, it would be executed on `executor`. Is this ok? Can partition destroy process be heavy?



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/compaction/Compactor.java:
##########
@@ -401,38 +415,24 @@ void mergeDeltaFileToMainFile(
     }
 
     /**
-     * Callback on destruction of the partition of the corresponding group.
+     * Prepares the compacter to destroy a partition.

Review Comment:
   ```suggestion
        * Prepares the compactor to destroy a partition.
   ```



##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/AbstractPageMemoryTableStorage.java:
##########
@@ -49,6 +51,8 @@ public abstract class AbstractPageMemoryTableStorage implements MvTableStorage {
 
     private volatile AtomicReferenceArray<AbstractPageMemoryMvPartitionStorage> mvPartitions;
 
+    protected final ConcurrentMap<Integer, CompletableFuture<Void>> partitionIdDestroyFuture = new ConcurrentHashMap<>();

Review Comment:
   I suggest to name this field in plular form (futures) or append 'Map' to the end so that the name gives a hint that it's not about just one future



-- 
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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/hash/PageMemoryHashIndexStorage.java:
##########
@@ -17,22 +17,35 @@
 
 package org.apache.ignite.internal.storage.pagememory.index.hash;
 
+import java.lang.invoke.MethodHandles;
+import java.lang.invoke.VarHandle;
 import org.apache.ignite.internal.schema.BinaryTuple;
 import org.apache.ignite.internal.storage.RowId;
+import org.apache.ignite.internal.storage.StorageClosedException;
 import org.apache.ignite.internal.storage.StorageException;
 import org.apache.ignite.internal.storage.index.HashIndexDescriptor;
 import org.apache.ignite.internal.storage.index.HashIndexStorage;
 import org.apache.ignite.internal.storage.index.IndexRow;
 import org.apache.ignite.internal.storage.pagememory.index.freelist.IndexColumns;
 import org.apache.ignite.internal.storage.pagememory.index.freelist.IndexColumnsFreeList;
 import org.apache.ignite.internal.util.Cursor;
-import org.apache.ignite.internal.util.CursorUtils;
+import org.apache.ignite.internal.util.IgniteSpinBusyLock;
 import org.apache.ignite.lang.IgniteInternalCheckedException;
 
 /**
- * Hash index storage implementation.
+ * Implementation of Hash index storage using Page Memory.
  */
 public class PageMemoryHashIndexStorage implements HashIndexStorage {
+    private static final VarHandle STARTED;
+
+    static {
+        try {
+            STARTED = MethodHandles.lookup().findVarHandle(PageMemoryHashIndexStorage.class, "started", boolean.class);

Review Comment:
   I think we should leave it as it is, we use less memory, and the complexity here is minimal.



-- 
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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/GroupPageStoresMap.java:
##########
@@ -47,52 +50,156 @@ public GroupPageStoresMap(LongOperationAsyncExecutor longOperationAsyncExecutor)
     }
 
     /**
-     * Puts the page stores for the group.
+     * Puts the page store for the group partition.
+     *
+     * @param groupId Group ID.
+     * @param partitionId Partition ID.
+     * @param pageStore Page store.
+     * @return Previous page store.
+     */
+    public @Nullable T put(Integer groupId, Integer partitionId, T pageStore) {
+        return longOperationAsyncExecutor.afterAsyncCompletion(() -> {
+                    PartitionPageStore<T> previous = groupIdPageStores
+                            .computeIfAbsent(groupId, id -> new GroupPageStores<>(groupId))
+                            .partitionIdPageStore
+                            .put(partitionId, new PartitionPageStore<>(partitionId, pageStore));
+
+                    return previous == null ? null : previous.pageStore;
+                }
+        );
+    }
+
+    /**
+     * Removes the page store for the group partition.
      *
-     * @param grpId Group ID.
-     * @param pageStores Page stores.
-     * @return Previous page stores.
+     * @param groupId Group ID.
+     * @param partitionId Partition ID.
+     * @return Removed page store.
      */
-    public @Nullable List<T> put(Integer grpId, List<T> pageStores) {
-        return longOperationAsyncExecutor.afterAsyncCompletion(() -> groupPageStores.put(grpId, pageStores));
+    public @Nullable T remove(Integer groupId, Integer partitionId) {
+        AtomicReference<PartitionPageStore<T>> partitionPageStoreRef = new AtomicReference<>();
+
+        groupIdPageStores.compute(groupId, (id, groupPageStores) -> {

Review Comment:
   Tried to 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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/storage-page-memory/src/test/java/org/apache/ignite/internal/storage/pagememory/PersistentPageMemoryMvTableStorageTest.java:
##########
@@ -101,4 +96,16 @@ public void testAbortRebalanceMvPartition() throws Exception {
     public void testFinishRebalanceMvPartition() throws Exception {
         super.testFinishRebalanceMvPartition();
     }
+
+    @Test
+    @Override
+    public void testDestroyPartition() throws Exception {
+        super.testDestroyPartition();
+
+        // Let's make sure that the checkpoint doesn't failed.

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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStoreManager.java:
##########
@@ -230,78 +269,86 @@ public long allocatePage(int grpId, int partId, byte flags) throws IgniteInterna
     }
 
     /**
-     * Initializing the file page stores for a group.
+     * Initialization of the page storage for the group partition.
      *
      * @param tableName Table name.
      * @param tableId Integer table id.
-     * @param partitions Partition number, must be greater than {@code 0} and less {@link PageIdAllocator#MAX_PARTITION_ID}.
+     * @param partitionId Partition ID, must be between {@code 0} (inclusive) and {@link PageIdAllocator#MAX_PARTITION_ID} (inclusive).
      * @throws IgniteInternalCheckedException If failed.
      */
-    public void initialize(String tableName, int tableId, int partitions) throws IgniteInternalCheckedException {
-        assert partitions > 0 && partitions < MAX_PARTITION_ID : partitions;
+    public void initialize(String tableName, int tableId, int partitionId) throws IgniteInternalCheckedException {
+        assert partitionId >= 0 && partitionId <= MAX_PARTITION_ID : partitionId;
 
-        initGroupDirLock.lock(tableId);
+        stripedLock.lock(tableId + partitionId);
 
         try {
-            if (!groupPageStores.containsPageStores(tableId)) {
-                List<FilePageStore> partitionFilePageStores = createFilePageStores(tableId, partitions);
+            if (!groupPageStores.contains(tableId, partitionId)) {
+                Path tableWorkDir = ensureGroupWorkDir(tableId);

Review Comment:
   Created a technical debt ticket https://issues.apache.org/jira/browse/IGNITE-18245



-- 
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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStoreManager.java:
##########
@@ -139,31 +149,63 @@ public void start() throws IgniteInternalCheckedException {
             throw new IgniteInternalCheckedException("Could not create work directory for page stores: " + dbDir, e);
         }
 
-        if (log.isWarnEnabled()) {
+        if (LOG.isWarnEnabled()) {
             String tmpDir = System.getProperty("java.io.tmpdir");
 
             if (tmpDir != null && this.dbDir.startsWith(tmpDir)) {
-                log.warn("Persistence store directory is in the temp directory and may be cleaned. "
+                LOG.warn("Persistence store directory is in the temp directory and may be cleaned. "
                         + "To avoid this change location of persistence directories [currentDir={}]", this.dbDir);
             }
         }
 
+        List<Path> toDelete = new ArrayList<>();
+
         try (Stream<Path> tmpFileStream = Files.find(
                 dbDir,
                 Integer.MAX_VALUE,
-                (path, basicFileAttributes) -> path.getFileName().toString().endsWith(TMP_FILE_SUFFIX)
-        )) {
-            List<Path> tmpFiles = tmpFileStream.collect(toList());
+                (path, basicFileAttributes) -> path.getFileName().toString().endsWith(TMP_FILE_SUFFIX))
+        ) {
+            toDelete.addAll(tmpFileStream.collect(toList()));
+        } catch (IOException e) {
+            throw new IgniteInternalCheckedException("Error while searching temporary files:" + dbDir, e);
+        }
+
+        Pattern delPartitionFilePatter = Pattern.compile(DEL_PART_FILE_REGEXP);
+
+        try (Stream<Path> delFileStream = Files.find(
+                dbDir,
+                Integer.MAX_VALUE,
+                (path, basicFileAttributes) -> path.getFileName().toString().endsWith(DEL_FILE_SUFFIX))
+        ) {
+            delFileStream.forEach(delFilePath -> {
+                Matcher matcher = delPartitionFilePatter.matcher(delFilePath.getFileName().toString());
 
-            if (!tmpFiles.isEmpty()) {
-                if (log.isInfoEnabled()) {
-                    log.info("Temporary files to be deleted: {}", tmpFiles.size());
+                if (!matcher.matches()) {
+                    throw new IgniteInternalException("Unknown file: " + delFilePath);
                 }
 
-                tmpFiles.forEach(IgniteUtils::deleteIfExists);
-            }
+                Path tableWorkDir = delFilePath.getParent();
+
+                int partitionId = Integer.parseInt(matcher.group(1));
+
+                toDelete.add(tableWorkDir.resolve(String.format(PART_FILE_TEMPLATE, partitionId)));
+
+                try {
+                    toDelete.addAll(List.of(findPartitionDeltaFiles(tableWorkDir, partitionId)));
+                } catch (IgniteInternalCheckedException e) {
+                    throw new IgniteInternalException("Error when searching delta files for partition:" + delFilePath, e);
+                }
+
+                toDelete.add(delFilePath);
+            });
         } catch (IOException e) {
-            throw new IgniteInternalCheckedException("Could not create work directory for page stores: " + dbDir, e);
+            throw new IgniteInternalCheckedException("Error while searching temporary files:" + dbDir, e);
+        }
+
+        if (!toDelete.isEmpty()) {
+            LOG.info("Files to be deleted: {}", toDelete);

Review Comment:
   Maybe this information is not needed by the user and let it be at the debug level?



-- 
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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/compaction/Compactor.java:
##########
@@ -65,13 +66,17 @@ public class Compactor extends IgniteWorker {
 
     private final @Nullable ThreadPoolExecutor threadPoolExecutor;
 
-    private final AtomicInteger deltaFileCount = new AtomicInteger();
+    /** Guarded by {@link #mux}. */
+    private boolean addedDeltaFiles;
 
     private final FilePageStoreManager filePageStoreManager;
 
     /** Thread local with buffers for the compaction threads. */
     private final ThreadLocal<ByteBuffer> threadBuf;

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 commented on a diff in pull request #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/compaction/Compactor.java:
##########
@@ -164,77 +170,90 @@ void waitDeltaFiles() {
     }
 
     /**
-     * Adds the number of delta files to compact.
-     *
-     * @param count Number of delta files.
+     * Callback on adding delta files so we can start compacting them.
      */
-    public void addDeltaFiles(int count) {
-        assert count >= 0;
-
-        if (count > 0) {
-            deltaFileCount.addAndGet(count);
+    public void onAddingDeltaFiles() {
+        synchronized (mux) {
+            addedDeltaFiles = true;
 
-            synchronized (mux) {
-                mux.notifyAll();
-            }
+            mux.notifyAll();
         }
     }
 
     /**
      * Merges delta files with partition files.
      */
     void doCompaction() {
-        // Let's collect one delta file for each partition.
-        Queue<IgniteBiTuple<FilePageStore, DeltaFilePageStoreIo>> queue = filePageStoreManager.allPageStores().stream()
-                .flatMap(List::stream)
-                .map(filePageStore -> {
+        while (true) {
+            // Let's collect one delta file for each partition.
+            ConcurrentLinkedQueue<DeltaFileToCompaction> queue = new ConcurrentLinkedQueue<>();
+
+            for (GroupPageStores<FilePageStore> groupPageStores : filePageStoreManager.allPageStores()) {
+                for (PartitionPageStore<FilePageStore> partitionPageStore : groupPageStores.getAll()) {
+                    FilePageStore filePageStore = partitionPageStore.pageStore();
+
                     DeltaFilePageStoreIo deltaFileToCompaction = filePageStore.getDeltaFileToCompaction();
 
-                    return deltaFileToCompaction == null ? null : new IgniteBiTuple<>(filePageStore, deltaFileToCompaction);
-                })
-                .filter(Objects::nonNull)
-                .collect(toCollection(ConcurrentLinkedQueue::new));
+                    if (!filePageStore.isMarkedToDestroy() && deltaFileToCompaction != null) {
+                        queue.add(new DeltaFileToCompaction(
+                                groupPageStores.groupId(),
+                                partitionPageStore.partitionId(),
+                                filePageStore,
+                                deltaFileToCompaction
+                        ));
+                    }
+                }
+            }
+
+            if (queue.isEmpty()) {

Review Comment:
   > All threads must wait until the last thread finishes its delta file, and only then the next batch is created.
   
   This is probably the most important sentence in my comment. Threads in thread-pool will wait until the last delta file is merged, and only then the next portion of delta files will be presented to the thread-pool.
   Is that intentionally planned? What was the reasoning?



-- 
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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/GroupPageStoresMap.java:
##########
@@ -47,52 +50,156 @@ public GroupPageStoresMap(LongOperationAsyncExecutor longOperationAsyncExecutor)
     }
 
     /**
-     * Puts the page stores for the group.
+     * Puts the page store for the group partition.
+     *
+     * @param groupId Group ID.
+     * @param partitionId Partition ID.
+     * @param pageStore Page store.
+     * @return Previous page store.
+     */
+    public @Nullable T put(Integer groupId, Integer partitionId, T pageStore) {
+        return longOperationAsyncExecutor.afterAsyncCompletion(() -> {
+                    PartitionPageStore<T> previous = groupIdPageStores

Review Comment:
   "minimal effort, until checkstyle passes" is not the best practice for coding.
   12 spaces is not a standard padding, why did you chose 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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/PartitionProcessingCounter.java:
##########
@@ -0,0 +1,83 @@
+/*
+ * 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;
+
+import java.lang.invoke.MethodHandles;
+import java.lang.invoke.VarHandle;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Helper class for tracking the completion of partition processing.
+ *
+ * <p>At the start of partition processing, you need to call {@link #onStartPartitionProcessing()}, at the end
+ * {@link #onFinishPartitionProcessing()}. When all partition processing is completed, the {@link #future()} will be completed.
+ *
+ * <p>It is recommended to use external synchronization for the correct operation of the {@link #counter partition processing counter} and
+ * the {@link #future future}.
+ */
+public class PartitionProcessingCounter {
+    private static final VarHandle COUNTER;
+
+    static {
+        try {
+            COUNTER = MethodHandles.lookup().findVarHandle(PartitionProcessingCounter.class, "counter", int.class);
+        } catch (ReflectiveOperationException e) {
+            throw new ExceptionInInitializerError(e);
+        }
+    }
+
+    /** Partition processing counter must be greater than or equal to zero. */
+    @SuppressWarnings("unused")
+    private volatile int counter;
+
+    /** Future that will be completed when the {@link #counter} is zero. */
+    private final CompletableFuture<Void> future = new CompletableFuture<>();
+
+    /**
+     * Callback at the start of partition processing.

Review Comment:
   Tried to 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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/checkpoint/Checkpointer.java:
##########
@@ -452,9 +456,21 @@ private void syncUpdatedPageStores(
                     return;
                 }
 
-                fsyncDeltaFilePageStoreOnCheckpointThread(entry.getKey(), entry.getValue());
+                GroupPartitionId partitionId = entry.getKey();
+
+                FilePageStore filePageStore = filePageStoreManager.getStore(partitionId.getGroupId(), partitionId.getPartitionId());
+
+                if (filePageStore == null || filePageStore.isMarkedToDestroy()) {
+                    continue;
+                }
+
+                currentCheckpointProgress.onStartPartitionProcessing(partitionId.getGroupId(), partitionId.getPartitionId());
+
+                fsyncDeltaFilePageStoreOnCheckpointThread(filePageStore, entry.getValue());
 
-                renameDeltaFileOnCheckpointThread(entry.getKey());
+                renameDeltaFileOnCheckpointThread(filePageStore, partitionId);
+
+                currentCheckpointProgress.onFinishPartitionProcessing(partitionId.getGroupId(), partitionId.getPartitionId());

Review Comment:
   Sounds reasonable, I'll 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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStoreManager.java:
##########
@@ -139,31 +149,63 @@ public void start() throws IgniteInternalCheckedException {
             throw new IgniteInternalCheckedException("Could not create work directory for page stores: " + dbDir, e);
         }
 
-        if (log.isWarnEnabled()) {
+        if (LOG.isWarnEnabled()) {
             String tmpDir = System.getProperty("java.io.tmpdir");
 
             if (tmpDir != null && this.dbDir.startsWith(tmpDir)) {
-                log.warn("Persistence store directory is in the temp directory and may be cleaned. "
+                LOG.warn("Persistence store directory is in the temp directory and may be cleaned. "
                         + "To avoid this change location of persistence directories [currentDir={}]", this.dbDir);
             }
         }
 
+        List<Path> toDelete = new ArrayList<>();
+
         try (Stream<Path> tmpFileStream = Files.find(
                 dbDir,
                 Integer.MAX_VALUE,
-                (path, basicFileAttributes) -> path.getFileName().toString().endsWith(TMP_FILE_SUFFIX)
-        )) {
-            List<Path> tmpFiles = tmpFileStream.collect(toList());
+                (path, basicFileAttributes) -> path.getFileName().toString().endsWith(TMP_FILE_SUFFIX))
+        ) {
+            toDelete.addAll(tmpFileStream.collect(toList()));
+        } catch (IOException e) {
+            throw new IgniteInternalCheckedException("Error while searching temporary files:" + dbDir, e);
+        }
+
+        Pattern delPartitionFilePatter = Pattern.compile(DEL_PART_FILE_REGEXP);
+
+        try (Stream<Path> delFileStream = Files.find(
+                dbDir,
+                Integer.MAX_VALUE,
+                (path, basicFileAttributes) -> path.getFileName().toString().endsWith(DEL_FILE_SUFFIX))
+        ) {
+            delFileStream.forEach(delFilePath -> {
+                Matcher matcher = delPartitionFilePatter.matcher(delFilePath.getFileName().toString());
 
-            if (!tmpFiles.isEmpty()) {
-                if (log.isInfoEnabled()) {
-                    log.info("Temporary files to be deleted: {}", tmpFiles.size());
+                if (!matcher.matches()) {
+                    throw new IgniteInternalException("Unknown file: " + delFilePath);
                 }
 
-                tmpFiles.forEach(IgniteUtils::deleteIfExists);
-            }
+                Path tableWorkDir = delFilePath.getParent();
+
+                int partitionId = Integer.parseInt(matcher.group(1));
+
+                toDelete.add(tableWorkDir.resolve(String.format(PART_FILE_TEMPLATE, partitionId)));
+
+                try {
+                    toDelete.addAll(List.of(findPartitionDeltaFiles(tableWorkDir, partitionId)));
+                } catch (IgniteInternalCheckedException e) {
+                    throw new IgniteInternalException("Error when searching delta files for partition:" + delFilePath, e);
+                }
+
+                toDelete.add(delFilePath);
+            });
         } catch (IOException e) {
-            throw new IgniteInternalCheckedException("Could not create work directory for page stores: " + dbDir, e);
+            throw new IgniteInternalCheckedException("Error while searching temporary files:" + dbDir, e);
+        }
+
+        if (!toDelete.isEmpty()) {
+            LOG.info("Files to be deleted: {}", toDelete);

Review Comment:
   There can be many files and for all tables, I don't think it's worth changing 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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/compaction/Compactor.java:
##########
@@ -357,14 +387,70 @@ void mergeDeltaFileToMainFile(
             return;
         }
 
+        if (filePageStore.isMarkedToDestroy()) {
+            return;
+        }
+
         boolean removed = filePageStore.removeDeltaFile(deltaFilePageStore);
 
         assert removed : filePageStore.filePath();
 
         deltaFilePageStore.markMergedToFilePageStore();
 
         deltaFilePageStore.stop(true);
+    }
+
+    /**
+     * Callback on destruction of the partition of the corresponding group.
+     *
+     * <p>If the partition compaction is in progress, then we will wait until it is completed so that there are no errors when we want to
+     * destroy the partition file and its delta file, and at this time its compaction occurs.
+     *
+     * @param groupId Group ID.
+     * @param partitionId Partition ID.
+     */
+    public void onPartitionDestruction(int groupId, int partitionId) throws IgniteInternalCheckedException {
+        CompletableFuture<Void> partitionProcessingFuture = processedPartitionMap.getProcessedPartitionFuture(groupId, partitionId);
+
+        if (partitionProcessingFuture != null) {
+            try {
+                // Time is taken arbitrarily, but long enough to allow time for the future to complete.
+                partitionProcessingFuture.get(10, SECONDS);
+            } catch (Exception e) {
+                throw new IgniteInternalCheckedException(
+                        IgniteStringFormatter.format(
+                                "Error waiting for partition processing to complete on compaction: [groupId={}, partitionId={}]",
+                                groupId,
+                                partitionId
+                        ),
+                        e
+                );
+            }
+        }
+    }
+
+    /**
+     * Delta file for compaction.
+     */
+    private static class DeltaFileToCompaction {

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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStoreManager.java:
##########
@@ -320,9 +369,9 @@ void stopAllGroupFilePageStores(boolean cleanFiles) {
             try {
                 stopGroupFilePageStores(partitionPageStores, cleanFiles);
 
-                log.info("Cleanup cache stores [total={}, cleanFiles={}]", partitionPageStores.size(), cleanFiles);
+                LOG.info("Cleanup cache stores [total={}, cleanFiles={}]", partitionPageStores.size(), cleanFiles);
             } catch (Exception e) {
-                log.info("Failed to gracefully stop page store managers", e);
+                LOG.info("Failed to gracefully stop page store managers", e);

Review Comment:
   Yep



-- 
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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/storage-api/src/main/java/org/apache/ignite/internal/storage/engine/MvTableStorage.java:
##########
@@ -59,6 +60,10 @@ public interface MvTableStorage {
     /**
      * Destroys a partition and all associated indices.
      *
+     * <p>This method will do nothing if there is no partition by ID, when trying to call methods to read or write (as well as all previous
+     * open cursors) for {@link MvPartitionStorage}, {@link HashIndexStorage} and {@link SortedIndexStorage}, {@link StorageClosedException}
+     * will be thrown.
+     *

Review Comment:
   What exactly is not clear?



-- 
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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/PersistentPageMemoryTableStorage.java:
##########
@@ -418,4 +430,43 @@ public CompletableFuture<Void> finishRebalanceMvPartition(int partitionId) {
         // TODO: IGNITE-18029 Implement
         throw new UnsupportedOperationException();
     }
+
+    @Override
+    public void destroyMvPartitionStorage(AbstractPageMemoryMvPartitionStorage mvPartitionStorage) throws StorageException {
+        int partitionId = mvPartitionStorage.partitionId();
+
+        CompletableFuture<Void> previousFuture = destroyFutureByPartitionId.put(partitionId, new CompletableFuture<>());
+
+        assert previousFuture == null : "Previous destruction of the partition has not completed: " + partitionId;

Review Comment:
   I don’t understand why return the current future while no one uses it, but later they will be on the assignment recalculation.



-- 
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 merged pull request #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


-- 
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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/compaction/Compactor.java:
##########
@@ -164,77 +170,90 @@ void waitDeltaFiles() {
     }
 
     /**
-     * Adds the number of delta files to compact.
-     *
-     * @param count Number of delta files.
+     * Callback on adding delta files so we can start compacting them.
      */
-    public void addDeltaFiles(int count) {
-        assert count >= 0;
-
-        if (count > 0) {
-            deltaFileCount.addAndGet(count);
+    public void onAddingDeltaFiles() {
+        synchronized (mux) {
+            addedDeltaFiles = true;
 
-            synchronized (mux) {
-                mux.notifyAll();
-            }
+            mux.notifyAll();
         }
     }
 
     /**
      * Merges delta files with partition files.
      */
     void doCompaction() {
-        // Let's collect one delta file for each partition.
-        Queue<IgniteBiTuple<FilePageStore, DeltaFilePageStoreIo>> queue = filePageStoreManager.allPageStores().stream()
-                .flatMap(List::stream)
-                .map(filePageStore -> {
+        while (true) {

Review Comment:
   I think this will become clearer from the new description of the method.



-- 
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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/PersistentPageMemoryTableStorage.java:
##########
@@ -86,41 +97,29 @@ public PersistentPageMemoryDataRegion dataRegion() {
         return dataRegion;
     }
 
-    /** {@inheritDoc} */
     @Override
     public boolean isVolatile() {
         return false;
     }
 
-    /** {@inheritDoc} */
-    @Override
-    public void start() throws StorageException {
-        super.start();
-
-        TableView tableView = tableCfg.value();
-
-        try {
-            dataRegion.filePageStoreManager().initialize(tableView.name(), tableView.tableId(), tableView.partitions());
-
-            int deltaFileCount = dataRegion.filePageStoreManager().getStores(tableView.tableId()).stream()
-                    .mapToInt(FilePageStore::deltaFileCount)
-                    .sum();
-
-            dataRegion.checkpointManager().addDeltaFileCountForCompaction(deltaFileCount);
-        } catch (IgniteInternalCheckedException e) {
-            throw new StorageException("Error initializing file page stores for table: " + tableView.name(), e);
-        }
-    }
-
-    /** {@inheritDoc} */
     @Override
     public void destroy() throws StorageException {
         close(true);
     }
 
-    /** {@inheritDoc} */
     @Override
     public PersistentPageMemoryMvPartitionStorage createMvPartitionStorage(int partitionId) {
+        CompletableFuture<Void> partitionDestroyFuture = destroyFutureByPartitionId.get(partitionId);
+
+        if (partitionDestroyFuture != null) {
+            try {
+                // Time is chosen randomly (long enough) so as not to call #join().
+                partitionDestroyFuture.get(10, TimeUnit.SECONDS);

Review Comment:
   Tried to 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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStoreManager.java:
##########
@@ -139,31 +149,63 @@ public void start() throws IgniteInternalCheckedException {
             throw new IgniteInternalCheckedException("Could not create work directory for page stores: " + dbDir, e);
         }
 
-        if (log.isWarnEnabled()) {
+        if (LOG.isWarnEnabled()) {
             String tmpDir = System.getProperty("java.io.tmpdir");
 
             if (tmpDir != null && this.dbDir.startsWith(tmpDir)) {
-                log.warn("Persistence store directory is in the temp directory and may be cleaned. "
+                LOG.warn("Persistence store directory is in the temp directory and may be cleaned. "
                         + "To avoid this change location of persistence directories [currentDir={}]", this.dbDir);
             }
         }
 
+        List<Path> toDelete = new ArrayList<>();
+
         try (Stream<Path> tmpFileStream = Files.find(
                 dbDir,
                 Integer.MAX_VALUE,
-                (path, basicFileAttributes) -> path.getFileName().toString().endsWith(TMP_FILE_SUFFIX)
-        )) {
-            List<Path> tmpFiles = tmpFileStream.collect(toList());
+                (path, basicFileAttributes) -> path.getFileName().toString().endsWith(TMP_FILE_SUFFIX))
+        ) {
+            toDelete.addAll(tmpFileStream.collect(toList()));
+        } catch (IOException e) {
+            throw new IgniteInternalCheckedException("Error while searching temporary files:" + dbDir, e);
+        }
+
+        Pattern delPartitionFilePatter = Pattern.compile(DEL_PART_FILE_REGEXP);
+
+        try (Stream<Path> delFileStream = Files.find(
+                dbDir,
+                Integer.MAX_VALUE,
+                (path, basicFileAttributes) -> path.getFileName().toString().endsWith(DEL_FILE_SUFFIX))
+        ) {
+            delFileStream.forEach(delFilePath -> {
+                Matcher matcher = delPartitionFilePatter.matcher(delFilePath.getFileName().toString());
 
-            if (!tmpFiles.isEmpty()) {
-                if (log.isInfoEnabled()) {
-                    log.info("Temporary files to be deleted: {}", tmpFiles.size());
+                if (!matcher.matches()) {
+                    throw new IgniteInternalException("Unknown file: " + delFilePath);
                 }
 
-                tmpFiles.forEach(IgniteUtils::deleteIfExists);
-            }
+                Path tableWorkDir = delFilePath.getParent();
+
+                int partitionId = Integer.parseInt(matcher.group(1));
+
+                toDelete.add(tableWorkDir.resolve(String.format(PART_FILE_TEMPLATE, partitionId)));
+
+                try {
+                    toDelete.addAll(List.of(findPartitionDeltaFiles(tableWorkDir, partitionId)));
+                } catch (IgniteInternalCheckedException e) {
+                    throw new IgniteInternalException("Error when searching delta files for partition:" + delFilePath, e);
+                }
+
+                toDelete.add(delFilePath);
+            });
         } catch (IOException e) {
-            throw new IgniteInternalCheckedException("Could not create work directory for page stores: " + dbDir, e);
+            throw new IgniteInternalCheckedException("Error while searching temporary files:" + dbDir, e);
+        }
+
+        if (!toDelete.isEmpty()) {
+            LOG.info("Files to be deleted: {}", toDelete);

Review Comment:
   User may not understand, why files are suddenly being deleted.



-- 
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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/compaction/Compactor.java:
##########
@@ -42,15 +43,13 @@
 import org.apache.ignite.internal.util.IgniteUtils;
 import org.apache.ignite.internal.util.worker.IgniteWorker;
 import org.apache.ignite.internal.util.worker.IgniteWorkerListener;
-import org.apache.ignite.lang.IgniteInternalCheckedException;
 import org.apache.ignite.lang.IgniteInternalException;
-import org.apache.ignite.lang.IgniteStringFormatter;
 import org.jetbrains.annotations.Nullable;
 
 /**
  * Entity to compact delta files.
  *
- * <p>To start compacting delta files, you need to notify about the appearance of {@link #onAddingDeltaFiles() delta files ready for
+ * <p>To start compacting delta files, you need to notify about the appearance of {@link #triggerCompaction() delta files ready for

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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/GroupPageStoresMap.java:
##########
@@ -47,52 +50,156 @@ public GroupPageStoresMap(LongOperationAsyncExecutor longOperationAsyncExecutor)
     }
 
     /**
-     * Puts the page stores for the group.
+     * Puts the page store for the group partition.
+     *
+     * @param groupId Group ID.
+     * @param partitionId Partition ID.
+     * @param pageStore Page store.
+     * @return Previous page store.
+     */
+    public @Nullable T put(Integer groupId, Integer partitionId, T pageStore) {
+        return longOperationAsyncExecutor.afterAsyncCompletion(() -> {
+                    PartitionPageStore<T> previous = groupIdPageStores

Review Comment:
   This is not a bug, my idea is so formatted and does not go beyond code style checks.



-- 
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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/storage-api/src/main/java/org/apache/ignite/internal/storage/engine/MvTableStorage.java:
##########
@@ -59,6 +60,10 @@ public interface MvTableStorage {
     /**
      * Destroys a partition and all associated indices.
      *
+     * <p>This method will do nothing if there is no partition by ID, when trying to call methods to read or write (as well as all previous
+     * open cursors) for {@link MvPartitionStorage}, {@link HashIndexStorage} and {@link SortedIndexStorage}, {@link StorageClosedException}
+     * will be thrown.
+     *

Review Comment:
   Deleted this comment.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

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


[GitHub] [ignite-3] rpuch commented on a diff in pull request #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/compaction/Compactor.java:
##########
@@ -348,6 +374,10 @@ void mergeDeltaFileToMainFile(
             return;
         }
 
+        if (filePageStore.isMarkedToDestroy()) {

Review Comment:
   To make sure that we always do these checks together in this method. After modifications someone might accidentally add just one check instead of 2.



-- 
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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/AbstractPageMemoryTableStorage.java:
##########
@@ -185,7 +212,7 @@ protected void close(boolean destroy) throws StorageException {
             AbstractPageMemoryMvPartitionStorage partition = mvPartitions.getAndUpdate(i, p -> null);
 
             if (partition != null) {
-                closeables.add(destroy ? partition::destroy : partition::close);
+                closeables.add(destroy ? () -> destroyMvPartitionStorage(partition) : partition::close);

Review Comment:
   You're right, I need to fix this.



-- 
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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/checkpoint/Checkpointer.java:
##########
@@ -442,7 +444,8 @@ boolean writePages(
     }
 
     private void syncUpdatedPageStores(
-            ConcurrentMap<GroupPartitionId, LongAdder> updatedPartitions
+            ConcurrentMap<GroupPartitionId, LongAdder> updatedPartitions,
+            CheckpointProgressImpl currentCheckpointProgress

Review Comment:
   It should be used, fixed 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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/impl/TestMvPartitionStorage.java:
##########
@@ -398,23 +421,35 @@ public ReadResult next() {
 
     @Override
     public @Nullable RowId closestRowId(RowId lowerBound) throws StorageException {
+        checkClosed();
+
         return map.ceilingKey(lowerBound);
     }
 
-    /** {@inheritDoc} */
     @Override
     public long rowsCount() {
+        checkClosed();
+
         return map.size();
     }
 
-    /** {@inheritDoc} */
     @Override
     public void close() {
-        // No-op.
+        started = false;

Review Comment:
   Change 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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/AbstractMvTableStorageTest.java:
##########
@@ -419,11 +505,69 @@ private static void createTestTable(TableConfiguration tableConfig) {
         assertThat(createTableFuture, willCompleteSuccessfully());
     }
 
-    private static <T> List<T> getAll(Cursor<T> cursor) {
+    private static <T> List<T> getAll(Cursor<T> cursor) throws Exception {
         try (cursor) {
             return cursor.stream().collect(Collectors.toList());
-        } catch (Exception e) {
-            throw new RuntimeException(e);
         }
     }
+
+    private void checkStorageDestroyed(MvPartitionStorage storage) {
+        int partId = PARTITION_ID;
+
+        assertThrows(StorageClosedException.class, () -> storage.runConsistently(() -> null));
+
+        assertThrows(StorageClosedException.class, storage::flush);
+
+        assertThrows(StorageClosedException.class, storage::lastAppliedIndex);
+        assertThrows(StorageClosedException.class, storage::lastAppliedTerm);
+        assertThrows(StorageClosedException.class, storage::persistedIndex);

Review Comment:
   Add 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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/compaction/Compactor.java:
##########
@@ -164,77 +170,90 @@ void waitDeltaFiles() {
     }
 
     /**
-     * Adds the number of delta files to compact.
-     *
-     * @param count Number of delta files.
+     * Callback on adding delta files so we can start compacting them.
      */
-    public void addDeltaFiles(int count) {
-        assert count >= 0;
-
-        if (count > 0) {
-            deltaFileCount.addAndGet(count);
+    public void onAddingDeltaFiles() {
+        synchronized (mux) {
+            addedDeltaFiles = true;
 
-            synchronized (mux) {
-                mux.notifyAll();
-            }
+            mux.notifyAll();
         }
     }
 
     /**
      * Merges delta files with partition files.
      */
     void doCompaction() {
-        // Let's collect one delta file for each partition.
-        Queue<IgniteBiTuple<FilePageStore, DeltaFilePageStoreIo>> queue = filePageStoreManager.allPageStores().stream()
-                .flatMap(List::stream)
-                .map(filePageStore -> {
+        while (true) {
+            // Let's collect one delta file for each partition.
+            ConcurrentLinkedQueue<DeltaFileToCompaction> queue = new ConcurrentLinkedQueue<>();
+
+            for (GroupPageStores<FilePageStore> groupPageStores : filePageStoreManager.allPageStores()) {
+                for (PartitionPageStore<FilePageStore> partitionPageStore : groupPageStores.getAll()) {
+                    FilePageStore filePageStore = partitionPageStore.pageStore();
+
                     DeltaFilePageStoreIo deltaFileToCompaction = filePageStore.getDeltaFileToCompaction();
 
-                    return deltaFileToCompaction == null ? null : new IgniteBiTuple<>(filePageStore, deltaFileToCompaction);
-                })
-                .filter(Objects::nonNull)
-                .collect(toCollection(ConcurrentLinkedQueue::new));
+                    if (!filePageStore.isMarkedToDestroy() && deltaFileToCompaction != null) {
+                        queue.add(new DeltaFileToCompaction(
+                                groupPageStores.groupId(),
+                                partitionPageStore.partitionId(),
+                                filePageStore,
+                                deltaFileToCompaction
+                        ));
+                    }
+                }
+            }
+
+            if (queue.isEmpty()) {
+                break;
+            }
+
+            updateHeartbeat();
 
-        assert !queue.isEmpty();
+            int threads = threadPoolExecutor == null ? 1 : threadPoolExecutor.getMaximumPoolSize();

Review Comment:
   If you configured 1 thread, then we do not need a pool.



-- 
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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/storage-page-memory/src/test/java/org/apache/ignite/internal/storage/pagememory/PersistentPageMemoryMvTableStorageTest.java:
##########
@@ -31,13 +33,13 @@
 import org.junit.jupiter.api.AfterEach;
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.Test;
 import org.junit.jupiter.api.extension.ExtendWith;
 
 /**
  * Tests for {@link PersistentPageMemoryTableStorage} class.
  */
-@ExtendWith(WorkDirectoryExtension.class)
-@ExtendWith(ConfigurationExtension.class)
+@ExtendWith({ConfigurationExtension.class, WorkDirectoryExtension.class})

Review Comment:
   It's just that I like it better, I suggest leaving it as it is.



-- 
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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/storage-api/src/main/java/org/apache/ignite/internal/storage/engine/MvTableStorage.java:
##########
@@ -59,6 +60,10 @@ public interface MvTableStorage {
     /**
      * Destroys a partition and all associated indices.
      *
+     * <p>This method will do nothing if there is no partition by ID, when trying to call methods to read or write (as well as all previous
+     * open cursors) for {@link MvPartitionStorage}, {@link HashIndexStorage} and {@link SortedIndexStorage}, {@link StorageClosedException}
+     * will be thrown.
+     *

Review Comment:
   The grammatical structure is very convoluted. Maybe something like
   ```
   This method will ...  if:
    - blah
    - foo
    - bar
   ```
   will be easier to read



-- 
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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/checkpoint/Checkpointer.java:
##########
@@ -452,9 +456,21 @@ private void syncUpdatedPageStores(
                     return;
                 }
 
-                fsyncDeltaFilePageStoreOnCheckpointThread(entry.getKey(), entry.getValue());
+                GroupPartitionId partitionId = entry.getKey();
+
+                FilePageStore filePageStore = filePageStoreManager.getStore(partitionId.getGroupId(), partitionId.getPartitionId());
+
+                if (filePageStore == null || filePageStore.isMarkedToDestroy()) {

Review Comment:
   How can it have 332 lines if github shows lines 460-463 in this particular comment? I agree with Roman, it looks very much like code duplication



-- 
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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/GroupPageStoresMap.java:
##########
@@ -47,52 +50,156 @@ public GroupPageStoresMap(LongOperationAsyncExecutor longOperationAsyncExecutor)
     }
 
     /**
-     * Puts the page stores for the group.
+     * Puts the page store for the group partition.
+     *
+     * @param groupId Group ID.
+     * @param partitionId Partition ID.
+     * @param pageStore Page store.
+     * @return Previous page store.
+     */
+    public @Nullable T put(Integer groupId, Integer partitionId, T pageStore) {
+        return longOperationAsyncExecutor.afterAsyncCompletion(() -> {
+                    PartitionPageStore<T> previous = groupIdPageStores
+                            .computeIfAbsent(groupId, id -> new GroupPageStores<>(groupId))
+                            .partitionIdPageStore
+                            .put(partitionId, new PartitionPageStore<>(partitionId, pageStore));
+
+                    return previous == null ? null : previous.pageStore;
+                }
+        );
+    }
+
+    /**
+     * Removes the page store for the group partition.
      *
-     * @param grpId Group ID.
-     * @param pageStores Page stores.
-     * @return Previous page stores.
+     * @param groupId Group ID.
+     * @param partitionId Partition ID.
+     * @return Removed page store.
      */
-    public @Nullable List<T> put(Integer grpId, List<T> pageStores) {
-        return longOperationAsyncExecutor.afterAsyncCompletion(() -> groupPageStores.put(grpId, pageStores));
+    public @Nullable T remove(Integer groupId, Integer partitionId) {
+        AtomicReference<PartitionPageStore<T>> partitionPageStoreRef = new AtomicReference<>();
+
+        groupIdPageStores.compute(groupId, (id, groupPageStores) -> {

Review Comment:
   What can be used under the hood is not important, in this case the map.



-- 
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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/PartitionProcessingCounterMap.java:
##########
@@ -0,0 +1,90 @@
+/*
+ * 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;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Helper class for thread-safe work with {@link PartitionProcessingCounter} for any partition of any group.
+ */
+public class PartitionProcessingCounterMap {
+    private final ConcurrentMap<GroupPartitionId, PartitionProcessingCounter> processedPartitions = new ConcurrentHashMap<>();
+
+    /**
+     * Callback at the beginning of checkpoint processing of a partition, for example, when writing dirty pages or executing a fsync.
+     *
+     * @param groupId Group ID.
+     * @param partitionId Partition ID.
+     */
+    public void onStartPartitionProcessing(int groupId, int partitionId) {
+        GroupPartitionId groupPartitionId = new GroupPartitionId(groupId, partitionId);
+
+        processedPartitions.compute(groupPartitionId, (id, partitionProcessingCounter) -> {
+            if (partitionProcessingCounter == null) {
+                PartitionProcessingCounter counter = new PartitionProcessingCounter();
+
+                counter.onStartPartitionProcessing();
+
+                return counter;
+            }
+
+            partitionProcessingCounter.onStartPartitionProcessing();
+
+            return partitionProcessingCounter;
+        });
+    }

Review Comment:
   Not suitable for us, you can see the reason in the description for `org.apache.ignite.internal.pagememory.persistence.PartitionProcessingCounter`



-- 
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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/VolatilePageMemoryTableStorage.java:
##########
@@ -157,4 +155,10 @@ public CompletableFuture<Void> finishRebalanceMvPartition(int partitionId) {
         // TODO: IGNITE-18028 Implement
         throw new UnsupportedOperationException();
     }
+
+    @Override
+    public void destroyMvPartitionStorage(AbstractPageMemoryMvPartitionStorage mvPartitionStorage) throws StorageException {

Review Comment:
   why?



-- 
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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/PersistentPageMemoryTableStorage.java:
##########
@@ -418,4 +430,43 @@ public CompletableFuture<Void> finishRebalanceMvPartition(int partitionId) {
         // TODO: IGNITE-18029 Implement
         throw new UnsupportedOperationException();
     }
+
+    @Override
+    public void destroyMvPartitionStorage(AbstractPageMemoryMvPartitionStorage mvPartitionStorage) throws StorageException {
+        int partitionId = mvPartitionStorage.partitionId();
+
+        CompletableFuture<Void> previousFuture = destroyFutureByPartitionId.put(partitionId, new CompletableFuture<>());
+
+        assert previousFuture == null : "Previous destruction of the partition has not completed: " + partitionId;

Review Comment:
   I see, didn't notice the other usage



-- 
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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/compaction/Compactor.java:
##########
@@ -338,6 +360,10 @@ void mergeDeltaFileToMainFile(
                 return;
             }
 
+            if (filePageStore.isMarkedToDestroy()) {

Review Comment:
   I think that this does not apply to the task of removing the partition, if necessary, we will fix it in another ticket.



-- 
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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/storage-api/src/main/java/org/apache/ignite/internal/storage/engine/MvTableStorage.java:
##########
@@ -59,6 +60,10 @@ public interface MvTableStorage {
     /**
      * Destroys a partition and all associated indices.
      *
+     * <p>This method will do nothing if there is no partition by ID, when trying to call methods to read or write (as well as all previous

Review Comment:
   Try 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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/GroupPageStoresMap.java:
##########
@@ -47,52 +50,156 @@ public GroupPageStoresMap(LongOperationAsyncExecutor longOperationAsyncExecutor)
     }
 
     /**
-     * Puts the page stores for the group.
+     * Puts the page store for the group partition.
+     *
+     * @param groupId Group ID.
+     * @param partitionId Partition ID.
+     * @param pageStore Page store.
+     * @return Previous page store.
+     */
+    public @Nullable T put(Integer groupId, Integer partitionId, T pageStore) {
+        return longOperationAsyncExecutor.afterAsyncCompletion(() -> {
+                    PartitionPageStore<T> previous = groupIdPageStores
+                            .computeIfAbsent(groupId, id -> new GroupPageStores<>(groupId))
+                            .partitionIdPageStore
+                            .put(partitionId, new PartitionPageStore<>(partitionId, pageStore));
+
+                    return previous == null ? null : previous.pageStore;
+                }
+        );
+    }
+
+    /**
+     * Removes the page store for the group partition.
      *
-     * @param grpId Group ID.
-     * @param pageStores Page stores.
-     * @return Previous page stores.
+     * @param groupId Group ID.
+     * @param partitionId Partition ID.
+     * @return Removed page store.
      */
-    public @Nullable List<T> put(Integer grpId, List<T> pageStores) {
-        return longOperationAsyncExecutor.afterAsyncCompletion(() -> groupPageStores.put(grpId, pageStores));
+    public @Nullable T remove(Integer groupId, Integer partitionId) {
+        AtomicReference<PartitionPageStore<T>> partitionPageStoreRef = new AtomicReference<>();
+
+        groupIdPageStores.compute(groupId, (id, groupPageStores) -> {
+            if (groupPageStores == null) {
+                return null;
+            }
+
+            partitionPageStoreRef.set(groupPageStores.partitionIdPageStore.remove(partitionId));
+
+            if (groupPageStores.partitionIdPageStore.isEmpty()) {
+                return null;
+            }
+
+            return groupPageStores;
+        });
+
+        PartitionPageStore<T> partitionPageStore = partitionPageStoreRef.get();
+
+        return partitionPageStore == null ? null : partitionPageStore.pageStore;
     }
 
     /**
      * Returns the page stores for the group.
      *
-     * @param grpId Group ID.
+     * @param groupId Group ID.
      */
-    public @Nullable List<T> get(Integer grpId) {
-        return groupPageStores.get(grpId);
+    public @Nullable GroupPageStores<T> get(Integer groupId) {
+        return groupIdPageStores.get(groupId);
     }
 
     /**
-     * Returns {@code true} if a page stores exists for the group.
+     * Returns {@code true} if a page store exists for the group partition.
      *
-     * @param grpId Group ID.
+     * @param groupId Group ID.
+     * @param partitionId Partition ID.
      */
-    public boolean containsPageStores(Integer grpId) {
-        return groupPageStores.containsKey(grpId);
+    public boolean contains(Integer groupId, Integer partitionId) {
+        GroupPageStores<T> groupPageStores = groupIdPageStores.get(groupId);
+
+        return groupPageStores != null && groupPageStores.partitionIdPageStore.containsKey(partitionId);
     }
 
     /**
-     * Returns all page stores of all groups.
+     * Returns a view of all page stores of all groups.
      */
-    public Collection<List<T>> allPageStores() {
-        return groupPageStores.values();
+    public Collection<GroupPageStores<T>> getAll() {
+        return unmodifiableCollection(groupIdPageStores.values());
     }
 
     /**
      * Clears all page stores of all groups.
      */
     public void clear() {
-        groupPageStores.clear();
+        groupIdPageStores.clear();
     }
 
     /**
      * Returns the number of groups for which there are page stores.
      */
     public int groupCount() {
-        return groupPageStores.size();
+        return groupIdPageStores.size();
+    }
+
+    /**
+     * Group partition page stores.
+     */
+    public static class GroupPageStores<T extends PageStore> {
+        private final int groupId;
+
+        private final ConcurrentMap<Integer, PartitionPageStore<T>> partitionIdPageStore = new ConcurrentHashMap<>();
+
+        private GroupPageStores(int groupId) {
+            this.groupId = groupId;
+        }
+
+        /**
+         * Returns the group ID.
+         */
+        public int groupId() {
+            return groupId;
+        }
+
+        /**
+         * Returns the partition's page store.
+         */
+        @Nullable
+        public PartitionPageStore<T> get(Integer partitionId) {
+            return partitionIdPageStore.get(partitionId);
+        }
+
+        /**
+         * Returns a view of the group's partition page stores.
+         */
+        public Collection<PartitionPageStore<T>> getAll() {
+            return unmodifiableCollection(partitionIdPageStore.values());
+        }
+    }
+
+    /**
+     * Partition page store.
+     */
+    public static class PartitionPageStore<T extends PageStore> {
+        private final int partitionId;
+
+        private final T pageStore;

Review Comment:
   We don't have a field there.



-- 
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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/checkpoint/Checkpointer.java:
##########
@@ -452,9 +456,21 @@ private void syncUpdatedPageStores(
                     return;
                 }
 
-                fsyncDeltaFilePageStoreOnCheckpointThread(entry.getKey(), entry.getValue());
+                GroupPartitionId partitionId = entry.getKey();
+
+                FilePageStore filePageStore = filePageStoreManager.getStore(partitionId.getGroupId(), partitionId.getPartitionId());
+
+                if (filePageStore == null || filePageStore.isMarkedToDestroy()) {

Review Comment:
   fix code duplication



-- 
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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/checkpoint/Checkpointer.java:
##########
@@ -500,13 +501,13 @@ private void syncUpdatedPageStores(
                             FilePageStore filePageStore = filePageStoreManager.getStore(partId.getGroupId(), partId.getPartitionId());
 
                             if (filePageStore != null && !filePageStore.isMarkedToDestroy()) {
-                                currentCheckpointProgress.onStartPartitionProcessing(partId.getGroupId(), partId.getPartitionId());
+                                currentCheckpointProgress.onStartPartitionProcessing(partId);
 
                                 fsyncDeltaFilePageStoreOnCheckpointThread(filePageStore, entry.getValue());
 
                                 renameDeltaFileOnCheckpointThread(filePageStore, partId);
 
-                                currentCheckpointProgress.onFinishPartitionProcessing(partId.getGroupId(), partId.getPartitionId());
+                                currentCheckpointProgress.onFinishPartitionProcessing(partId);

Review Comment:
   That's how it is in the latest code.



-- 
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] rpuch commented on a diff in pull request #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/AbstractPageMemoryTableStorage.java:
##########
@@ -185,7 +212,7 @@ protected void close(boolean destroy) throws StorageException {
             AbstractPageMemoryMvPartitionStorage partition = mvPartitions.getAndUpdate(i, p -> null);
 
             if (partition != null) {
-                closeables.add(destroy ? partition::destroy : partition::close);
+                closeables.add(destroy ? () -> destroyMvPartitionStorage(partition) : partition::close);

Review Comment:
   Is it ok that we don't wait for any partition to be actually destroyed (as we just ignore the returned futures)?



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/checkpoint/Checkpointer.java:
##########
@@ -442,7 +444,8 @@ boolean writePages(
     }
 
     private void syncUpdatedPageStores(
-            ConcurrentMap<GroupPartitionId, LongAdder> updatedPartitions
+            ConcurrentMap<GroupPartitionId, LongAdder> updatedPartitions,
+            CheckpointProgressImpl currentCheckpointProgress

Review Comment:
   This parameter does not seem to be used



##########
modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java:
##########
@@ -378,21 +381,38 @@ public RocksDbMvPartitionStorage getOrCreateMvPartition(int partitionId) throws
     }
 
     @Override
-    public void destroyPartition(int partitionId) throws StorageException {
+    public CompletableFuture<Void> destroyPartition(int partitionId) {
         checkPartitionId(partitionId);
 
+        CompletableFuture<Void> destroyPartitionFuture = new CompletableFuture<>();
+
+        CompletableFuture<Void> previousDestroyPartitionFuture = partitionIdDestroyFutureMap.putIfAbsent(
+                partitionId,
+                destroyPartitionFuture
+        );
+
+        if (previousDestroyPartitionFuture != null) {
+            return previousDestroyPartitionFuture;
+        }
+
         RocksDbMvPartitionStorage mvPartition = partitions.getAndSet(partitionId, null);
 
         if (mvPartition != null) {
-            //TODO IGNITE-17626 Destroy indexes as well...
-            mvPartition.destroy();
-
             try {
+                //TODO IGNITE-17626 Destroy indexes as well...
+                mvPartition.destroy();

Review Comment:
   Can this operation take a long time? If not, please add a comment stating this; if yes, probably a TODO should be added to address this later as `MvTableStorage#destroyPartition()` should be async after the latest changes.



-- 
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 #1325: IGNITE-17132 [Native Persistence 3.0] Implement partition destruction for persistent PageMemory

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/compaction/Compactor.java:
##########
@@ -348,6 +374,10 @@ void mergeDeltaFileToMainFile(
             return;
         }
 
+        if (filePageStore.isMarkedToDestroy()) {

Review Comment:
   I think we do not need a separate method for two slightly different checks, this complicates the code a bit.



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